diff --git a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/HttpClient.java b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/HttpClient.java index 9e51feb090a2..b455fde0ff98 100644 --- a/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/HttpClient.java +++ b/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/http/HttpClient.java @@ -75,12 +75,7 @@ public Thread newThread(final Runnable r) { } }); - taskExecutor.scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - peerSelector.refresh(); - } - }, 0, 5, TimeUnit.SECONDS); + taskExecutor.scheduleWithFixedDelay(peerSelector::refresh, 0, 5, TimeUnit.SECONDS); } diff --git a/nifi-commons/nifi-write-ahead-log/src/test/java/org/apache/nifi/wali/TestSequentialAccessWriteAheadLog.java b/nifi-commons/nifi-write-ahead-log/src/test/java/org/apache/nifi/wali/TestSequentialAccessWriteAheadLog.java index 84d42dd54604..d37677cdd39d 100644 --- a/nifi-commons/nifi-write-ahead-log/src/test/java/org/apache/nifi/wali/TestSequentialAccessWriteAheadLog.java +++ b/nifi-commons/nifi-write-ahead-log/src/test/java/org/apache/nifi/wali/TestSequentialAccessWriteAheadLog.java @@ -332,19 +332,16 @@ public void testUpdatePerformance() throws IOException, InterruptedException { for (int j = 0; j < 2; j++) { for (int i = 0; i < numThreads; i++) { - final Thread t = new Thread(new Runnable() { - @Override - public void run() { - final List batch = new ArrayList<>(); - for (int i = 0; i < updateCountPerThread / batchSize; i++) { - batch.clear(); - for (int j = 0; j < batchSize; j++) { - final DummyRecord record = new DummyRecord(String.valueOf(i), UpdateType.CREATE); - batch.add(record); - } - - assertThrows(Throwable.class, () -> repo.update(batch, false)); + final Thread t = new Thread(() -> { + final List batch = new ArrayList<>(); + for (int i1 = 0; i1 < updateCountPerThread / batchSize; i1++) { + batch.clear(); + for (int j1 = 0; j1 < batchSize; j1++) { + final DummyRecord record = new DummyRecord(String.valueOf(i1), UpdateType.CREATE); + batch.add(record); } + + assertThrows(Throwable.class, () -> repo.update(batch, false)); } }); diff --git a/nifi-extension-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/TestChannel.java b/nifi-extension-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/TestChannel.java index 94041c15d86e..a5cd534359ca 100644 --- a/nifi-extension-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/TestChannel.java +++ b/nifi-extension-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/test/java/org/apache/nifi/amqp/processors/TestChannel.java @@ -148,7 +148,7 @@ public void close() throws IOException, TimeoutException { } @Override - public void close(int closeCode, String closeMessage) throws IOException, TimeoutException { + public void close(int closeCode, String closeMessage) { throw new UnsupportedOperationException("This method is not currently supported as it is not used by current API in testing"); } @@ -285,14 +285,11 @@ private void discard(final String exchange, final String routingKey, boolean man final byte[] body) { // NO ROUTE. Invoke return listener async for (final ReturnListener listener : returnListeners) { - this.executorService.execute(new Runnable() { - @Override - public void run() { - try { - listener.handleReturn(-9, "Rejecting", exchange, routingKey, props, body); - } catch (Exception e) { - throw new IllegalStateException("Failed to send return message", e); - } + this.executorService.execute(() -> { + try { + listener.handleReturn(-9, "Rejecting", exchange, routingKey, props, body); + } catch (Exception e) { + throw new IllegalStateException("Failed to send return message", e); } }); } @@ -509,7 +506,7 @@ public void basicReject(long deliveryTag, boolean requeue) throws IOException { } @Override - public String basicConsume(String queue, Consumer callback) throws IOException { + public String basicConsume(String queue, Consumer callback) { throw new UnsupportedOperationException("This method is not currently supported as it is not used by current API in testing"); } diff --git a/nifi-extension-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ExtractAvroMetadata.java b/nifi-extension-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ExtractAvroMetadata.java index 341ec72c39af..bcb2cbb57bd2 100644 --- a/nifi-extension-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ExtractAvroMetadata.java +++ b/nifi-extension-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ExtractAvroMetadata.java @@ -17,7 +17,6 @@ package org.apache.nifi.processors.avro; import java.io.BufferedInputStream; -import java.io.IOException; import java.io.InputStream; import java.security.NoSuchAlgorithmException; import java.util.HashMap; @@ -51,7 +50,6 @@ import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; @SideEffectFree @@ -163,44 +161,41 @@ public void onTrigger(final ProcessContext context, final ProcessSession session } try { - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(InputStream rawIn) throws IOException { - try (final InputStream in = new BufferedInputStream(rawIn); - final DataFileStream reader = new DataFileStream<>(in, new GenericDatumReader<>())) { - - final Schema schema = reader.getSchema(); - if (schema == null) { - throw new ProcessException("Avro schema was null"); - } + session.read(flowFile, rawIn -> { + try (final InputStream in = new BufferedInputStream(rawIn); + final DataFileStream reader = new DataFileStream<>(in, new GenericDatumReader<>())) { - for (String key : reader.getMetaKeys()) { - if (requestedMetadataKeys.contains(key)) { - avroMetadata.put(key, reader.getMetaString(key)); - } - } + final Schema schema = reader.getSchema(); + if (schema == null) { + throw new ProcessException("Avro schema was null"); + } - try { - final byte[] rawFingerprint = SchemaNormalization.parsingFingerprint(fingerprintAlgorithm, schema); - avroMetadata.put(SCHEMA_FINGERPRINT_ATTR, Hex.encodeHexString(rawFingerprint)); - avroMetadata.put(SCHEMA_TYPE_ATTR, schema.getType().getName()); - avroMetadata.put(SCHEMA_NAME_ATTR, schema.getName()); - } catch (NoSuchAlgorithmException e) { - // shouldn't happen since allowable values are valid algorithms - throw new ProcessException(e); + for (String key : reader.getMetaKeys()) { + if (requestedMetadataKeys.contains(key)) { + avroMetadata.put(key, reader.getMetaString(key)); } + } + + try { + final byte[] rawFingerprint = SchemaNormalization.parsingFingerprint(fingerprintAlgorithm, schema); + avroMetadata.put(SCHEMA_FINGERPRINT_ATTR, Hex.encodeHexString(rawFingerprint)); + avroMetadata.put(SCHEMA_TYPE_ATTR, schema.getType().getName()); + avroMetadata.put(SCHEMA_NAME_ATTR, schema.getName()); + } catch (NoSuchAlgorithmException e) { + // shouldn't happen since allowable values are valid algorithms + throw new ProcessException(e); + } - if (countRecords) { - long recordCount = reader.getBlockCount(); - try { - while (reader.nextBlock() != null) { - recordCount += reader.getBlockCount(); - } - } catch (NoSuchElementException e) { - // happens at end of file + if (countRecords) { + long recordCount = reader.getBlockCount(); + try { + while (reader.nextBlock() != null) { + recordCount += reader.getBlockCount(); } - avroMetadata.put(ITEM_COUNT_ATTR, String.valueOf(recordCount)); + } catch (NoSuchElementException e) { + // happens at end of file } + avroMetadata.put(ITEM_COUNT_ATTR, String.valueOf(recordCount)); } } }); diff --git a/nifi-extension-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java b/nifi-extension-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java index b2dc64dd04ca..38f19662113b 100644 --- a/nifi-extension-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java +++ b/nifi-extension-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/SplitAvro.java @@ -64,8 +64,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; @SideEffectFree @@ -247,55 +245,49 @@ public List split(final ProcessSession session, final FlowFile origina final List childFlowFiles = new ArrayList<>(); final AtomicReference recordHolder = new AtomicReference<>(null); - session.read(originalFlowFile, new InputStreamCallback() { - @Override - public void process(InputStream rawIn) throws IOException { - try (final InputStream in = new BufferedInputStream(rawIn); - final DataFileStream reader = new DataFileStream<>(in, new GenericDatumReader<>())) { + session.read(originalFlowFile, rawIn -> { + try (final InputStream in = new BufferedInputStream(rawIn); + final DataFileStream reader = new DataFileStream<>(in, new GenericDatumReader<>())) { - final AtomicReference codec = new AtomicReference<>(reader.getMetaString(DataFileConstants.CODEC)); - if (codec.get() == null) { - codec.set(DataFileConstants.NULL_CODEC); - } + final AtomicReference codec = new AtomicReference<>(reader.getMetaString(DataFileConstants.CODEC)); + if (codec.get() == null) { + codec.set(DataFileConstants.NULL_CODEC); + } - // while records are left, start a new split by spawning a FlowFile - final AtomicReference hasNextHolder = new AtomicReference<>(reader.hasNext()); - while (hasNextHolder.get()) { - FlowFile childFlowFile = session.create(originalFlowFile); - childFlowFile = session.write(childFlowFile, new OutputStreamCallback() { - @Override - public void process(OutputStream rawOut) throws IOException { - try (final BufferedOutputStream out = new BufferedOutputStream(rawOut)) { - splitWriter.init(reader, codec.get(), out); - - // append to the current FlowFile until no more records, or splitSize is reached - int recordCount = 0; - while (hasNextHolder.get() && recordCount < splitSize) { - recordHolder.set(reader.next(recordHolder.get())); - splitWriter.write(recordHolder.get()); - recordCount++; - hasNextHolder.set(reader.hasNext()); - } - - splitWriter.flush(); - } finally { - splitWriter.close(); - } + // while records are left, start a new split by spawning a FlowFile + final AtomicReference hasNextHolder = new AtomicReference<>(reader.hasNext()); + while (hasNextHolder.get()) { + FlowFile childFlowFile = session.create(originalFlowFile); + childFlowFile = session.write(childFlowFile, rawOut -> { + try (final BufferedOutputStream out = new BufferedOutputStream(rawOut)) { + splitWriter.init(reader, codec.get(), out); + + // append to the current FlowFile until no more records, or splitSize is reached + int recordCount = 0; + while (hasNextHolder.get() && recordCount < splitSize) { + recordHolder.set(reader.next(recordHolder.get())); + splitWriter.write(recordHolder.get()); + recordCount++; + hasNextHolder.set(reader.hasNext()); } - }); - - // would prefer this to be part of the SplitWriter, but putting the metadata in FlowFile attributes - // can't be done inside of an OutputStream callback which is where the splitWriter is used - if (splitWriter instanceof BareRecordSplitWriter && transferMetadata) { - final Map metadata = new HashMap<>(); - for (String metaKey : reader.getMetaKeys()) { - metadata.put(metaKey, reader.getMetaString(metaKey)); - } - childFlowFile = session.putAllAttributes(childFlowFile, metadata); - } - childFlowFiles.add(childFlowFile); + splitWriter.flush(); + } finally { + splitWriter.close(); + } + }); + + // would prefer this to be part of the SplitWriter, but putting the metadata in FlowFile attributes + // can't be done inside of an OutputStream callback which is where the splitWriter is used + if (splitWriter instanceof BareRecordSplitWriter && transferMetadata) { + final Map metadata = new HashMap<>(); + for (String metaKey : reader.getMetaKeys()) { + metadata.put(metaKey, reader.getMetaString(metaKey)); + } + childFlowFile = session.putAllAttributes(childFlowFile, metadata); } + + childFlowFiles.add(childFlowFile); } } }); diff --git a/nifi-extension-bundles/nifi-extension-utils/nifi-event-listen/src/main/java/org/apache/nifi/processor/util/listen/AbstractListenEventBatchingProcessor.java b/nifi-extension-bundles/nifi-extension-utils/nifi-event-listen/src/main/java/org/apache/nifi/processor/util/listen/AbstractListenEventBatchingProcessor.java index bdba71623762..8192b11b668a 100644 --- a/nifi-extension-bundles/nifi-extension-utils/nifi-event-listen/src/main/java/org/apache/nifi/processor/util/listen/AbstractListenEventBatchingProcessor.java +++ b/nifi-extension-bundles/nifi-extension-utils/nifi-event-listen/src/main/java/org/apache/nifi/processor/util/listen/AbstractListenEventBatchingProcessor.java @@ -24,11 +24,9 @@ import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.listen.event.Event; import java.io.IOException; -import java.io.OutputStream; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -185,15 +183,12 @@ protected Map getBatches(final ProcessSession sessio final boolean writeDemarcator = (i > 0); try { final byte[] rawMessage = event.getData(); - FlowFile appendedFlowFile = session.append(batch.getFlowFile(), new OutputStreamCallback() { - @Override - public void process(final OutputStream out) throws IOException { - if (writeDemarcator) { - out.write(messageDemarcatorBytes); - } - - out.write(rawMessage); + FlowFile appendedFlowFile = session.append(batch.getFlowFile(), out -> { + if (writeDemarcator) { + out.write(messageDemarcatorBytes); } + + out.write(rawMessage); }); // update the FlowFile reference in the batch object diff --git a/nifi-extension-bundles/nifi-extension-utils/nifi-event-listen/src/main/java/org/apache/nifi/processor/util/listen/ListenerProperties.java b/nifi-extension-bundles/nifi-extension-utils/nifi-event-listen/src/main/java/org/apache/nifi/processor/util/listen/ListenerProperties.java index 91a18aafa50c..feb53db3f3b7 100644 --- a/nifi-extension-bundles/nifi-extension-utils/nifi-event-listen/src/main/java/org/apache/nifi/processor/util/listen/ListenerProperties.java +++ b/nifi-extension-bundles/nifi-extension-utils/nifi-event-listen/src/main/java/org/apache/nifi/processor/util/listen/ListenerProperties.java @@ -17,9 +17,7 @@ package org.apache.nifi.processor.util.listen; import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.components.Validator; import org.apache.nifi.expression.AttributeExpression; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.processor.util.StandardValidators; @@ -51,37 +49,34 @@ public class ListenerProperties { public static final PropertyDescriptor NETWORK_INTF_NAME = new PropertyDescriptor.Builder() .name("Local Network Interface") .description("The name of a local network interface to be used to restrict listening to a specific LAN.") - .addValidator(new Validator() { - @Override - public ValidationResult validate(String subject, String input, ValidationContext context) { - ValidationResult result = new ValidationResult.Builder() - .subject("Local Network Interface").valid(true).input(input).build(); - if (interfaceSet.contains(input.toLowerCase())) { - return result; - } - - String message; - String realValue = input; - try { - if (context.isExpressionLanguagePresent(input)) { - AttributeExpression ae = context.newExpressionLanguageCompiler().compile(input); - realValue = ae.evaluate(); - } - - if (interfaceSet.contains(realValue.toLowerCase())) { - return result; - } + .addValidator((subject, input, context) -> { + ValidationResult result = new ValidationResult.Builder() + .subject("Local Network Interface").valid(true).input(input).build(); + if (interfaceSet.contains(input.toLowerCase())) { + return result; + } - message = realValue + " is not a valid network name. Valid names are " + interfaceSet.toString(); + String message; + String realValue = input; + try { + if (context.isExpressionLanguagePresent(input)) { + AttributeExpression ae = context.newExpressionLanguageCompiler().compile(input); + realValue = ae.evaluate(); + } - } catch (IllegalArgumentException e) { - message = "Not a valid AttributeExpression: " + e.getMessage(); + if (interfaceSet.contains(realValue.toLowerCase())) { + return result; } - result = new ValidationResult.Builder().subject("Local Network Interface") - .valid(false).input(input).explanation(message).build(); - return result; + message = realValue + " is not a valid network name. Valid names are " + interfaceSet; + + } catch (IllegalArgumentException e) { + message = "Not a valid AttributeExpression: " + e.getMessage(); } + result = new ValidationResult.Builder().subject("Local Network Interface") + .valid(false).input(input).explanation(message).build(); + + return result; }) .expressionLanguageSupported(ExpressionLanguageScope.ENVIRONMENT) .build(); diff --git a/nifi-extension-bundles/nifi-extension-utils/nifi-file-transfer/src/main/java/org/apache/nifi/processor/util/file/transfer/PutFileTransfer.java b/nifi-extension-bundles/nifi-extension-utils/nifi-file-transfer/src/main/java/org/apache/nifi/processor/util/file/transfer/PutFileTransfer.java index 2888a3bc99f1..a5ce9d8c2032 100644 --- a/nifi-extension-bundles/nifi-extension-utils/nifi-file-transfer/src/main/java/org/apache/nifi/processor/util/file/transfer/PutFileTransfer.java +++ b/nifi-extension-bundles/nifi-extension-utils/nifi-file-transfer/src/main/java/org/apache/nifi/processor/util/file/transfer/PutFileTransfer.java @@ -25,7 +25,6 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.FlowFileAccessException; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.util.StopWatch; import org.apache.nifi.util.StringUtils; @@ -120,16 +119,13 @@ public void onTrigger(final ProcessContext context, final ProcessSession session beforePut(flowFile, context, transfer); final FlowFile flowFileToTransfer = flowFile; final AtomicReference fullPathRef = new AtomicReference<>(null); - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - try (final InputStream bufferedIn = new BufferedInputStream(in)) { - if (workingDirPath != null && context.getProperty(FileTransfer.CREATE_DIRECTORY).asBoolean()) { - transfer.ensureDirectoryExists(flowFileToTransfer, new File(workingDirPath)); - } - - fullPathRef.set(transfer.put(flowFileToTransfer, workingDirPath, conflictResult.getFileName(), bufferedIn)); + session.read(flowFile, in -> { + try (final InputStream bufferedIn = new BufferedInputStream(in)) { + if (workingDirPath != null && context.getProperty(FileTransfer.CREATE_DIRECTORY).asBoolean()) { + transfer.ensureDirectoryExists(flowFileToTransfer, new File(workingDirPath)); } + + fullPathRef.set(transfer.put(flowFileToTransfer, workingDirPath, conflictResult.getFileName(), bufferedIn)); } }); afterPut(flowFile, context, transfer); diff --git a/nifi-extension-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/flow/GroovyProcessSessionWrap.java b/nifi-extension-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/flow/GroovyProcessSessionWrap.java index 62814cc8cd32..d9a838aab0ea 100644 --- a/nifi-extension-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/flow/GroovyProcessSessionWrap.java +++ b/nifi-extension-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/flow/GroovyProcessSessionWrap.java @@ -51,21 +51,19 @@ public SessionFile wrap(FlowFile f) { * true - accept and continue, false - reject and continue, null - reject and stop, or any FlowFileFilterResult value. */ public List get(Closure filter) { - return this.get(new FlowFileFilter() { - @SuppressWarnings("ConstantConditions") - public FlowFileFilterResult filter(FlowFile flowFile) { - Object res = filter.call(wrap(flowFile)); - if (res == null) { - return FlowFileFilterResult.REJECT_AND_TERMINATE; - } - if (res instanceof Boolean) { - return ((Boolean) res ? FlowFileFilterResult.ACCEPT_AND_CONTINUE : FlowFileFilterResult.REJECT_AND_CONTINUE); - } - if (res instanceof FlowFileFilterResult) { - return (FlowFileFilterResult) res; - } - return (org.codehaus.groovy.runtime.DefaultGroovyMethods.asBoolean(res) ? FlowFileFilterResult.ACCEPT_AND_CONTINUE : FlowFileFilterResult.REJECT_AND_CONTINUE); + return this.get(flowFile -> { + Object res = filter.call(wrap(flowFile)); + if (res == null) { + return FlowFileFilter.FlowFileFilterResult.REJECT_AND_TERMINATE; } + if (res instanceof Boolean) { + return ((Boolean) res ? FlowFileFilter.FlowFileFilterResult.ACCEPT_AND_CONTINUE : FlowFileFilter.FlowFileFilterResult.REJECT_AND_CONTINUE); + } + if (res instanceof FlowFileFilter.FlowFileFilterResult) { + return (FlowFileFilter.FlowFileFilterResult) res; + } + return (org.codehaus.groovy.runtime.DefaultGroovyMethods.asBoolean(res) + ? FlowFileFilter.FlowFileFilterResult.ACCEPT_AND_CONTINUE : FlowFileFilter.FlowFileFilterResult.REJECT_AND_CONTINUE); }); } } diff --git a/nifi-extension-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/flow/GroovySessionFile.java b/nifi-extension-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/flow/GroovySessionFile.java index 6afcec6ef79f..2cccf0447117 100644 --- a/nifi-extension-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/flow/GroovySessionFile.java +++ b/nifi-extension-bundles/nifi-groovyx-bundle/nifi-groovyx-processors/src/main/java/org/apache/nifi/processors/groovyx/flow/GroovySessionFile.java @@ -17,9 +17,6 @@ package org.apache.nifi.processors.groovyx.flow; import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.processor.io.OutputStreamCallback; -import org.apache.nifi.processor.io.StreamCallback; -import org.apache.nifi.processor.io.InputStreamCallback; import groovy.lang.Writable; import groovy.lang.Closure; @@ -110,13 +107,11 @@ public Object invokeMethod(String name, Object args) { * @return reference to self */ public GroovySessionFile write(String charset, Closure c) { - this.write(new OutputStreamCallback() { - public void process(OutputStream out) throws IOException { - Writer w = new OutputStreamWriter(out, charset); - c.call(w); - w.flush(); - w.close(); - } + this.write(out -> { + Writer w = new OutputStreamWriter(out, charset); + c.call(w); + w.flush(); + w.close(); }); return this; } @@ -129,13 +124,11 @@ public void process(OutputStream out) throws IOException { * @return reference to self */ public GroovySessionFile write(String charset, CharSequence c) { - this.write(new OutputStreamCallback() { - public void process(OutputStream out) throws IOException { - Writer w = new OutputStreamWriter(out, charset); - w.append(c); - w.flush(); - w.close(); - } + this.write(out -> { + Writer w = new OutputStreamWriter(out, charset); + w.append(c); + w.flush(); + w.close(); }); return this; } @@ -148,13 +141,11 @@ public void process(OutputStream out) throws IOException { * @return reference to self */ public GroovySessionFile write(String charset, Writable c) { - this.write(new OutputStreamCallback() { - public void process(OutputStream out) throws IOException { - Writer w = new OutputStreamWriter(out, charset); - c.writeTo(w); - w.flush(); - w.close(); - } + this.write(out -> { + Writer w = new OutputStreamWriter(out, charset); + c.writeTo(w); + w.flush(); + w.close(); }); return this; } @@ -168,17 +159,9 @@ public void process(OutputStream out) throws IOException { */ public GroovySessionFile write(Closure c) { if (c.getMaximumNumberOfParameters() == 1) { - this.write(new OutputStreamCallback() { - public void process(OutputStream out) throws IOException { - c.call(out); - } - }); + this.write(out -> c.call(out)); } else { - this.write(new StreamCallback() { - public void process(InputStream in, OutputStream out) throws IOException { - c.call(in, out); - } - }); + this.write((in, out) -> c.call(in, out)); } return this; } @@ -190,11 +173,7 @@ public void process(InputStream in, OutputStream out) throws IOException { * @return reference to self */ public GroovySessionFile append(Closure c) { - this.append(new OutputStreamCallback() { - public void process(OutputStream out) throws IOException { - c.call(out); - } - }); + this.append(c::call); return this; } @@ -206,13 +185,11 @@ public void process(OutputStream out) throws IOException { * @return reference to self */ public GroovySessionFile append(String charset, Writable c) { - this.append(new OutputStreamCallback() { - public void process(OutputStream out) throws IOException { - Writer w = new OutputStreamWriter(out, charset); - c.writeTo(w); - w.flush(); - w.close(); - } + this.append(out -> { + Writer w = new OutputStreamWriter(out, charset); + c.writeTo(w); + w.flush(); + w.close(); }); return this; } @@ -225,13 +202,11 @@ public void process(OutputStream out) throws IOException { * @return reference to self */ public GroovySessionFile append(String charset, Closure c) { - this.append(new OutputStreamCallback() { - public void process(OutputStream out) throws IOException { - Writer w = new OutputStreamWriter(out, charset); - c.call(w); - w.flush(); - w.close(); - } + this.append(out -> { + Writer w = new OutputStreamWriter(out, charset); + c.call(w); + w.flush(); + w.close(); }); return this; } @@ -244,13 +219,11 @@ public void process(OutputStream out) throws IOException { * @return reference to self */ public GroovySessionFile append(String charset, CharSequence c) { - this.append(new OutputStreamCallback() { - public void process(OutputStream out) throws IOException { - Writer w = new OutputStreamWriter(out, charset); - w.append(c); - w.flush(); - w.close(); - } + this.append(out -> { + Writer w = new OutputStreamWriter(out, charset); + w.append(c); + w.flush(); + w.close(); }); return this; } @@ -261,11 +234,7 @@ public void process(OutputStream out) throws IOException { * @param c Closure with one parameter InputStream. */ public void read(Closure c) { - this.read(new InputStreamCallback() { - public void process(InputStream in) { - c.call(in); - } - }); + this.read(c::call); } /** @@ -275,12 +244,10 @@ public void process(InputStream in) { * @param c Closure with one parameter Reader. */ public void read(String charset, Closure c) { - this.read(new InputStreamCallback() { - public void process(InputStream in) throws IOException { - InputStreamReader r = new InputStreamReader(in, charset); - c.call(r); - r.close(); - } + this.read(in -> { + InputStreamReader r = new InputStreamReader(in, charset); + c.call(r); + r.close(); }); } diff --git a/nifi-extension-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/SequenceFileWriterImpl.java b/nifi-extension-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/SequenceFileWriterImpl.java index efc1faceea59..5a61e2a71285 100644 --- a/nifi-extension-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/SequenceFileWriterImpl.java +++ b/nifi-extension-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/SequenceFileWriterImpl.java @@ -28,7 +28,6 @@ import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processors.hadoop.util.ByteFilteringOutputStream; import org.apache.nifi.processors.hadoop.util.InputStreamWritable; import org.apache.nifi.processors.hadoop.util.SequenceFileWriter; @@ -39,8 +38,7 @@ import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; -import java.io.OutputStream; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; public class SequenceFileWriterImpl implements SequenceFileWriter { @@ -64,46 +62,37 @@ public FlowFile writeSequenceFile(final FlowFile flowFile, final ProcessSession // read via the BytesWritable class) while allowing us to stream the data rather than buffering // entire files in memory. final byte[] toReplace, replaceWith; - try { - toReplace = InputStreamWritable.class.getCanonicalName().getBytes("UTF-8"); - replaceWith = BytesWritable.class.getCanonicalName().getBytes("UTF-8"); - } catch (final UnsupportedEncodingException e) { - // This won't happen. - throw new RuntimeException("UTF-8 is not a supported Character Format"); - } + toReplace = InputStreamWritable.class.getCanonicalName().getBytes(StandardCharsets.UTF_8); + replaceWith = BytesWritable.class.getCanonicalName().getBytes(StandardCharsets.UTF_8); final StopWatch watch = new StopWatch(true); - FlowFile sfFlowFile = session.write(flowFile, new StreamCallback() { - - @Override - public void process(InputStream in, OutputStream out) throws IOException { - // Use a FilterableOutputStream to change 'InputStreamWritable' to 'BytesWritable' - see comment - // above for an explanation of why we want to do this. - final ByteFilteringOutputStream bwos = new ByteFilteringOutputStream(out); + FlowFile sfFlowFile = session.write(flowFile, (in, out) -> { + // Use a FilterableOutputStream to change 'InputStreamWritable' to 'BytesWritable' - see comment + // above for an explanation of why we want to do this. + final ByteFilteringOutputStream bwos = new ByteFilteringOutputStream(out); - // TODO: Adding this filter could be dangerous... A Sequence File's header contains 3 bytes: "SEQ", - // followed by 1 byte that is the Sequence File version, followed by 2 "entries." These "entries" - // contain the size of the Key/Value type and the Key/Value type. So, we will be writing the - // value type as InputStreamWritable -- which we need to change to BytesWritable. This means that - // we must also change the "size" that is written, but replacing this single byte could be - // dangerous. However, we know exactly what will be written to the header, and we limit this at one - // replacement, so we should be just fine. - bwos.addFilter(toReplace, replaceWith, 1); - bwos.addFilter((byte) InputStreamWritable.class.getCanonicalName().length(), - (byte) BytesWritable.class.getCanonicalName().length(), 1); + // TODO: Adding this filter could be dangerous... A Sequence File's header contains 3 bytes: "SEQ", + // followed by 1 byte that is the Sequence File version, followed by 2 "entries." These "entries" + // contain the size of the Key/Value type and the Key/Value type. So, we will be writing the + // value type as InputStreamWritable -- which we need to change to BytesWritable. This means that + // we must also change the "size" that is written, but replacing this single byte could be + // dangerous. However, we know exactly what will be written to the header, and we limit this at one + // replacement, so we should be just fine. + bwos.addFilter(toReplace, replaceWith, 1); + bwos.addFilter((byte) InputStreamWritable.class.getCanonicalName().length(), + (byte) BytesWritable.class.getCanonicalName().length(), 1); - try (final FSDataOutputStream fsDataOutputStream = new FSDataOutputStream(bwos, new Statistics("")); - final SequenceFile.Writer writer = SequenceFile.createWriter(configuration, - SequenceFile.Writer.stream(fsDataOutputStream), - SequenceFile.Writer.keyClass(Text.class), - SequenceFile.Writer.valueClass(InputStreamWritable.class), - SequenceFile.Writer.compression(compressionType, compressionCodec))) { + try (final FSDataOutputStream fsDataOutputStream = new FSDataOutputStream(bwos, new Statistics("")); + final Writer writer = SequenceFile.createWriter(configuration, + Writer.stream(fsDataOutputStream), + Writer.keyClass(Text.class), + Writer.valueClass(InputStreamWritable.class), + Writer.compression(compressionType, compressionCodec))) { - processInputStream(in, flowFile, writer); + processInputStream(in, flowFile, writer); - } finally { - watch.stop(); - } + } finally { + watch.stop(); } }); logger.debug("Wrote Sequence File {} ({}).", diff --git a/nifi-extension-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/inotify/GetHDFSEvents.java b/nifi-extension-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/inotify/GetHDFSEvents.java index c169854a1a31..9ad2cb427e42 100644 --- a/nifi-extension-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/inotify/GetHDFSEvents.java +++ b/nifi-extension-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/inotify/GetHDFSEvents.java @@ -44,7 +44,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.hadoop.AbstractHadoopProcessor; import org.apache.nifi.processors.hadoop.FetchHDFS; @@ -53,7 +52,6 @@ import org.apache.nifi.processors.hadoop.PutHDFS; import java.io.IOException; -import java.io.OutputStream; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -200,12 +198,7 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), "application/json"); flowFile = session.putAttribute(flowFile, EventAttributes.EVENT_TYPE, e.getEventType().name()); flowFile = session.putAttribute(flowFile, EventAttributes.EVENT_PATH, path); - flowFile = session.write(flowFile, new OutputStreamCallback() { - @Override - public void process(OutputStream out) throws IOException { - out.write(OBJECT_MAPPER.writeValueAsBytes(e)); - } - }); + flowFile = session.write(flowFile, out -> out.write(OBJECT_MAPPER.writeValueAsBytes(e))); flowFiles.add(flowFile); } diff --git a/nifi-extension-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java b/nifi-extension-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java index b19e4a0df8b9..3a965ba6fc69 100644 --- a/nifi-extension-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java +++ b/nifi-extension-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/ExtractHL7Attributes.java @@ -33,8 +33,7 @@ import ca.uhn.hl7v2.parser.PipeParser; import ca.uhn.hl7v2.validation.ValidationContext; import ca.uhn.hl7v2.validation.impl.ValidationContextFactory; -import java.io.IOException; -import java.io.InputStream; + import java.nio.charset.Charset; import java.util.Collections; import java.util.HashMap; @@ -61,7 +60,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.stream.io.StreamUtils; @@ -176,12 +174,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final String inputVersion = context.getProperty(HL7_INPUT_VERSION).getValue(); final byte[] buffer = new byte[(int) flowFile.getSize()]; - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - StreamUtils.fillBuffer(in, buffer); - } - }); + session.read(flowFile, in -> StreamUtils.fillBuffer(in, buffer)); @SuppressWarnings("resource") final HapiContext hapiContext = new DefaultHapiContext(); diff --git a/nifi-extension-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java b/nifi-extension-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java index 7b155b6f52ca..0d08210fb5a5 100644 --- a/nifi-extension-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java +++ b/nifi-extension-bundles/nifi-hl7-bundle/nifi-hl7-processors/src/main/java/org/apache/nifi/processors/hl7/RouteHL7.java @@ -16,8 +16,6 @@ */ package org.apache.nifi.processors.hl7; -import java.io.IOException; -import java.io.InputStream; import java.nio.charset.Charset; import java.util.HashMap; import java.util.HashSet; @@ -51,7 +49,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.stream.io.StreamUtils; @@ -156,12 +153,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).evaluateAttributeExpressions(flowFile).getValue()); final byte[] buffer = new byte[(int) flowFile.getSize()]; - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - StreamUtils.fillBuffer(in, buffer); - } - }); + session.read(flowFile, in -> StreamUtils.fillBuffer(in, buffer)); @SuppressWarnings("resource") final HapiContext hapiContext = new DefaultHapiContext(); diff --git a/nifi-extension-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSPublisher.java b/nifi-extension-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSPublisher.java index f84c21c1f4fd..5dd39a31b245 100644 --- a/nifi-extension-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSPublisher.java +++ b/nifi-extension-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSPublisher.java @@ -20,7 +20,6 @@ import org.apache.nifi.logging.ComponentLog; import org.springframework.jms.connection.CachingConnectionFactory; import org.springframework.jms.core.JmsTemplate; -import org.springframework.jms.core.MessageCreator; import org.springframework.jms.support.JmsHeaders; import jakarta.jms.BytesMessage; @@ -50,25 +49,19 @@ void publish(String destinationName, byte[] messageBytes) { } void publish(final String destinationName, final byte[] messageBytes, final Map flowFileAttributes) { - this.jmsTemplate.send(destinationName, new MessageCreator() { - @Override - public Message createMessage(Session session) throws JMSException { - BytesMessage message = session.createBytesMessage(); - message.writeBytes(messageBytes); - setMessageHeaderAndProperties(session, message, flowFileAttributes); - return message; - } + this.jmsTemplate.send(destinationName, session -> { + BytesMessage message = session.createBytesMessage(); + message.writeBytes(messageBytes); + setMessageHeaderAndProperties(session, message, flowFileAttributes); + return message; }); } void publish(String destinationName, String messageText, final Map flowFileAttributes) { - this.jmsTemplate.send(destinationName, new MessageCreator() { - @Override - public Message createMessage(Session session) throws JMSException { - TextMessage message = session.createTextMessage(messageText); - setMessageHeaderAndProperties(session, message, flowFileAttributes); - return message; - } + this.jmsTemplate.send(destinationName, session -> { + TextMessage message = session.createTextMessage(messageText); + setMessageHeaderAndProperties(session, message, flowFileAttributes); + return message; }); } diff --git a/nifi-extension-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ExtractImageMetadata.java b/nifi-extension-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ExtractImageMetadata.java index b2e39eddd270..89584381d14d 100644 --- a/nifi-extension-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ExtractImageMetadata.java +++ b/nifi-extension-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/image/ExtractImageMetadata.java @@ -16,8 +16,6 @@ */ package org.apache.nifi.processors.image; -import java.io.IOException; -import java.io.InputStream; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -42,7 +40,6 @@ import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import com.drew.imaging.ImageMetadataReader; @@ -120,15 +117,12 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final Integer max = context.getProperty(MAX_NUMBER_OF_ATTRIBUTES).asInteger(); try { - session.read(flowfile, new InputStreamCallback() { - @Override - public void process(InputStream in) throws IOException { - try { - Metadata imageMetadata = ImageMetadataReader.readMetadata(in); - value.set(imageMetadata); - } catch (ImageProcessingException ex) { - throw new ProcessException(ex); - } + session.read(flowfile, in -> { + try { + Metadata imageMetadata = ImageMetadataReader.readMetadata(in); + value.set(imageMetadata); + } catch (ImageProcessingException ex) { + throw new ProcessException(ex); } }); diff --git a/nifi-extension-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java b/nifi-extension-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java index ff71652f7a31..2e557b713a63 100644 --- a/nifi-extension-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java +++ b/nifi-extension-bundles/nifi-media-bundle/nifi-media-processors/src/main/java/org/apache/nifi/processors/media/ExtractMediaMetadata.java @@ -46,7 +46,6 @@ import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.tika.exception.TikaException; @@ -180,15 +179,12 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final String prefix = context.getProperty(METADATA_KEY_PREFIX).evaluateAttributeExpressions(flowFile).getValue(); try { - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(InputStream in) throws IOException { - try { - Map results = tika_parse(in, prefix, maxAttribCount, maxAttribLength); - value.set(results); - } catch (SAXException | TikaException e) { - throw new IOException(e); - } + session.read(flowFile, in -> { + try { + Map results = tika_parse(in, prefix, maxAttribCount, maxAttribLength); + value.set(results); + } catch (SAXException | TikaException e) { + throw new IOException(e); } }); diff --git a/nifi-extension-bundles/nifi-network-bundle/nifi-network-processors/src/main/java/org/apache/nifi/processors/network/ParseNetflowv5.java b/nifi-extension-bundles/nifi-network-bundle/nifi-network-processors/src/main/java/org/apache/nifi/processors/network/ParseNetflowv5.java index 53f9fcbd8653..dd1af40868a6 100644 --- a/nifi-extension-bundles/nifi-network-bundle/nifi-network-processors/src/main/java/org/apache/nifi/processors/network/ParseNetflowv5.java +++ b/nifi-extension-bundles/nifi-network-bundle/nifi-network-processors/src/main/java/org/apache/nifi/processors/network/ParseNetflowv5.java @@ -38,12 +38,10 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processors.network.parser.Netflowv5Parser; import java.io.BufferedOutputStream; import java.io.ByteArrayOutputStream; -import java.io.IOException; import java.io.OutputStream; import java.util.ArrayList; import java.util.HashMap; @@ -177,12 +175,9 @@ int record = 0; // Add JSON Objects generateJSONUtil(results, parser, record++); - recordFlowFile = session.write(recordFlowFile, new OutputStreamCallback() { - @Override - public void process(OutputStream out) throws IOException { - try (OutputStream outputStream = new BufferedOutputStream(out)) { - outputStream.write(MAPPER.writeValueAsBytes(results)); - } + recordFlowFile = session.write(recordFlowFile, out -> { + try (OutputStream outputStream = new BufferedOutputStream(out)) { + outputStream.write(MAPPER.writeValueAsBytes(results)); } }); // Adjust the FlowFile mime.type attribute diff --git a/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java b/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java index e8f1e2ca5534..bbb0930e8a9b 100644 --- a/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java +++ b/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedPartitionRecord.java @@ -31,7 +31,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; @@ -47,7 +46,6 @@ import javax.script.ScriptEngine; import javax.script.ScriptException; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.HashMap; @@ -157,76 +155,73 @@ private boolean partition( final RecordCounts counts = new RecordCounts(); try { - session.read(incomingFlowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - try ( - final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger()) - ) { - final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema()); - final RecordSet recordSet = reader.createRecordSet(); - final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet); - - final Map outgoingFlowFiles = new HashMap<>(); - final Map recordSetWriters = new HashMap<>(); - - // Reading in records and evaluate script - while (pushBackSet.isAnotherRecord()) { - final Record record = pushBackSet.next(); - final Object evaluatedValue = evaluator.evaluate(record, counts.getRecordCount()); - getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, counts.getRecordCount(), evaluatedValue); - counts.incrementRecordCount(); - - final Object partition = (evaluatedValue instanceof Object[]) ? Arrays.asList((Object[]) evaluatedValue) : evaluatedValue; - RecordSetWriter writer = recordSetWriters.get(partition); - - if (writer == null) { - final FlowFile outgoingFlowFile = session.create(incomingFlowFile); - final OutputStream out = session.write(outgoingFlowFile); - writer = writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile); - - writer.beginRecordSet(); - outgoingFlowFiles.put(partition, outgoingFlowFile); - recordSetWriters.put(partition, writer); - } - - writer.write(record); + session.read(incomingFlowFile, in -> { + try ( + final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger()) + ) { + final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema()); + final RecordSet recordSet = reader.createRecordSet(); + final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet); + + final Map outgoingFlowFiles = new HashMap<>(); + final Map recordSetWriters = new HashMap<>(); + + // Reading in records and evaluate script + while (pushBackSet.isAnotherRecord()) { + final Record record = pushBackSet.next(); + final Object evaluatedValue = evaluator.evaluate(record, counts.getRecordCount()); + getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, counts.getRecordCount(), evaluatedValue); + counts.incrementRecordCount(); + + final Object partition = (evaluatedValue instanceof Object[]) ? Arrays.asList((Object[]) evaluatedValue) : evaluatedValue; + RecordSetWriter writer = recordSetWriters.get(partition); + + if (writer == null) { + final FlowFile outgoingFlowFile = session.create(incomingFlowFile); + final OutputStream out = session.write(outgoingFlowFile); + writer = writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile); + + writer.beginRecordSet(); + outgoingFlowFiles.put(partition, outgoingFlowFile); + recordSetWriters.put(partition, writer); } - // Sending outgoing flow files - int fragmentIndex = 0; - - for (final Object partition : outgoingFlowFiles.keySet()) { - final RecordSetWriter writer = recordSetWriters.get(partition); - final FlowFile outgoingFlowFile = outgoingFlowFiles.get(partition); - - final Map attributes = new HashMap<>(incomingFlowFile.getAttributes()); - attributes.put("mime.type", writer.getMimeType()); - attributes.put("partition", partition == null ? "" : partition.toString()); - attributes.put("fragment.index", String.valueOf(fragmentIndex)); - attributes.put("fragment.count", String.valueOf(outgoingFlowFiles.size())); - - try { - final WriteResult finalResult = writer.finishRecordSet(); - final int outgoingFlowFileRecords = finalResult.getRecordCount(); - attributes.put("record.count", String.valueOf(outgoingFlowFileRecords)); - writer.close(); - } catch (final IOException e) { - throw new ProcessException("Resources used for record writing might not be closed", e); - } - - session.putAllAttributes(outgoingFlowFile, attributes); - session.transfer(outgoingFlowFile, RELATIONSHIP_SUCCESS); - fragmentIndex++; - } + writer.write(record); + } - final long millis = System.currentTimeMillis() - startMillis; - session.adjustCounter("Records Processed", counts.getRecordCount(), true); - session.getProvenanceReporter().fork(incomingFlowFile, outgoingFlowFiles.values(), "Processed " + counts.getRecordCount() + " Records", millis); + // Sending outgoing flow files + int fragmentIndex = 0; + + for (final Object partition : outgoingFlowFiles.keySet()) { + final RecordSetWriter writer = recordSetWriters.get(partition); + final FlowFile outgoingFlowFile = outgoingFlowFiles.get(partition); + + final Map attributes = new HashMap<>(incomingFlowFile.getAttributes()); + attributes.put("mime.type", writer.getMimeType()); + attributes.put("partition", partition == null ? "" : partition.toString()); + attributes.put("fragment.index", String.valueOf(fragmentIndex)); + attributes.put("fragment.count", String.valueOf(outgoingFlowFiles.size())); + + try { + final WriteResult finalResult = writer.finishRecordSet(); + final int outgoingFlowFileRecords = finalResult.getRecordCount(); + attributes.put("record.count", String.valueOf(outgoingFlowFileRecords)); + writer.close(); + } catch (final IOException e) { + throw new ProcessException("Resources used for record writing might not be closed", e); + } - } catch (final ScriptException | SchemaNotFoundException | MalformedRecordException e) { - throw new ProcessException("After processing " + counts.getRecordCount() + " Records, encountered failure when attempting to process " + incomingFlowFile, e); + session.putAllAttributes(outgoingFlowFile, attributes); + session.transfer(outgoingFlowFile, RELATIONSHIP_SUCCESS); + fragmentIndex++; } + + final long millis = System.currentTimeMillis() - startMillis; + session.adjustCounter("Records Processed", counts.getRecordCount(), true); + session.getProvenanceReporter().fork(incomingFlowFile, outgoingFlowFiles.values(), "Processed " + counts.getRecordCount() + " Records", millis); + + } catch (final ScriptException | SchemaNotFoundException | MalformedRecordException e) { + throw new ProcessException("After processing " + counts.getRecordCount() + " Records, encountered failure when attempting to process " + incomingFlowFile, e); } }); diff --git a/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java b/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java index 4a17dc439a5e..2323ea0afe08 100644 --- a/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java +++ b/nifi-extension-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/ScriptedRouterProcessor.java @@ -30,7 +30,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; @@ -46,7 +45,6 @@ import javax.script.ScriptEngine; import javax.script.ScriptException; import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.HashMap; import java.util.List; @@ -125,72 +123,69 @@ private boolean route( final RecordCounts counts = new RecordCounts(); try { - session.read(incomingFlowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - try ( - final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger()) - ) { - final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema()); - final RecordSet recordSet = reader.createRecordSet(); - final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet); - final Map outgoingFlowFiles = new HashMap<>(); - final Map recordSetWriters = new HashMap<>(); - - // Reading in records and evaluate script - while (pushBackSet.isAnotherRecord()) { - final Record record = pushBackSet.next(); - final Object evaluatedValue = evaluator.evaluate(record, counts.getRecordCount()); - getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, counts.getRecordCount(), evaluatedValue); - counts.incrementRecordCount(); - - if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) { - final Optional outgoingRelationship = resolveRelationship(scriptResultType.cast(evaluatedValue)); - - if (outgoingRelationship.isPresent()) { - if (!outgoingFlowFiles.containsKey(outgoingRelationship.get())) { - final FlowFile outgoingFlowFile = session.create(incomingFlowFile); - final OutputStream out = session.write(outgoingFlowFile); - final RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile); - writer.beginRecordSet(); - - outgoingFlowFiles.put(outgoingRelationship.get(), outgoingFlowFile); - recordSetWriters.put(outgoingRelationship.get(), writer); - } - - recordSetWriters.get(outgoingRelationship.get()).write(record); - } else { - getLogger().debug("Record with evaluated value {} has no outgoing relationship determined", String.valueOf(evaluatedValue)); + session.read(incomingFlowFile, in -> { + try ( + final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, incomingFlowFile.getSize(), getLogger()) + ) { + final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema()); + final RecordSet recordSet = reader.createRecordSet(); + final PushBackRecordSet pushBackSet = new PushBackRecordSet(recordSet); + final Map outgoingFlowFiles = new HashMap<>(); + final Map recordSetWriters = new HashMap<>(); + + // Reading in records and evaluate script + while (pushBackSet.isAnotherRecord()) { + final Record record = pushBackSet.next(); + final Object evaluatedValue = evaluator.evaluate(record, counts.getRecordCount()); + getLogger().debug("Evaluated scripted against {} (index {}), producing result of {}", record, counts.getRecordCount(), evaluatedValue); + counts.incrementRecordCount(); + + if (evaluatedValue != null && scriptResultType.isInstance(evaluatedValue)) { + final Optional outgoingRelationship = resolveRelationship(scriptResultType.cast(evaluatedValue)); + + if (outgoingRelationship.isPresent()) { + if (!outgoingFlowFiles.containsKey(outgoingRelationship.get())) { + final FlowFile outgoingFlowFile = session.create(incomingFlowFile); + final OutputStream out = session.write(outgoingFlowFile); + final RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, out, outgoingFlowFile); + writer.beginRecordSet(); + + outgoingFlowFiles.put(outgoingRelationship.get(), outgoingFlowFile); + recordSetWriters.put(outgoingRelationship.get(), writer); } + + recordSetWriters.get(outgoingRelationship.get()).write(record); } else { - throw new ProcessException("Script returned a value of " + evaluatedValue - + " but this Processor requires that the object returned be an instance of " + scriptResultType.getSimpleName()); + getLogger().debug("Record with evaluated value {} has no outgoing relationship determined", String.valueOf(evaluatedValue)); } + } else { + throw new ProcessException("Script returned a value of " + evaluatedValue + + " but this Processor requires that the object returned be an instance of " + scriptResultType.getSimpleName()); } + } - // Sending outgoing flow files - for (final Relationship relationship : outgoingFlowFiles.keySet()) { - final RecordSetWriter writer = recordSetWriters.get(relationship); - final FlowFile outgoingFlowFile = outgoingFlowFiles.get(relationship); - - final Map attributes = new HashMap<>(incomingFlowFile.getAttributes()); - attributes.put("mime.type", writer.getMimeType()); - - try { - final WriteResult finalResult = writer.finishRecordSet(); - final int outgoingFlowFileRecords = finalResult.getRecordCount(); - attributes.put("record.count", String.valueOf(outgoingFlowFileRecords)); - writer.close(); - } catch (final IOException e) { - throw new ProcessException("Resources used for record writing might not be closed", e); - } - - session.putAllAttributes(outgoingFlowFile, attributes); - session.transfer(outgoingFlowFile, relationship); + // Sending outgoing flow files + for (final Relationship relationship : outgoingFlowFiles.keySet()) { + final RecordSetWriter writer = recordSetWriters.get(relationship); + final FlowFile outgoingFlowFile = outgoingFlowFiles.get(relationship); + + final Map attributes = new HashMap<>(incomingFlowFile.getAttributes()); + attributes.put("mime.type", writer.getMimeType()); + + try { + final WriteResult finalResult = writer.finishRecordSet(); + final int outgoingFlowFileRecords = finalResult.getRecordCount(); + attributes.put("record.count", String.valueOf(outgoingFlowFileRecords)); + writer.close(); + } catch (final IOException e) { + throw new ProcessException("Resources used for record writing might not be closed", e); } - } catch (final ScriptException | SchemaNotFoundException | MalformedRecordException e) { - throw new ProcessException("After processing " + counts.getRecordCount() + " Records, encountered failure when attempting to process " + incomingFlowFile, e); + + session.putAllAttributes(outgoingFlowFile, attributes); + session.transfer(outgoingFlowFile, relationship); } + } catch (final ScriptException | SchemaNotFoundException | MalformedRecordException e) { + throw new ProcessException("After processing " + counts.getRecordCount() + " Records, encountered failure when attempting to process " + incomingFlowFile, e); } }); diff --git a/nifi-extension-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java b/nifi-extension-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java index 44ffc38b919f..e82dca417e22 100644 --- a/nifi-extension-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java +++ b/nifi-extension-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/GetSplunk.java @@ -48,7 +48,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.scheduling.SchedulingStrategy; import org.apache.nifi.ssl.SSLContextProvider; @@ -56,7 +55,6 @@ import java.io.BufferedOutputStream; import java.io.IOException; import java.io.InputStream; -import java.io.OutputStream; import java.time.OffsetDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; @@ -494,12 +492,9 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro final InputStream exportSearch = export; FlowFile flowFile = session.create(); - flowFile = session.write(flowFile, new OutputStreamCallback() { - @Override - public void process(OutputStream rawOut) throws IOException { - try (BufferedOutputStream out = new BufferedOutputStream(rawOut)) { - IOUtils.copyLarge(exportSearch, out); - } + flowFile = session.write(flowFile, rawOut -> { + try (BufferedOutputStream out = new BufferedOutputStream(rawOut)) { + IOUtils.copyLarge(exportSearch, out); } }); diff --git a/nifi-extension-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunk.java b/nifi-extension-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunk.java index 67f14d3939ab..855592a5aa9f 100644 --- a/nifi-extension-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunk.java +++ b/nifi-extension-bundles/nifi-splunk-bundle/nifi-splunk-processors/src/main/java/org/apache/nifi/processors/splunk/PutSplunk.java @@ -46,7 +46,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.put.AbstractPutEventProcessor; import org.apache.nifi.stream.io.ByteCountingInputStream; import org.apache.nifi.stream.io.util.NonThreadSafeCircularBuffer; @@ -184,53 +183,50 @@ private void processDelimitedMessages(final ProcessContext context, final Proces activeBatches.add(messageBatch); try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream rawIn) throws IOException { - byte[] data = null; // contents of a single message - boolean streamFinished = false; + session.read(flowFile, rawIn -> { + byte[] data = null; // contents of a single message + boolean streamFinished = false; - int nextByte; - try (final InputStream bufferedIn = new BufferedInputStream(rawIn); - final ByteCountingInputStream in = new ByteCountingInputStream(bufferedIn)) { + int nextByte; + try (final InputStream bufferedIn = new BufferedInputStream(rawIn); + final ByteCountingInputStream in = new ByteCountingInputStream(bufferedIn)) { - long messageStartOffset = in.getBytesConsumed(); + long messageStartOffset = in.getBytesConsumed(); - // read until we're out of data. - while (!streamFinished) { - nextByte = in.read(); + // read until we're out of data. + while (!streamFinished) { + nextByte = in.read(); - if (nextByte > -1) { - baos.write(nextByte); - } + if (nextByte > -1) { + baos.write(nextByte); + } - if (nextByte == -1) { - // we ran out of data. This message is complete. - data = getMessage(baos, baos.size(), protocol); - streamFinished = true; - } else if (buffer.addAndCompare((byte) nextByte)) { - // we matched our delimiter. This message is complete. We want all of the bytes from the - // underlying BAOS except for the last 'delimiterBytes.length' bytes because we don't want - // the delimiter itself to be sent. - data = getMessage(baos, baos.size() - delimiterBytes.length, protocol); - } + if (nextByte == -1) { + // we ran out of data. This message is complete. + data = getMessage(baos, baos.size(), protocol); + streamFinished = true; + } else if (buffer.addAndCompare((byte) nextByte)) { + // we matched our delimiter. This message is complete. We want all of the bytes from the + // underlying BAOS except for the last 'delimiterBytes.length' bytes because we don't want + // the delimiter itself to be sent. + data = getMessage(baos, baos.size() - delimiterBytes.length, protocol); + } + + if (data != null) { + final long messageEndOffset = in.getBytesConsumed(); - if (data != null) { - final long messageEndOffset = in.getBytesConsumed(); - - // If the message has no data, ignore it. - if (data.length != 0) { - final long rangeStart = messageStartOffset; - eventSender.sendEvent(data); - messageBatch.addSuccessfulRange(rangeStart, messageEndOffset); - messagesSent.incrementAndGet(); - } - - // reset BAOS so that we can start a new message. - baos.reset(); - data = null; - messageStartOffset = in.getBytesConsumed(); + // If the message has no data, ignore it. + if (data.length != 0) { + final long rangeStart = messageStartOffset; + eventSender.sendEvent(data); + messageBatch.addSuccessfulRange(rangeStart, messageEndOffset); + messagesSent.incrementAndGet(); } + + // reset BAOS so that we can start a new message. + baos.reset(); + data = null; + messageStartOffset = in.getBytesConsumed(); } } } diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java index c2ecfef702be..86e02564c94b 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractJsonPathProcessor.java @@ -31,13 +31,10 @@ import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessSession; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.StringUtils; import java.io.BufferedInputStream; -import java.io.IOException; -import java.io.InputStream; import java.util.List; import java.util.Map; import java.util.Objects; @@ -90,23 +87,20 @@ static Configuration createConfiguration(final int maxStringLength) { static DocumentContext validateAndEstablishJsonContext(ProcessSession processSession, FlowFile flowFile, Configuration jsonPathConfiguration) { // Parse the document once into an associated context to support multiple path evaluations if specified final AtomicReference contextHolder = new AtomicReference<>(null); - processSession.read(flowFile, new InputStreamCallback() { - @Override - public void process(InputStream in) throws IOException { - try (BufferedInputStream bufferedInputStream = new BufferedInputStream(in)) { - DocumentContext ctx = JsonPath.using(jsonPathConfiguration).parse(bufferedInputStream); - contextHolder.set(ctx); - } catch (IllegalArgumentException iae) { - // The JsonPath.parse() above first parses the json, then creates a context object from the parsed - // json. It is possible for the json parsing to complete without error, but produce a null object. - // In this case the context creation will fail and throw an IllegalArgumentException. This is in - // my opinion a bug in the JsonPath library, as it doesn't really throw the correct exception - // contextually. - // The general handling in derived classes handles InvalidJsonException. - // The best thing to do here, is to re-throw with the proper exception, such that the calling logic - // can route. - throw new InvalidJsonException(iae); - } + processSession.read(flowFile, in -> { + try (BufferedInputStream bufferedInputStream = new BufferedInputStream(in)) { + DocumentContext ctx = JsonPath.using(jsonPathConfiguration).parse(bufferedInputStream); + contextHolder.set(ctx); + } catch (IllegalArgumentException iae) { + // The JsonPath.parse() above first parses the json, then creates a context object from the parsed + // json. It is possible for the json parsing to complete without error, but produce a null object. + // In this case the context creation will fail and throw an IllegalArgumentException. This is in + // my opinion a bug in the JsonPath library, as it doesn't really throw the correct exception + // contextually. + // The general handling in derived classes handles InvalidJsonException. + // The best thing to do here, is to re-throw with the proper exception, such that the calling logic + // can route. + throw new InvalidJsonException(iae); } }); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java index 26add07f8f20..7351b8f96cd4 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java @@ -27,7 +27,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; @@ -38,9 +37,6 @@ import org.apache.nifi.serialization.record.Record; import org.apache.nifi.serialization.record.RecordSchema; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -119,57 +115,54 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final FlowFile original = flowFile; final Map originalAttributes = flowFile.getAttributes(); try { - flowFile = session.write(flowFile, new StreamCallback() { - @Override - public void process(final InputStream in, final OutputStream out) throws IOException { - - try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { - - // Get the first record and process it before we create the Record Writer. We do this so that if the Processor - // updates the Record's schema, we can provide an updated schema to the Record Writer. If there are no records, - // then we can simply create the Writer with the Reader's schema and begin & end the Record Set. - Record firstRecord = reader.nextRecord(); - if (firstRecord == null) { - final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, reader.getSchema()); - try (final RecordSetWriter writer = writerFactory.createWriter(getLogger(), writeSchema, out, originalAttributes)) { - writer.beginRecordSet(); - - final WriteResult writeResult = writer.finishRecordSet(); - attributes.put("record.count", String.valueOf(writeResult.getRecordCount())); - attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType()); - attributes.putAll(writeResult.getAttributes()); - recordCount.set(writeResult.getRecordCount()); - } - - return; - } + flowFile = session.write(flowFile, (in, out) -> { - firstRecord = AbstractRecordProcessor.this.process(firstRecord, original, context, 1L); + try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { - final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, firstRecord.getSchema()); + // Get the first record and process it before we create the Record Writer. We do this so that if the Processor + // updates the Record's schema, we can provide an updated schema to the Record Writer. If there are no records, + // then we can simply create the Writer with the Reader's schema and begin & end the Record Set. + Record firstRecord = reader.nextRecord(); + if (firstRecord == null) { + final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, reader.getSchema()); try (final RecordSetWriter writer = writerFactory.createWriter(getLogger(), writeSchema, out, originalAttributes)) { writer.beginRecordSet(); - writer.write(firstRecord); - - Record record; - long count = 1L; - while ((record = reader.nextRecord()) != null) { - final Record processed = AbstractRecordProcessor.this.process(record, original, context, ++count); - writer.write(processed); - } - final WriteResult writeResult = writer.finishRecordSet(); attributes.put("record.count", String.valueOf(writeResult.getRecordCount())); attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType()); attributes.putAll(writeResult.getAttributes()); recordCount.set(writeResult.getRecordCount()); } - } catch (final SchemaNotFoundException e) { - throw new ProcessException(e.getLocalizedMessage(), e); - } catch (final MalformedRecordException e) { - throw new ProcessException("Could not parse incoming data", e); + + return; + } + + firstRecord = AbstractRecordProcessor.this.process(firstRecord, original, context, 1L); + + final RecordSchema writeSchema = writerFactory.getSchema(originalAttributes, firstRecord.getSchema()); + try (final RecordSetWriter writer = writerFactory.createWriter(getLogger(), writeSchema, out, originalAttributes)) { + writer.beginRecordSet(); + + writer.write(firstRecord); + + Record record; + long count = 1L; + while ((record = reader.nextRecord()) != null) { + final Record processed = AbstractRecordProcessor.this.process(record, original, context, ++count); + writer.write(processed); + } + + final WriteResult writeResult = writer.finishRecordSet(); + attributes.put("record.count", String.valueOf(writeResult.getRecordCount())); + attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType()); + attributes.putAll(writeResult.getAttributes()); + recordCount.set(writeResult.getRecordCount()); } + } catch (final SchemaNotFoundException e) { + throw new ProcessException(e.getLocalizedMessage(), e); + } catch (final MalformedRecordException e) { + throw new ProcessException("Could not parse incoming data", e); } }); } catch (final Exception e) { diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java index 27504ad3240c..0d12bb473f8a 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CompressContent.java @@ -53,7 +53,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.stream.io.GZIPOutputStream; import org.apache.nifi.util.StopWatch; import org.tukaani.xz.LZMA2Options; @@ -300,112 +299,109 @@ public void onTrigger(final ProcessContext context, final ProcessSession session }; try { - flowFile = session.write(flowFile, new StreamCallback() { - @Override - public void process(final InputStream rawIn, final OutputStream rawOut) throws IOException { - final OutputStream compressionOut; - final InputStream compressionIn; - - final OutputStream bufferedOut = new BufferedOutputStream(rawOut, 65536); - final InputStream bufferedIn = new BufferedInputStream(rawIn, 65536); - - try { - if (MODE_COMPRESS.equalsIgnoreCase(compressionMode)) { - compressionIn = bufferedIn; - - switch (compressionFormat.toLowerCase()) { - case COMPRESSION_FORMAT_GZIP: { - int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger(); - compressionOut = new GZIPOutputStream(bufferedOut, compressionLevel); - mimeTypeRef.set("application/gzip"); - break; - } - case COMPRESSION_FORMAT_DEFLATE: { - final int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger(); - compressionOut = new DeflaterOutputStream(bufferedOut, new Deflater(compressionLevel)); - mimeTypeRef.set("application/gzip"); - break; - } - case COMPRESSION_FORMAT_LZMA: - compressionOut = new LzmaOutputStream.Builder(bufferedOut).build(); - mimeTypeRef.set("application/x-lzma"); - break; - case COMPRESSION_FORMAT_XZ_LZMA2: - final int xzCompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger(); - compressionOut = new XZOutputStream(bufferedOut, new LZMA2Options(xzCompressionLevel)); - mimeTypeRef.set("application/x-xz"); - break; - case COMPRESSION_FORMAT_SNAPPY: - compressionOut = new SnappyOutputStream(bufferedOut); - mimeTypeRef.set("application/x-snappy"); - break; - case COMPRESSION_FORMAT_SNAPPY_HADOOP: - compressionOut = new SnappyHadoopCompatibleOutputStream(bufferedOut); - mimeTypeRef.set("application/x-snappy-hadoop"); - break; - case COMPRESSION_FORMAT_SNAPPY_FRAMED: - compressionOut = new SnappyFramedOutputStream(bufferedOut); - mimeTypeRef.set("application/x-snappy-framed"); - break; - case COMPRESSION_FORMAT_LZ4_FRAMED: - mimeTypeRef.set("application/x-lz4-framed"); - compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut); - break; - case COMPRESSION_FORMAT_ZSTD: - final int zstdCompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger() * 2; - compressionOut = new ZstdCompressorOutputStream(bufferedOut, zstdCompressionLevel); - mimeTypeRef.set("application/zstd"); - break; - case COMPRESSION_FORMAT_BROTLI: { - Brotli4jLoader.ensureAvailability(); - final int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger(); - Encoder.Parameters params = new Encoder.Parameters().setQuality(compressionLevel); - compressionOut = new BrotliOutputStream(bufferedOut, params); - mimeTypeRef.set("application/x-brotli"); - break; - } - case COMPRESSION_FORMAT_BZIP2: - default: - mimeTypeRef.set("application/x-bzip2"); - compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut); - break; + flowFile = session.write(flowFile, (rawIn, rawOut) -> { + final OutputStream compressionOut; + final InputStream compressionIn; + + final OutputStream bufferedOut = new BufferedOutputStream(rawOut, 65536); + final InputStream bufferedIn = new BufferedInputStream(rawIn, 65536); + + try { + if (MODE_COMPRESS.equalsIgnoreCase(compressionMode)) { + compressionIn = bufferedIn; + + switch (compressionFormat.toLowerCase()) { + case COMPRESSION_FORMAT_GZIP: { + int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger(); + compressionOut = new GZIPOutputStream(bufferedOut, compressionLevel); + mimeTypeRef.set("application/gzip"); + break; } - } else { - compressionOut = bufferedOut; - compressionIn = switch (compressionFormat.toLowerCase()) { - case COMPRESSION_FORMAT_LZMA -> new LzmaInputStream(bufferedIn, new Decoder()); - case COMPRESSION_FORMAT_XZ_LZMA2 -> new XZInputStream(bufferedIn); - case COMPRESSION_FORMAT_BZIP2 -> - // need this two-arg constructor to support concatenated streams - new BZip2CompressorInputStream(bufferedIn, true); - case COMPRESSION_FORMAT_GZIP -> new GzipCompressorInputStream(bufferedIn, true); - case COMPRESSION_FORMAT_DEFLATE -> new InflaterInputStream(bufferedIn); - case COMPRESSION_FORMAT_SNAPPY -> new SnappyInputStream(bufferedIn); - case COMPRESSION_FORMAT_SNAPPY_HADOOP -> throw new Exception("Cannot decompress snappy-hadoop."); - case COMPRESSION_FORMAT_SNAPPY_FRAMED -> new SnappyFramedInputStream(bufferedIn); - case COMPRESSION_FORMAT_LZ4_FRAMED -> new FramedLZ4CompressorInputStream(bufferedIn, true); - case COMPRESSION_FORMAT_ZSTD -> new ZstdCompressorInputStream(bufferedIn); - case COMPRESSION_FORMAT_BROTLI -> { - Brotli4jLoader.ensureAvailability(); - yield new BrotliInputStream(bufferedIn); - } - default -> new CompressorStreamFactory().createCompressorInputStream(compressionFormat.toLowerCase(), bufferedIn); - }; + case COMPRESSION_FORMAT_DEFLATE: { + final int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger(); + compressionOut = new DeflaterOutputStream(bufferedOut, new Deflater(compressionLevel)); + mimeTypeRef.set("application/gzip"); + break; + } + case COMPRESSION_FORMAT_LZMA: + compressionOut = new LzmaOutputStream.Builder(bufferedOut).build(); + mimeTypeRef.set("application/x-lzma"); + break; + case COMPRESSION_FORMAT_XZ_LZMA2: + final int xzCompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger(); + compressionOut = new XZOutputStream(bufferedOut, new LZMA2Options(xzCompressionLevel)); + mimeTypeRef.set("application/x-xz"); + break; + case COMPRESSION_FORMAT_SNAPPY: + compressionOut = new SnappyOutputStream(bufferedOut); + mimeTypeRef.set("application/x-snappy"); + break; + case COMPRESSION_FORMAT_SNAPPY_HADOOP: + compressionOut = new SnappyHadoopCompatibleOutputStream(bufferedOut); + mimeTypeRef.set("application/x-snappy-hadoop"); + break; + case COMPRESSION_FORMAT_SNAPPY_FRAMED: + compressionOut = new SnappyFramedOutputStream(bufferedOut); + mimeTypeRef.set("application/x-snappy-framed"); + break; + case COMPRESSION_FORMAT_LZ4_FRAMED: + mimeTypeRef.set("application/x-lz4-framed"); + compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut); + break; + case COMPRESSION_FORMAT_ZSTD: + final int zstdCompressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger() * 2; + compressionOut = new ZstdCompressorOutputStream(bufferedOut, zstdCompressionLevel); + mimeTypeRef.set("application/zstd"); + break; + case COMPRESSION_FORMAT_BROTLI: { + Brotli4jLoader.ensureAvailability(); + final int compressionLevel = context.getProperty(COMPRESSION_LEVEL).asInteger(); + Encoder.Parameters params = new Encoder.Parameters().setQuality(compressionLevel); + compressionOut = new BrotliOutputStream(bufferedOut, params); + mimeTypeRef.set("application/x-brotli"); + break; + } + case COMPRESSION_FORMAT_BZIP2: + default: + mimeTypeRef.set("application/x-bzip2"); + compressionOut = new CompressorStreamFactory().createCompressorOutputStream(compressionFormat.toLowerCase(), bufferedOut); + break; } - } catch (final Exception e) { - closeQuietly(bufferedOut); - throw new IOException(e); + } else { + compressionOut = bufferedOut; + compressionIn = switch (compressionFormat.toLowerCase()) { + case COMPRESSION_FORMAT_LZMA -> new LzmaInputStream(bufferedIn, new Decoder()); + case COMPRESSION_FORMAT_XZ_LZMA2 -> new XZInputStream(bufferedIn); + case COMPRESSION_FORMAT_BZIP2 -> + // need this two-arg constructor to support concatenated streams + new BZip2CompressorInputStream(bufferedIn, true); + case COMPRESSION_FORMAT_GZIP -> new GzipCompressorInputStream(bufferedIn, true); + case COMPRESSION_FORMAT_DEFLATE -> new InflaterInputStream(bufferedIn); + case COMPRESSION_FORMAT_SNAPPY -> new SnappyInputStream(bufferedIn); + case COMPRESSION_FORMAT_SNAPPY_HADOOP -> throw new Exception("Cannot decompress snappy-hadoop."); + case COMPRESSION_FORMAT_SNAPPY_FRAMED -> new SnappyFramedInputStream(bufferedIn); + case COMPRESSION_FORMAT_LZ4_FRAMED -> new FramedLZ4CompressorInputStream(bufferedIn, true); + case COMPRESSION_FORMAT_ZSTD -> new ZstdCompressorInputStream(bufferedIn); + case COMPRESSION_FORMAT_BROTLI -> { + Brotli4jLoader.ensureAvailability(); + yield new BrotliInputStream(bufferedIn); + } + default -> new CompressorStreamFactory().createCompressorInputStream(compressionFormat.toLowerCase(), bufferedIn); + }; } + } catch (final Exception e) { + closeQuietly(bufferedOut); + throw new IOException(e); + } - try (final InputStream in = compressionIn; - final OutputStream out = compressionOut) { - final byte[] buffer = new byte[8192]; - int len; - while ((len = in.read(buffer)) > 0) { - out.write(buffer, 0, len); - } - out.flush(); + try (final InputStream in = compressionIn; + final OutputStream out = compressionOut) { + final byte[] buffer = new byte[8192]; + int len; + while ((len = in.read(buffer)) > 0) { + out.write(buffer, 0, len); } + out.flush(); } }); stopWatch.stop(); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java index cc5553863086..b625ece1dbcc 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertCharacterSet.java @@ -31,16 +31,12 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.StopWatch; import java.io.BufferedReader; import java.io.BufferedWriter; -import java.io.IOException; -import java.io.InputStream; import java.io.InputStreamReader; -import java.io.OutputStream; import java.io.OutputStreamWriter; import java.nio.CharBuffer; import java.nio.charset.Charset; @@ -140,19 +136,16 @@ public void onTrigger(final ProcessContext context, final ProcessSession session try { final StopWatch stopWatch = new StopWatch(true); - flowFile = session.write(flowFile, new StreamCallback() { - @Override - public void process(final InputStream rawIn, final OutputStream rawOut) throws IOException { - try (final BufferedReader reader = new BufferedReader(new InputStreamReader(rawIn, decoder), MAX_BUFFER_SIZE); - final BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(rawOut, encoder), MAX_BUFFER_SIZE)) { - int charsRead; - while ((charsRead = reader.read(charBuffer)) != -1) { - charBuffer.flip(); - writer.write(charBuffer.array(), 0, charsRead); - } - - writer.flush(); + flowFile = session.write(flowFile, (rawIn, rawOut) -> { + try (final BufferedReader reader = new BufferedReader(new InputStreamReader(rawIn, decoder), MAX_BUFFER_SIZE); + final BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(rawOut, encoder), MAX_BUFFER_SIZE)) { + int charsRead; + while ((charsRead = reader.read(charBuffer)) != -1) { + charBuffer.flip(); + writer.write(charBuffer.array(), 0, charsRead); } + + writer.flush(); } }); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DebugFlow.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DebugFlow.java index 28273d81d1f0..a50b1a8974b0 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DebugFlow.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DebugFlow.java @@ -35,11 +35,8 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; -import java.io.IOException; -import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.util.Collection; import java.util.List; @@ -448,12 +445,7 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro final byte[] data = new byte[context.getProperty(CONTENT_SIZE).asDataSize(DataUnit.B).intValue()]; random.nextBytes(data); - ff = session.write(ff, new OutputStreamCallback() { - @Override - public void process(final OutputStream out) throws IOException { - out.write(data); - } - }); + ff = session.write(ff, out -> out.write(data)); } } diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java index 5dee52ed9ff8..218a9f3e8fe7 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExtractText.java @@ -38,12 +38,10 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.stream.io.StreamUtils; import java.io.IOException; -import java.io.InputStream; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; @@ -399,12 +397,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session try { final byte[] byteBuffer = buffer; - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(InputStream in) throws IOException { - StreamUtils.fillBuffer(in, byteBuffer, false); - } - }); + session.read(flowFile, in -> StreamUtils.fillBuffer(in, byteBuffer, false)); final long len = Math.min(byteBuffer.length, flowFile.getSize()); contentString = new String(byteBuffer, 0, (int) len, charset); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java index eaa9b3993548..257c1d244a6d 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateFlowFile.java @@ -38,12 +38,9 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.scheduling.SchedulingStrategy; -import java.io.IOException; -import java.io.OutputStream; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; @@ -243,12 +240,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session FlowFile flowFile = session.create(); final byte[] writtenData = uniqueData ? generateData(context) : data; if (writtenData.length > 0) { - flowFile = session.write(flowFile, new OutputStreamCallback() { - @Override - public void process(final OutputStream out) throws IOException { - out.write(writtenData); - } - }); + flowFile = session.write(flowFile, out -> out.write(writtenData)); } flowFile = session.putAllAttributes(flowFile, generatedAttributes); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java index b03116e22014..5509e0ff14da 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/LookupRecord.java @@ -45,7 +45,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.record.path.FieldValue; import org.apache.nifi.record.path.RecordPath; @@ -386,41 +385,38 @@ public void onTrigger(final ProcessContext context, final ProcessSession session } try { - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { - - final Map writeSchemas = new HashMap<>(); - - Record record; - while ((record = reader.nextRecord()) != null) { - final List subRecords = getSubRecords(record, rootRecordPath); - final Set matchResults = new HashSet<>(); - for (final Record subRecord : subRecords) { - final MatchResult matchResult = replacementStrategy.lookup(subRecord, context, lookupContext); - matchResults.add(matchResult); - } - record.incorporateInactiveFields(); - - final Set relationships = getRelationships(matchResults); + session.read(flowFile, in -> { + try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { + + final Map writeSchemas = new HashMap<>(); + + Record record; + while ((record = reader.nextRecord()) != null) { + final List subRecords = getSubRecords(record, rootRecordPath); + final Set matchResults = new HashSet<>(); + for (final Record subRecord : subRecords) { + final MatchResult matchResult = replacementStrategy.lookup(subRecord, context, lookupContext); + matchResults.add(matchResult); + } + record.incorporateInactiveFields(); - for (final Relationship relationship : relationships) { - // Determine the Write Schema to use for each relationship - RecordSchema writeSchema = writeSchemas.get(relationship); - if (writeSchema == null) { - final RecordSchema outputSchema = enrichedSchema == null ? record.getSchema() : enrichedSchema; - writeSchema = writerFactory.getSchema(originalAttributes, outputSchema); - writeSchemas.put(relationship, writeSchema); - } + final Set relationships = getRelationships(matchResults); - final RecordSetWriter writer = lookupContext.getRecordWriterForRelationship(relationship, writeSchema); - writer.write(record); + for (final Relationship relationship : relationships) { + // Determine the Write Schema to use for each relationship + RecordSchema writeSchema = writeSchemas.get(relationship); + if (writeSchema == null) { + final RecordSchema outputSchema = enrichedSchema == null ? record.getSchema() : enrichedSchema; + writeSchema = writerFactory.getSchema(originalAttributes, outputSchema); + writeSchemas.put(relationship, writeSchema); } + + final RecordSetWriter writer = lookupContext.getRecordWriterForRelationship(relationship, writeSchema); + writer.write(record); } - } catch (final SchemaNotFoundException | MalformedRecordException e) { - throw new ProcessException("Could not parse incoming data", e); } + } catch (final SchemaNotFoundException | MalformedRecordException e) { + throw new ProcessException("Could not parse incoming data", e); } }); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java index 16045d5d93a8..cab590eb4abb 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ModifyBytes.java @@ -32,15 +32,10 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.OutputStreamCallback; -import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.util.StopWatch; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -117,19 +112,11 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro final StopWatch stopWatch = new StopWatch(true); if (newFileSize <= 0) { - ff = session.write(ff, new OutputStreamCallback() { - @Override - public void process(final OutputStream out) throws IOException { - out.write(new byte[0]); - } - }); + ff = session.write(ff, out -> out.write(new byte[0])); } else { - ff = session.write(ff, new StreamCallback() { - @Override - public void process(final InputStream in, final OutputStream out) throws IOException { - in.skipNBytes(startOffset); - StreamUtils.copy(in, out, newFileSize); - } + ff = session.write(ff, (in, out) -> { + in.skipNBytes(startOffset); + StreamUtils.copy(in, out, newFileSize); }); } diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ParseSyslog.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ParseSyslog.java index b76d44864789..2850ae2580b2 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ParseSyslog.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ParseSyslog.java @@ -33,15 +33,12 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.stream.io.StreamUtils; import org.apache.nifi.syslog.attributes.SyslogAttributes; import org.apache.nifi.syslog.events.SyslogEvent; import org.apache.nifi.syslog.parsers.SyslogParser; -import java.io.IOException; -import java.io.InputStream; import java.nio.charset.Charset; import java.util.HashMap; import java.util.List; @@ -120,12 +117,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session } final byte[] buffer = new byte[(int) flowFile.getSize()]; - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - StreamUtils.fillBuffer(in, buffer); - } - }); + session.read(flowFile, in -> StreamUtils.fillBuffer(in, buffer)); final SyslogEvent event; try { diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java index 5822651cd40b..00135aebf877 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java @@ -512,12 +512,9 @@ public FlowFile replace(FlowFile flowFile, final ProcessSession session, final P final String replacementValue = context.getProperty(REPLACEMENT_VALUE).evaluateAttributeExpressions(flowFile).getValue(); if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) { - flowFile = session.write(flowFile, new StreamCallback() { - @Override - public void process(final InputStream in, final OutputStream out) throws IOException { - out.write(replacementValue.getBytes(charset)); - IOUtils.copy(in, out); - } + flowFile = session.write(flowFile, (in, out) -> { + out.write(replacementValue.getBytes(charset)); + IOUtils.copy(in, out); }); } else { flowFile = session.write(flowFile, new StreamReplaceCallback(charset, maxBufferSize, context.getProperty(LINE_BY_LINE_EVALUATION_MODE).getValue(), @@ -548,16 +545,13 @@ public FlowFile replace(FlowFile flowFile, final ProcessSession session, final P final String appendValue = context.getProperty(appendValueDescriptor).evaluateAttributeExpressions(flowFile).getValue(); if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) { - flowFile = session.write(flowFile, new StreamCallback() { - @Override - public void process(final InputStream in, final OutputStream out) throws IOException { - if (prependValue != null && !prependValue.isEmpty()) { - out.write(prependValue.getBytes(charset)); - } - - IOUtils.copy(in, out); - out.write(appendValue.getBytes(charset)); + flowFile = session.write(flowFile, (in, out) -> { + if (prependValue != null && !prependValue.isEmpty()) { + out.write(prependValue.getBytes(charset)); } + + IOUtils.copy(in, out); + out.write(appendValue.getBytes(charset)); }); } else { flowFile = session.write(flowFile, new StreamReplaceCallback(charset, maxBufferSize, context.getProperty(LINE_BY_LINE_EVALUATION_MODE).getValue(), @@ -605,12 +599,9 @@ private static class RegexReplace implements ReplacementStrategyExecutor { private final int numCapturingGroups; // back references are not supported in the evaluated expression - private final AttributeValueDecorator escapeBackRefDecorator = new AttributeValueDecorator() { - @Override - public String decorate(final String attributeValue) { - // when we encounter a '$[0-9+]' replace it with '\$[0-9+]' - return attributeValue.replaceAll("(\\$\\d+?)", "\\\\$1"); - } + private final AttributeValueDecorator escapeBackRefDecorator = attributeValue -> { + // when we encounter a '$[0-9+]' replace it with '\$[0-9+]' + return attributeValue.replaceAll("(\\$\\d+?)", "\\\\$1"); }; public RegexReplace(final String regex) { @@ -718,15 +709,12 @@ public FlowFile replace(FlowFile flowFile, final ProcessSession session, final P final int bufferSize = Math.min(maxBufferSize, flowFileSize); final byte[] buffer = new byte[bufferSize]; - flowFile = session.write(flowFile, new StreamCallback() { - @Override - public void process(final InputStream in, final OutputStream out) throws IOException { - StreamUtils.fillBuffer(in, buffer, false); - final String contentString = new String(buffer, 0, flowFileSize, charset); - // Interpreting the search and replacement values as char sequences - final String updatedValue = contentString.replace(searchValue, replacementValue); - out.write(updatedValue.getBytes(charset)); - } + flowFile = session.write(flowFile, (in, out) -> { + StreamUtils.fillBuffer(in, buffer, false); + final String contentString = new String(buffer, 0, flowFileSize, charset); + // Interpreting the search and replacement values as char sequences + final String updatedValue = contentString.replace(searchValue, replacementValue); + out.write(updatedValue.getBytes(charset)); }); } else { final Pattern searchPattern = Pattern.compile(searchValue, Pattern.LITERAL); @@ -772,14 +760,11 @@ public FlowFile replace(FlowFile flowFile, final ProcessSession session, final P final int bufferSize = Math.min(maxBufferSize, flowFileSize); final byte[] buffer = new byte[bufferSize]; - flowFile = session.write(flowFile, new StreamCallback() { - @Override - public void process(final InputStream in, final OutputStream out) throws IOException { - StreamUtils.fillBuffer(in, buffer, false); - final String originalContent = new String(buffer, 0, flowFileSize, charset); - final String substitutedContent = StringSubstitutor.replace(originalContent, flowFileAttributes); - out.write(substitutedContent.getBytes(charset)); - } + flowFile = session.write(flowFile, (in, out) -> { + StreamUtils.fillBuffer(in, buffer, false); + final String originalContent = new String(buffer, 0, flowFileSize, charset); + final String substitutedContent = StringSubstitutor.replace(originalContent, flowFileAttributes); + out.write(substitutedContent.getBytes(charset)); }); } else { flowFile = session.write(flowFile, new StreamReplaceCallback(charset, maxBufferSize, context.getProperty(LINE_BY_LINE_EVALUATION_MODE).getValue(), diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java index ff1509b6a398..9c2cad339633 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteOnContent.java @@ -34,12 +34,9 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.stream.io.StreamUtils; -import java.io.IOException; -import java.io.InputStream; import java.nio.charset.Charset; import java.util.HashMap; import java.util.HashSet; @@ -155,12 +152,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session return; } - final AttributeValueDecorator quoteDecorator = new AttributeValueDecorator() { - @Override - public String decorate(final String attributeValue) { - return (attributeValue == null) ? null : Pattern.quote(attributeValue); - } - }; + final AttributeValueDecorator quoteDecorator = attributeValue -> (attributeValue == null) ? null : Pattern.quote(attributeValue); final Map> flowFileDestinationMap = new HashMap<>(); final ComponentLog logger = getLogger(); @@ -172,12 +164,7 @@ public String decorate(final String attributeValue) { flowFileDestinationMap.put(flowFile, destinations); final AtomicInteger bufferedByteCount = new AtomicInteger(0); - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - bufferedByteCount.set(StreamUtils.fillBuffer(in, buffer, false)); - } - }); + session.read(flowFile, in -> bufferedByteCount.set(StreamUtils.fillBuffer(in, buffer, false))); final String contentString = new String(buffer, 0, bufferedByteCount.get(), charset); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java index 19ad4c254e7f..7c5338cb3172 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RouteText.java @@ -45,14 +45,9 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -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.stream.io.util.LineDemarcator; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; @@ -431,88 +426,85 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final Map> flowFileMap = new HashMap<>(); final Pattern groupPattern = groupingRegex; - session.read(originalFlowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - try (final LineDemarcator demarcator = new LineDemarcator(in, charset, Integer.MAX_VALUE, 8192)) { + session.read(originalFlowFile, in -> { + try (final LineDemarcator demarcator = new LineDemarcator(in, charset, Integer.MAX_VALUE, 8192)) { + + final Map variables = new HashMap<>(2); + + int lineCount = 0; + String line; + while ((line = demarcator.nextLine()) != null) { + + final String matchLine; + if (trim) { + matchLine = line.trim(); + } else { + // Always trim off the new-line and carriage return characters before evaluating the line. + // The NLKBufferedReader maintains these characters so that when we write the line out we can maintain + // these characters. However, we don't actually want to match against these characters. + final String lineWithoutEndings; + final int indexOfCR = line.indexOf("\r"); + final int indexOfNL = line.indexOf("\n"); + if (indexOfCR > 0 && indexOfNL > 0) { + lineWithoutEndings = line.substring(0, Math.min(indexOfCR, indexOfNL)); + } else if (indexOfCR > 0) { + lineWithoutEndings = line.substring(0, indexOfCR); + } else if (indexOfNL > 0) { + lineWithoutEndings = line.substring(0, indexOfNL); + } else { + lineWithoutEndings = line; + } - final Map variables = new HashMap<>(2); + matchLine = lineWithoutEndings; + } - int lineCount = 0; - String line; - while ((line = demarcator.nextLine()) != null) { + variables.put("line", line); + variables.put("lineNo", String.valueOf(++lineCount)); - final String matchLine; - if (trim) { - matchLine = line.trim(); - } else { - // Always trim off the new-line and carriage return characters before evaluating the line. - // The NLKBufferedReader maintains these characters so that when we write the line out we can maintain - // these characters. However, we don't actually want to match against these characters. - final String lineWithoutEndings; - final int indexOfCR = line.indexOf("\r"); - final int indexOfNL = line.indexOf("\n"); - if (indexOfCR > 0 && indexOfNL > 0) { - lineWithoutEndings = line.substring(0, Math.min(indexOfCR, indexOfNL)); - } else if (indexOfCR > 0) { - lineWithoutEndings = line.substring(0, indexOfCR); - } else if (indexOfNL > 0) { - lineWithoutEndings = line.substring(0, indexOfNL); - } else { - lineWithoutEndings = line; - } - - matchLine = lineWithoutEndings; + int propertiesThatMatchedLine = 0; + for (final Map.Entry entry : propValueMap.entrySet()) { + boolean lineMatchesProperty = lineMatches(matchLine, entry.getValue(), matchStrategy, ignoreCase, originalFlowFile, variables); + if (lineMatchesProperty) { + propertiesThatMatchedLine++; } - variables.put("line", line); - variables.put("lineNo", String.valueOf(++lineCount)); - - int propertiesThatMatchedLine = 0; - for (final Map.Entry entry : propValueMap.entrySet()) { - boolean lineMatchesProperty = lineMatches(matchLine, entry.getValue(), matchStrategy, ignoreCase, originalFlowFile, variables); - if (lineMatchesProperty) { - propertiesThatMatchedLine++; - } - - if (lineMatchesProperty && ROUTE_TO_MATCHING_PROPERTY_NAME.getValue().equals(routeStrategy)) { - // route each individual line to each Relationship that matches. This one matches. - final Relationship relationship = entry.getKey(); - - final Group group = getGroup(matchLine, groupPattern); - appendLine(session, flowFileMap, relationship, originalFlowFile, line, charset, group); - continue; - } - - // break as soon as possible to avoid calculating things we don't need to calculate. - if (lineMatchesProperty && ROUTE_TO_MATCHED_WHEN_ANY_PROPERTY_MATCHES.getValue().equals(routeStrategy)) { - break; - } - - if (!lineMatchesProperty && ROUTE_TO_MATCHED_WHEN_ALL_PROPERTIES_MATCH.getValue().equals(routeStrategy)) { - break; - } + if (lineMatchesProperty && ROUTE_TO_MATCHING_PROPERTY_NAME.getValue().equals(routeStrategy)) { + // route each individual line to each Relationship that matches. This one matches. + final Relationship relationship = entry.getKey(); + + final Group group = getGroup(matchLine, groupPattern); + appendLine(session, flowFileMap, relationship, originalFlowFile, line, charset, group); + continue; } - final Relationship relationship; - if (ROUTE_TO_MATCHING_PROPERTY_NAME.getValue().equals(routeStrategy) && propertiesThatMatchedLine > 0) { - // Set relationship to null so that we do not append the line to each FlowFile again. #appendLine is called - // above within the loop, as the line may need to go to multiple different FlowFiles. - relationship = null; - } else if (ROUTE_TO_MATCHED_WHEN_ANY_PROPERTY_MATCHES.getValue().equals(routeStrategy) && propertiesThatMatchedLine > 0) { - relationship = REL_MATCH; - } else if (ROUTE_TO_MATCHED_WHEN_ALL_PROPERTIES_MATCH.getValue().equals(routeStrategy) && propertiesThatMatchedLine == propValueMap.size()) { - relationship = REL_MATCH; - } else { - relationship = REL_NO_MATCH; + // break as soon as possible to avoid calculating things we don't need to calculate. + if (lineMatchesProperty && ROUTE_TO_MATCHED_WHEN_ANY_PROPERTY_MATCHES.getValue().equals(routeStrategy)) { + break; } - // If the target relationship (usually REL_NO_MATCH) is auto-terminated, don't bother creating the flowfile or writing to it. - if (relationship != null && !context.isAutoTerminated(relationship)) { - final Group group = getGroup(matchLine, groupPattern); - appendLine(session, flowFileMap, relationship, originalFlowFile, line, charset, group); + if (!lineMatchesProperty && ROUTE_TO_MATCHED_WHEN_ALL_PROPERTIES_MATCH.getValue().equals(routeStrategy)) { + break; } } + + final Relationship relationship; + if (ROUTE_TO_MATCHING_PROPERTY_NAME.getValue().equals(routeStrategy) && propertiesThatMatchedLine > 0) { + // Set relationship to null so that we do not append the line to each FlowFile again. #appendLine is called + // above within the loop, as the line may need to go to multiple different FlowFiles. + relationship = null; + } else if (ROUTE_TO_MATCHED_WHEN_ANY_PROPERTY_MATCHES.getValue().equals(routeStrategy) && propertiesThatMatchedLine > 0) { + relationship = REL_MATCH; + } else if (ROUTE_TO_MATCHED_WHEN_ALL_PROPERTIES_MATCH.getValue().equals(routeStrategy) && propertiesThatMatchedLine == propValueMap.size()) { + relationship = REL_MATCH; + } else { + relationship = REL_NO_MATCH; + } + + // If the target relationship (usually REL_NO_MATCH) is auto-terminated, don't bother creating the flowfile or writing to it. + if (relationship != null && !context.isAutoTerminated(relationship)) { + final Group group = getGroup(matchLine, groupPattern); + appendLine(session, flowFileMap, relationship, originalFlowFile, line, charset, group); + } } } }); @@ -571,12 +563,7 @@ private void appendLine(final ProcessSession session, final Map out.write(line.getBytes(charset))); groupToFlowFileMap.put(group, flowFile); } diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java index 22fe514c94ec..ac090bdba656 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ScanContent.java @@ -33,7 +33,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.util.file.monitor.LastModifiedMonitor; import org.apache.nifi.util.file.monitor.SynchronousFileWatcher; import org.apache.nifi.util.search.Search; @@ -209,14 +208,11 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final AtomicReference> termRef = new AtomicReference<>(null); termRef.set(null); - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream rawIn) throws IOException { - try (final InputStream in = new BufferedInputStream(rawIn)) { - final SearchState searchResult = finalSearch.search(in, false); - if (searchResult.foundMatch()) { - termRef.set(searchResult.getResults().keySet().iterator().next()); - } + session.read(flowFile, rawIn -> { + try (final InputStream in = new BufferedInputStream(rawIn)) { + final SearchState searchResult = finalSearch.search(in, false); + if (searchResult.foundMatch()) { + termRef.set(searchResult.getResults().keySet().iterator().next()); } } }); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java index 5328e7435e37..a7d2af90e3b2 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java @@ -43,13 +43,11 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.NaiveSearchRingBuffer; import org.apache.nifi.util.Tuple; import java.io.BufferedInputStream; -import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -208,39 +206,36 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final List> splits = new ArrayList<>(); final NaiveSearchRingBuffer buffer = new NaiveSearchRingBuffer(byteSequence); - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream rawIn) throws IOException { - long bytesRead = 0L; - long startOffset = 0L; - - try (final InputStream in = new BufferedInputStream(rawIn)) { - while (true) { - final int nextByte = in.read(); - if (nextByte == -1) { - return; + session.read(flowFile, rawIn -> { + long bytesRead = 0L; + long startOffset = 0L; + + try (final InputStream in = new BufferedInputStream(rawIn)) { + while (true) { + final int nextByte = in.read(); + if (nextByte == -1) { + return; + } + + bytesRead++; + boolean matched = buffer.addAndCompare((byte) (nextByte & 0xFF)); + if (matched) { + long splitLength; + + if (keepTrailingSequence) { + splitLength = bytesRead - startOffset; + } else { + splitLength = bytesRead - startOffset - byteSequence.length; } - bytesRead++; - boolean matched = buffer.addAndCompare((byte) (nextByte & 0xFF)); - if (matched) { - long splitLength; - - if (keepTrailingSequence) { - splitLength = bytesRead - startOffset; - } else { - splitLength = bytesRead - startOffset - byteSequence.length; - } - - if (keepLeadingSequence && startOffset > 0) { - splitLength += byteSequence.length; - } - - final long splitStart = (keepLeadingSequence && startOffset > 0) ? startOffset - byteSequence.length : startOffset; - splits.add(new Tuple<>(splitStart, splitLength)); - startOffset = bytesRead; - buffer.clear(); + if (keepLeadingSequence && startOffset > 0) { + splitLength += byteSequence.length; } + + final long splitStart = (keepLeadingSequence && startOffset > 0) ? startOffset - byteSequence.length : startOffset; + splits.add(new Tuple<>(splitStart, splitLength)); + startOffset = bytesRead; + buffer.clear(); } } } diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java index 451333f836f3..deb69bcef709 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitRecord.java @@ -35,7 +35,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; @@ -49,8 +48,6 @@ import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.RecordSet; -import java.io.IOException; -import java.io.InputStream; import java.io.OutputStream; import java.util.ArrayList; import java.util.HashMap; @@ -151,53 +148,50 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final Map originalAttributes = original.getAttributes(); final String fragmentId = UUID.randomUUID().toString(); try { - session.read(original, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { - - final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema()); - - final RecordSet recordSet = reader.createRecordSet(); - final PushBackRecordSet pushbackSet = new PushBackRecordSet(recordSet); - - int fragmentIndex = 0; - while (pushbackSet.isAnotherRecord()) { - FlowFile split = session.create(original); - - try { - final Map attributes = new HashMap<>(); - final WriteResult writeResult; - - try (final OutputStream out = session.write(split); - final RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, out, split)) { - if (maxRecords == 1) { - final Record record = pushbackSet.next(); - writeResult = writer.write(record); - } else { - final RecordSet limitedSet = pushbackSet.limit(maxRecords); - writeResult = writer.write(limitedSet); - } - - attributes.put("record.count", String.valueOf(writeResult.getRecordCount())); - attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType()); - attributes.put(FRAGMENT_INDEX, String.valueOf(fragmentIndex)); - attributes.put(FRAGMENT_ID, fragmentId); - attributes.put(SEGMENT_ORIGINAL_FILENAME, original.getAttribute(CoreAttributes.FILENAME.key())); - attributes.putAll(writeResult.getAttributes()); - - session.adjustCounter("Records Split", writeResult.getRecordCount(), false); - } - - split = session.putAllAttributes(split, attributes); - } finally { - splits.add(split); + session.read(original, in -> { + try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, original.getSize(), getLogger())) { + + final RecordSchema schema = writerFactory.getSchema(originalAttributes, reader.getSchema()); + + final RecordSet recordSet = reader.createRecordSet(); + final PushBackRecordSet pushbackSet = new PushBackRecordSet(recordSet); + + int fragmentIndex = 0; + while (pushbackSet.isAnotherRecord()) { + FlowFile split = session.create(original); + + try { + final Map attributes = new HashMap<>(); + final WriteResult writeResult; + + try (final OutputStream out = session.write(split); + final RecordSetWriter writer = writerFactory.createWriter(getLogger(), schema, out, split)) { + if (maxRecords == 1) { + final Record record = pushbackSet.next(); + writeResult = writer.write(record); + } else { + final RecordSet limitedSet = pushbackSet.limit(maxRecords); + writeResult = writer.write(limitedSet); + } + + attributes.put("record.count", String.valueOf(writeResult.getRecordCount())); + attributes.put(CoreAttributes.MIME_TYPE.key(), writer.getMimeType()); + attributes.put(FRAGMENT_INDEX, String.valueOf(fragmentIndex)); + attributes.put(FRAGMENT_ID, fragmentId); + attributes.put(SEGMENT_ORIGINAL_FILENAME, original.getAttribute(CoreAttributes.FILENAME.key())); + attributes.putAll(writeResult.getAttributes()); + + session.adjustCounter("Records Split", writeResult.getRecordCount(), false); } - fragmentIndex++; + + split = session.putAllAttributes(split, attributes); + } finally { + splits.add(split); } - } catch (final SchemaNotFoundException | MalformedRecordException e) { - throw new ProcessException("Failed to parse incoming data", e); + fragmentIndex++; } + } catch (final SchemaNotFoundException | MalformedRecordException e) { + throw new ProcessException("Failed to parse incoming data", e); } }); } catch (final ProcessException pe) { diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java index 5936f129ddb4..3b309c0590ab 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitText.java @@ -41,15 +41,12 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.stream.io.util.TextLineDemarcator; import org.apache.nifi.stream.io.util.TextLineDemarcator.OffsetInfo; import java.io.IOException; import java.io.InputStream; -import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; @@ -203,46 +200,43 @@ public void onTrigger(ProcessContext context, ProcessSession processSession) thr AtomicBoolean error = new AtomicBoolean(); List computedSplitsInfo = new ArrayList<>(); AtomicReference headerSplitInfoRef = new AtomicReference<>(); - processSession.read(sourceFlowFile, new InputStreamCallback() { - @Override - public void process(InputStream in) throws IOException { - TextLineDemarcator demarcator = new TextLineDemarcator(in); - SplitInfo splitInfo = null; - long startOffset = 0; - - // Compute fragment representing the header (if available) - long start = System.nanoTime(); - try { - if (SplitText.this.headerLineCount > 0) { - splitInfo = SplitText.this.computeHeader(demarcator, startOffset, SplitText.this.headerLineCount, null, null); - if ((splitInfo != null) && (splitInfo.lineCount < SplitText.this.headerLineCount)) { - error.set(true); - getLogger().error("Unable to split {} due to insufficient amount of header lines. Required {} but was {}. Routing to failure.", - sourceFlowFile, SplitText.this.headerLineCount, splitInfo.lineCount); - } - } else if (SplitText.this.headerMarker != null) { - splitInfo = SplitText.this.computeHeader(demarcator, startOffset, Long.MAX_VALUE, SplitText.this.headerMarker.getBytes(StandardCharsets.UTF_8), null); + processSession.read(sourceFlowFile, in -> { + TextLineDemarcator demarcator = new TextLineDemarcator(in); + SplitInfo splitInfo = null; + long startOffset = 0; + + // Compute fragment representing the header (if available) + long start = System.nanoTime(); + try { + if (SplitText.this.headerLineCount > 0) { + splitInfo = SplitText.this.computeHeader(demarcator, startOffset, SplitText.this.headerLineCount, null, null); + if ((splitInfo != null) && (splitInfo.lineCount < SplitText.this.headerLineCount)) { + error.set(true); + getLogger().error("Unable to split {} due to insufficient amount of header lines. Required {} but was {}. Routing to failure.", + sourceFlowFile, SplitText.this.headerLineCount, splitInfo.lineCount); } - headerSplitInfoRef.set(splitInfo); - } catch (IllegalStateException e) { - error.set(true); - getLogger().error("Routing to failure.", e); + } else if (SplitText.this.headerMarker != null) { + splitInfo = SplitText.this.computeHeader(demarcator, startOffset, Long.MAX_VALUE, SplitText.this.headerMarker.getBytes(StandardCharsets.UTF_8), null); } + headerSplitInfoRef.set(splitInfo); + } catch (IllegalStateException e) { + error.set(true); + getLogger().error("Routing to failure.", e); + } - // Compute and collect fragments representing the individual splits - if (!error.get()) { - if (headerSplitInfoRef.get() != null) { - startOffset = headerSplitInfoRef.get().length; - } - long preAccumulatedLength = startOffset; - while ((splitInfo = SplitText.this.nextSplit(demarcator, startOffset, SplitText.this.lineCount, splitInfo, preAccumulatedLength)) != null) { - computedSplitsInfo.add(splitInfo); - startOffset += splitInfo.length; - } - long stop = System.nanoTime(); - if (getLogger().isDebugEnabled()) { - getLogger().debug("Computed splits in {} milliseconds.", (stop - start)); - } + // Compute and collect fragments representing the individual splits + if (!error.get()) { + if (headerSplitInfoRef.get() != null) { + startOffset = headerSplitInfoRef.get().length; + } + long preAccumulatedLength = startOffset; + while ((splitInfo = SplitText.this.nextSplit(demarcator, startOffset, SplitText.this.lineCount, splitInfo, preAccumulatedLength)) != null) { + computedSplitsInfo.add(splitInfo); + startOffset += splitInfo.length; + } + long stop = System.nanoTime(); + if (getLogger().isDebugEnabled()) { + getLogger().debug("Computed splits in {} milliseconds.", (stop - start)); } } }); @@ -357,12 +351,9 @@ private List generateSplitFlowFiles(String fragmentId, FlowFile source private FlowFile concatenateContents(FlowFile sourceFlowFile, ProcessSession session, FlowFile... flowFiles) { FlowFile mergedFlowFile = session.create(sourceFlowFile); for (FlowFile flowFile : flowFiles) { - mergedFlowFile = session.append(mergedFlowFile, new OutputStreamCallback() { - @Override - public void process(OutputStream out) throws IOException { - try (InputStream is = session.read(flowFile)) { - IOUtils.copy(is, out); - } + mergedFlowFile = session.append(mergedFlowFile, out -> { + try (InputStream is = session.read(flowFile)) { + IOUtils.copy(is, out); } }); } diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java index 1cbdc663651a..3a13f4328eba 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TailFile.java @@ -47,7 +47,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.processors.standard.TailFile.TailFileState.StateKeys; import org.apache.nifi.scheduling.SchedulingStrategy; @@ -897,15 +896,12 @@ private void processTailFile(final ProcessContext context, final ProcessSession final boolean reReadOnNul = context.getProperty(REREAD_ON_NUL).asBoolean(); AtomicReference abort = new AtomicReference<>(); - flowFile = session.write(flowFile, new OutputStreamCallback() { - @Override - public void process(final OutputStream rawOut) throws IOException { - try (final OutputStream out = new BufferedOutputStream(rawOut)) { - positionHolder.set(readLines(fileReader, currentState.getBuffer(), out, chksum, reReadOnNul)); - } catch (NulCharacterEncounteredException e) { - positionHolder.set(e.getRePos()); - abort.set(e); - } + flowFile = session.write(flowFile, rawOut -> { + try (final OutputStream out = new BufferedOutputStream(rawOut)) { + positionHolder.set(readLines(fileReader, currentState.getBuffer(), out, chksum, reReadOnNul)); + } catch (NulCharacterEncounteredException e) { + positionHolder.set(e.getRePos()); + abort.set(e); } }); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java index 223ce22fda52..f83155a22baf 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitJson.java @@ -20,14 +20,12 @@ import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; import org.junit.jupiter.api.Test; import java.io.BufferedOutputStream; -import java.io.IOException; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.nio.file.Path; @@ -61,7 +59,7 @@ public void testInvalidJsonDocument() throws Exception { testRunner.run(); testRunner.assertAllFlowFilesTransferred(SplitJson.REL_FAILURE, 1); - final MockFlowFile out = testRunner.getFlowFilesForRelationship(SplitJson.REL_FAILURE).get(0); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(SplitJson.REL_FAILURE).getFirst(); // Verify that the content was unchanged out.assertContentEquals(XML_SNIPPET); } @@ -77,7 +75,7 @@ public void testSplit_nonArrayResult() throws Exception { Relationship expectedRel = SplitJson.REL_FAILURE; testRunner.assertAllFlowFilesTransferred(expectedRel, 1); - final MockFlowFile out = testRunner.getFlowFilesForRelationship(expectedRel).get(0); + final MockFlowFile out = testRunner.getFlowFilesForRelationship(expectedRel).getFirst(); out.assertContentEquals(JSON_SNIPPET); } @@ -90,10 +88,10 @@ public void testSplit_arrayResult_oneValue() throws Exception { testRunner.run(); testRunner.assertTransferCount(SplitJson.REL_ORIGINAL, 1); - testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0).assertAttributeEquals(FRAGMENT_COUNT.key(), "1"); + testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).getFirst().assertAttributeEquals(FRAGMENT_COUNT.key(), "1"); testRunner.assertTransferCount(SplitJson.REL_SPLIT, 1); - testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0).assertContentEquals(JSON_SNIPPET); - testRunner.getFlowFilesForRelationship(SplitJson.REL_SPLIT).get(0).assertContentEquals("0"); + testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).getFirst().assertContentEquals(JSON_SNIPPET); + testRunner.getFlowFilesForRelationship(SplitJson.REL_SPLIT).getFirst().assertContentEquals("0"); } @Test @@ -107,9 +105,9 @@ public void testSplit_arrayResult_multipleValues() throws Exception { int numSplitsExpected = 10; testRunner.assertTransferCount(SplitJson.REL_ORIGINAL, 1); - testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0).assertAttributeEquals(FRAGMENT_COUNT.key(), String.valueOf(numSplitsExpected)); + testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).getFirst().assertAttributeEquals(FRAGMENT_COUNT.key(), String.valueOf(numSplitsExpected)); testRunner.assertTransferCount(SplitJson.REL_SPLIT, numSplitsExpected); - final MockFlowFile originalOut = testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0); + final MockFlowFile originalOut = testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).getFirst(); originalOut.assertContentEquals(JSON_SNIPPET); } @@ -124,9 +122,9 @@ public void testSplit_change_jsonpath() throws Exception { int numSplitsExpected = 10; testRunner.assertTransferCount(SplitJson.REL_ORIGINAL, 1); - testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0).assertAttributeEquals(FRAGMENT_COUNT.key(), String.valueOf(numSplitsExpected)); + testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).getFirst().assertAttributeEquals(FRAGMENT_COUNT.key(), String.valueOf(numSplitsExpected)); testRunner.assertTransferCount(SplitJson.REL_SPLIT, numSplitsExpected); - final MockFlowFile originalOut = testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0); + final MockFlowFile originalOut = testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).getFirst(); originalOut.assertContentEquals(JSON_SNIPPET); // Change JsonPath Expression, verify it is being applied correctly @@ -137,12 +135,12 @@ public void testSplit_change_jsonpath() throws Exception { testRunner.run(); testRunner.assertTransferCount(SplitJson.REL_ORIGINAL, 1); - final MockFlowFile originalFlowFile = testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0); + final MockFlowFile originalFlowFile = testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).getFirst(); originalFlowFile.assertAttributeExists(FRAGMENT_ID.key()); originalFlowFile.assertAttributeEquals(FRAGMENT_COUNT.key(), "7"); originalFlowFile.assertContentEquals(JSON_SNIPPET); testRunner.assertTransferCount(SplitJson.REL_SPLIT, 7); - MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(SplitJson.REL_SPLIT).get(0); + MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(SplitJson.REL_SPLIT).getFirst(); flowFile.assertContentEquals("{\"first\":\"Shaffer\",\"last\":\"Pearson\"}"); } @@ -155,12 +153,12 @@ public void testSplit_arrayResult_nonScalarValues() throws Exception { testRunner.run(); testRunner.assertTransferCount(SplitJson.REL_ORIGINAL, 1); - final MockFlowFile originalFlowFile = testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).get(0); + final MockFlowFile originalFlowFile = testRunner.getFlowFilesForRelationship(SplitJson.REL_ORIGINAL).getFirst(); originalFlowFile.assertAttributeExists(FRAGMENT_ID.key()); originalFlowFile.assertAttributeEquals(FRAGMENT_COUNT.key(), "7"); originalFlowFile.assertContentEquals(JSON_SNIPPET); testRunner.assertTransferCount(SplitJson.REL_SPLIT, 7); - MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(SplitJson.REL_SPLIT).get(0); + MockFlowFile flowFile = testRunner.getFlowFilesForRelationship(SplitJson.REL_SPLIT).getFirst(); flowFile.assertContentEquals("{\"first\":\"Shaffer\",\"last\":\"Pearson\"}"); flowFile.assertAttributeEquals(FRAGMENT_COUNT.key(), "7"); flowFile.assertAttributeEquals(FRAGMENT_INDEX.key(), "0"); @@ -181,7 +179,7 @@ public void testSplit_pathNotFound() throws Exception { testRunner.run(); testRunner.assertTransferCount(SplitJson.REL_FAILURE, 1); - testRunner.getFlowFilesForRelationship(SplitJson.REL_FAILURE).get(0).assertContentEquals(JSON_SNIPPET); + testRunner.getFlowFilesForRelationship(SplitJson.REL_FAILURE).getFirst().assertContentEquals(JSON_SNIPPET); } @Test @@ -192,12 +190,9 @@ public void testSplit_pathToNullValue() { ProcessSession session = testRunner.getProcessSessionFactory().createSession(); FlowFile ff = session.create(); - ff = session.write(ff, new OutputStreamCallback() { - @Override - public void process(OutputStream out) throws IOException { - try (OutputStream outputStream = new BufferedOutputStream(out)) { - outputStream.write("{\"stringField\": \"String Value\", \"nullField\": null}".getBytes(StandardCharsets.UTF_8)); - } + ff = session.write(ff, out -> { + try (OutputStream outputStream = new BufferedOutputStream(out)) { + outputStream.write("{\"stringField\": \"String Value\", \"nullField\": null}".getBytes(StandardCharsets.UTF_8)); } }); @@ -215,12 +210,9 @@ public void testSplit_pathToArrayWithNulls_emptyStringRepresentation() { ProcessSession session = testRunner.getProcessSessionFactory().createSession(); FlowFile ff = session.create(); - ff = session.write(ff, new OutputStreamCallback() { - @Override - public void process(OutputStream out) throws IOException { - try (OutputStream outputStream = new BufferedOutputStream(out)) { - outputStream.write("{\"stringField\": \"String Value\", \"arrayOfNulls\": [null, null, null]}".getBytes(StandardCharsets.UTF_8)); - } + ff = session.write(ff, out -> { + try (OutputStream outputStream = new BufferedOutputStream(out)) { + outputStream.write("{\"stringField\": \"String Value\", \"arrayOfNulls\": [null, null, null]}".getBytes(StandardCharsets.UTF_8)); } }); @@ -245,12 +237,9 @@ public void testSplit_pathToArrayWithNulls_nullStringRepresentation() { ProcessSession session = testRunner.getProcessSessionFactory().createSession(); FlowFile ff = session.create(); - ff = session.write(ff, new OutputStreamCallback() { - @Override - public void process(OutputStream out) throws IOException { - try (OutputStream outputStream = new BufferedOutputStream(out)) { - outputStream.write("{\"stringField\": \"String Value\", \"arrayOfNulls\": [null, null, null]}".getBytes(StandardCharsets.UTF_8)); - } + ff = session.write(ff, out -> { + try (OutputStream outputStream = new BufferedOutputStream(out)) { + outputStream.write("{\"stringField\": \"String Value\", \"arrayOfNulls\": [null, null, null]}".getBytes(StandardCharsets.UTF_8)); } }); @@ -271,12 +260,9 @@ public void testSplit_pathToInputStringNullValue() { testRunner.setProperty(SplitJson.ARRAY_JSON_PATH_EXPRESSION, "$.*"); ProcessSession session = testRunner.getProcessSessionFactory().createSession(); FlowFile ff = session.create(); - ff = session.write(ff, new OutputStreamCallback() { - @Override - public void process(OutputStream out) throws IOException { - try (OutputStream outputStream = new BufferedOutputStream(out)) { - outputStream.write("null".getBytes(StandardCharsets.UTF_8)); - } + ff = session.write(ff, out -> { + try (OutputStream outputStream = new BufferedOutputStream(out)) { + outputStream.write("null".getBytes(StandardCharsets.UTF_8)); } }); testRunner.enqueue(ff); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java index 7f3ddb59f46b..772e0f2d6ceb 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTailFile.java @@ -37,7 +37,6 @@ import java.io.BufferedWriter; import java.io.File; import java.io.FileOutputStream; -import java.io.FilenameFilter; import java.io.IOException; import java.io.OutputStreamWriter; import java.io.RandomAccessFile; @@ -182,7 +181,7 @@ public void testNULContentWithReReadOnNulTrue() throws Exception { Integer reposition = "first_line_with_nul".length(); String content2 = " overwrite_nul_and_continue_first_line\n"; - List expected = Arrays.asList("first_line_with_nul overwrite_nul_and_continue_first_line\n"); + List expected = List.of("first_line_with_nul overwrite_nul_and_continue_first_line\n"); // WHEN // THEN @@ -303,7 +302,7 @@ public void testStartPositionCurrentTime() throws IOException { runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - final MockFlowFile out = runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0); + final MockFlowFile out = runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst(); out.assertContentEquals("6\n"); } @@ -327,13 +326,13 @@ public void testFileWrittenToAfterRollover() throws IOException, InterruptedExce raf.write("a\nb\n".getBytes()); runner.run(1, false, true); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("a\nb\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("a\nb\n"); runner.clearTransferState(); raf.write("c\n".getBytes()); runner.run(1, false, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("c\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("c\n"); runner.clearTransferState(); // Write additional data to file, then roll file over @@ -352,7 +351,7 @@ public void testFileWrittenToAfterRollover() throws IOException, InterruptedExce // Trigger processor and verify data is consumed properly runner.run(1, false, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("d\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("d\n"); runner.clearTransferState(); // Write to the file and trigger again. @@ -360,7 +359,7 @@ public void testFileWrittenToAfterRollover() throws IOException, InterruptedExce runner.run(1, false, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("e\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("e\n"); runner.clearTransferState(); // Write out some more characters and then write NUL characters. This should result in the processor not consuming the data. @@ -371,7 +370,7 @@ public void testFileWrittenToAfterRollover() throws IOException, InterruptedExce runner.run(1, false, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("f\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("f\n"); runner.clearTransferState(); // Truncate the NUL bytes and replace with additional data, ending with a new line. This should ingest the entire line of text. @@ -380,7 +379,7 @@ public void testFileWrittenToAfterRollover() throws IOException, InterruptedExce runner.run(1, false, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("g\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("g\n"); runner.clearTransferState(); // Ensure that no data comes in for a bit, since the last modified date on the rolled over file isn't old enough. @@ -414,7 +413,7 @@ public void testConsumeAfterTruncationStartAtBeginningOfFile() throws IOExceptio raf.write("hello\n".getBytes()); runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello\n"); runner.clearTransferState(); // roll over the file @@ -433,7 +432,7 @@ public void testConsumeAfterTruncationStartAtBeginningOfFile() throws IOExceptio runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("HELLO\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("HELLO\n"); } @Test @@ -446,7 +445,7 @@ public void testConsumeAfterTruncationStartAtCurrentTime() throws IOException, I raf.write("hello\n".getBytes()); runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello\n"); runner.clearTransferState(); // truncate and then write same number of bytes @@ -462,17 +461,17 @@ public void testConsumeAfterTruncationStartAtCurrentTime() throws IOException, I runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("HELLO\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("HELLO\n"); } @Test - public void testStartAtBeginningOfFile() throws IOException, InterruptedException { + public void testStartAtBeginningOfFile() throws IOException { runner.setProperty(TailFile.START_POSITION, TailFile.START_CURRENT_FILE.getValue()); raf.write("hello world\n".getBytes()); runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello world\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello world\n"); } @Test @@ -486,7 +485,7 @@ public void testStartAtCurrentTime() throws IOException, InterruptedException { } @Test - public void testStartAtBeginningOfTime() throws IOException, InterruptedException { + public void testStartAtBeginningOfTime() throws IOException { raf.write("hello".getBytes()); raf.close(); file.renameTo(new File(file.getParentFile(), file.getName() + ".previous")); @@ -526,7 +525,7 @@ public void testRemainderOfFileRecoveredAfterRestart() throws IOException { raf.write("hello\n".getBytes()); runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello\n"); runner.clearTransferState(); raf.write("world".getBytes()); @@ -552,7 +551,7 @@ public void testRemainderOfFileRecoveredIfRolledOverWhileRunning() throws IOExce raf.write("hello\n".getBytes()); runner.run(1, false, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello\n"); runner.clearTransferState(); raf.write("world".getBytes()); @@ -583,7 +582,7 @@ public void testRolloverAfterHavingReadAllData() throws IOException, Interrupted raf.write("hello\n".getBytes()); runner.run(1, true, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello\n"); runner.clearTransferState(); raf.write("world".getBytes()); @@ -606,7 +605,7 @@ public void testRolloverAfterHavingReadAllData() throws IOException, Interrupted } @Test - public void testRolloverWriteMoreDataThanPrevious() throws IOException, InterruptedException { + public void testRolloverWriteMoreDataThanPrevious() throws IOException { // If we have read all data in a file, and that file does not end with a new-line, then the last line // in the file will have been read, added to the checksum, and then we would re-seek to "unread" that // last line since it didn't have a new-line. We need to ensure that if the data is then rolled over @@ -620,7 +619,7 @@ public void testRolloverWriteMoreDataThanPrevious() throws IOException, Interrup raf.write("hello\n".getBytes()); runner.run(1, true, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello\n"); runner.clearTransferState(); raf.write("world".getBytes()); @@ -651,7 +650,7 @@ public void testMultipleRolloversAfterHavingReadAllData() throws IOException, In raf.write("hello\n".getBytes()); runner.run(1, true, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello\n"); runner.clearTransferState(); raf.write("world".getBytes()); @@ -698,7 +697,7 @@ public void testMultipleRolloversAfterHavingReadAllDataWhileStillRunning() throw raf.write("hello\n".getBytes()); runner.run(1, false, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello\n"); runner.clearTransferState(); raf.write("world".getBytes()); @@ -745,7 +744,7 @@ public void testMultipleRolloversWithLongerFileLength() throws IOException, Inte raf.write("hello\n".getBytes()); runner.run(1, false, false); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("hello\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("hello\n"); runner.clearTransferState(); raf.write("world".getBytes()); @@ -800,7 +799,7 @@ public void testConsumeWhenNewLineFound() throws IOException, InterruptedExcepti assertEquals("target/log.txt", state.getFilename()); assertTrue(state.getTimestamp() <= System.currentTimeMillis()); assertEquals(14, state.getPosition()); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("Hello, World\r\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("Hello, World\r\n"); runner.clearTransferState(); @@ -811,19 +810,19 @@ public void testConsumeWhenNewLineFound() throws IOException, InterruptedExcepti raf.write("\n".getBytes()); runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("12345\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("12345\n"); runner.clearTransferState(); raf.write("carriage\rreturn\r".getBytes()); runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("carriage\r"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("carriage\r"); runner.clearTransferState(); raf.write("\r\n".getBytes()); runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("return\r\r\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("return\r\r\n"); } @Test @@ -875,7 +874,7 @@ private void testMultiLineWaitsForRegexMatch(final boolean shutdownBetweenReads) runner.run(1, shutdownBetweenReads, shutdownBetweenReads); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - final MockFlowFile multiLineOutputFile = runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0); + final MockFlowFile multiLineOutputFile = runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst(); multiLineOutputFile.assertContentEquals("<3>Start of multi-line\n0\n1\n2\n3\n4\n5\n6\n7\n8\n9\n"); runner.clearTransferState(); @@ -887,7 +886,7 @@ private void testMultiLineWaitsForRegexMatch(final boolean shutdownBetweenReads) runner.run(1, shutdownBetweenReads, shutdownBetweenReads); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - final MockFlowFile finalOutputFile = runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0); + final MockFlowFile finalOutputFile = runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst(); finalOutputFile.assertContentEquals("<4>Last One\n"); } @@ -913,7 +912,7 @@ public void testRolloverAndUpdateAtSameTime() throws IOException { // written to log.txt before it rolled, and then we should get some data from the new log.txt. runner.run(1, false, true); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 2); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("another"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("another"); // If we run again, we should get nothing. // We did have an issue where we were recognizing the previously rolled over file again because the timestamps @@ -945,7 +944,7 @@ public void testRolloverWhenNoRollingPattern() throws IOException { // from the tailed file runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("new file\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("new file\n"); runner.clearTransferState(); // in the unlikely case where more data is written after the file is moved @@ -958,7 +957,7 @@ public void testRolloverWhenNoRollingPattern() throws IOException { runner.run(); runner.assertAllFlowFilesTransferred(TailFile.REL_SUCCESS, 1); - runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).get(0).assertContentEquals("with longer data in the new file\n"); + runner.getFlowFilesForRelationship(TailFile.REL_SUCCESS).getFirst().assertContentEquals("with longer data in the new file\n"); runner.clearTransferState(); } @@ -1052,7 +1051,7 @@ public void testMultipleFiles() throws IOException, InterruptedException { } @Test - public void testDetectNewFile() throws IOException, InterruptedException { + public void testDetectNewFile() throws IOException { runner.setProperty(TailFile.BASE_DIRECTORY, "target"); runner.setProperty(TailFile.MODE, TailFile.MODE_MULTIFILE); runner.setProperty(TailFile.LOOKUP_FREQUENCY, "1 sec"); @@ -1115,7 +1114,7 @@ public void testHandleRemovedFile() throws IOException { } @Test - public void testMultipleFilesWithBasedirAndFilenameEL() throws IOException, InterruptedException { + public void testMultipleFilesWithBasedirAndFilenameEL() throws IOException { runner.setEnvironmentVariableValue("vrBaseDirectory", "target"); runner.setProperty(TailFile.BASE_DIRECTORY, "${vrBaseDirectory}"); runner.setProperty(TailFile.MODE, TailFile.MODE_MULTIFILE); @@ -1309,12 +1308,7 @@ private void assertFilenamesInStateMap(Collection expectedFilenames) thr private void cleanFiles(String directory) { final File targetDir = new File(directory); if (targetDir.exists()) { - final File[] files = targetDir.listFiles(new FilenameFilter() { - @Override - public boolean accept(final File dir, final String name) { - return name.startsWith("log") || name.endsWith("log"); - } - }); + final File[] files = targetDir.listFiles((dir, name) -> name.startsWith("log") || name.endsWith("log")); for (final File file : files) { file.delete(); @@ -1339,7 +1333,7 @@ private RandomAccessFile initializeFile(String path, String data) throws IOExcep return randomAccessFile; } - private void deleteFile(String path) throws IOException { + private void deleteFile(String path) { File file = new File(path); assertTrue(file.delete()); } diff --git a/nifi-extension-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/java/org/apache/nifi/http/StandardHttpContextMap.java b/nifi-extension-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/java/org/apache/nifi/http/StandardHttpContextMap.java index edb369be4065..4518f8473c3d 100644 --- a/nifi-extension-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/java/org/apache/nifi/http/StandardHttpContextMap.java +++ b/nifi-extension-bundles/nifi-standard-services/nifi-http-context-map-bundle/nifi-http-context-map/src/main/java/org/apache/nifi/http/StandardHttpContextMap.java @@ -24,7 +24,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import jakarta.servlet.AsyncContext; @@ -86,13 +85,10 @@ protected List getSupportedPropertyDescriptors() { @OnEnabled public void onConfigured(final ConfigurationContext context) { maxSize = context.getProperty(MAX_OUTSTANDING_REQUESTS).asInteger(); - executor = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() { - @Override - public Thread newThread(final Runnable r) { - final Thread thread = Executors.defaultThreadFactory().newThread(r); - thread.setName("StandardHttpContextMap-" + getIdentifier()); - return thread; - } + executor = Executors.newSingleThreadScheduledExecutor(r -> { + final Thread thread = Executors.defaultThreadFactory().newThread(r); + thread.setName("StandardHttpContextMap-" + getIdentifier()); + return thread; }); maxRequestNanos = context.getProperty(REQUEST_EXPIRATION).asTimePeriod(TimeUnit.NANOSECONDS); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java index ab34a2a52d0b..f28f8d7a333a 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/StandardClusterCoordinationProtocolSender.java @@ -285,48 +285,45 @@ public Thread newThread(final Runnable r) { }); for (final NodeIdentifier nodeId : nodesToNotify) { - executor.submit(new Runnable() { - @Override - public void run() { - final int attempts = 5; - final int retrySeconds = 1; - Exception lastException = null; - - for (int i = 0; i < attempts; i++) { - try (final Socket socket = createSocket(nodeId, true)) { - // marshal message to output stream - final OutputStream out = socket.getOutputStream(); - out.write(msgBytes); - } catch (final Exception e) { - if (e instanceof ProtocolException && e.getCause() instanceof ConnectException && nodeId.equals(msg.getNodeId())) { - // We treat Connect Exceptions different because it means we're not able to reach the node at all. If that is the case and the - // node is the affected node, we don't want to retry. This is common when trying to remove a node that has been terminated, etc. - // and retrying is not likely to help; additionally, when the node reconnects to the cluster, it will get the update at that point - // and does not need the update until then. - logger.warn("Failed to send Node Status Change message to {} because unable to connect to node. Will not retry.", nodeId, e); - return; - } - - logger.warn("Failed to send Node Status Change message to {}", nodeId, e); - - lastException = e; - - try { - Thread.sleep(retrySeconds * 1000L); - } catch (final InterruptedException ie) { - Thread.currentThread().interrupt(); - return; - } - - continue; + executor.submit(() -> { + final int attempts = 5; + final int retrySeconds = 1; + Exception lastException = null; + + for (int i = 0; i < attempts; i++) { + try (final Socket socket = createSocket(nodeId, true)) { + // marshal message to output stream + final OutputStream out = socket.getOutputStream(); + out.write(msgBytes); + } catch (final Exception e) { + if (e instanceof ProtocolException && e.getCause() instanceof ConnectException && nodeId.equals(msg.getNodeId())) { + // We treat Connect Exceptions different because it means we're not able to reach the node at all. If that is the case and the + // node is the affected node, we don't want to retry. This is common when trying to remove a node that has been terminated, etc. + // and retrying is not likely to help; additionally, when the node reconnects to the cluster, it will get the update at that point + // and does not need the update until then. + logger.warn("Failed to send Node Status Change message to {} because unable to connect to node. Will not retry.", nodeId, e); + return; } - logger.debug("Notified {} of status change {}", nodeId, msg); - return; + logger.warn("Failed to send Node Status Change message to {}", nodeId, e); + + lastException = e; + + try { + Thread.sleep(retrySeconds * 1000L); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + return; + } + + continue; } - throw new ProtocolException("Failed to send Node Status Change message to " + nodeId, lastException); + logger.debug("Notified {} of status change {}", nodeId, msg); + return; } + + throw new ProtocolException("Failed to send Node Status Change message to " + nodeId, lastException); }); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/io/socket/SocketListener.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/io/socket/SocketListener.java index 61a1e81fa8f8..fb81773c883c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/io/socket/SocketListener.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster-protocol/src/main/java/org/apache/nifi/io/socket/SocketListener.java @@ -97,49 +97,43 @@ public Thread newThread(final Runnable r) { final ExecutorService runnableExecServiceRef = executorService; final ServerSocket runnableServerSocketRef = serverSocket; - final Thread t = new Thread(new Runnable() { - @Override - public void run() { - while (runnableExecServiceRef.isShutdown() == false) { - Socket socket = null; + final Thread t = new Thread(() -> { + while (!runnableExecServiceRef.isShutdown()) { + Socket socket = null; + try { try { - try { - socket = runnableServerSocketRef.accept(); - if (configuration.getSocketTimeout() != null) { - socket.setSoTimeout(configuration.getSocketTimeout()); - } - } catch (final SocketTimeoutException ste) { - // nobody connected to us. Go ahead and call closeQuietly just to make sure we don't leave - // any sockets lingering - SocketUtils.closeQuietly(socket); - continue; - } catch (final SocketException se) { - logger.warn("Failed to communicate with {}", (socket == null ? "Unknown Host" : socket.getInetAddress().getHostName()), se); - SocketUtils.closeQuietly(socket); - continue; - } catch (final Throwable t) { - logger.warn("Socket Listener encountered exception", t); - SocketUtils.closeQuietly(socket); - continue; + socket = runnableServerSocketRef.accept(); + if (configuration.getSocketTimeout() != null) { + socket.setSoTimeout(configuration.getSocketTimeout()); } - - final Socket finalSocket = socket; - runnableExecServiceRef.execute(new Runnable() { - @Override - public void run() { - try { - dispatchRequest(finalSocket); - } catch (final Throwable t) { - logger.warn("Dispatching socket request encountered exception", t); - } finally { - SocketUtils.closeQuietly(finalSocket); - } - } - }); - } catch (final Throwable t) { - logger.error("Socket Listener encountered exception", t); + } catch (final SocketTimeoutException ste) { + // nobody connected to us. Go ahead and call closeQuietly just to make sure we don't leave + // any sockets lingering + SocketUtils.closeQuietly(socket); + continue; + } catch (final SocketException se) { + logger.warn("Failed to communicate with {}", (socket == null ? "Unknown Host" : socket.getInetAddress().getHostName()), se); SocketUtils.closeQuietly(socket); + continue; + } catch (final Throwable t12) { + logger.warn("Socket Listener encountered exception", t12); + SocketUtils.closeQuietly(socket); + continue; } + + final Socket finalSocket = socket; + runnableExecServiceRef.execute(() -> { + try { + dispatchRequest(finalSocket); + } catch (final Throwable t1) { + logger.warn("Dispatching socket request encountered exception", t1); + } finally { + SocketUtils.closeQuietly(finalSocket); + } + }); + } catch (final Throwable t12) { + logger.error("Socket Listener encountered exception", t12); + SocketUtils.closeQuietly(socket); } } }); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java index d154e07e4547..88a5dae601be 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/heartbeat/AbstractHeartbeatMonitor.java @@ -77,15 +77,12 @@ public synchronized final void start() { logger.error("Failed to start Heartbeat Monitor", e); } - this.future = flowEngine.scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - try { - monitorHeartbeats(); - } catch (final Exception e) { - clusterCoordinator.reportEvent(null, Severity.ERROR, "Failed to process heartbeats from nodes due to " + e.toString()); - logger.error("Failed to process heartbeats", e); - } + this.future = flowEngine.scheduleWithFixedDelay(() -> { + try { + monitorHeartbeats(); + } catch (final Exception e) { + clusterCoordinator.reportEvent(null, Severity.ERROR, "Failed to process heartbeats from nodes due to " + e); + logger.error("Failed to process heartbeats", e); } }, heartbeatIntervalMillis, heartbeatIntervalMillis, TimeUnit.MILLISECONDS); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java index d74b5a1a6007..1bbf43d1d61e 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/coordination/node/NodeClusterCoordinator.java @@ -958,38 +958,35 @@ void notifyOthersOfNodeStatusChange(final NodeConnectionStatus updatedStatus, fi private Future offloadAsynchronously(final OffloadMessage request, final int attempts, final int retrySeconds) { final CompletableFuture future = new CompletableFuture<>(); - final Thread offloadThread = new Thread(new Runnable() { - @Override - public void run() { - final NodeIdentifier nodeId = request.getNodeId(); + final Thread offloadThread = new Thread(() -> { + final NodeIdentifier nodeId = request.getNodeId(); - Exception lastException = null; - for (int i = 0; i < attempts; i++) { - try { - senderListener.offload(request); - reportEvent(nodeId, Severity.INFO, "Node was offloaded due to " + request.getExplanation()); + Exception lastException = null; + for (int i = 0; i < attempts; i++) { + try { + senderListener.offload(request); + reportEvent(nodeId, Severity.INFO, "Node was offloaded due to " + request.getExplanation()); + + future.complete(null); + return; + } catch (final Exception e) { + logger.error("Failed to notify {} that it has been offloaded due to {}", request.getNodeId(), request.getExplanation(), e); + lastException = e; - future.complete(null); + try { + Thread.sleep(retrySeconds * 1000L); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); return; - } catch (final Exception e) { - logger.error("Failed to notify {} that it has been offloaded due to {}", request.getNodeId(), request.getExplanation(), e); - lastException = e; - - try { - Thread.sleep(retrySeconds * 1000L); - } catch (final InterruptedException ie) { - Thread.currentThread().interrupt(); - return; - } } } + } - updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.DISCONNECTED, null, - "Attempted to offload node but failed to notify node that it was to offload its data. State reset to disconnected.")); - addNodeEvent(nodeId, "Failed to initiate node offload: " + lastException); + updateNodeStatus(new NodeConnectionStatus(nodeId, NodeConnectionState.DISCONNECTED, null, + "Attempted to offload node but failed to notify node that it was to offload its data. State reset to disconnected.")); + addNodeEvent(nodeId, "Failed to initiate node offload: " + lastException); - future.completeExceptionally(lastException); - } + future.completeExceptionally(lastException); }, "Offload " + request.getNodeId()); offloadThread.start(); @@ -999,47 +996,44 @@ public void run() { private Future disconnectAsynchronously(final DisconnectMessage request, final int attempts, final int retrySeconds) { final CompletableFuture future = new CompletableFuture<>(); - final Thread disconnectThread = new Thread(new Runnable() { - @Override - public void run() { - final NodeIdentifier nodeId = request.getNodeId(); - - Exception lastException = null; - for (int i = 0; i < attempts; i++) { - // If the node is restarted, it will attempt to reconnect. In that case, we don't want to disconnect the node - // again. So we instead log the fact that the state has now transitioned to this point and consider the task completed. - final NodeConnectionState currentConnectionState = getConnectionState(nodeId); - if (currentConnectionState == NodeConnectionState.CONNECTING || currentConnectionState == NodeConnectionState.CONNECTED) { - reportEvent(nodeId, Severity.INFO, String.format( - "State of Node %s has now transitioned from DISCONNECTED to %s so will no longer attempt to notify node that it is disconnected.", nodeId, currentConnectionState)); - future.completeExceptionally(new IllegalStateException("Node was marked as disconnected but its state transitioned from DISCONNECTED back to " + currentConnectionState + - " before the node could be notified. This typically indicates that the node was restarted.")); + final Thread disconnectThread = new Thread(() -> { + final NodeIdentifier nodeId = request.getNodeId(); + + Exception lastException = null; + for (int i = 0; i < attempts; i++) { + // If the node is restarted, it will attempt to reconnect. In that case, we don't want to disconnect the node + // again. So we instead log the fact that the state has now transitioned to this point and consider the task completed. + final NodeConnectionState currentConnectionState = getConnectionState(nodeId); + if (currentConnectionState == NodeConnectionState.CONNECTING || currentConnectionState == NodeConnectionState.CONNECTED) { + reportEvent(nodeId, Severity.INFO, String.format( + "State of Node %s has now transitioned from DISCONNECTED to %s so will no longer attempt to notify node that it is disconnected.", nodeId, currentConnectionState)); + future.completeExceptionally(new IllegalStateException("Node was marked as disconnected but its state transitioned from DISCONNECTED back to " + currentConnectionState + + " before the node could be notified. This typically indicates that the node was restarted.")); + + return; + } - return; - } + // Try to send disconnect notice to the node + try { + senderListener.disconnect(request); + reportEvent(nodeId, Severity.INFO, "Node disconnected due to " + request.getExplanation()); + future.complete(null); + return; + } catch (final Exception e) { + logger.error("Failed to notify {} that it has been disconnected from the cluster due to {}", request.getNodeId(), request.getExplanation()); + lastException = e; - // Try to send disconnect notice to the node try { - senderListener.disconnect(request); - reportEvent(nodeId, Severity.INFO, "Node disconnected due to " + request.getExplanation()); - future.complete(null); + Thread.sleep(retrySeconds * 1000L); + } catch (final InterruptedException ie) { + future.completeExceptionally(ie); + Thread.currentThread().interrupt(); return; - } catch (final Exception e) { - logger.error("Failed to notify {} that it has been disconnected from the cluster due to {}", request.getNodeId(), request.getExplanation()); - lastException = e; - - try { - Thread.sleep(retrySeconds * 1000L); - } catch (final InterruptedException ie) { - future.completeExceptionally(ie); - Thread.currentThread().interrupt(); - return; - } } } - - future.completeExceptionally(lastException); } + + future.completeExceptionally(lastException); }, "Disconnect " + request.getNodeId()); disconnectThread.start(); @@ -1063,62 +1057,59 @@ public void validateHeartbeat(final NodeHeartbeat heartbeat) { } private void requestReconnectionAsynchronously(final ReconnectionRequestMessage request, final int reconnectionAttempts, final int retrySeconds, final boolean includeDataFlow) { - final Thread reconnectionThread = new Thread(new Runnable() { - @Override - public void run() { - // create the request - while (flowService == null) { - try { - Thread.sleep(100L); - } catch (final InterruptedException ie) { - logger.info("Could not send Reconnection request to {} because thread was " - + "interrupted before FlowService was made available", request.getNodeId()); - Thread.currentThread().interrupt(); - return; - } + final Thread reconnectionThread = new Thread(() -> { + // create the request + while (flowService == null) { + try { + Thread.sleep(100L); + } catch (final InterruptedException ie) { + logger.info("Could not send Reconnection request to {} because thread was " + + "interrupted before FlowService was made available", request.getNodeId()); + Thread.currentThread().interrupt(); + return; } + } - for (int i = 0; i < reconnectionAttempts; i++) { - try { - if (NodeConnectionState.CONNECTING != getConnectionState(request.getNodeId())) { - // the node status has changed. It's no longer appropriate to attempt reconnection. - return; - } + for (int i = 0; i < reconnectionAttempts; i++) { + try { + if (NodeConnectionState.CONNECTING != getConnectionState(request.getNodeId())) { + // the node status has changed. It's no longer appropriate to attempt reconnection. + return; + } - if (includeDataFlow) { - request.setDataFlow(new StandardDataFlow(flowService.createDataFlowFromController())); - } + if (includeDataFlow) { + request.setDataFlow(new StandardDataFlow(flowService.createDataFlowFromController())); + } - request.setNodeConnectionStatuses(getConnectionStatuses()); - final ComponentRevisionSnapshot componentRevisionSnapshot = ComponentRevisionSnapshot.fromRevisionSnapshot(revisionManager.getAllRevisions()); - request.setComponentRevisions(componentRevisionSnapshot); + request.setNodeConnectionStatuses(getConnectionStatuses()); + final ComponentRevisionSnapshot componentRevisionSnapshot = ComponentRevisionSnapshot.fromRevisionSnapshot(revisionManager.getAllRevisions()); + request.setComponentRevisions(componentRevisionSnapshot); - // Issue a reconnection request to the node. - senderListener.requestReconnection(request); + // Issue a reconnection request to the node. + senderListener.requestReconnection(request); - // successfully told node to reconnect -- we're done! - logger.info("Successfully requested that {} join the cluster", request.getNodeId()); + // successfully told node to reconnect -- we're done! + logger.info("Successfully requested that {} join the cluster", request.getNodeId()); - return; - } catch (final Exception e) { - logger.warn("Problem encountered issuing reconnection request to node {}", request.getNodeId(), e); - eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, "Problem encountered issuing reconnection request to node " - + request.getNodeId() + " due to: " + e); - } - - try { - Thread.sleep(1000L * retrySeconds); - } catch (final InterruptedException ie) { - break; - } + return; + } catch (final Exception e) { + logger.warn("Problem encountered issuing reconnection request to node {}", request.getNodeId(), e); + eventReporter.reportEvent(Severity.WARNING, EVENT_CATEGORY, "Problem encountered issuing reconnection request to node " + + request.getNodeId() + " due to: " + e); } - // We failed to reconnect too many times. We must now mark node as disconnected. - if (NodeConnectionState.CONNECTING == getConnectionState(request.getNodeId())) { - requestNodeDisconnect(request.getNodeId(), DisconnectionCode.UNABLE_TO_COMMUNICATE, - "Attempted to request that node reconnect to cluster but could not communicate with node"); + try { + Thread.sleep(1000L * retrySeconds); + } catch (final InterruptedException ie) { + break; } } + + // We failed to reconnect too many times. We must now mark node as disconnected. + if (NodeConnectionState.CONNECTING == getConnectionState(request.getNodeId())) { + requestNodeDisconnect(request.getNodeId(), DisconnectionCode.UNABLE_TO_COMMUNICATE, + "Attempted to request that node reconnect to cluster but could not communicate with node"); + } }, "Reconnect " + request.getNodeId()); reconnectionThread.start(); @@ -1405,7 +1396,7 @@ public void afterRequest(final String uriPath, final String method, final Set failedNodeIds = problematicNodeResponses.stream().map(response -> response.getNodeId()).collect(Collectors.toSet()); + final Set failedNodeIds = problematicNodeResponses.stream().map(NodeResponse::getNodeId).collect(Collectors.toSet()); logger.warn("The following nodes failed to process URI {} '{}'. Requesting each node reconnect to cluster.", uriPath, failedNodeIds); for (final NodeIdentifier nodeId : failedNodeIds) { // Update the node to 'CONNECTING' status and request that the node connect diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java index d0ce10d6f564..878d1b501423 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java @@ -25,14 +25,12 @@ import jakarta.ws.rs.HttpMethod; import jakarta.ws.rs.ProcessingException; -import jakarta.ws.rs.WebApplicationException; import jakarta.ws.rs.core.Response; import jakarta.ws.rs.core.Response.ResponseBuilder; import jakarta.ws.rs.core.Response.Status; import jakarta.ws.rs.core.StreamingOutput; -import java.io.IOException; + import java.io.InputStream; -import java.io.OutputStream; import java.net.URI; import java.util.List; import java.util.Objects; @@ -255,12 +253,7 @@ private Response createResponse() { if (!HttpMethod.HEAD.equalsIgnoreCase(httpMethod)) { // set the entity if (updatedEntity == null) { - responseBuilder.entity(new StreamingOutput() { - @Override - public void write(final OutputStream output) throws IOException, WebApplicationException { - IOUtils.copy(getInputStream(), output); - } - }); + responseBuilder.entity((StreamingOutput) output -> IOUtils.copy(getInputStream(), output)); } else { responseBuilder.entity(updatedEntity); } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java index 3de9b11ea85e..51f55fd1a6ce 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/StandardProcessorNode.java @@ -1755,16 +1755,13 @@ private void initiateStart(final ScheduledExecutorService taskScheduler, final l // to do this, we use #scheduleWithFixedDelay and then make that Future available to the task // itself by placing it into an AtomicReference. final AtomicReference> futureRef = new AtomicReference<>(); - final Runnable monitoringTask = new Runnable() { - @Override - public void run() { - Future monitoringFuture = futureRef.get(); - if (monitoringFuture == null) { // Future is not yet available. Just return and wait for the next invocation. - return; - } - - monitorAsyncTask(taskFuture, monitoringFuture, completionTimestampRef.get()); + final Runnable monitoringTask = () -> { + Future monitoringFuture = futureRef.get(); + if (monitoringFuture == null) { // Future is not yet available. Just return and wait for the next invocation. + return; } + + monitorAsyncTask(taskFuture, monitoringFuture, completionTimestampRef.get()); }; final Future future = taskScheduler.scheduleWithFixedDelay(monitoringTask, 1, 10, TimeUnit.MILLISECONDS); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java index d21cd2586b71..2bee7fe62b83 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceNode.java @@ -729,19 +729,16 @@ public CompletableFuture disable(ScheduledExecutorService scheduler) { if (this.stateTransition.transitionToDisabling(ControllerServiceState.ENABLED, future)) { final ConfigurationContext configContext = new StandardConfigurationContext(this, this.serviceProvider, null); - scheduler.execute(new Runnable() { - @Override - public void run() { - try { - invokeDisable(configContext); - } finally { - stateTransition.disable(); - - // Now all components that reference this service will be invalid. Trigger validation to occur so that - // this is reflected in any response that may go back to a user/client. - for (final ComponentNode component : getReferences().getReferencingComponents()) { - component.performValidation(); - } + scheduler.execute(() -> { + try { + invokeDisable(configContext); + } finally { + stateTransition.disable(); + + // Now all components that reference this service will be invalid. Trigger validation to occur so that + // this is reflected in any response that may go back to a user/client. + for (final ComponentNode component : getReferences().getReferencingComponents()) { + component.performValidation(); } } }); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java index 2c2ad21744cc..f99969e37108 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java @@ -1049,12 +1049,7 @@ public void removeRemoteProcessGroup(final RemoteProcessGroup remoteProcessGroup remoteGroup.getInputPorts().forEach(scheduler::onPortRemoved); remoteGroup.getOutputPorts().forEach(scheduler::onPortRemoved); - scheduler.submitFrameworkTask(new Runnable() { - @Override - public void run() { - stateManagerProvider.onComponentRemoved(remoteGroup.getIdentifier()); - } - }); + scheduler.submitFrameworkTask(() -> stateManagerProvider.onComponentRemoved(remoteGroup.getIdentifier())); remoteGroups.remove(remoteGroupId); LOG.info("{} removed from flow", remoteProcessGroup); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java index cbe56ebd0664..7de7dd839f43 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FileSystemSwapManager.java @@ -51,7 +51,6 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.FileOutputStream; -import java.io.FilenameFilter; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -208,12 +207,7 @@ public SwapContents peek(final String swapLocation, final FlowFileQueue flowFile @Override public void purge() { - final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() { - @Override - public boolean accept(final File dir, final String name) { - return SWAP_FILE_PATTERN.matcher(name).matches() || TEMP_SWAP_FILE_PATTERN.matcher(name).matches(); - } - }); + final File[] swapFiles = storageDirectory.listFiles((dir, name) -> SWAP_FILE_PATTERN.matcher(name).matches() || TEMP_SWAP_FILE_PATTERN.matcher(name).matches()); for (final File file : swapFiles) { if (!file.delete()) { @@ -255,12 +249,7 @@ private String getOwnerPartition(final File swapFile) { @Override public Set getSwappedPartitionNames(final FlowFileQueue queue) { - final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() { - @Override - public boolean accept(final File dir, final String name) { - return SWAP_FILE_PATTERN.matcher(name).matches(); - } - }); + final File[] swapFiles = storageDirectory.listFiles((dir, name) -> SWAP_FILE_PATTERN.matcher(name).matches()); if (swapFiles == null) { return Collections.emptySet(); @@ -277,12 +266,7 @@ public boolean accept(final File dir, final String name) { @Override public List recoverSwapLocations(final FlowFileQueue flowFileQueue, final String partitionName) throws IOException { - final File[] swapFiles = storageDirectory.listFiles(new FilenameFilter() { - @Override - public boolean accept(final File dir, final String name) { - return SWAP_FILE_PATTERN.matcher(name).matches() || TEMP_SWAP_FILE_PATTERN.matcher(name).matches(); - } - }); + final File[] swapFiles = storageDirectory.listFiles((dir, name) -> SWAP_FILE_PATTERN.matcher(name).matches() || TEMP_SWAP_FILE_PATTERN.matcher(name).matches()); if (swapFiles == null) { return Collections.emptyList(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index ee50266061bd..6310e01afbb7 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -73,7 +73,6 @@ import org.apache.nifi.controller.leader.election.LeaderElectionStateChangeListener; import org.apache.nifi.controller.queue.FlowFileQueue; import org.apache.nifi.controller.queue.FlowFileQueueFactory; -import org.apache.nifi.controller.queue.LoadBalanceStrategy; import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.controller.queue.StandardFlowFileQueue; import org.apache.nifi.controller.queue.clustered.ContentRepositoryFlowFileAccess; @@ -722,22 +721,19 @@ private FlowController( analyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, statusHistoryRepository, statusAnalyticsModelMapFactory, predictionIntervalMillis, queryIntervalMillis, modelScoreName, modelScoreThreshold); - timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - try { - Long startTs = System.currentTimeMillis(); - RepositoryStatusReport statusReport = flowFileEventRepository.reportTransferEvents(startTs); - flowManager.findAllConnections().forEach(connection -> { - ConnectionStatusAnalytics connectionStatusAnalytics = ((ConnectionStatusAnalytics) analyticsEngine.getStatusAnalytics(connection.getIdentifier())); - connectionStatusAnalytics.refresh(); - connectionStatusAnalytics.loadPredictions(statusReport); - }); - Long endTs = System.currentTimeMillis(); - LOG.debug("Time Elapsed for Prediction for loading all predictions: {}", endTs - startTs); - } catch (final Exception e) { - LOG.error("Failed to generate predictions", e); - } + timerDrivenEngineRef.get().scheduleWithFixedDelay(() -> { + try { + Long startTs = System.currentTimeMillis(); + RepositoryStatusReport statusReport = flowFileEventRepository.reportTransferEvents(startTs); + flowManager.findAllConnections().forEach(connection -> { + ConnectionStatusAnalytics connectionStatusAnalytics = ((ConnectionStatusAnalytics) analyticsEngine.getStatusAnalytics(connection.getIdentifier())); + connectionStatusAnalytics.refresh(); + connectionStatusAnalytics.loadPredictions(statusReport); + }); + Long endTs = System.currentTimeMillis(); + LOG.debug("Time Elapsed for Prediction for loading all predictions: {}", endTs - startTs); + } catch (final Exception e) { + LOG.error("Failed to generate predictions", e); } }, 0L, 15, TimeUnit.SECONDS); @@ -746,14 +742,11 @@ public void run() { eventAccess = new StandardEventAccess(flowManager, flowFileEventRepository, processScheduler, authorizer, provenanceRepository, auditService, analyticsEngine, flowFileRepository, contentRepository); - timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - try { - statusHistoryRepository.capture(getNodeStatusSnapshot(), eventAccess.getControllerStatus(), getGarbageCollectionStatus(), new Date()); - } catch (final Exception e) { - LOG.error("Failed to capture component stats for Stats History", e); - } + timerDrivenEngineRef.get().scheduleWithFixedDelay(() -> { + try { + statusHistoryRepository.capture(getNodeStatusSnapshot(), eventAccess.getControllerStatus(), getGarbageCollectionStatus(), new Date()); + } catch (final Exception e) { + LOG.error("Failed to capture component stats for Stats History", e); } }, snapshotMillis, snapshotMillis, TimeUnit.MILLISECONDS); @@ -1066,30 +1059,24 @@ public void initializeFlow(final QueueProvider queueProvider) throws IOException notifyComponentsConfigurationRestored(); - timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - try { - updateRemoteProcessGroups(); - } catch (final Throwable t) { - LOG.warn("Unable to update Remote Process Groups", t); - } + timerDrivenEngineRef.get().scheduleWithFixedDelay(() -> { + try { + updateRemoteProcessGroups(); + } catch (final Throwable t) { + LOG.warn("Unable to update Remote Process Groups", t); } }, 0L, 30L, TimeUnit.SECONDS); - timerDrivenEngineRef.get().scheduleWithFixedDelay(new Runnable() { - @Override - public void run() { - final ProcessGroup rootGroup = flowManager.getRootGroup(); - final List allGroups = rootGroup.findAllProcessGroups(); - allGroups.add(rootGroup); - - for (final ProcessGroup group : allGroups) { - try { - group.synchronizeWithFlowRegistry(flowManager); - } catch (final Exception e) { - LOG.error("Failed to synchronize {} with Flow Registry", group, e); - } + timerDrivenEngineRef.get().scheduleWithFixedDelay(() -> { + final ProcessGroup rootGroup = flowManager.getRootGroup(); + final List allGroups = rootGroup.findAllProcessGroups(); + allGroups.add(rootGroup); + + for (final ProcessGroup group : allGroups) { + try { + group.synchronizeWithFlowRegistry(flowManager); + } catch (final Exception e) { + LOG.error("Failed to synchronize {} with Flow Registry", group, e); } } }, 5, 60, TimeUnit.SECONDS); @@ -2327,26 +2314,23 @@ public EventReporter getEventReporter() { swapManager.initialize(initializationContext); } - final FlowFileQueueFactory flowFileQueueFactory = new FlowFileQueueFactory() { - @Override - public FlowFileQueue createFlowFileQueue(final LoadBalanceStrategy loadBalanceStrategy, final String partitioningAttribute, final ProcessGroup processGroup) { - final FlowFileQueue flowFileQueue; - - if (clusterCoordinator == null) { - flowFileQueue = new StandardFlowFileQueue(id, flowFileRepository, provenanceRepository, processScheduler, swapManager, - eventReporter, nifiProperties.getQueueSwapThreshold(), - processGroup.getDefaultFlowFileExpiration(), processGroup.getDefaultBackPressureObjectThreshold(), processGroup.getDefaultBackPressureDataSizeThreshold()); - } else { - flowFileQueue = new SocketLoadBalancedFlowFileQueue(id, processScheduler, flowFileRepository, provenanceRepository, contentRepository, - clusterCoordinator, loadBalanceClientRegistry, swapManager, nifiProperties.getQueueSwapThreshold(), eventReporter); + final FlowFileQueueFactory flowFileQueueFactory = (loadBalanceStrategy, partitioningAttribute, processGroup) -> { + final FlowFileQueue flowFileQueue; - flowFileQueue.setFlowFileExpiration(processGroup.getDefaultFlowFileExpiration()); - flowFileQueue.setBackPressureObjectThreshold(processGroup.getDefaultBackPressureObjectThreshold()); - flowFileQueue.setBackPressureDataSizeThreshold(processGroup.getDefaultBackPressureDataSizeThreshold()); - } + if (clusterCoordinator == null) { + flowFileQueue = new StandardFlowFileQueue(id, flowFileRepository, provenanceRepository, processScheduler, swapManager, + eventReporter, nifiProperties.getQueueSwapThreshold(), + processGroup.getDefaultFlowFileExpiration(), processGroup.getDefaultBackPressureObjectThreshold(), processGroup.getDefaultBackPressureDataSizeThreshold()); + } else { + flowFileQueue = new SocketLoadBalancedFlowFileQueue(id, processScheduler, flowFileRepository, provenanceRepository, contentRepository, + clusterCoordinator, loadBalanceClientRegistry, swapManager, nifiProperties.getQueueSwapThreshold(), eventReporter); - return flowFileQueue; + flowFileQueue.setFlowFileExpiration(processGroup.getDefaultFlowFileExpiration()); + flowFileQueue.setBackPressureObjectThreshold(processGroup.getDefaultBackPressureObjectThreshold()); + flowFileQueue.setBackPressureDataSizeThreshold(processGroup.getDefaultBackPressureDataSizeThreshold()); } + + return flowFileQueue; }; final Connection connection = builder.id(requireNonNull(id).intern()) @@ -3442,8 +3426,8 @@ private NodeStatus getNodeStatusSnapshot() { result.setTimerDrivenThreads(getActiveTimerDrivenThreadCount()); result.setFlowFileRepositoryFreeSpace(systemDiagnostics.getFlowFileRepositoryStorageUsage().getFreeSpace()); result.setFlowFileRepositoryUsedSpace(systemDiagnostics.getFlowFileRepositoryStorageUsage().getUsedSpace()); - result.setContentRepositories(systemDiagnostics.getContentRepositoryStorageUsage().entrySet().stream().map(e -> getStorageStatus(e)).collect(Collectors.toList())); - result.setProvenanceRepositories(systemDiagnostics.getProvenanceRepositoryStorageUsage().entrySet().stream().map(e -> getStorageStatus(e)).collect(Collectors.toList())); + result.setContentRepositories(systemDiagnostics.getContentRepositoryStorageUsage().entrySet().stream().map(FlowController::getStorageStatus).collect(Collectors.toList())); + result.setProvenanceRepositories(systemDiagnostics.getProvenanceRepositoryStorageUsage().entrySet().stream().map(FlowController::getStorageStatus).collect(Collectors.toList())); return result; } diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java index 9ef6554eab1a..bdbdb0519c73 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowService.java @@ -388,26 +388,18 @@ public ProtocolMessage handle(final ProtocolMessage request, final Set n // may still be held, causing this node to take a long time to respond to requests. controller.suspendHeartbeats(); - final Thread t = new Thread(new Runnable() { - @Override - public void run() { - handleReconnectionRequest((ReconnectionRequestMessage) request); - } - }, "Reconnect to Cluster"); + final Thread t = new Thread(() -> handleReconnectionRequest((ReconnectionRequestMessage) request), "Reconnect to Cluster"); t.setDaemon(true); t.start(); return new ReconnectionResponseMessage(); } case OFFLOAD_REQUEST: { - final Thread t = new Thread(new Runnable() { - @Override - public void run() { - try { - handleOffloadRequest((OffloadMessage) request); - } catch (InterruptedException e) { - throw new ProtocolException("Could not complete offload request", e); - } + final Thread t = new Thread(() -> { + try { + handleOffloadRequest((OffloadMessage) request); + } catch (InterruptedException e) { + throw new ProtocolException("Could not complete offload request", e); } }, "Offload Flow Files from Node"); t.setDaemon(true); @@ -416,12 +408,7 @@ public void run() { return null; } case DISCONNECTION_REQUEST: { - final Thread t = new Thread(new Runnable() { - @Override - public void run() { - handleDisconnectionRequest((DisconnectMessage) request); - } - }, "Disconnect from Cluster"); + final Thread t = new Thread(() -> handleDisconnectionRequest((DisconnectMessage) request), "Disconnect from Cluster"); t.setDaemon(true); t.start(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java index f27f2b2d6405..a33cd99a4737 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/queue/AbstractFlowFileQueue.java @@ -289,12 +289,7 @@ public DropFlowFileStatus dropFlowFiles(final String requestIdentifier, final St return dropRequest; } - final Thread t = new Thread(new Runnable() { - @Override - public void run() { - dropFlowFiles(dropRequest, requestor); - } - }, "Drop FlowFiles for Connection " + getIdentifier()); + final Thread t = new Thread(() -> dropFlowFiles(dropRequest, requestor), "Drop FlowFiles for Connection " + getIdentifier()); t.setDaemon(true); t.start(); diff --git a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/test/processors/DataGeneratorTestProcessor.java b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/test/processors/DataGeneratorTestProcessor.java index d3bf26216bbe..74aca738d01c 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/test/processors/DataGeneratorTestProcessor.java +++ b/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/test/processors/DataGeneratorTestProcessor.java @@ -16,8 +16,6 @@ */ package org.apache.nifi.test.processors; -import java.io.IOException; -import java.io.OutputStream; import java.util.Collections; import java.util.HashSet; import java.util.Random; @@ -29,7 +27,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.io.OutputStreamCallback; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,12 +63,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final byte[] data = new byte[4096]; random.nextBytes(data); - flowFile = session.write(flowFile, new OutputStreamCallback() { - @Override - public void process(final OutputStream out) throws IOException { - out.write(data); - } - }); + flowFile = session.write(flowFile, out -> out.write(data)); LOG.info("{} transferring {} to success", this, flowFile); session.transfer(flowFile, REL_SUCCESS); diff --git a/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java b/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java index 8a0ef868740a..4c0b31eff226 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java +++ b/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/HttpRemoteSiteListener.java @@ -31,7 +31,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import static org.apache.nifi.util.NiFiProperties.DEFAULT_SITE_TO_SITE_HTTP_TRANSACTION_TTL; @@ -50,14 +49,11 @@ public class HttpRemoteSiteListener implements RemoteSiteListener { private HttpRemoteSiteListener(final NiFiProperties nifiProperties) { super(); - taskExecutor = Executors.newScheduledThreadPool(1, new ThreadFactory() { - @Override - public Thread newThread(final Runnable r) { - final Thread thread = Executors.defaultThreadFactory().newThread(r); - thread.setName("Http Site-to-Site Transaction Maintenance"); - thread.setDaemon(true); - return thread; - } + taskExecutor = Executors.newScheduledThreadPool(1, r -> { + final Thread thread = Executors.defaultThreadFactory().newThread(r); + thread.setName("Http Site-to-Site Transaction Maintenance"); + thread.setDaemon(true); + return thread; }); int txTtlSec; diff --git a/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java b/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java index 0f37df96e29f..db0f6cbb883f 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java +++ b/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRemoteGroupPort.java @@ -34,7 +34,6 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.remote.client.SiteToSiteClient; import org.apache.nifi.remote.client.SiteToSiteClientConfig; import org.apache.nifi.remote.exception.PortNotRunningException; @@ -56,7 +55,6 @@ import javax.net.ssl.SSLContext; import java.io.IOException; -import java.io.InputStream; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -318,12 +316,9 @@ private int transferFlowFiles(final Transaction transaction, final ProcessContex final long startNanos = System.nanoTime(); // call codec.encode within a session callback so that we have the InputStream to read the FlowFile final FlowFile toWrap = flowFile; - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - final DataPacket dataPacket = new StandardDataPacket(toWrap.getAttributes(), in, toWrap.getSize()); - transaction.send(dataPacket); - } + session.read(flowFile, in -> { + final DataPacket dataPacket = new StandardDataPacket(toWrap.getAttributes(), in, toWrap.getSize()); + transaction.send(dataPacket); }); final long transferNanos = System.nanoTime() - startNanos; @@ -377,7 +372,7 @@ public void process(final InputStream in) throws IOException { } - private int receiveFlowFiles(final Transaction transaction, final ProcessContext context, final ProcessSession session) throws IOException, ProtocolException { + private int receiveFlowFiles(final Transaction transaction, final ProcessContext context, final ProcessSession session) throws IOException { final String userDn = transaction.getCommunicant().getDistinguishedName(); final StopWatch stopWatch = new StopWatch(true); diff --git a/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/AbstractFlowFileServerProtocol.java b/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/AbstractFlowFileServerProtocol.java index 4b62569b897b..bfe7a2671885 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/AbstractFlowFileServerProtocol.java +++ b/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/protocol/AbstractFlowFileServerProtocol.java @@ -25,7 +25,6 @@ import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; -import org.apache.nifi.processor.io.InputStreamCallback; import org.apache.nifi.remote.Peer; import org.apache.nifi.remote.PortAuthorizationResult; import org.apache.nifi.remote.PublicPort; @@ -254,12 +253,9 @@ public int transferFlowFiles(final Peer peer, final ProcessContext context, fina final StopWatch transferWatch = new StopWatch(true); final FlowFile toSend = flowFile; - session.read(flowFile, new InputStreamCallback() { - @Override - public void process(final InputStream in) throws IOException { - final DataPacket dataPacket = new StandardDataPacket(toSend.getAttributes(), in, toSend.getSize()); - codec.encode(dataPacket, checkedOutputStream); - } + session.read(flowFile, in -> { + final DataPacket dataPacket = new StandardDataPacket(toSend.getAttributes(), in, toSend.getSize()); + codec.encode(dataPacket, checkedOutputStream); }); final long transmissionMillis = transferWatch.getElapsed(TimeUnit.MILLISECONDS); diff --git a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsyncRequestManager.java b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsyncRequestManager.java index 5ac56ce4d8cd..d318b2df74f7 100644 --- a/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsyncRequestManager.java +++ b/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/concurrent/AsyncRequestManager.java @@ -108,22 +108,19 @@ public void submitRequest(final String type, final String requestId, final Async logger.debug("Submitted request {}", key); - threadPool.submit(new Runnable() { - @Override - public void run() { - try { - // set the user authentication token - final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(request.getUser())); - SecurityContextHolder.getContext().setAuthentication(authentication); - - task.accept(request); - } catch (final Exception e) { - logger.error("Failed to perform asynchronous task", e); - request.fail("Encountered unexpected error when performing asynchronous task: " + e); - } finally { - // clear the authentication token - SecurityContextHolder.getContext().setAuthentication(null); - } + threadPool.submit(() -> { + try { + // set the user authentication token + final Authentication authentication = new NiFiAuthenticationToken(new NiFiUserDetails(request.getUser())); + SecurityContextHolder.getContext().setAuthentication(authentication); + + task.accept(request); + } catch (final Exception e) { + logger.error("Failed to perform asynchronous task", e); + request.fail("Encountered unexpected error when performing asynchronous task: " + e); + } finally { + // clear the authentication token + SecurityContextHolder.getContext().setAuthentication(null); } }); } diff --git a/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java b/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java index ca45b9750375..20d3e6950c94 100644 --- a/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java +++ b/nifi-mock/src/test/java/org/apache/nifi/util/TestStandardProcessorTestRunner.java @@ -31,7 +31,6 @@ import org.apache.nifi.reporting.InitializationException; import org.junit.jupiter.api.Test; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -125,12 +124,7 @@ public void testFlowFileValidator() { runner.assertTransferCount(AddAttributeProcessor.REL_SUCCESS, 3); runner.assertTransferCount(AddAttributeProcessor.REL_FAILURE, 2); runner.assertAllFlowFilesContainAttribute(AddAttributeProcessor.REL_SUCCESS, AddAttributeProcessor.KEY); - runner.assertAllFlowFiles(AddAttributeProcessor.REL_SUCCESS, new FlowFileValidator() { - @Override - public void assertFlowFile(FlowFile f) { - assertEquals("value", f.getAttribute(AddAttributeProcessor.KEY)); - } - }); + runner.assertAllFlowFiles(AddAttributeProcessor.REL_SUCCESS, f -> assertEquals("value", f.getAttribute(AddAttributeProcessor.KEY))); } @Test @@ -139,9 +133,7 @@ public void testFailFlowFileValidator() { final TestRunner runner = TestRunners.newTestRunner(proc); runner.run(5, true); - assertThrows(AssertionError.class, () -> { - runner.assertAllFlowFiles(f -> assertEquals("value", f.getAttribute(AddAttributeProcessor.KEY))); - }); + assertThrows(AssertionError.class, () -> runner.assertAllFlowFiles(f -> assertEquals("value", f.getAttribute(AddAttributeProcessor.KEY)))); } @Test @@ -339,7 +331,7 @@ private static class SimpleTestService extends AbstractControllerService { private boolean opmCalled = false; protected List getSupportedPropertyDescriptors() { - return Arrays.asList(namePropertyDescriptor); + return List.of(namePropertyDescriptor); } public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { @@ -363,7 +355,7 @@ private static class RequiredPropertyTestService extends AbstractControllerServi .build(); protected List getSupportedPropertyDescriptors() { - return Arrays.asList(namePropertyDescriptor); + return List.of(namePropertyDescriptor); } } @@ -376,7 +368,7 @@ public void testErrorLogMessageArguments() { logger.error("expected test error", t); final List log = logger.getErrorMessages(); - final LogMessage msg = log.get(0); + final LogMessage msg = log.getFirst(); assertTrue(msg.getMsg().contains("expected test error")); assertNotNull(msg.getThrowable()); diff --git a/nifi-registry/nifi-registry-core/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/NiFiRegistryListener.java b/nifi-registry/nifi-registry-core/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/NiFiRegistryListener.java index f2ead2e4ec21..dfc2603791ca 100644 --- a/nifi-registry/nifi-registry-core/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/NiFiRegistryListener.java +++ b/nifi-registry/nifi-registry-core/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/NiFiRegistryListener.java @@ -25,7 +25,6 @@ import java.net.Socket; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; public class NiFiRegistryListener { @@ -64,14 +63,11 @@ private class Listener implements Runnable { public Listener(final ServerSocket serverSocket, final RunNiFiRegistry runner) { this.serverSocket = serverSocket; - this.executor = Executors.newFixedThreadPool(2, new ThreadFactory() { - @Override - public Thread newThread(final Runnable runnable) { - final Thread t = Executors.defaultThreadFactory().newThread(runnable); - t.setDaemon(true); - t.setName("NiFi Registry Bootstrap Command Listener"); - return t; - } + this.executor = Executors.newFixedThreadPool(2, runnable -> { + final Thread t = Executors.defaultThreadFactory().newThread(runnable); + t.setDaemon(true); + t.setName("NiFi Registry Bootstrap Command Listener"); + return t; }); this.runner = runner; @@ -108,26 +104,23 @@ public void run() { throw ioe; } - executor.submit(new Runnable() { - @Override - public void run() { + executor.submit(() -> { + try { + // we want to ensure that we don't try to read data from an InputStream directly + // by a BufferedReader because any user on the system could open a socket and send + // a multi-gigabyte file without any new lines in order to crash the Bootstrap, + // which in turn may cause the Shutdown Hook to shutdown NiFi. + // So we will limit the amount of data to read to 4 KB + final InputStream limitingIn = new LimitingInputStream(socket.getInputStream(), 4096); + final BootstrapCodec codec = new BootstrapCodec(runner, limitingIn, socket.getOutputStream()); + codec.communicate(); + } catch (final Throwable t) { + System.out.println("Failed to communicate with NiFi Registry due to " + t); + t.printStackTrace(); + } finally { try { - // we want to ensure that we don't try to read data from an InputStream directly - // by a BufferedReader because any user on the system could open a socket and send - // a multi-gigabyte file without any new lines in order to crash the Bootstrap, - // which in turn may cause the Shutdown Hook to shutdown NiFi. - // So we will limit the amount of data to read to 4 KB - final InputStream limitingIn = new LimitingInputStream(socket.getInputStream(), 4096); - final BootstrapCodec codec = new BootstrapCodec(runner, limitingIn, socket.getOutputStream()); - codec.communicate(); - } catch (final Throwable t) { - System.out.println("Failed to communicate with NiFi Registry due to " + t); - t.printStackTrace(); - } finally { - try { - socket.close(); - } catch (final IOException ioe) { - } + socket.close(); + } catch (final IOException ignored) { } } }); diff --git a/nifi-registry/nifi-registry-core/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/RunNiFiRegistry.java b/nifi-registry/nifi-registry-core/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/RunNiFiRegistry.java index dd9454b2f1f0..54662eb0af79 100644 --- a/nifi-registry/nifi-registry-core/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/RunNiFiRegistry.java +++ b/nifi-registry/nifi-registry-core/nifi-registry-bootstrap/src/main/java/org/apache/nifi/registry/bootstrap/RunNiFiRegistry.java @@ -26,7 +26,6 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.FileOutputStream; -import java.io.FilenameFilter; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -50,7 +49,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; -import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; @@ -121,14 +119,11 @@ public class RunNiFiRegistry { public RunNiFiRegistry(final File bootstrapConfigFile, final boolean verbose) { this.bootstrapConfigFile = bootstrapConfigFile; - loggingExecutor = Executors.newFixedThreadPool(2, new ThreadFactory() { - @Override - public Thread newThread(final Runnable runnable) { - final Thread t = Executors.defaultThreadFactory().newThread(runnable); - t.setDaemon(true); - t.setName("NiFi logging handler"); - return t; - } + loggingExecutor = Executors.newFixedThreadPool(2, runnable -> { + final Thread t = Executors.defaultThreadFactory().newThread(runnable); + t.setDaemon(true); + t.setName("NiFi logging handler"); + return t; }); } @@ -858,23 +853,13 @@ public void start() throws IOException { } } - final File[] libSharedFiles = libSharedDir.listFiles(new FilenameFilter() { - @Override - public boolean accept(final File dir, final String filename) { - return filename.toLowerCase().endsWith(".jar"); - } - }); + final File[] libSharedFiles = libSharedDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar")); if (libSharedFiles == null || libSharedFiles.length == 0) { throw new RuntimeException("Could not find lib shared directory at " + libSharedDir.getAbsolutePath()); } - final File[] libFiles = libDir.listFiles(new FilenameFilter() { - @Override - public boolean accept(final File dir, final String filename) { - return filename.toLowerCase().endsWith(".jar"); - } - }); + final File[] libFiles = libDir.listFiles((dir, filename) -> filename.toLowerCase().endsWith(".jar")); if (libFiles == null || libFiles.length == 0) { throw new RuntimeException("Could not find lib directory at " + libDir.getAbsolutePath()); @@ -1050,35 +1035,29 @@ private void handleLogging(final Process process) { } } - final Future stdOutFuture = loggingExecutor.submit(new Runnable() { - @Override - public void run() { - final Logger stdOutLogger = LoggerFactory.getLogger("org.apache.nifi.registry.StdOut"); - final InputStream in = process.getInputStream(); - try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) { - String line; - while ((line = reader.readLine()) != null) { - stdOutLogger.info(line); - } - } catch (IOException e) { - defaultLogger.error("Failed to read from NiFi Registry's Standard Out stream", e); + final Future stdOutFuture = loggingExecutor.submit(() -> { + final Logger stdOutLogger = LoggerFactory.getLogger("org.apache.nifi.registry.StdOut"); + final InputStream in = process.getInputStream(); + try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) { + String line; + while ((line = reader.readLine()) != null) { + stdOutLogger.info(line); } + } catch (IOException e) { + defaultLogger.error("Failed to read from NiFi Registry's Standard Out stream", e); } }); - final Future stdErrFuture = loggingExecutor.submit(new Runnable() { - @Override - public void run() { - final Logger stdErrLogger = LoggerFactory.getLogger("org.apache.nifi.registry.StdErr"); - final InputStream in = process.getErrorStream(); - try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) { - String line; - while ((line = reader.readLine()) != null) { - stdErrLogger.error(line); - } - } catch (IOException e) { - defaultLogger.error("Failed to read from NiFi Registry's Standard Error stream", e); + final Future stdErrFuture = loggingExecutor.submit(() -> { + final Logger stdErrLogger = LoggerFactory.getLogger("org.apache.nifi.registry.StdErr"); + final InputStream in = process.getErrorStream(); + try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) { + String line; + while ((line = reader.readLine()) != null) { + stdErrLogger.error(line); } + } catch (IOException e) { + defaultLogger.error("Failed to read from NiFi Registry's Standard Error stream", e); } }); diff --git a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessSchedulingAgent.java b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessSchedulingAgent.java index 235be1da1d31..31b8d10544d2 100644 --- a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessSchedulingAgent.java +++ b/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StatelessSchedulingAgent.java @@ -60,19 +60,16 @@ public void onEvent(final Connectable connectable) { @Override public void schedule(final ReportingTaskNode taskNode, final LifecycleState scheduleState) { final long schedulingMillis = taskNode.getSchedulingPeriod(TimeUnit.MILLISECONDS); - final Thread thread = new Thread(new Runnable() { - @Override - public void run() { - while (true) { - try { - Thread.sleep(schedulingMillis); - } catch (final InterruptedException e) { - logger.info("Interrupted while waiting to trigger {}. Will no longer trigger Reporting Task to run", taskNode); - return; - } - - triggerReportingTask(taskNode, scheduleState); + final Thread thread = new Thread(() -> { + while (true) { + try { + Thread.sleep(schedulingMillis); + } catch (final InterruptedException e) { + logger.info("Interrupted while waiting to trigger {}. Will no longer trigger Reporting Task to run", taskNode); + return; } + + triggerReportingTask(taskNode, scheduleState); } }); diff --git a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/performance/StatelessPerformanceIT.java b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/performance/StatelessPerformanceIT.java index b62fef3cffb9..b5c1cd1e1d53 100644 --- a/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/performance/StatelessPerformanceIT.java +++ b/nifi-system-tests/nifi-stateless-system-test-suite/src/test/java/org/apache/nifi/stateless/performance/StatelessPerformanceIT.java @@ -47,25 +47,22 @@ public void testCreateDestroyPerf() throws InterruptedException { final List threads = new ArrayList<>(); final int numThreads = 6; for (int threadIndex = 0; threadIndex < numThreads; threadIndex++) { - final Thread t = new Thread(new Runnable() { - @Override - public void run() { - try { - final StatelessDataflow dataflow = loadDataflow(builder.getFlowSnapshot()); + final Thread t = new Thread(() -> { + try { + final StatelessDataflow dataflow = loadDataflow(builder.getFlowSnapshot()); - final int iterations = 5_000_000; - final long start = System.currentTimeMillis(); - for (int i = 0; i < iterations; i++) { - final DataflowTrigger trigger = dataflow.trigger(); - final TriggerResult result = trigger.getResult(); + final int iterations = 5_000_000; + final long start = System.currentTimeMillis(); + for (int i = 0; i < iterations; i++) { + final DataflowTrigger trigger = dataflow.trigger(); + final TriggerResult result = trigger.getResult(); - result.acknowledge(); - } - final long millis = System.currentTimeMillis() - start; - logger.info("Took {} millis to run {} iterations", millis, iterations); - } catch (final Exception e) { - e.printStackTrace(); + result.acknowledge(); } + final long millis = System.currentTimeMillis() - start; + logger.info("Took {} millis to run {} iterations", millis, iterations); + } catch (final Exception e) { + e.printStackTrace(); } }); diff --git a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/ReOrderFlowFiles.java b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/ReOrderFlowFiles.java index 3f671149fee3..dfddb96648b2 100644 --- a/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/ReOrderFlowFiles.java +++ b/nifi-system-tests/nifi-system-test-extensions-bundle/nifi-system-test-extensions/src/main/java/org/apache/nifi/processors/tests/system/ReOrderFlowFiles.java @@ -67,20 +67,12 @@ protected List getSupportedPropertyDescriptors() { public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { final PropertyValue selectionValue = context.getProperty(FIRST_SELECTION_CRITERIA); - final List matching = session.get(new FlowFileFilter() { - @Override - public FlowFileFilterResult filter(final FlowFile flowFile) { - final boolean selected = selectionValue.evaluateAttributeExpressions(flowFile).asBoolean(); - return selected ? FlowFileFilterResult.ACCEPT_AND_CONTINUE : FlowFileFilterResult.REJECT_AND_CONTINUE; - } + final List matching = session.get(flowFile -> { + final boolean selected = selectionValue.evaluateAttributeExpressions(flowFile).asBoolean(); + return selected ? FlowFileFilter.FlowFileFilterResult.ACCEPT_AND_CONTINUE : FlowFileFilter.FlowFileFilterResult.REJECT_AND_CONTINUE; }); - final List unmatched = session.get(new FlowFileFilter() { - @Override - public FlowFileFilterResult filter(final FlowFile flowFile) { - return FlowFileFilterResult.ACCEPT_AND_CONTINUE; - } - }); + final List unmatched = session.get(flowFile -> FlowFileFilter.FlowFileFilterResult.ACCEPT_AND_CONTINUE); session.transfer(matching, REL_SUCCESS); session.transfer(unmatched, REL_SUCCESS);