You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2015/01/16 17:57:46 UTC
[1/8] incubator-nifi git commit: NIFI-273: Moved
getAvailableRelationships from ProcessSession to ProcessContext
Repository: incubator-nifi
Updated Branches:
refs/heads/develop db2360998 -> 73384b23d
NIFI-273: Moved getAvailableRelationships from ProcessSession to ProcessContext
Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/3a7b8de0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/3a7b8de0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/3a7b8de0
Branch: refs/heads/develop
Commit: 3a7b8de0e30e7d4dac877611d0aa9bf67bfadffd
Parents: 94a06fc
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Jan 16 11:27:22 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Jan 16 11:27:22 2015 -0500
----------------------------------------------------------------------
.../apache/nifi/controller/StandardFunnel.java | 4 +--
.../repository/BatchingSessionFactory.java | 5 ----
.../repository/StandardProcessSession.java | 5 ----
.../scheduling/ConnectableProcessContext.java | 21 +++++++++++++++
.../nifi/processor/StandardProcessContext.java | 28 ++++++++++++++++++++
.../processor/StandardSchedulingContext.java | 6 +++++
.../protocol/socket/SocketClientProtocol.java | 2 +-
.../socket/SocketFlowFileServerProtocol.java | 2 +-
.../processors/standard/DistributeLoad.java | 16 +++++------
.../nifi/processors/standard/ListenHTTP.java | 2 ++
.../nifi/processors/standard/ListenUDP.java | 2 +-
.../processors/standard/PutFileTransfer.java | 2 +-
.../standard/servlets/ListenHTTPServlet.java | 7 ++++-
.../processors/standard/TestDistributeLoad.java | 1 +
.../apache/nifi/processor/ProcessContext.java | 8 ++++++
.../apache/nifi/processor/ProcessSession.java | 6 -----
.../apache/nifi/util/MockProcessContext.java | 23 ++++++++++++++++
.../apache/nifi/util/MockProcessSession.java | 4 ---
.../apache/nifi/util/SharedSessionState.java | 21 +--------------
.../nifi/util/StandardProcessorTestRunner.java | 8 +++---
20 files changed, 114 insertions(+), 59 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
index e516f20..e34e043 100644
--- a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
+++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
@@ -348,7 +348,7 @@ public class StandardFunnel implements Funnel {
private void onTrigger(final ProcessContext context, final ProcessSession session) {
readLock.lock();
try {
- Set<Relationship> available = session.getAvailableRelationships();
+ Set<Relationship> available = context.getAvailableRelationships();
int transferred = 0;
while (!available.isEmpty()) {
final List<FlowFile> flowFiles = session.get(10);
@@ -359,7 +359,7 @@ public class StandardFunnel implements Funnel {
transferred += flowFiles.size();
session.transfer(flowFiles, Relationship.ANONYMOUS);
session.commit();
- available = session.getAvailableRelationships();
+ available = context.getAvailableRelationships();
}
if (transferred == 0) {
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
index eae2550..d5dba82 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
@@ -98,11 +98,6 @@ public class BatchingSessionFactory implements ProcessSessionFactory {
}
@Override
- public Set<Relationship> getAvailableRelationships() {
- return session.getAvailableRelationships();
- }
-
- @Override
public FlowFile create() {
return session.create();
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index 3d3e854..dcb461c 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@ -1237,11 +1237,6 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
}
@Override
- public Set<Relationship> getAvailableRelationships() {
- return context.getAvailableRelationships();
- }
-
- @Override
public FlowFile create() {
final Map<String, String> attrs = new HashMap<>();
attrs.put(CoreAttributes.FILENAME.key(), String.valueOf(System.nanoTime()));
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
index 8c60e4b..acb3a01 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
@@ -16,13 +16,18 @@
*/
package org.apache.nifi.controller.scheduling;
+import java.util.Collection;
+import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Map;
+import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.encrypt.StringEncryptor;
@@ -30,6 +35,7 @@ import org.apache.nifi.expression.AttributeValueDecorator;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
/**
@@ -170,4 +176,19 @@ public class ConnectableProcessContext implements ProcessContext {
public ControllerServiceLookup getControllerServiceLookup() {
return null;
}
+
+ @Override
+ public Set<Relationship> getAvailableRelationships() {
+ for ( final Connection connection : connectable.getConnections() ) {
+ if ( connection.getFlowFileQueue().isFull() ) {
+ return Collections.emptySet();
+ }
+ }
+
+ final Collection<Relationship> relationships = connectable.getRelationships();
+ if ( relationships instanceof Set ) {
+ return (Set<Relationship>) relationships;
+ }
+ return new HashSet<>(connectable.getRelationships());
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
index 93a8c6b..cd0d31c 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
@@ -16,7 +16,9 @@
*/
package org.apache.nifi.processor;
+import java.util.Collection;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@@ -24,6 +26,7 @@ import org.apache.nifi.attribute.expression.language.PreparedQuery;
import org.apache.nifi.attribute.expression.language.Query;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
import org.apache.nifi.controller.ProcessorNode;
@@ -142,4 +145,29 @@ public class StandardProcessContext implements ProcessContext, ControllerService
public ControllerServiceLookup getControllerServiceLookup() {
return this;
}
+
+ @Override
+ public Set<Relationship> getAvailableRelationships() {
+ final Set<Relationship> set = new HashSet<>();
+ for (final Relationship relationship : procNode.getRelationships()) {
+ final Collection<Connection> connections = procNode.getConnections(relationship);
+ if (connections.isEmpty()) {
+ set.add(relationship);
+ } else {
+ boolean available = true;
+ for (final Connection connection : connections) {
+ if (connection.getFlowFileQueue().isFull()) {
+ available = false;
+ }
+ }
+
+ if (available) {
+ set.add(relationship);
+ }
+ }
+ }
+
+ return set;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
index 0fe08c9..318901f 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
@@ -17,6 +17,7 @@
package org.apache.nifi.processor;
import java.util.Map;
+import java.util.Set;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
@@ -104,4 +105,9 @@ public class StandardSchedulingContext implements SchedulingContext {
public ControllerServiceLookup getControllerServiceLookup() {
return processContext.getControllerServiceLookup();
}
+
+ @Override
+ public Set<Relationship> getAvailableRelationships() {
+ return processContext.getAvailableRelationships();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java b/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
index 22ec983..d4b4f61 100644
--- a/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
+++ b/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
@@ -327,7 +327,7 @@ public class SocketClientProtocol implements ClientProtocol {
// Commit the session so that we have persisted the data
session.commit();
- if ( session.getAvailableRelationships().isEmpty() ) {
+ if ( context.getAvailableRelationships().isEmpty() ) {
// Confirm that we received the data and the peer can now discard it but that the peer should not
// send any more data for a bit
logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java b/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
index 88b6a41..5edd4f9 100644
--- a/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
+++ b/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
@@ -482,7 +482,7 @@ public class SocketFlowFileServerProtocol implements ServerProtocol {
// Commit the session so that we have persisted the data
session.commit();
- if ( session.getAvailableRelationships().isEmpty() ) {
+ if ( context.getAvailableRelationships().isEmpty() ) {
// Confirm that we received the data and the peer can now discard it but that the peer should not
// send any more data for a bit
logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
index a755b1a..3ac55d2 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
@@ -354,14 +354,14 @@ public class DistributeLoad extends AbstractProcessor {
}
final DistributionStrategy strategy = strategyRef.get();
- final Set<Relationship> available = session.getAvailableRelationships();
+ final Set<Relationship> available = context.getAvailableRelationships();
final int numRelationships = context.getProperty(NUM_RELATIONSHIPS).asInteger();
final boolean allDestinationsAvailable = (available.size() == numRelationships);
if (!allDestinationsAvailable && strategy.requiresAllDestinationsAvailable()) {
return;
}
- final Relationship relationship = strategy.mapToRelationship(session, flowFile);
+ final Relationship relationship = strategy.mapToRelationship(context, flowFile);
if (relationship == null) {
// can't transfer the FlowFiles. Roll back and yield
session.rollback();
@@ -403,7 +403,7 @@ public class DistributeLoad extends AbstractProcessor {
* @param flowFiles
* @return
*/
- Relationship mapToRelationship(ProcessSession session, FlowFile flowFile);
+ Relationship mapToRelationship(ProcessContext context, FlowFile flowFile);
boolean requiresAllDestinationsAvailable();
}
@@ -413,7 +413,7 @@ public class DistributeLoad extends AbstractProcessor {
private final AtomicLong counter = new AtomicLong(0L);
@Override
- public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
+ public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
final int numRelationships = relationshipList.size();
@@ -427,7 +427,7 @@ public class DistributeLoad extends AbstractProcessor {
final long counterValue = counter.getAndIncrement();
final int idx = (int) (counterValue % numRelationships);
relationship = relationshipList.get(idx);
- foundFreeRelationship = session.getAvailableRelationships().contains(relationship);
+ foundFreeRelationship = context.getAvailableRelationships().contains(relationship);
if (++attempts % numRelationships == 0 && !foundFreeRelationship) {
return null;
}
@@ -448,7 +448,7 @@ public class DistributeLoad extends AbstractProcessor {
private final AtomicLong counter = new AtomicLong(0L);
@Override
- public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
+ public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
final long counterValue = counter.getAndIncrement();
final int idx = (int) (counterValue % relationshipList.size());
@@ -467,7 +467,7 @@ public class DistributeLoad extends AbstractProcessor {
private final AtomicLong counter = new AtomicLong(0L);
@Override
- public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
+ public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
final int numRelationships = relationshipList.size();
@@ -481,7 +481,7 @@ public class DistributeLoad extends AbstractProcessor {
final long counterValue = counter.getAndIncrement();
final int idx = (int) (counterValue % numRelationships);
relationship = relationshipList.get(idx);
- foundFreeRelationship = session.getAvailableRelationships().contains(relationship);
+ foundFreeRelationship = context.getAvailableRelationships().contains(relationship);
if (++attempts % numRelationships == 0 && !foundFreeRelationship) {
return null;
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
index 2b0b437..b7fe97a 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
@@ -117,6 +117,7 @@ public class ListenHTTP extends AbstractSessionFactoryProcessor {
public static final String CONTEXT_ATTRIBUTE_PROCESSOR = "processor";
public static final String CONTEXT_ATTRIBUTE_LOGGER = "logger";
public static final String CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER = "sessionFactoryHolder";
+ public static final String CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER = "processContextHolder";
public static final String CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN = "authorityPattern";
public static final String CONTEXT_ATTRIBUTE_HEADER_PATTERN = "headerPattern";
public static final String CONTEXT_ATTRIBUTE_FLOWFILE_MAP = "flowFileMap";
@@ -240,6 +241,7 @@ public class ListenHTTP extends AbstractSessionFactoryProcessor {
contextHandler.setAttribute(CONTEXT_ATTRIBUTE_PROCESSOR, this);
contextHandler.setAttribute(CONTEXT_ATTRIBUTE_LOGGER, getLogger());
contextHandler.setAttribute(CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER, sessionFactoryReference);
+ contextHandler.setAttribute(CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER, context);
contextHandler.setAttribute(CONTEXT_ATTRIBUTE_FLOWFILE_MAP, flowFileMap);
contextHandler.setAttribute(CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN, Pattern.compile(context.getProperty(AUTHORIZED_DN_PATTERN).getValue()));
contextHandler.setAttribute(CONTEXT_ATTRIBUTE_STREAM_THROTTLER, streamThrottler);
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
index 65b3c66..43d8395 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
@@ -377,7 +377,7 @@ public class ListenUDP extends AbstractSessionFactoryProcessor {
// this will throttle the processing of the received datagrams. If there are no more
// buffers to read into because none have been returned to the pool via consumer.process(),
// then the desired back pressure on the channel is created.
- if (session.getAvailableRelationships().size() > 0) {
+ if (context.getAvailableRelationships().size() > 0) {
consumer.process();
if (flowFileCount == newFlowFiles.size()) {
// no new datagrams received, need to throttle this thread back so it does
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
index 31e5105..da80546 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
@@ -148,7 +148,7 @@ public abstract class PutFileTransfer<T extends FileTransfer> extends AbstractPr
session.transfer(flowFile, conflictResult.getRelationship());
session.commit();
- } while (isScheduled() && (getRelationships().size() == session.getAvailableRelationships().size()) && (++fileCount < maxNumberOfFiles) && ((flowFile = session.get()) != null));
+ } while (isScheduled() && (getRelationships().size() == context.getAvailableRelationships().size()) && (++fileCount < maxNumberOfFiles) && ((flowFile = session.get()) != null));
} catch (final IOException e) {
context.yield();
logger.error("Unable to transfer {} to remote host {} due to {}", new Object[]{flowFile, hostname, e});
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
index cae61f0..1cf5f1f 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
@@ -48,6 +48,7 @@ import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.stream.io.StreamThrottler;
import org.apache.nifi.logging.ProcessorLog;
+import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.io.OutputStreamCallback;
@@ -87,6 +88,7 @@ public class ListenHTTPServlet extends HttpServlet {
private ProcessorLog logger;
private AtomicReference<ProcessSessionFactory> sessionFactoryHolder;
+ private volatile ProcessContext processContext;
private Pattern authorizedPattern;
private Pattern headerPattern;
private ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap;
@@ -103,6 +105,7 @@ public class ListenHTTPServlet extends HttpServlet {
final ServletContext context = config.getServletContext();
this.logger = (ProcessorLog) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_LOGGER);
this.sessionFactoryHolder = (AtomicReference<ProcessSessionFactory>) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER);
+ this.processContext = (ProcessContext) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER);
this.authorizedPattern = (Pattern) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN);
this.headerPattern = (Pattern) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_HEADER_PATTERN);
this.flowFileMap = (ConcurrentMap<String, FlowFileEntryTimeWrapper>) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_FLOWFILE_MAP);
@@ -118,6 +121,8 @@ public class ListenHTTPServlet extends HttpServlet {
@Override
protected void doPost(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
+ final ProcessContext context = processContext;
+
ProcessSessionFactory sessionFactory;
do {
sessionFactory = sessionFactoryHolder.get();
@@ -136,7 +141,7 @@ public class ListenHTTPServlet extends HttpServlet {
try {
final long n = filesReceived.getAndIncrement() % FILES_BEFORE_CHECKING_DESTINATION_SPACE;
if (n == 0 || !spaceAvailable.get()) {
- if (session.getAvailableRelationships().isEmpty()) {
+ if (context.getAvailableRelationships().isEmpty()) {
spaceAvailable.set(false);
if (logger.isDebugEnabled()) {
logger.debug("Received request from " + request.getRemoteHost() + " but no space available; Indicating Service Unavailable");
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java b/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
index a6402e4..ab4c978 100644
--- a/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
+++ b/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
@@ -132,6 +132,7 @@ public class TestDistributeLoad {
testRunner.assertQueueEmpty();
for (int i = 1; i <= 100; i++) {
+ System.out.println(i);
testRunner.assertTransferCount(String.valueOf(i), (i == 50) ? 0 : 1);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
index 9e04439..7fa183f 100644
--- a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
+++ b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
@@ -17,6 +17,7 @@
package org.apache.nifi.processor;
import java.util.Map;
+import java.util.Set;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.PropertyValue;
@@ -121,4 +122,11 @@ public interface ProcessContext {
* @return
*/
ControllerServiceLookup getControllerServiceLookup();
+
+ /**
+ * @return the set of all relationships for which space is available to
+ * receive new objects
+ */
+ Set<Relationship> getAvailableRelationships();
+
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
index 09d1bd2..d3de916 100644
--- a/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
+++ b/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
@@ -166,12 +166,6 @@ public interface ProcessSession {
QueueSize getQueueSize();
/**
- * @return the set of all relationships for which space is available to
- * receive new objects
- */
- Set<Relationship> getAvailableRelationships();
-
- /**
* Creates a new FlowFile in the repository with no content and without any
* linkage to a parent FlowFile. This method is appropriate only when data
* is received or created from an external system. Otherwise, this method
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
index 6e5f65d..15591d7 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
@@ -21,10 +21,12 @@ import static java.util.Objects.requireNonNull;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
+import java.util.Set;
import org.apache.nifi.components.ConfigurableComponent;
import org.apache.nifi.components.PropertyDescriptor;
@@ -32,6 +34,8 @@ import org.apache.nifi.components.PropertyValue;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.controller.ControllerService;
import org.apache.nifi.controller.ControllerServiceLookup;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SchedulingContext;
import org.junit.Assert;
@@ -45,6 +49,8 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
private boolean enableExpressionValidation = false;
private boolean allowExpressionValidation = true;
+ private volatile Set<Relationship> unavailableRelationships = new HashSet<>();
+
/**
* Creates a new MockProcessContext for the given Processor
*
@@ -258,4 +264,21 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
public void leaseControllerService(final String identifier) {
}
+ public Set<Relationship> getAvailableRelationships() {
+ if ( !(component instanceof Processor) ) {
+ return Collections.emptySet();
+ }
+
+ final Set<Relationship> relationships = new HashSet<>(((Processor) component).getRelationships());
+ relationships.removeAll(unavailableRelationships);
+ return relationships;
+ }
+
+ public void setUnavailableRelationships(final Set<Relationship> relationships) {
+ this.unavailableRelationships = Collections.unmodifiableSet(new HashSet<>(relationships));
+ }
+
+ public Set<Relationship> getUnavailableRelationships() {
+ return unavailableRelationships;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
index 552780c..ea55b34 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
@@ -329,10 +329,6 @@ public class MockProcessSession implements ProcessSession {
return newFlowFile;
}
- @Override
- public Set<Relationship> getAvailableRelationships() {
- return sharedState.getAvailableRelationships();
- }
@Override
public MockFlowFile merge(final Collection<FlowFile> sources, final FlowFile destination) {
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java b/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
index 96bef71..13a87de 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
@@ -16,49 +16,30 @@
*/
package org.apache.nifi.util;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
import org.apache.nifi.provenance.ProvenanceReporter;
public class SharedSessionState {
private final MockFlowFileQueue flowFileQueue;
private final ProvenanceReporter provenanceReporter;
+ @SuppressWarnings("unused")
private final Processor processor;
private final AtomicLong flowFileIdGenerator;
private final ConcurrentMap<String, AtomicLong> counterMap = new ConcurrentHashMap<>();
- private volatile Set<Relationship> unavailableRelationships;
public SharedSessionState(final Processor processor, final AtomicLong flowFileIdGenerator) {
flowFileQueue = new MockFlowFileQueue();
provenanceReporter = new MockProvenanceReporter();
- unavailableRelationships = new HashSet<>();
this.flowFileIdGenerator = flowFileIdGenerator;
this.processor = processor;
}
- public Set<Relationship> getAvailableRelationships() {
- final Set<Relationship> relationships = new HashSet<>(processor.getRelationships());
- relationships.removeAll(unavailableRelationships);
- return relationships;
- }
-
- public void setUnavailableRelationships(final Set<Relationship> relationships) {
- this.unavailableRelationships = Collections.unmodifiableSet(new HashSet<>(relationships));
- }
-
- public Set<Relationship> getUnavailableRelationships() {
- return unavailableRelationships;
- }
-
public MockFlowFileQueue getFlowFileQueue() {
return flowFileQueue;
}
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/3a7b8de0/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
index 54b611d..40d5035 100644
--- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
+++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
@@ -426,9 +426,9 @@ public class StandardProcessorTestRunner implements TestRunner {
@Override
public void setRelationshipAvailable(final Relationship relationship) {
- final Set<Relationship> unavailable = new HashSet<>(sharedState.getUnavailableRelationships());
+ final Set<Relationship> unavailable = new HashSet<>(context.getUnavailableRelationships());
unavailable.remove(relationship);
- sharedState.setUnavailableRelationships(unavailable);
+ context.setUnavailableRelationships(unavailable);
}
@Override
@@ -438,9 +438,9 @@ public class StandardProcessorTestRunner implements TestRunner {
@Override
public void setRelationshipUnavailable(final Relationship relationship) {
- final Set<Relationship> unavailable = new HashSet<>(sharedState.getUnavailableRelationships());
+ final Set<Relationship> unavailable = new HashSet<>(context.getUnavailableRelationships());
unavailable.add(relationship);
- sharedState.setUnavailableRelationships(unavailable);
+ context.setUnavailableRelationships(unavailable);
}
@Override
[5/8] incubator-nifi git commit: Merge branch 'develop' of
https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
index 0000000,8c60e4b..acb3a01
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
@@@ -1,0 -1,173 +1,194 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.controller.scheduling;
+
++import java.util.Collection;
++import java.util.Collections;
+ import java.util.HashMap;
++import java.util.HashSet;
+ import java.util.Map;
++import java.util.Set;
+ import java.util.concurrent.TimeUnit;
+
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.connectable.Connectable;
++import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.controller.ControllerService;
+ import org.apache.nifi.controller.ControllerServiceLookup;
+ import org.apache.nifi.encrypt.StringEncryptor;
+ import org.apache.nifi.expression.AttributeValueDecorator;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.processor.DataUnit;
+ import org.apache.nifi.processor.ProcessContext;
++import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.exception.ProcessException;
+
+ /**
+ * This class is essentially an empty shell for {@link Connectable}s that are
+ * not Processors
+ */
+ public class ConnectableProcessContext implements ProcessContext {
+
+ private final Connectable connectable;
+ private final StringEncryptor encryptor;
+
+ public ConnectableProcessContext(final Connectable connectable, final StringEncryptor encryptor) {
+ this.connectable = connectable;
+ this.encryptor = encryptor;
+ }
+
+ @Override
+ public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+ return getProperty(descriptor.getName());
+ }
+
+ @Override
+ public PropertyValue getProperty(final String propertyName) {
+ return new PropertyValue() {
+ @Override
+ public String getValue() {
+ return null;
+ }
+
+ @Override
+ public Integer asInteger() {
+ return null;
+ }
+
+ @Override
+ public Long asLong() {
+ return null;
+ }
+
+ @Override
+ public Boolean asBoolean() {
+ return null;
+ }
+
+ @Override
+ public Float asFloat() {
+ return null;
+ }
+
+ @Override
+ public Double asDouble() {
+ return null;
+ }
+
+ @Override
+ public Long asTimePeriod(final TimeUnit timeUnit) {
+ return null;
+ }
+
+ @Override
+ public Double asDataSize(final DataUnit dataUnit) {
+ return null;
+ }
+
+ @Override
+ public PropertyValue evaluateAttributeExpressions() throws ProcessException {
+ return this;
+ }
+
+ @Override
+ public PropertyValue evaluateAttributeExpressions(final FlowFile flowFile) throws ProcessException {
+ return this;
+ }
+
+ @Override
+ public PropertyValue evaluateAttributeExpressions(final AttributeValueDecorator decorator) throws ProcessException {
+ return this;
+ }
+
+ @Override
+ public PropertyValue evaluateAttributeExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException {
+ return this;
+ }
+
+ @Override
+ public ControllerService asControllerService() {
+ return null;
+ }
+
+ @Override
+ public <T extends ControllerService> T asControllerService(Class<T> serviceType) throws IllegalArgumentException {
+ return null;
+ }
+
+ @Override
+ public boolean isSet() {
+ return false;
+ }
+ };
+ }
+
+ @Override
+ public PropertyValue newPropertyValue(String rawValue) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void yield() {
+ connectable.yield();
+ }
+
+ @Override
+ public int getMaxConcurrentTasks() {
+ return connectable.getMaxConcurrentTasks();
+ }
+
+ @Override
+ public String getAnnotationData() {
+ return null;
+ }
+
+ @Override
+ public Map<PropertyDescriptor, String> getProperties() {
+ return new HashMap<>();
+ }
+
+ @Override
+ public String decrypt(String encrypted) {
+ return encryptor.decrypt(encrypted);
+ }
+
+ @Override
+ public String encrypt(String unencrypted) {
+ return encryptor.encrypt(unencrypted);
+ }
+
+ @Override
+ public ControllerServiceLookup getControllerServiceLookup() {
+ return null;
+ }
++
++ @Override
++ public Set<Relationship> getAvailableRelationships() {
++ for ( final Connection connection : connectable.getConnections() ) {
++ if ( connection.getFlowFileQueue().isFull() ) {
++ return Collections.emptySet();
++ }
++ }
++
++ final Collection<Relationship> relationships = connectable.getRelationships();
++ if ( relationships instanceof Set ) {
++ return (Set<Relationship>) relationships;
++ }
++ return new HashSet<>(connectable.getRelationships());
++ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
index 0000000,93a8c6b..cd0d31c
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
@@@ -1,0 -1,145 +1,173 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processor;
+
++import java.util.Collection;
+ import java.util.HashMap;
++import java.util.HashSet;
+ import java.util.Map;
+ import java.util.Set;
+
+ import org.apache.nifi.attribute.expression.language.PreparedQuery;
+ import org.apache.nifi.attribute.expression.language.Query;
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
++import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.controller.ControllerService;
+ import org.apache.nifi.controller.ControllerServiceLookup;
+ import org.apache.nifi.controller.ProcessorNode;
+ import org.apache.nifi.controller.service.ControllerServiceProvider;
+ import org.apache.nifi.encrypt.StringEncryptor;
+
+ public class StandardProcessContext implements ProcessContext, ControllerServiceLookup {
+
+ private final ProcessorNode procNode;
+ private final ControllerServiceProvider controllerServiceProvider;
+ private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
+ private final StringEncryptor encryptor;
+
+ public StandardProcessContext(final ProcessorNode processorNode, final ControllerServiceProvider controllerServiceProvider, final StringEncryptor encryptor) {
+ this.procNode = processorNode;
+ this.controllerServiceProvider = controllerServiceProvider;
+ this.encryptor = encryptor;
+
+ preparedQueries = new HashMap<>();
+ for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getProperties().entrySet()) {
+ final PropertyDescriptor desc = entry.getKey();
+ String value = entry.getValue();
+ if (value == null) {
+ value = desc.getDefaultValue();
+ }
+
+ final PreparedQuery pq = Query.prepare(value);
+ preparedQueries.put(desc, pq);
+ }
+ }
+
+ @Override
+ public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+ return getProperty(descriptor.getName());
+ }
+
+ /**
+ * <p>
+ * Returns the currently configured value for the property with the given
+ * name.
+ * </p>
+ */
+ @Override
+ public PropertyValue getProperty(final String propertyName) {
+ final Processor processor = procNode.getProcessor();
+ final PropertyDescriptor descriptor = processor.getPropertyDescriptor(propertyName);
+ if (descriptor == null) {
+ return null;
+ }
+
+ final String setPropertyValue = procNode.getProperty(descriptor);
+ final String propValue = (setPropertyValue == null) ? descriptor.getDefaultValue() : setPropertyValue;
+
+ return new StandardPropertyValue(propValue, this, preparedQueries.get(descriptor));
+ }
+
+ @Override
+ public PropertyValue newPropertyValue(final String rawValue) {
+ return new StandardPropertyValue(rawValue, this, Query.prepare(rawValue));
+ }
+
+ @Override
+ public void yield() {
+ procNode.yield();
+ }
+
+ @Override
+ public ControllerService getControllerService(final String serviceIdentifier) {
+ return controllerServiceProvider.getControllerService(serviceIdentifier);
+ }
+
+ @Override
+ public int getMaxConcurrentTasks() {
+ return procNode.getMaxConcurrentTasks();
+ }
+
+ @Override
+ public String getAnnotationData() {
+ return procNode.getAnnotationData();
+ }
+
+ @Override
+ public Map<PropertyDescriptor, String> getProperties() {
+ return procNode.getProperties();
+ }
+
+ @Override
+ public String encrypt(final String unencrypted) {
+ return encryptor.encrypt(unencrypted);
+ }
+
+ @Override
+ public String decrypt(final String encrypted) {
+ return encryptor.decrypt(encrypted);
+ }
+
+ @Override
+ public Set<String> getControllerServiceIdentifiers(final Class<? extends ControllerService> serviceType) {
+ if (!serviceType.isInterface()) {
+ throw new IllegalArgumentException("ControllerServices may be referenced only via their interfaces; " + serviceType + " is not an interface");
+ }
+ return controllerServiceProvider.getControllerServiceIdentifiers(serviceType);
+ }
+
+ @Override
+ public boolean isControllerServiceEnabled(final ControllerService service) {
+ return controllerServiceProvider.isControllerServiceEnabled(service);
+ }
+
+ @Override
+ public boolean isControllerServiceEnabled(final String serviceIdentifier) {
+ return controllerServiceProvider.isControllerServiceEnabled(serviceIdentifier);
+ }
+
+ @Override
+ public ControllerServiceLookup getControllerServiceLookup() {
+ return this;
+ }
++
++ @Override
++ public Set<Relationship> getAvailableRelationships() {
++ final Set<Relationship> set = new HashSet<>();
++ for (final Relationship relationship : procNode.getRelationships()) {
++ final Collection<Connection> connections = procNode.getConnections(relationship);
++ if (connections.isEmpty()) {
++ set.add(relationship);
++ } else {
++ boolean available = true;
++ for (final Connection connection : connections) {
++ if (connection.getFlowFileQueue().isFull()) {
++ available = false;
++ }
++ }
++
++ if (available) {
++ set.add(relationship);
++ }
++ }
++ }
++
++ return set;
++ }
++
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
index 0000000,0fe08c9..318901f
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
@@@ -1,0 -1,107 +1,113 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processor;
+
+ import java.util.Map;
++import java.util.Set;
+
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.controller.ControllerServiceLookup;
+ import org.apache.nifi.controller.ProcessorNode;
+ import org.apache.nifi.controller.service.ControllerServiceNode;
+ import org.apache.nifi.controller.service.ControllerServiceProvider;
+
+ public class StandardSchedulingContext implements SchedulingContext {
+
+ private final ProcessContext processContext;
+ private final ControllerServiceProvider serviceProvider;
+ private final ProcessorNode processorNode;
+
+ public StandardSchedulingContext(final ProcessContext processContext, final ControllerServiceProvider serviceProvider, final ProcessorNode processorNode) {
+ this.processContext = processContext;
+ this.serviceProvider = serviceProvider;
+ this.processorNode = processorNode;
+ }
+
+ @Override
+ public void leaseControllerService(final String identifier) {
+ final ControllerServiceNode serviceNode = serviceProvider.getControllerServiceNode(identifier);
+ if (serviceNode == null) {
+ throw new IllegalArgumentException("Cannot lease Controller Service because no Controller Service exists with identifier " + identifier);
+ }
+
+ if (serviceNode.isDisabled()) {
+ throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getControllerService() + " is currently disabled");
+ }
+
+ if (!serviceNode.isValid()) {
+ throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getControllerService() + " is not currently valid");
+ }
+
+ serviceNode.addReference(processorNode);
+ }
+
+ @Override
+ public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+ return processContext.getProperty(descriptor);
+ }
+
+ @Override
+ public PropertyValue getProperty(final String propertyName) {
+ return processContext.getProperty(propertyName);
+ }
+
+ @Override
+ public PropertyValue newPropertyValue(final String rawValue) {
+ return processContext.newPropertyValue(rawValue);
+ }
+
+ @Override
+ public void yield() {
+ processContext.yield();
+ }
+
+ @Override
+ public int getMaxConcurrentTasks() {
+ return processContext.getMaxConcurrentTasks();
+ }
+
+ @Override
+ public String getAnnotationData() {
+ return processContext.getAnnotationData();
+ }
+
+ @Override
+ public Map<PropertyDescriptor, String> getProperties() {
+ return processContext.getProperties();
+ }
+
+ @Override
+ public String encrypt(final String unencrypted) {
+ return processContext.encrypt(unencrypted);
+ }
+
+ @Override
+ public String decrypt(final String encrypted) {
+ return processContext.decrypt(encrypted);
+ }
+
+ @Override
+ public ControllerServiceLookup getControllerServiceLookup() {
+ return processContext.getControllerServiceLookup();
+ }
++
++ @Override
++ public Set<Relationship> getAvailableRelationships() {
++ return processContext.getAvailableRelationships();
++ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
index 0000000,22ec983..d4b4f61
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketClientProtocol.java
@@@ -1,0 -1,510 +1,510 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.remote.protocol.socket;
+
+ import java.io.DataInputStream;
+ import java.io.DataOutputStream;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.TimeUnit;
+ import java.util.zip.CRC32;
+ import java.util.zip.CheckedInputStream;
+ import java.util.zip.CheckedOutputStream;
+
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.remote.Peer;
+ import org.apache.nifi.remote.PeerStatus;
+ import org.apache.nifi.remote.RemoteGroupPort;
+ import org.apache.nifi.remote.RemoteResourceFactory;
+ import org.apache.nifi.remote.StandardVersionNegotiator;
+ import org.apache.nifi.remote.VersionNegotiator;
+ import org.apache.nifi.remote.codec.FlowFileCodec;
+ import org.apache.nifi.remote.codec.StandardFlowFileCodec;
+ import org.apache.nifi.remote.exception.HandshakeException;
+ import org.apache.nifi.remote.exception.ProtocolException;
+ import org.apache.nifi.remote.io.CompressionInputStream;
+ import org.apache.nifi.remote.io.CompressionOutputStream;
+ import org.apache.nifi.remote.protocol.ClientProtocol;
+ import org.apache.nifi.remote.protocol.CommunicationsSession;
+ import org.apache.nifi.remote.protocol.RequestType;
+ import org.apache.nifi.util.FormatUtils;
+ import org.apache.nifi.util.StopWatch;
+
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+
+ public class SocketClientProtocol implements ClientProtocol {
+ private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(4, 3, 2, 1);
+
+
+ private RemoteGroupPort port;
+ private boolean useCompression;
+
+ private String commsIdentifier;
+ private boolean handshakeComplete = false;
+
+ private final Logger logger = LoggerFactory.getLogger(SocketClientProtocol.class);
+
+ private Response handshakeResponse = null;
+ private boolean readyForFileTransfer = false;
+ private String transitUriPrefix = null;
+
+ private static final long BATCH_SEND_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
+
+ public SocketClientProtocol() {
+ }
+
+ public void setPort(final RemoteGroupPort port) {
+ this.port = port;
+ this.useCompression = port.isUseCompression();
+ }
+
+ @Override
+ public void handshake(final Peer peer) throws IOException, HandshakeException {
+ if ( handshakeComplete ) {
+ throw new IllegalStateException("Handshake has already been completed");
+ }
+ commsIdentifier = UUID.randomUUID().toString();
+ logger.debug("{} handshaking with {}", this, peer);
+
+ final Map<HandshakeProperty, String> properties = new HashMap<>();
+ properties.put(HandshakeProperty.GZIP, String.valueOf(useCompression));
+ properties.put(HandshakeProperty.PORT_IDENTIFIER, port.getIdentifier());
+ properties.put(HandshakeProperty.REQUEST_EXPIRATION_MILLIS, String.valueOf(
+ port.getRemoteProcessGroup().getCommunicationsTimeout(TimeUnit.MILLISECONDS)) );
+
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ commsSession.setTimeout(port.getRemoteProcessGroup().getCommunicationsTimeout(TimeUnit.MILLISECONDS));
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ dos.writeUTF(commsIdentifier);
+
+ if ( versionNegotiator.getVersion() >= 3 ) {
+ dos.writeUTF(peer.getUrl());
+ transitUriPrefix = peer.getUrl();
+
+ if ( !transitUriPrefix.endsWith("/") ) {
+ transitUriPrefix = transitUriPrefix + "/";
+ }
+ }
+
+ dos.writeInt(properties.size());
+ for ( final Map.Entry<HandshakeProperty, String> entry : properties.entrySet() ) {
+ dos.writeUTF(entry.getKey().name());
+ dos.writeUTF(entry.getValue());
+ }
+
+ dos.flush();
+
+ try {
+ handshakeResponse = Response.read(dis);
+ } catch (final ProtocolException e) {
+ throw new HandshakeException(e);
+ }
+
+ switch (handshakeResponse.getCode()) {
+ case PORT_NOT_IN_VALID_STATE:
+ case UNKNOWN_PORT:
+ case PORTS_DESTINATION_FULL:
+ break;
+ case PROPERTIES_OK:
+ readyForFileTransfer = true;
+ break;
+ default:
+ logger.error("{} received unexpected response {} from {} when negotiating Codec", new Object[] {
+ this, handshakeResponse, peer});
+ peer.close();
+ throw new HandshakeException("Received unexpected response " + handshakeResponse);
+ }
+
+ logger.debug("{} Finished handshake with {}", this, peer);
+ handshakeComplete = true;
+ }
+
+ public boolean isReadyForFileTransfer() {
+ return readyForFileTransfer;
+ }
+
+ public boolean isPortInvalid() {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not completed successfully");
+ }
+ return handshakeResponse.getCode() == ResponseCode.PORT_NOT_IN_VALID_STATE;
+ }
+
+ public boolean isPortUnknown() {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not completed successfully");
+ }
+ return handshakeResponse.getCode() == ResponseCode.UNKNOWN_PORT;
+ }
+
+ public boolean isDestinationFull() {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not completed successfully");
+ }
+ return handshakeResponse.getCode() == ResponseCode.PORTS_DESTINATION_FULL;
+ }
+
+ @Override
+ public Set<PeerStatus> getPeerStatuses(final Peer peer) throws IOException {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not been performed");
+ }
+
+ logger.debug("{} Get Peer Statuses from {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ RequestType.REQUEST_PEER_LIST.writeRequestType(dos);
+ dos.flush();
+ final int numPeers = dis.readInt();
+ final Set<PeerStatus> peers = new HashSet<>(numPeers);
+ for (int i=0; i < numPeers; i++) {
+ final String hostname = dis.readUTF();
+ final int port = dis.readInt();
+ final boolean secure = dis.readBoolean();
+ final int flowFileCount = dis.readInt();
+ peers.add(new PeerStatus(hostname, port, secure, flowFileCount));
+ }
+
+ logger.debug("{} Received {} Peer Statuses from {}", this, peers.size(), peer);
+ return peers;
+ }
+
+ @Override
+ public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, ProtocolException {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not been performed");
+ }
+
+ logger.debug("{} Negotiating Codec with {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ RequestType.NEGOTIATE_FLOWFILE_CODEC.writeRequestType(dos);
+
+ FlowFileCodec codec = new StandardFlowFileCodec();
+ try {
+ codec = (FlowFileCodec) RemoteResourceFactory.initiateResourceNegotiation(codec, dis, dos);
+ } catch (HandshakeException e) {
+ throw new ProtocolException(e.toString());
+ }
+ logger.debug("{} negotiated FlowFileCodec {} with {}", new Object[] {this, codec, commsSession});
+
+ return codec;
+ }
+
+
+ @Override
+ public void receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not been performed");
+ }
+ if ( !readyForFileTransfer ) {
+ throw new IllegalStateException("Cannot receive files; handshake resolution was " + handshakeResponse);
+ }
+
+ logger.debug("{} Receiving FlowFiles from {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ String userDn = commsSession.getUserDn();
+ if ( userDn == null ) {
+ userDn = "none";
+ }
+
+ // Indicate that we would like to have some data
+ RequestType.RECEIVE_FLOWFILES.writeRequestType(dos);
+ dos.flush();
+
+ // Determine if Peer will send us data or has no data to send us
+ final Response dataAvailableCode = Response.read(dis);
+ switch (dataAvailableCode.getCode()) {
+ case MORE_DATA:
+ logger.debug("{} {} Indicates that data is available", this, peer);
+ break;
+ case NO_MORE_DATA:
+ logger.debug("{} No data available from {}", peer);
+ return;
+ default:
+ throw new ProtocolException("Got unexpected response when asking for data: " + dataAvailableCode);
+ }
+
+ final StopWatch stopWatch = new StopWatch(true);
+ final Set<FlowFile> flowFilesReceived = new HashSet<>();
+ long bytesReceived = 0L;
+ final CRC32 crc = new CRC32();
+
+ // Peer has data. Decode the bytes into FlowFiles until peer says he's finished sending data.
+ boolean continueTransaction = true;
+ String calculatedCRC = "";
+ while (continueTransaction) {
+ final InputStream flowFileInputStream = useCompression ? new CompressionInputStream(dis) : dis;
+ final CheckedInputStream checkedIn = new CheckedInputStream(flowFileInputStream, crc);
+
+ final long startNanos = System.nanoTime();
+ FlowFile flowFile = codec.decode(checkedIn, session);
+ final long transmissionNanos = System.nanoTime() - startNanos;
+ final long transmissionMillis = TimeUnit.MILLISECONDS.convert(transmissionNanos, TimeUnit.NANOSECONDS);
+
+ final String sourceFlowFileIdentifier = flowFile.getAttribute(CoreAttributes.UUID.key());
+ flowFile = session.putAttribute(flowFile, CoreAttributes.UUID.key(), UUID.randomUUID().toString());
+
+ final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceFlowFileIdentifier;
+ session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceFlowFileIdentifier, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transmissionMillis);
+
+ session.transfer(flowFile, Relationship.ANONYMOUS);
+ bytesReceived += flowFile.getSize();
+ flowFilesReceived.add(flowFile);
+ logger.debug("{} Received {} from {}", this, flowFile, peer);
+
+ final Response transactionCode = Response.read(dis);
+ switch (transactionCode.getCode()) {
+ case CONTINUE_TRANSACTION:
+ logger.trace("{} Received ContinueTransaction indicator from {}", this, peer);
+ break;
+ case FINISH_TRANSACTION:
+ logger.trace("{} Received FinishTransaction indicator from {}", this, peer);
+ continueTransaction = false;
+ calculatedCRC = String.valueOf(checkedIn.getChecksum().getValue());
+ break;
+ default:
+ throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionCode);
+ }
+ }
+
+ // we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
+ // to peer so that we can verify that the connection is still open. This is a two-phase commit,
+ // which helps to prevent the chances of data duplication. Without doing this, we may commit the
+ // session and then when we send the response back to the peer, the peer may have timed out and may not
+ // be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
+ // Critical Section involved in this transaction so that rather than the Critical Section being the
+ // time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
+ logger.trace("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
+ ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
+
+ final Response confirmTransactionResponse = Response.read(dis);
+ logger.trace("{} Received {} from {}", this, confirmTransactionResponse, peer);
+
+ switch (confirmTransactionResponse.getCode()) {
+ case CONFIRM_TRANSACTION:
+ break;
+ case BAD_CHECKSUM:
+ session.rollback();
+ throw new IOException(this + " Received a BadChecksum response from peer " + peer);
+ default:
+ throw new ProtocolException(this + " Received unexpected Response from peer " + peer + " : " + confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
+ }
+
+ // Commit the session so that we have persisted the data
+ session.commit();
+
- if ( session.getAvailableRelationships().isEmpty() ) {
++ if ( context.getAvailableRelationships().isEmpty() ) {
+ // Confirm that we received the data and the peer can now discard it but that the peer should not
+ // send any more data for a bit
+ logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
+ ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(dos);
+ } else {
+ // Confirm that we received the data and the peer can now discard it
+ logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
+ ResponseCode.TRANSACTION_FINISHED.writeResponse(dos);
+ }
+
+ stopWatch.stop();
+ final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles";
+ final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived);
+ final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+ final String dataSize = FormatUtils.formatDataSize(bytesReceived);
+ logger.info("{} Successfully receveied {} ({}) from {} in {} milliseconds at a rate of {}", new Object[] {
+ this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+ }
+
+ @Override
+ public void transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+ if ( !handshakeComplete ) {
+ throw new IllegalStateException("Handshake has not been performed");
+ }
+ if ( !readyForFileTransfer ) {
+ throw new IllegalStateException("Cannot transfer files; handshake resolution was " + handshakeResponse);
+ }
+
+ FlowFile flowFile = session.get();
+ if ( flowFile == null ) {
+ return;
+ }
+
+ logger.debug("{} Sending FlowFiles to {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ String userDn = commsSession.getUserDn();
+ if ( userDn == null ) {
+ userDn = "none";
+ }
+
+ // Indicate that we would like to have some data
+ RequestType.SEND_FLOWFILES.writeRequestType(dos);
+ dos.flush();
+
+ final StopWatch stopWatch = new StopWatch(true);
+ final CRC32 crc = new CRC32();
+
+ long bytesSent = 0L;
+ final Set<FlowFile> flowFilesSent = new HashSet<>();
+ boolean continueTransaction = true;
+ String calculatedCRC = "";
+ final long startSendingNanos = System.nanoTime();
+ while (continueTransaction) {
+ final OutputStream flowFileOutputStream = useCompression ? new CompressionOutputStream(dos) : dos;
+ logger.debug("{} Sending {} to {}", this, flowFile, peer);
+
+ final CheckedOutputStream checkedOutStream = new CheckedOutputStream(flowFileOutputStream, crc);
+
+ final long startNanos = System.nanoTime();
+ flowFile = codec.encode(flowFile, session, checkedOutStream);
+ final long transferNanos = System.nanoTime() - startNanos;
+ final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
+
+ // need to close the CompressionOutputStream in order to force it write out any remaining bytes.
+ // Otherwise, do NOT close it because we don't want to close the underlying stream
+ // (CompressionOutputStream will not close the underlying stream when it's closed)
+ if ( useCompression ) {
+ checkedOutStream.close();
+ }
+
+ flowFilesSent.add(flowFile);
+ bytesSent += flowFile.getSize();
+ logger.debug("{} Sent {} to {}", this, flowFile, peer);
+
+ final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + flowFile.getAttribute(CoreAttributes.UUID.key());
+ session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + userDn, transferMillis, false);
+ session.remove(flowFile);
+
+ final long sendingNanos = System.nanoTime() - startSendingNanos;
+ if ( sendingNanos < BATCH_SEND_NANOS ) {
+ flowFile = session.get();
+ } else {
+ flowFile = null;
+ }
+
+ continueTransaction = (flowFile != null);
+ if ( continueTransaction ) {
+ logger.debug("{} Sent CONTINUE_TRANSACTION indicator to {}", this, peer);
+ ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
+ } else {
+ logger.debug("{} Sent FINISH_TRANSACTION indicator to {}", this, peer);
+ ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
+
+ calculatedCRC = String.valueOf( checkedOutStream.getChecksum().getValue() );
+ }
+ }
+
+ // we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
+ final Response transactionConfirmationResponse = Response.read(dis);
+ if ( transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION ) {
+ // Confirm checksum and echo back the confirmation.
+ logger.trace("{} Received {} from {}", this, transactionConfirmationResponse, peer);
+ final String receivedCRC = transactionConfirmationResponse.getMessage();
+
+ if ( versionNegotiator.getVersion() > 3 ) {
+ if ( !receivedCRC.equals(calculatedCRC) ) {
+ ResponseCode.BAD_CHECKSUM.writeResponse(dos);
+ session.rollback();
+ throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as " + calculatedCRC + " while peer calculated CRC32 Checksum as " + receivedCRC + "; canceling transaction and rolling back session");
+ }
+ }
+
+ ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
+ } else {
+ throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse);
+ }
+
+ final String flowFileDescription = (flowFilesSent.size() < 20) ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
+
+ final Response transactionResponse;
+ try {
+ transactionResponse = Response.read(dis);
+ } catch (final IOException e) {
+ logger.error("{} Failed to receive a response from {} when expecting a TransactionFinished Indicator." +
+ " It is unknown whether or not the peer successfully received/processed the data." +
+ " Therefore, {} will be rolled back, possibly resulting in data duplication of {}",
+ this, peer, session, flowFileDescription);
+ session.rollback();
+ throw e;
+ }
+
+ logger.debug("{} Received {} from {}", this, transactionResponse, peer);
+ if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
+ peer.penalize(port.getYieldPeriod(TimeUnit.MILLISECONDS));
+ } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
+ throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
+ }
+
+ // consume input stream entirely, ignoring its contents. If we
+ // don't do this, the Connection will not be returned to the pool
+ stopWatch.stop();
+ final String uploadDataRate = stopWatch.calculateDataRate(bytesSent);
+ final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+ final String dataSize = FormatUtils.formatDataSize(bytesSent);
+
+ session.commit();
+
+ logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {
+ this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+ }
+
+ @Override
+ public VersionNegotiator getVersionNegotiator() {
+ return versionNegotiator;
+ }
+
+ @Override
+ public void shutdown(final Peer peer) throws IOException {
+ readyForFileTransfer = false;
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ logger.debug("{} Shutting down with {}", this, peer);
+ // Indicate that we would like to have some data
+ RequestType.SHUTDOWN.writeRequestType(dos);
+ dos.flush();
+ }
+
+ @Override
+ public String getResourceName() {
+ return "SocketFlowFileProtocol";
+ }
+
+ @Override
+ public String toString() {
+ return "SocketClientProtocol[CommsID=" + commsIdentifier + "]";
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
index 0000000,88b6a41..5edd4f9
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
+++ b/nifi/nar-bundles/framework-bundle/framework/site-to-site/src/main/java/org/apache/nifi/remote/protocol/socket/SocketFlowFileServerProtocol.java
@@@ -1,0 -1,581 +1,581 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.remote.protocol.socket;
+
+ import java.io.DataInputStream;
+ import java.io.DataOutputStream;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.net.InetAddress;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.TimeUnit;
+ import java.util.zip.CRC32;
+ import java.util.zip.CheckedInputStream;
+ import java.util.zip.CheckedOutputStream;
+
+ import org.apache.nifi.cluster.NodeInformant;
+ import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.connectable.Port;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.groups.ProcessGroup;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.remote.Peer;
+ import org.apache.nifi.remote.PortAuthorizationResult;
+ import org.apache.nifi.remote.RemoteResourceFactory;
+ import org.apache.nifi.remote.RootGroupPort;
+ import org.apache.nifi.remote.StandardVersionNegotiator;
+ import org.apache.nifi.remote.VersionNegotiator;
+ import org.apache.nifi.remote.codec.FlowFileCodec;
+ import org.apache.nifi.remote.exception.HandshakeException;
+ import org.apache.nifi.remote.exception.ProtocolException;
+ import org.apache.nifi.remote.io.CompressionInputStream;
+ import org.apache.nifi.remote.io.CompressionOutputStream;
+ import org.apache.nifi.remote.protocol.CommunicationsSession;
+ import org.apache.nifi.remote.protocol.RequestType;
+ import org.apache.nifi.remote.protocol.ServerProtocol;
+ import org.apache.nifi.util.FormatUtils;
+ import org.apache.nifi.util.NiFiProperties;
+ import org.apache.nifi.util.StopWatch;
+
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+
+ public class SocketFlowFileServerProtocol implements ServerProtocol {
+ public static final String RESOURCE_NAME = "SocketFlowFileProtocol";
+
+ private ProcessGroup rootGroup;
+ private String commsIdentifier;
+ private boolean handshakeCompleted;
+
+ private Boolean useGzip;
+ private long requestExpirationMillis;
+ private RootGroupPort port;
+ private boolean shutdown = false;
+ private FlowFileCodec negotiatedFlowFileCodec = null;
+ private String transitUriPrefix = null;
+
+ private final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(4, 3, 2, 1);
+ private final Logger logger = LoggerFactory.getLogger(SocketFlowFileServerProtocol.class);
+
+ private static final long BATCH_NANOS = TimeUnit.SECONDS.toNanos(5L); // send batches of up to 5 seconds
+
+
+ @Override
+ public void setRootProcessGroup(final ProcessGroup group) {
+ if ( !group.isRootGroup() ) {
+ throw new IllegalArgumentException();
+ }
+ this.rootGroup = group;
+ }
+
+ @Override
+ public void handshake(final Peer peer) throws IOException, HandshakeException {
+ if ( handshakeCompleted ) {
+ throw new IllegalStateException("Handshake has already been completed");
+ }
+ if ( shutdown ) {
+ throw new IllegalStateException("Protocol is shutdown");
+ }
+
+ logger.debug("{} Handshaking with {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ commsIdentifier = dis.readUTF();
+
+ if ( versionNegotiator.getVersion() >= 3 ) {
+ transitUriPrefix = dis.readUTF();
+ if ( !transitUriPrefix.endsWith("/") ) {
+ transitUriPrefix = transitUriPrefix + "/";
+ }
+ }
+
+ final Map<String, String> properties = new HashMap<>();
+ final int numProperties = dis.readInt();
+ for (int i=0; i < numProperties; i++) {
+ final String propertyName = dis.readUTF();
+ final String propertyValue = dis.readUTF();
+ properties.put(propertyName, propertyValue);
+ }
+
+ // evaluate the properties received
+ boolean responseWritten = false;
+ for ( final Map.Entry<String, String> entry : properties.entrySet() ) {
+ final String propertyName = entry.getKey();
+ final String value = entry.getValue();
+
+ final HandshakeProperty property;
+ try {
+ property = HandshakeProperty.valueOf(propertyName);
+ } catch (final Exception e) {
+ ResponseCode.UNKNOWN_PROPERTY_NAME.writeResponse(dos, "Unknown Property Name: " + propertyName);
+ throw new HandshakeException("Received unknown property: " + propertyName);
+ }
+
+ switch (property) {
+ case GZIP: {
+ useGzip = Boolean.parseBoolean(value);
+ break;
+ }
+ case REQUEST_EXPIRATION_MILLIS:
+ requestExpirationMillis = Long.parseLong(value);
+ break;
+ case PORT_IDENTIFIER: {
+ Port receivedPort = rootGroup.getInputPort(value);
+ if ( receivedPort == null ) {
+ receivedPort = rootGroup.getOutputPort(value);
+ }
+ if ( receivedPort == null ) {
+ logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
+ ResponseCode.UNKNOWN_PORT.writeResponse(dos);
+ throw new HandshakeException("Received unknown port identifier: " + value);
+ }
+ if ( !(receivedPort instanceof RootGroupPort) ) {
+ logger.debug("Responding with ResponseCode UNKNOWN_PORT for identifier {}", value);
+ ResponseCode.UNKNOWN_PORT.writeResponse(dos);
+ throw new HandshakeException("Received port identifier " + value + ", but this Port is not a RootGroupPort");
+ }
+
+ this.port = (RootGroupPort) receivedPort;
+ final PortAuthorizationResult portAuthResult = this.port.checkUserAuthorization(peer.getCommunicationsSession().getUserDn());
+ if ( !portAuthResult.isAuthorized() ) {
+ logger.debug("Responding with ResponseCode UNAUTHORIZED: ", portAuthResult.getExplanation());
+ ResponseCode.UNAUTHORIZED.writeResponse(dos, portAuthResult.getExplanation());
+ responseWritten = true;
+ break;
+ }
+
+ if ( !receivedPort.isValid() ) {
+ logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
+ ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port is not valid");
+ responseWritten = true;
+ break;
+ }
+
+ if ( !receivedPort.isRunning() ) {
+ logger.debug("Responding with ResponseCode PORT_NOT_IN_VALID_STATE for {}", receivedPort);
+ ResponseCode.PORT_NOT_IN_VALID_STATE.writeResponse(dos, "Port not running");
+ responseWritten = true;
+ break;
+ }
+
+ // PORTS_DESTINATION_FULL was introduced in version 2. If version 1, just ignore this
+ // we we will simply not service the request but the sender will timeout
+ if ( getVersionNegotiator().getVersion() > 1 ) {
+ for ( final Connection connection : port.getConnections() ) {
+ if ( connection.getFlowFileQueue().isFull() ) {
+ logger.debug("Responding with ResponseCode PORTS_DESTINATION_FULL for {}", receivedPort);
+ ResponseCode.PORTS_DESTINATION_FULL.writeResponse(dos);
+ responseWritten = true;
+ break;
+ }
+ }
+ }
+
+ break;
+ }
+ }
+ }
+
+ if ( useGzip == null ) {
+ logger.debug("Responding with ResponseCode MISSING_PROPERTY because GZIP Property missing");
+ ResponseCode.MISSING_PROPERTY.writeResponse(dos, HandshakeProperty.GZIP.name());
+ throw new HandshakeException("Missing Property " + HandshakeProperty.GZIP.name());
+ }
+ if ( port == null ) {
+ logger.debug("Responding with ResponseCode MISSING_PROPERTY because Port Identifier property is missing");
+ ResponseCode.MISSING_PROPERTY.writeResponse(dos, HandshakeProperty.PORT_IDENTIFIER.name());
+ throw new HandshakeException("Missing Property " + HandshakeProperty.PORT_IDENTIFIER.name());
+ }
+
+ // send "OK" response
+ if ( !responseWritten ) {
+ ResponseCode.PROPERTIES_OK.writeResponse(dos);
+ }
+
+ logger.debug("{} Finished handshake with {}", this, peer);
+ handshakeCompleted = true;
+ }
+
+ @Override
+ public boolean isHandshakeSuccessful() {
+ return handshakeCompleted;
+ }
+
+ @Override
+ public RootGroupPort getPort() {
+ return port;
+ }
+
+ @Override
+ public FlowFileCodec negotiateCodec(final Peer peer) throws IOException, ProtocolException {
+ if ( !handshakeCompleted ) {
+ throw new IllegalStateException("Handshake has not been completed");
+ }
+ if ( shutdown ) {
+ throw new IllegalStateException("Protocol is shutdown");
+ }
+
+ logger.debug("{} Negotiating Codec with {} using {}", new Object[] {this, peer, peer.getCommunicationsSession()});
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ // Negotiate the FlowFileCodec to use.
+ try {
+ negotiatedFlowFileCodec = RemoteResourceFactory.receiveCodecNegotiation(dis, dos);
+ logger.debug("{} Negotiated Codec {} with {}", new Object[] {this, negotiatedFlowFileCodec, peer});
+ return negotiatedFlowFileCodec;
+ } catch (final HandshakeException e) {
+ throw new ProtocolException(e.toString());
+ }
+ }
+
+ @Override
+ public FlowFileCodec getPreNegotiatedCodec() {
+ return negotiatedFlowFileCodec;
+ }
+
+
+ @Override
+ public int transferFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+ if ( !handshakeCompleted ) {
+ throw new IllegalStateException("Handshake has not been completed");
+ }
+ if ( shutdown ) {
+ throw new IllegalStateException("Protocol is shutdown");
+ }
+
+ logger.debug("{} Sending FlowFiles to {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ String remoteDn = commsSession.getUserDn();
+ if ( remoteDn == null ) {
+ remoteDn = "none";
+ }
+
+ FlowFile flowFile = session.get();
+ if ( flowFile == null ) {
+ // we have no data to send. Notify the peer.
+ logger.debug("{} No data to send to {}", this, peer);
+ ResponseCode.NO_MORE_DATA.writeResponse(dos);
+ return 0;
+ }
+
+ // we have data to send.
+ logger.debug("{} Data is available to send to {}", this, peer);
+ ResponseCode.MORE_DATA.writeResponse(dos);
+
+ final StopWatch stopWatch = new StopWatch(true);
+ long bytesSent = 0L;
+ final Set<FlowFile> flowFilesSent = new HashSet<>();
+ final CRC32 crc = new CRC32();
+
+ // send data until we reach some batch size
+ boolean continueTransaction = true;
+ final long startNanos = System.nanoTime();
+ String calculatedCRC = "";
+ while (continueTransaction) {
+ final OutputStream flowFileOutputStream = useGzip ? new CompressionOutputStream(dos) : dos;
+ logger.debug("{} Sending {} to {}", new Object[] {this, flowFile, peer});
+
+ final CheckedOutputStream checkedOutputStream = new CheckedOutputStream(flowFileOutputStream, crc);
+
+ final StopWatch transferWatch = new StopWatch(true);
+ flowFile = codec.encode(flowFile, session, checkedOutputStream);
+ final long transmissionMillis = transferWatch.getElapsed(TimeUnit.MILLISECONDS);
+
+ // need to close the CompressionOutputStream in order to force it write out any remaining bytes.
+ // Otherwise, do NOT close it because we don't want to close the underlying stream
+ // (CompressionOutputStream will not close the underlying stream when it's closed)
+ if ( useGzip ) {
+ checkedOutputStream.close();
+ }
+
+ flowFilesSent.add(flowFile);
+ bytesSent += flowFile.getSize();
+
+ final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + flowFile.getAttribute(CoreAttributes.UUID.key());
+ session.getProvenanceReporter().send(flowFile, transitUri, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transmissionMillis, false);
+ session.remove(flowFile);
+
+ final long sendingNanos = System.nanoTime() - startNanos;
+ if ( sendingNanos < BATCH_NANOS ) {
+ flowFile = session.get();
+ } else {
+ flowFile = null;
+ }
+
+ continueTransaction = (flowFile != null);
+ if ( continueTransaction ) {
+ logger.debug("{} Sending ContinueTransaction indicator to {}", this, peer);
+ ResponseCode.CONTINUE_TRANSACTION.writeResponse(dos);
+ } else {
+ logger.debug("{} Sending FinishTransaction indicator to {}", this, peer);
+ ResponseCode.FINISH_TRANSACTION.writeResponse(dos);
+ calculatedCRC = String.valueOf(checkedOutputStream.getChecksum().getValue());
+ }
+ }
+
+ // we've sent a FINISH_TRANSACTION. Now we'll wait for the peer to send a 'Confirm Transaction' response
+ final Response transactionConfirmationResponse = Response.read(dis);
+ if ( transactionConfirmationResponse.getCode() == ResponseCode.CONFIRM_TRANSACTION ) {
+ // Confirm Checksum and echo back the confirmation.
+ logger.debug("{} Received {} from {}", this, transactionConfirmationResponse, peer);
+ final String receivedCRC = transactionConfirmationResponse.getMessage();
+
+ if ( versionNegotiator.getVersion() > 3 ) {
+ if ( !receivedCRC.equals(calculatedCRC) ) {
+ ResponseCode.BAD_CHECKSUM.writeResponse(dos);
+ session.rollback();
+ throw new IOException(this + " Sent data to peer " + peer + " but calculated CRC32 Checksum as " + calculatedCRC + " while peer calculated CRC32 Checksum as " + receivedCRC + "; canceling transaction and rolling back session");
+ }
+ }
+
+ ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, "");
+ } else {
+ throw new ProtocolException("Expected to receive 'Confirm Transaction' response from peer " + peer + " but received " + transactionConfirmationResponse);
+ }
+
+ final String flowFileDescription = flowFilesSent.size() < 20 ? flowFilesSent.toString() : flowFilesSent.size() + " FlowFiles";
+
+ final Response transactionResponse;
+ try {
+ transactionResponse = Response.read(dis);
+ } catch (final IOException e) {
+ logger.error("{} Failed to receive a response from {} when expecting a TransactionFinished Indicator." +
+ " It is unknown whether or not the peer successfully received/processed the data." +
+ " Therefore, {} will be rolled back, possibly resulting in data duplication of {}",
+ this, peer, session, flowFileDescription);
+ session.rollback();
+ throw e;
+ }
+
+ logger.debug("{} received {} from {}", new Object[] {this, transactionResponse, peer});
+ if ( transactionResponse.getCode() == ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL ) {
+ peer.penalize(port.getYieldPeriod(TimeUnit.MILLISECONDS));
+ } else if ( transactionResponse.getCode() != ResponseCode.TRANSACTION_FINISHED ) {
+ throw new ProtocolException("After sending data, expected TRANSACTION_FINISHED response but got " + transactionResponse);
+ }
+
+ session.commit();
+
+ stopWatch.stop();
+ final String uploadDataRate = stopWatch.calculateDataRate(bytesSent);
+ final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+ final String dataSize = FormatUtils.formatDataSize(bytesSent);
+ logger.info("{} Successfully sent {} ({}) to {} in {} milliseconds at a rate of {}", new Object[] {
+ this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+
+ return flowFilesSent.size();
+ }
+
+
+ @Override
+ public int receiveFlowFiles(final Peer peer, final ProcessContext context, final ProcessSession session, final FlowFileCodec codec) throws IOException, ProtocolException {
+ if ( !handshakeCompleted ) {
+ throw new IllegalStateException("Handshake has not been completed");
+ }
+ if ( shutdown ) {
+ throw new IllegalStateException("Protocol is shutdown");
+ }
+
+ logger.debug("{} receiving FlowFiles from {}", this, peer);
+
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataInputStream dis = new DataInputStream(commsSession.getInput().getInputStream());
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+ String remoteDn = commsSession.getUserDn();
+ if ( remoteDn == null ) {
+ remoteDn = "none";
+ }
+
+ final StopWatch stopWatch = new StopWatch(true);
+ final CRC32 crc = new CRC32();
+
+ // Peer has data. Otherwise, we would not have been called, because they would not have sent
+ // a SEND_FLOWFILES request to use. Just decode the bytes into FlowFiles until peer says he's
+ // finished sending data.
+ final Set<FlowFile> flowFilesReceived = new HashSet<>();
+ long bytesReceived = 0L;
+ boolean continueTransaction = true;
+ String calculatedCRC = "";
+ while (continueTransaction) {
+ final long startNanos = System.nanoTime();
+ final InputStream flowFileInputStream = useGzip ? new CompressionInputStream(dis) : dis;
+ final CheckedInputStream checkedInputStream = new CheckedInputStream(flowFileInputStream, crc);
+
+ FlowFile flowFile = codec.decode(checkedInputStream, session);
+ final long transferNanos = System.nanoTime() - startNanos;
+ final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
+ final String sourceSystemFlowFileUuid = flowFile.getAttribute(CoreAttributes.UUID.key());
+ flowFile = session.putAttribute(flowFile, CoreAttributes.UUID.key(), UUID.randomUUID().toString());
+
+ final String transitUri = (transitUriPrefix == null) ? peer.getUrl() : transitUriPrefix + sourceSystemFlowFileUuid;
+ session.getProvenanceReporter().receive(flowFile, transitUri, "urn:nifi:" + sourceSystemFlowFileUuid, "Remote Host=" + peer.getHost() + ", Remote DN=" + remoteDn, transferMillis);
+ session.transfer(flowFile, Relationship.ANONYMOUS);
+ flowFilesReceived.add(flowFile);
+ bytesReceived += flowFile.getSize();
+
+ final Response transactionResponse = Response.read(dis);
+ switch (transactionResponse.getCode()) {
+ case CONTINUE_TRANSACTION:
+ logger.debug("{} Received ContinueTransaction indicator from {}", this, peer);
+ break;
+ case FINISH_TRANSACTION:
+ logger.debug("{} Received FinishTransaction indicator from {}", this, peer);
+ continueTransaction = false;
+ calculatedCRC = String.valueOf(checkedInputStream.getChecksum().getValue());
+ break;
+ default:
+ throw new ProtocolException("Received unexpected response from peer: when expecting Continue Transaction or Finish Transaction, received" + transactionResponse);
+ }
+ }
+
+ // we received a FINISH_TRANSACTION indicator. Send back a CONFIRM_TRANSACTION message
+ // to peer so that we can verify that the connection is still open. This is a two-phase commit,
+ // which helps to prevent the chances of data duplication. Without doing this, we may commit the
+ // session and then when we send the response back to the peer, the peer may have timed out and may not
+ // be listening. As a result, it will re-send the data. By doing this two-phase commit, we narrow the
+ // Critical Section involved in this transaction so that rather than the Critical Section being the
+ // time window involved in the entire transaction, it is reduced to a simple round-trip conversation.
+ logger.debug("{} Sending CONFIRM_TRANSACTION Response Code to {}", this, peer);
+ ResponseCode.CONFIRM_TRANSACTION.writeResponse(dos, calculatedCRC);
+
+ final Response confirmTransactionResponse = Response.read(dis);
+ logger.debug("{} Received {} from {}", this, confirmTransactionResponse, peer);
+
+ switch (confirmTransactionResponse.getCode()) {
+ case CONFIRM_TRANSACTION:
+ break;
+ case BAD_CHECKSUM:
+ session.rollback();
+ throw new IOException(this + " Received a BadChecksum response from peer " + peer);
+ default:
+ throw new ProtocolException(this + " Received unexpected Response Code from peer " + peer + " : " + confirmTransactionResponse + "; expected 'Confirm Transaction' Response Code");
+ }
+
+ // Commit the session so that we have persisted the data
+ session.commit();
+
- if ( session.getAvailableRelationships().isEmpty() ) {
++ if ( context.getAvailableRelationships().isEmpty() ) {
+ // Confirm that we received the data and the peer can now discard it but that the peer should not
+ // send any more data for a bit
+ logger.debug("{} Sending TRANSACTION_FINISHED_BUT_DESTINATION_FULL to {}", this, peer);
+ ResponseCode.TRANSACTION_FINISHED_BUT_DESTINATION_FULL.writeResponse(dos);
+ } else {
+ // Confirm that we received the data and the peer can now discard it
+ logger.debug("{} Sending TRANSACTION_FINISHED to {}", this, peer);
+ ResponseCode.TRANSACTION_FINISHED.writeResponse(dos);
+ }
+
+ stopWatch.stop();
+ final String flowFileDescription = flowFilesReceived.size() < 20 ? flowFilesReceived.toString() : flowFilesReceived.size() + " FlowFiles";
+ final String uploadDataRate = stopWatch.calculateDataRate(bytesReceived);
+ final long uploadMillis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+ final String dataSize = FormatUtils.formatDataSize(bytesReceived);
+ logger.info("{} Successfully received {} ({}) from {} in {} milliseconds at a rate of {}", new Object[] {
+ this, flowFileDescription, dataSize, peer, uploadMillis, uploadDataRate});
+
+ return flowFilesReceived.size();
+ }
+
+ @Override
+ public RequestType getRequestType(final Peer peer) throws IOException {
+ if ( !handshakeCompleted ) {
+ throw new IllegalStateException("Handshake has not been completed");
+ }
+ if ( shutdown ) {
+ throw new IllegalStateException("Protocol is shutdown");
+ }
+
+ logger.debug("{} Reading Request Type from {} using {}", new Object[] {this, peer, peer.getCommunicationsSession()});
+ final RequestType requestType = RequestType.readRequestType(new DataInputStream(peer.getCommunicationsSession().getInput().getInputStream()));
+ logger.debug("{} Got Request Type {} from {}", new Object[] {this, requestType, peer});
+
+ return requestType;
+ }
+
+ @Override
+ public VersionNegotiator getVersionNegotiator() {
+ return versionNegotiator;
+ }
+
+ @Override
+ public void shutdown(final Peer peer) {
+ logger.debug("{} Shutting down with {}", this, peer);
+ shutdown = true;
+ }
+
+ @Override
+ public boolean isShutdown() {
+ return shutdown;
+ }
+
+ @Override
+ public void sendPeerList(final Peer peer) throws IOException {
+ if ( !handshakeCompleted ) {
+ throw new IllegalStateException("Handshake has not been completed");
+ }
+ if ( shutdown ) {
+ throw new IllegalStateException("Protocol is shutdown");
+ }
+
+ logger.debug("{} Sending Peer List to {}", this, peer);
+ final CommunicationsSession commsSession = peer.getCommunicationsSession();
+ final DataOutputStream dos = new DataOutputStream(commsSession.getOutput().getOutputStream());
+
+ final NiFiProperties properties = NiFiProperties.getInstance();
+
+ // we have only 1 peer: ourselves.
+ dos.writeInt(1);
+ dos.writeUTF(InetAddress.getLocalHost().getHostName());
+ dos.writeInt(properties.getRemoteInputPort());
+ dos.writeBoolean(properties.isSiteToSiteSecure());
+ dos.writeInt(0); // doesn't matter how many FlowFiles we have, because we're the only host.
+ dos.flush();
+ }
+
+ @Override
+ public String getResourceName() {
+ return RESOURCE_NAME;
+ }
+
+ @Override
+ public void setNodeInformant(final NodeInformant nodeInformant) {
+ }
+
+ @Override
+ public long getRequestExpiration() {
+ return requestExpirationMillis;
+ }
+
+ @Override
+ public String toString() {
+ return "SocketFlowFileServerProtocol[CommsID=" + commsIdentifier + "]";
+ }
+ }
[7/8] incubator-nifi git commit: Merge branch 'develop' of
https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
index 0000000,e516f20..e34e043
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java
@@@ -1,0 -1,541 +1,541 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.controller;
+
+ import static java.util.Objects.requireNonNull;
+
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Set;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicLong;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.concurrent.locks.Lock;
+ import java.util.concurrent.locks.ReadWriteLock;
+ import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+ import org.apache.nifi.components.ValidationResult;
+ import org.apache.nifi.connectable.Connectable;
+ import org.apache.nifi.connectable.ConnectableType;
+ import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.connectable.Funnel;
+ import org.apache.nifi.connectable.Position;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.groups.ProcessGroup;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.scheduling.SchedulingStrategy;
+ import org.apache.nifi.util.FormatUtils;
+
+ import org.apache.commons.lang3.builder.ToStringBuilder;
+ import org.apache.commons.lang3.builder.ToStringStyle;
+
+ public class StandardFunnel implements Funnel {
+
+ public static final long MINIMUM_PENALIZATION_MILLIS = 0L;
+ public static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS;
+ public static final long MINIMUM_YIELD_MILLIS = 0L;
+ public static final long DEFAULT_YIELD_PERIOD = 1000L;
+ public static final TimeUnit DEFAULT_YIELD_TIME_UNIT = TimeUnit.MILLISECONDS;
+
+ private final String identifier;
+ private final Set<Connection> outgoingConnections;
+ private final List<Connection> incomingConnections;
+ private final List<Relationship> relationships;
+
+ private final AtomicReference<ProcessGroup> processGroupRef;
+ private final AtomicReference<Position> position;
+ private final AtomicReference<String> penalizationPeriod;
+ private final AtomicReference<String> yieldPeriod;
+ private final AtomicReference<String> schedulingPeriod;
+ private final AtomicReference<String> name;
+ private final AtomicLong schedulingNanos;
+ private final AtomicBoolean lossTolerant;
+ private final AtomicReference<ScheduledState> scheduledState;
+ private final AtomicLong yieldExpiration;
+
+ private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
+ private final Lock readLock = rwLock.readLock();
+ private final Lock writeLock = rwLock.writeLock();
+
+ public StandardFunnel(final String identifier, final ProcessGroup processGroup, final ProcessScheduler scheduler) {
+ this.identifier = identifier;
+ this.processGroupRef = new AtomicReference<>(processGroup);
+
+ outgoingConnections = new HashSet<>();
+ incomingConnections = new ArrayList<>();
+
+ final List<Relationship> relationships = new ArrayList<>();
+ relationships.add(Relationship.ANONYMOUS);
+ this.relationships = Collections.unmodifiableList(relationships);
+
+ lossTolerant = new AtomicBoolean(false);
+ position = new AtomicReference<>(new Position(0D, 0D));
+ scheduledState = new AtomicReference<>(ScheduledState.STOPPED);
+ penalizationPeriod = new AtomicReference<>("30 sec");
+ yieldPeriod = new AtomicReference<>("1 sec");
+ yieldExpiration = new AtomicLong(0L);
+ schedulingPeriod = new AtomicReference<>("0 millis");
+ schedulingNanos = new AtomicLong(30000);
+ name = new AtomicReference<>("Funnel");
+ }
+
+ @Override
+ public String getIdentifier() {
+ return identifier;
+ }
+
+ @Override
+ public Collection<Relationship> getRelationships() {
+ return relationships;
+ }
+
+ @Override
+ public Relationship getRelationship(final String relationshipName) {
+ return (Relationship.ANONYMOUS.getName().equals(relationshipName)) ? Relationship.ANONYMOUS : null;
+ }
+
+ @Override
+ public void addConnection(final Connection connection) throws IllegalArgumentException {
+ writeLock.lock();
+ try {
+ if (!requireNonNull(connection).getSource().equals(this) && !connection.getDestination().equals(this)) {
+ throw new IllegalArgumentException("Cannot add a connection to a Funnel for which the Funnel is neither the Source nor the Destination");
+ }
+ if (connection.getSource().equals(this) && connection.getDestination().equals(this)) {
+ throw new IllegalArgumentException("Cannot add a connection from a Funnel back to itself");
+ }
+
+ if (connection.getDestination().equals(this)) {
+ // don't add the connection twice. This may occur if we have a self-loop because we will be told
+ // to add the connection once because we are the source and again because we are the destination.
+ if (!incomingConnections.contains(connection)) {
+ incomingConnections.add(connection);
+ }
+ }
+
+ if (connection.getSource().equals(this)) {
+ // don't add the connection twice. This may occur if we have a self-loop because we will be told
+ // to add the connection once because we are the source and again because we are the destination.
+ if (!outgoingConnections.contains(connection)) {
+ for (final Relationship relationship : connection.getRelationships()) {
+ if (!relationship.equals(Relationship.ANONYMOUS)) {
+ throw new IllegalArgumentException("No relationship with name " + relationship + " exists for Funnels");
+ }
+ }
+
+ outgoingConnections.add(connection);
+ }
+ }
+ } finally {
+ writeLock.unlock();
+ }
+ }
+
+ @Override
+ public boolean hasIncomingConnection() {
+ readLock.lock();
+ try {
+ return !incomingConnections.isEmpty();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ @Override
+ public void updateConnection(final Connection connection) throws IllegalStateException {
+ if (requireNonNull(connection).getSource().equals(this)) {
+ writeLock.lock();
+ try {
+ if (!outgoingConnections.remove(connection)) {
+ throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port");
+ }
+ outgoingConnections.add(connection);
+ } finally {
+ writeLock.unlock();
+ }
+ }
+
+ if (connection.getDestination().equals(this)) {
+ writeLock.lock();
+ try {
+ if (!incomingConnections.remove(connection)) {
+ throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port");
+ }
+ incomingConnections.add(connection);
+ } finally {
+ writeLock.unlock();
+ }
+ }
+ }
+
+ @Override
+ public void removeConnection(final Connection connection) throws IllegalArgumentException, IllegalStateException {
+ writeLock.lock();
+ try {
+ if (!requireNonNull(connection).getSource().equals(this)) {
+ final boolean existed = incomingConnections.remove(connection);
+ if (!existed) {
+ throw new IllegalStateException("The given connection is not currently registered for this ProcessorNode");
+ }
+ return;
+ }
+
+ final boolean removed = outgoingConnections.remove(connection);
+ if (!removed) {
+ throw new IllegalStateException(connection + " is not registered with " + this);
+ }
+ } finally {
+ writeLock.unlock();
+ }
+ }
+
+ @Override
+ public Set<Connection> getConnections() {
+ readLock.lock();
+ try {
+ return Collections.unmodifiableSet(outgoingConnections);
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ @Override
+ public Set<Connection> getConnections(final Relationship relationship) {
+ readLock.lock();
+ try {
+ if (relationship.equals(Relationship.ANONYMOUS)) {
+ return Collections.unmodifiableSet(outgoingConnections);
+ }
+
+ throw new IllegalArgumentException("No relationship with name " + relationship.getName() + " exists for Funnels");
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ @Override
+ public List<Connection> getIncomingConnections() {
+ readLock.lock();
+ try {
+ return new ArrayList<>(incomingConnections);
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ @Override
+ public Position getPosition() {
+ return position.get();
+ }
+
+ @Override
+ public void setPosition(Position position) {
+ this.position.set(position);
+ }
+
+ @Override
+ public String getName() {
+ return name.get();
+ }
+
+ /**
+ * Throws {@link UnsupportedOperationException}
+ *
+ * @param name
+ */
+ @Override
+ public void setName(final String name) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public String getComments() {
+ return "";
+ }
+
+ @Override
+ public void setComments(final String comments) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ProcessGroup getProcessGroup() {
+ return processGroupRef.get();
+ }
+
+ @Override
+ public void setProcessGroup(final ProcessGroup group) {
+ processGroupRef.set(group);
+ }
+
+ @Override
+ public boolean isAutoTerminated(Relationship relationship) {
+ return false;
+ }
+
+ @Override
+ public boolean isRunning() {
+ return isRunning(this);
+ }
+
+ private boolean isRunning(final Connectable source) {
+ return getScheduledState() == ScheduledState.RUNNING;
+ }
+
+ @Override
+ public boolean isTriggerWhenEmpty() {
+ return false;
+ }
+
+ @Override
+ public ScheduledState getScheduledState() {
+ return scheduledState.get();
+ }
+
+ @Override
+ public boolean isLossTolerant() {
+ return lossTolerant.get();
+ }
+
+ @Override
+ public void setLossTolerant(final boolean lossTolerant) {
+ this.lossTolerant.set(lossTolerant);
+ }
+
+ @Override
+ public String toString() {
+ return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE).append("id", getIdentifier()).toString();
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
+ final ProcessSession session = sessionFactory.createSession();
+
+ try {
+ onTrigger(context, session);
+ session.commit();
+ } catch (final ProcessException e) {
+ session.rollback();
+ throw e;
+ } catch (final Throwable t) {
+ session.rollback();
+ throw new RuntimeException(t);
+ }
+ }
+
+ private void onTrigger(final ProcessContext context, final ProcessSession session) {
+ readLock.lock();
+ try {
- Set<Relationship> available = session.getAvailableRelationships();
++ Set<Relationship> available = context.getAvailableRelationships();
+ int transferred = 0;
+ while (!available.isEmpty()) {
+ final List<FlowFile> flowFiles = session.get(10);
+ if (flowFiles.isEmpty()) {
+ break;
+ }
+
+ transferred += flowFiles.size();
+ session.transfer(flowFiles, Relationship.ANONYMOUS);
+ session.commit();
- available = session.getAvailableRelationships();
++ available = context.getAvailableRelationships();
+ }
+
+ if (transferred == 0) {
+ context.yield();
+ }
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ /**
+ * Has no effect
+ */
+ @Override
+ public void setMaxConcurrentTasks(int taskCount) {
+ }
+
+ @Override
+ public int getMaxConcurrentTasks() {
+ return 1;
+ }
+
+ @Override
+ public void setScheduledState(final ScheduledState scheduledState) {
+ this.scheduledState.set(scheduledState);
+ }
+
+ @Override
+ public ConnectableType getConnectableType() {
+ return ConnectableType.FUNNEL;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public Collection<ValidationResult> getValidationErrors() {
+ return Collections.EMPTY_LIST;
+ }
+
+ /**
+ * Updates the amount of time that this processor should avoid being
+ * scheduled when the processor calls
+ * {@link nifi.processor.ProcessContext#yield() ProcessContext.yield()}
+ *
+ * @param yieldPeriod
+ */
+ @Override
+ public void setYieldPeriod(final String yieldPeriod) {
+ final long yieldMillis = FormatUtils.getTimeDuration(requireNonNull(yieldPeriod), TimeUnit.MILLISECONDS);
+ if (yieldMillis < 0) {
+ throw new IllegalArgumentException("Yield duration must be positive");
+ }
+ this.yieldPeriod.set(yieldPeriod);
+ }
+
+ /**
+ * @param schedulingPeriod
+ */
+ @Override
+ public void setScheduldingPeriod(final String schedulingPeriod) {
+ final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS);
+ if (schedulingNanos < 0) {
+ throw new IllegalArgumentException("Scheduling Period must be positive");
+ }
+
+ this.schedulingPeriod.set(schedulingPeriod);
+ this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos));
+ }
+
+ @Override
+ public long getPenalizationPeriod(final TimeUnit timeUnit) {
+ return FormatUtils.getTimeDuration(getPenalizationPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
+ }
+
+ @Override
+ public String getPenalizationPeriod() {
+ return penalizationPeriod.get();
+ }
+
+ /**
+ * Causes the processor not to be scheduled for some period of time. This
+ * duration can be obtained and set via the
+ * {@link #getYieldPeriod(TimeUnit)} and
+ * {@link #setYieldPeriod(long, TimeUnit)} methods.
+ */
+ @Override
+ public void yield() {
+ final long yieldMillis = getYieldPeriod(TimeUnit.MILLISECONDS);
+ yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis));
+ }
+
+ @Override
+ public long getYieldExpiration() {
+ return yieldExpiration.get();
+ }
+
+ @Override
+ public String getSchedulingPeriod() {
+ return schedulingPeriod.get();
+ }
+
+ @Override
+ public void setPenalizationPeriod(final String penalizationPeriod) {
+ this.penalizationPeriod.set(penalizationPeriod);
+ }
+
+ @Override
+ public String getYieldPeriod() {
+ return yieldPeriod.get();
+ }
+
+ @Override
+ public long getYieldPeriod(final TimeUnit timeUnit) {
+ return FormatUtils.getTimeDuration(getYieldPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit);
+ }
+
+ @Override
+ public long getSchedulingPeriod(final TimeUnit timeUnit) {
+ return timeUnit.convert(schedulingNanos.get(), TimeUnit.NANOSECONDS);
+ }
+
+ @Override
+ public boolean isSideEffectFree() {
+ return true;
+ }
+
+ @Override
+ public void verifyCanDelete(boolean ignoreConnections) throws IllegalStateException {
+ if (ignoreConnections) {
+ return;
+ }
+
+ readLock.lock();
+ try {
+ for (final Connection connection : outgoingConnections) {
+ connection.verifyCanDelete();
+ }
+
+ for (final Connection connection : incomingConnections) {
+ if (connection.getSource().equals(this)) {
+ connection.verifyCanDelete();
+ } else {
+ throw new IllegalStateException(this + " is the destination of another component");
+ }
+ }
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ @Override
+ public void verifyCanDelete() {
+ verifyCanDelete(false);
+ }
+
+ @Override
+ public void verifyCanStart() {
+ }
+
+ @Override
+ public void verifyCanStop() {
+ }
+
+ @Override
+ public void verifyCanUpdate() {
+ }
+
+ @Override
+ public void verifyCanEnable() {
+ }
+
+ @Override
+ public void verifyCanDisable() {
+ }
+
+ @Override
+ public SchedulingStrategy getSchedulingStrategy() {
+ return SchedulingStrategy.TIMER_DRIVEN;
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
index 0000000,eae2550..d5dba82
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/BatchingSessionFactory.java
@@@ -1,0 -1,247 +1,242 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.controller.repository;
+
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.nio.file.Path;
+ import java.util.Collection;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.regex.Pattern;
+
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.processor.FlowFileFilter;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.QueueSize;
+ 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.io.StreamCallback;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+
+ public class BatchingSessionFactory implements ProcessSessionFactory {
+
+ private final HighThroughputSession highThroughputSession;
+
+ public BatchingSessionFactory(final StandardProcessSession standardProcessSession) {
+ highThroughputSession = new HighThroughputSession(standardProcessSession);
+ }
+
+ @Override
+ public ProcessSession createSession() {
+ return highThroughputSession;
+ }
+
+ private class HighThroughputSession implements ProcessSession {
+
+ private final StandardProcessSession session;
+
+ public HighThroughputSession(final StandardProcessSession session) {
+ this.session = session;
+ }
+
+ @Override
+ public void commit() {
+ session.checkpoint();
+ }
+
+ @Override
+ public void rollback() {
+ session.rollback();
+ }
+
+ @Override
+ public void rollback(boolean penalize) {
+ session.rollback(penalize);
+ }
+
+ @Override
+ public void adjustCounter(String name, long delta, boolean immediate) {
+ session.adjustCounter(name, delta, immediate);
+ }
+
+ @Override
+ public FlowFile get() {
+ return session.get();
+ }
+
+ @Override
+ public List<FlowFile> get(int maxResults) {
+ return session.get(maxResults);
+ }
+
+ @Override
+ public List<FlowFile> get(FlowFileFilter filter) {
+ return session.get(filter);
+ }
+
+ @Override
+ public QueueSize getQueueSize() {
+ return session.getQueueSize();
+ }
+
+ @Override
- public Set<Relationship> getAvailableRelationships() {
- return session.getAvailableRelationships();
- }
-
- @Override
+ public FlowFile create() {
+ return session.create();
+ }
+
+ @Override
+ public FlowFile create(FlowFile parent) {
+ return session.create(parent);
+ }
+
+ @Override
+ public FlowFile create(Collection<FlowFile> parents) {
+ return session.create(parents);
+ }
+
+ @Override
+ public FlowFile clone(FlowFile example) {
+ return session.clone(example);
+ }
+
+ @Override
+ public FlowFile clone(FlowFile example, long offset, long size) {
+ return session.clone(example, offset, size);
+ }
+
+ @Override
+ public FlowFile penalize(FlowFile flowFile) {
+ return session.penalize(flowFile);
+ }
+
+ @Override
+ public FlowFile putAttribute(FlowFile flowFile, String key, String value) {
+ return session.putAttribute(flowFile, key, value);
+ }
+
+ @Override
+ public FlowFile putAllAttributes(FlowFile flowFile, Map<String, String> attributes) {
+ return session.putAllAttributes(flowFile, attributes);
+ }
+
+ @Override
+ public FlowFile removeAttribute(FlowFile flowFile, String key) {
+ return session.removeAttribute(flowFile, key);
+ }
+
+ @Override
+ public FlowFile removeAllAttributes(FlowFile flowFile, Set<String> keys) {
+ return session.removeAllAttributes(flowFile, keys);
+ }
+
+ @Override
+ public FlowFile removeAllAttributes(FlowFile flowFile, Pattern keyPattern) {
+ return session.removeAllAttributes(flowFile, keyPattern);
+ }
+
+ @Override
+ public void transfer(FlowFile flowFile, Relationship relationship) {
+ session.transfer(flowFile, relationship);
+ }
+
+ @Override
+ public void transfer(FlowFile flowFile) {
+ session.transfer(flowFile);
+ }
+
+ @Override
+ public void transfer(Collection<FlowFile> flowFiles) {
+ session.transfer(flowFiles);
+ }
+
+ @Override
+ public void transfer(Collection<FlowFile> flowFiles, Relationship relationship) {
+ session.transfer(flowFiles, relationship);
+ }
+
+ @Override
+ public void remove(FlowFile flowFile) {
+ session.remove(flowFile);
+ }
+
+ @Override
+ public void remove(Collection<FlowFile> flowFiles) {
+ session.remove(flowFiles);
+ }
+
+ @Override
+ public void read(FlowFile source, InputStreamCallback reader) {
+ session.read(source, reader);
+ }
+
+ @Override
+ public FlowFile merge(Collection<FlowFile> sources, FlowFile destination) {
+ return session.merge(sources, destination);
+ }
+
+ @Override
+ public FlowFile merge(Collection<FlowFile> sources, FlowFile destination, byte[] header, byte[] footer, byte[] demarcator) {
+ return session.merge(sources, destination, header, footer, demarcator);
+ }
+
+ @Override
+ public FlowFile write(FlowFile source, OutputStreamCallback writer) {
+ return session.write(source, writer);
+ }
+
+ @Override
+ public FlowFile write(FlowFile source, StreamCallback writer) {
+ return session.write(source, writer);
+ }
+
+ @Override
+ public FlowFile append(FlowFile source, OutputStreamCallback writer) {
+ return session.append(source, writer);
+ }
+
+ @Override
+ public FlowFile importFrom(Path source, boolean keepSourceFile, FlowFile destination) {
+ return session.importFrom(source, keepSourceFile, destination);
+ }
+
+ @Override
+ public FlowFile importFrom(InputStream source, FlowFile destination) {
+ return session.importFrom(source, destination);
+ }
+
+ @Override
+ public void exportTo(FlowFile flowFile, Path destination, boolean append) {
+ session.exportTo(flowFile, destination, append);
+ }
+
+ @Override
+ public void exportTo(FlowFile flowFile, OutputStream destination) {
+ session.exportTo(flowFile, destination);
+ }
+
+ @Override
+ public ProvenanceReporter getProvenanceReporter() {
+ return session.getProvenanceReporter();
+ }
+
+ }
+
+ }
[8/8] incubator-nifi git commit: Merge branch 'develop' of
https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Posted by ma...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/73384b23
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/73384b23
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/73384b23
Branch: refs/heads/develop
Commit: 73384b23d1de7c814cd6b8089de7c893db057244
Parents: 3a7b8de db23609
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Jan 16 11:29:35 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Jan 16 11:29:35 2015 -0500
----------------------------------------------------------------------
assembly/pom.xml | 458 ---
assembly/src/main/assembly/dependencies.xml | 140 -
commons/data-provenance-utils/.gitignore | 2 -
commons/data-provenance-utils/pom.xml | 40 -
.../nifi/provenance/AsyncLineageSubmission.java | 87 -
.../nifi/provenance/AsyncQuerySubmission.java | 81 -
.../nifi/provenance/NamedSearchableField.java | 95 -
.../nifi/provenance/SearchableFieldParser.java | 53 -
.../nifi/provenance/SearchableFields.java | 84 -
.../nifi/provenance/StandardLineageResult.java | 324 --
.../StandardProvenanceEventRecord.java | 752 ----
.../nifi/provenance/StandardQueryResult.java | 168 -
.../nifi/provenance/lineage/EdgeNode.java | 74 -
.../nifi/provenance/lineage/EventNode.java | 109 -
.../provenance/lineage/FlowFileLineage.java | 76 -
.../nifi/provenance/lineage/FlowFileNode.java | 83 -
commons/flowfile-packager/pom.xml | 41 -
.../org/apache/nifi/util/FlowFilePackager.java | 28 -
.../apache/nifi/util/FlowFilePackagerV1.java | 104 -
.../apache/nifi/util/FlowFilePackagerV2.java | 146 -
.../apache/nifi/util/FlowFilePackagerV3.java | 93 -
.../apache/nifi/util/FlowFileUnpackager.java | 30 -
.../apache/nifi/util/FlowFileUnpackagerV1.java | 155 -
.../apache/nifi/util/FlowFileUnpackagerV2.java | 143 -
.../apache/nifi/util/FlowFileUnpackagerV3.java | 161 -
.../java/org/apache/nifi/util/Unpackage.java | 119 -
.../nifi/util/TestPackageUnpackageV3.java | 56 -
commons/nifi-expression-language/pom.xml | 56 -
.../language/antlr/AttributeExpressionLexer.g | 210 -
.../language/antlr/AttributeExpressionParser.g | 139 -
.../output/AttributeExpressionLexer.tokens | 88 -
.../expression/language/EmptyPreparedQuery.java | 62 -
.../language/InvalidPreparedQuery.java | 71 -
.../expression/language/PreparedQuery.java | 39 -
.../attribute/expression/language/Query.java | 1186 ------
.../language/StandardAttributeExpression.java | 65 -
.../StandardExpressionLanguageCompiler.java | 58 -
.../language/StandardPreparedQuery.java | 83 -
.../language/evaluation/BooleanEvaluator.java | 32 -
.../language/evaluation/BooleanQueryResult.java | 43 -
.../language/evaluation/DateEvaluator.java | 34 -
.../language/evaluation/DateQueryResult.java | 45 -
.../language/evaluation/Evaluator.java | 32 -
.../language/evaluation/NumberEvaluator.java | 33 -
.../language/evaluation/NumberQueryResult.java | 43 -
.../language/evaluation/QueryResult.java | 26 -
.../language/evaluation/StringEvaluator.java | 32 -
.../language/evaluation/StringQueryResult.java | 43 -
.../evaluation/cast/BooleanCastEvaluator.java | 50 -
.../evaluation/cast/DateCastEvaluator.java | 117 -
.../evaluation/cast/NumberCastEvaluator.java | 72 -
.../evaluation/cast/StringCastEvaluator.java | 49 -
.../evaluation/functions/AndEvaluator.java | 60 -
.../evaluation/functions/AppendEvaluator.java | 50 -
.../functions/AttributeEvaluator.java | 45 -
.../evaluation/functions/ContainsEvaluator.java | 53 -
.../functions/DateToNumberEvaluator.java | 50 -
.../evaluation/functions/DivideEvaluator.java | 57 -
.../evaluation/functions/EndsWithEvaluator.java | 53 -
.../evaluation/functions/EqualsEvaluator.java | 89 -
.../functions/EqualsIgnoreCaseEvaluator.java | 60 -
.../evaluation/functions/FindEvaluator.java | 72 -
.../evaluation/functions/FormatEvaluator.java | 61 -
.../functions/GreaterThanEvaluator.java | 60 -
.../functions/GreaterThanOrEqualEvaluator.java | 60 -
.../evaluation/functions/HostnameEvaluator.java | 58 -
.../evaluation/functions/IPEvaluator.java | 46 -
.../evaluation/functions/IndexOfEvaluator.java | 53 -
.../evaluation/functions/IsEmptyEvaluator.java | 43 -
.../evaluation/functions/IsNullEvaluator.java | 45 -
.../functions/LastIndexOfEvaluator.java | 53 -
.../evaluation/functions/LengthEvaluator.java | 46 -
.../evaluation/functions/LessThanEvaluator.java | 60 -
.../functions/LessThanOrEqualEvaluator.java | 60 -
.../evaluation/functions/MatchesEvaluator.java | 71 -
.../evaluation/functions/MinusEvaluator.java | 57 -
.../evaluation/functions/ModEvaluator.java | 57 -
.../evaluation/functions/MultiplyEvaluator.java | 57 -
.../evaluation/functions/NotEvaluator.java | 49 -
.../evaluation/functions/NotNullEvaluator.java | 45 -
.../evaluation/functions/NowEvaluator.java | 39 -
.../functions/NumberToDateEvaluator.java | 52 -
.../functions/OneUpSequenceEvaluator.java | 41 -
.../evaluation/functions/OrEvaluator.java | 60 -
.../evaluation/functions/PlusEvaluator.java | 57 -
.../evaluation/functions/PrependEvaluator.java | 50 -
.../functions/ReplaceAllEvaluator.java | 55 -
.../functions/ReplaceEmptyEvaluator.java | 50 -
.../evaluation/functions/ReplaceEvaluator.java | 55 -
.../functions/ReplaceNullEvaluator.java | 47 -
.../functions/StartsWithEvaluator.java | 53 -
.../functions/StringToDateEvaluator.java | 65 -
.../functions/SubstringAfterEvaluator.java | 59 -
.../functions/SubstringAfterLastEvaluator.java | 55 -
.../functions/SubstringBeforeEvaluator.java | 58 -
.../functions/SubstringBeforeLastEvaluator.java | 55 -
.../functions/SubstringEvaluator.java | 65 -
.../evaluation/functions/ToLowerEvaluator.java | 45 -
.../evaluation/functions/ToNumberEvaluator.java | 46 -
.../evaluation/functions/ToRadixEvaluator.java | 77 -
.../evaluation/functions/ToStringEvaluator.java | 45 -
.../evaluation/functions/ToUpperEvaluator.java | 45 -
.../evaluation/functions/TrimEvaluator.java | 45 -
.../functions/UrlDecodeEvaluator.java | 55 -
.../functions/UrlEncodeEvaluator.java | 55 -
.../evaluation/functions/UuidEvaluator.java | 39 -
.../literals/BooleanLiteralEvaluator.java | 44 -
.../literals/NumberLiteralEvaluator.java | 44 -
.../literals/StringLiteralEvaluator.java | 77 -
.../evaluation/reduce/CountEvaluator.java | 56 -
.../evaluation/reduce/JoinEvaluator.java | 59 -
.../evaluation/reduce/ReduceEvaluator.java | 23 -
.../selection/AllAttributesEvaluator.java | 68 -
.../selection/AnyAttributeEvaluator.java | 68 -
.../AnyMatchingAttributeEvaluator.java | 21 -
.../selection/DelineatedAttributeEvaluator.java | 83 -
.../evaluation/selection/MappingEvaluator.java | 61 -
.../selection/MultiAttributeEvaluator.java | 24 -
.../selection/MultiMatchAttributeEvaluator.java | 82 -
.../selection/MultiNamedAttributeEvaluator.java | 64 -
.../AttributeExpressionLanguageException.java | 34 -
...ibuteExpressionLanguageParsingException.java | 34 -
.../exception/IllegalAttributeException.java | 29 -
.../expression/language/TestQuery.java | 1143 ------
.../language/TestStandardPreparedQuery.java | 92 -
commons/nifi-logging-utils/pom.xml | 36 -
.../java/org/apache/nifi/logging/NiFiLog.java | 367 --
commons/nifi-properties/.gitignore | 3 -
commons/nifi-properties/pom.xml | 29 -
.../org/apache/nifi/util/NiFiProperties.java | 876 -----
.../java/org/apache/nifi/util/StringUtils.java | 66 -
commons/nifi-security-utils/pom.xml | 40 -
.../nifi/security/util/CertificateUtils.java | 158 -
.../nifi/security/util/EncryptionMethod.java | 84 -
.../apache/nifi/security/util/KeystoreType.java | 26 -
.../nifi/security/util/SecurityStoreTypes.java | 144 -
.../nifi/security/util/SslContextFactory.java | 180 -
commons/nifi-socket-utils/pom.xml | 60 -
.../nifi/io/nio/AbstractChannelReader.java | 166 -
.../java/org/apache/nifi/io/nio/BufferPool.java | 114 -
.../apache/nifi/io/nio/ChannelDispatcher.java | 160 -
.../org/apache/nifi/io/nio/ChannelListener.java | 228 --
.../nifi/io/nio/DatagramChannelReader.java | 59 -
.../apache/nifi/io/nio/SocketChannelReader.java | 55 -
.../io/nio/consumer/AbstractStreamConsumer.java | 132 -
.../nifi/io/nio/consumer/StreamConsumer.java | 80 -
.../io/nio/consumer/StreamConsumerFactory.java | 27 -
.../nifi/io/socket/SSLContextFactory.java | 102 -
.../io/socket/ServerSocketConfiguration.java | 83 -
.../nifi/io/socket/SocketConfiguration.java | 116 -
.../apache/nifi/io/socket/SocketListener.java | 211 -
.../org/apache/nifi/io/socket/SocketUtils.java | 169 -
.../socket/multicast/DiscoverableService.java | 43 -
.../multicast/DiscoverableServiceImpl.java | 78 -
.../multicast/MulticastConfiguration.java | 99 -
.../io/socket/multicast/MulticastListener.java | 193 -
.../multicast/MulticastServiceDiscovery.java | 34 -
.../multicast/MulticastServicesBroadcaster.java | 33 -
.../socket/multicast/MulticastTimeToLive.java | 50 -
.../io/socket/multicast/MulticastUtils.java | 109 -
.../io/socket/multicast/ServiceDiscovery.java | 31 -
.../socket/multicast/ServicesBroadcaster.java | 56 -
.../apache/nifi/io/nio/example/ServerMain.java | 141 -
.../apache/nifi/io/nio/example/TCPClient.java | 86 -
.../apache/nifi/io/nio/example/UDPClient.java | 51 -
.../io/nio/example/UselessStreamConsumer.java | 43 -
.../src/test/resources/log4j.xml | 36 -
commons/nifi-utils/.gitignore | 8 -
commons/nifi-utils/pom.xml | 33 -
.../flowfile/attributes/CoreAttributes.java | 72 -
.../attributes/FlowFileAttributeKey.java | 21 -
.../nifi/remote/StandardVersionNegotiator.java | 81 -
.../apache/nifi/remote/VersionNegotiator.java | 65 -
.../TransmissionDisabledException.java | 25 -
.../nifi/remote/io/CompressionInputStream.java | 184 -
.../nifi/remote/io/CompressionOutputStream.java | 147 -
.../remote/io/InterruptableInputStream.java | 117 -
.../remote/io/InterruptableOutputStream.java | 81 -
.../remote/io/socket/BufferStateManager.java | 111 -
.../io/socket/SocketChannelInputStream.java | 157 -
.../io/socket/SocketChannelOutputStream.java | 113 -
.../remote/io/socket/ssl/SSLSocketChannel.java | 602 ---
.../socket/ssl/SSLSocketChannelInputStream.java | 62 -
.../ssl/SSLSocketChannelOutputStream.java | 53 -
.../nifi/stream/io/BufferedInputStream.java | 37 -
.../nifi/stream/io/BufferedOutputStream.java | 140 -
.../nifi/stream/io/ByteArrayInputStream.java | 250 --
.../nifi/stream/io/ByteArrayOutputStream.java | 250 --
.../nifi/stream/io/ByteCountingInputStream.java | 104 -
.../stream/io/ByteCountingOutputStream.java | 63 -
.../apache/nifi/stream/io/DataOutputStream.java | 417 --
.../apache/nifi/stream/io/GZIPOutputStream.java | 41 -
.../stream/io/LeakyBucketStreamThrottler.java | 331 --
.../nifi/stream/io/NonCloseableInputStream.java | 56 -
.../stream/io/NonCloseableOutputStream.java | 51 -
.../apache/nifi/stream/io/NullOutputStream.java | 46 -
.../apache/nifi/stream/io/StreamThrottler.java | 33 -
.../org/apache/nifi/stream/io/StreamUtils.java | 257 --
.../apache/nifi/stream/io/ZipOutputStream.java | 38 -
.../exception/BytePatternNotFoundException.java | 28 -
.../io/util/NonThreadSafeCircularBuffer.java | 69 -
.../org/apache/nifi/util/BooleanHolder.java | 25 -
.../java/org/apache/nifi/util/FormatUtils.java | 205 -
.../org/apache/nifi/util/IntegerHolder.java | 54 -
.../java/org/apache/nifi/util/LongHolder.java | 60 -
.../apache/nifi/util/NaiveSearchRingBuffer.java | 135 -
.../java/org/apache/nifi/util/ObjectHolder.java | 39 -
.../java/org/apache/nifi/util/RingBuffer.java | 292 --
.../java/org/apache/nifi/util/StopWatch.java | 127 -
.../main/java/org/apache/nifi/util/Tuple.java | 83 -
.../concurrency/DebugDisabledTimedLock.java | 67 -
.../util/concurrency/DebugEnabledTimedLock.java | 136 -
.../util/concurrency/DebuggableTimedLock.java | 30 -
.../apache/nifi/util/concurrency/TimedLock.java | 59 -
.../org/apache/nifi/util/file/FileUtils.java | 623 ---
.../file/monitor/CompoundUpdateMonitor.java | 115 -
.../util/file/monitor/LastModifiedMonitor.java | 30 -
.../nifi/util/file/monitor/MD5SumMonitor.java | 51 -
.../file/monitor/SynchronousFileWatcher.java | 123 -
.../nifi/util/file/monitor/UpdateMonitor.java | 25 -
.../org/apache/nifi/util/search/Search.java | 57 -
.../org/apache/nifi/util/search/SearchTerm.java | 141 -
.../util/search/ahocorasick/AhoCorasick.java | 155 -
.../nifi/util/search/ahocorasick/Node.java | 72 -
.../util/search/ahocorasick/SearchState.java | 63 -
.../nifi/util/timebuffer/EntityAccess.java | 26 -
.../nifi/util/timebuffer/LongEntityAccess.java | 43 -
.../nifi/util/timebuffer/TimedBuffer.java | 114 -
.../nifi/util/timebuffer/TimestampedLong.java | 35 -
.../io/TestCompressionInputOutputStreams.java | 153 -
.../stream/io/TestLeakyBucketThrottler.java | 147 -
.../nifi/util/TestNaiveSearchRingBuffer.java | 72 -
.../file/monitor/TestCompoundUpdateMonitor.java | 71 -
.../monitor/TestSynchronousFileWatcher.java | 61 -
.../nifi/util/timebuffer/TestRingBuffer.java | 182 -
.../nifi/util/timebuffer/TestTimedBuffer.java | 106 -
.../src/test/resources/logback-test.xml | 32 -
commons/nifi-web-utils/pom.xml | 56 -
.../org/apache/nifi/web/util/ClientUtils.java | 132 -
.../nifi/web/util/ObjectMapperResolver.java | 48 -
.../java/org/apache/nifi/web/util/WebUtils.java | 198 -
commons/pom.xml | 44 -
commons/processor-utilities/pom.xml | 45 -
.../nifi/processor/util/FlowFileFilters.java | 65 -
.../nifi/processor/util/SSLProperties.java | 226 --
.../nifi/processor/util/StandardValidators.java | 544 ---
.../apache/nifi/processor/TestFormatUtils.java | 40 -
.../processor/util/TestStandardValidators.java | 54 -
commons/wali/.gitignore | 2 -
commons/wali/pom.xml | 41 -
.../org/wali/MinimalLockingWriteAheadLog.java | 1008 -----
commons/wali/src/main/java/org/wali/SerDe.java | 128 -
.../src/main/java/org/wali/SyncListener.java | 62 -
.../wali/src/main/java/org/wali/UpdateType.java | 49 -
.../java/org/wali/WriteAheadRepository.java | 122 -
.../src/test/java/org/wali/DummyRecord.java | 61 -
.../test/java/org/wali/DummyRecordSerde.java | 107 -
.../wali/TestMinimalLockingWriteAheadLog.java | 298 --
maven-plugins/nar-maven-plugin/pom.xml | 75 +
.../src/main/java/nifi/NarMojo.java | 613 +++
.../resources/META-INF/plexus/components.xml | 52 +
maven-plugins/pom.xml | 346 ++
.../execute-script-processors/pom.xml | 81 -
.../nifi/processors/script/ExecuteScript.java | 566 ---
.../apache/nifi/scripting/ConverterScript.java | 131 -
.../nifi/scripting/JRubyScriptFactory.java | 46 -
.../nifi/scripting/JavaScriptScriptFactory.java | 56 -
.../nifi/scripting/JythonScriptFactory.java | 45 -
.../nifi/scripting/OutputStreamHandler.java | 24 -
.../org/apache/nifi/scripting/ReaderScript.java | 79 -
.../java/org/apache/nifi/scripting/Script.java | 303 --
.../nifi/scripting/ScriptEngineFactory.java | 117 -
.../apache/nifi/scripting/ScriptFactory.java | 269 --
.../org/apache/nifi/scripting/WriterScript.java | 67 -
.../org.apache.nifi.processor.Processor | 15 -
.../index.html | 264 --
.../processors/script/TestExecuteScript.java | 939 -----
.../src/test/resources/alwaysFail.js | 24 -
.../src/test/resources/alwaysFail.py | 19 -
.../src/test/resources/alwaysFail.rb | 21 -
.../src/test/resources/ffTest.js | 28 -
.../src/test/resources/ffTest.py | 22 -
.../src/test/resources/ffTest.rb | 30 -
.../src/test/resources/lib/Sub.py | 18 -
.../src/test/resources/lib/sub.js | 22 -
.../src/test/resources/lib/sub.rb | 17 -
.../src/test/resources/loadLocal.js | 30 -
.../src/test/resources/loadLocal.py | 26 -
.../src/test/resources/loadLocal.rb | 29 -
.../src/test/resources/log4j.xml | 54 -
.../src/test/resources/optionalValidators.js | 28 -
.../src/test/resources/optionalValidators.py | 22 -
.../src/test/resources/optionalValidators.rb | 39 -
.../src/test/resources/paramTest.js | 28 -
.../src/test/resources/paramTest.py | 26 -
.../src/test/resources/paramTest.rb | 31 -
.../src/test/resources/parseXml.js | 36 -
.../src/test/resources/readTest.js | 30 -
.../src/test/resources/readTest.py | 32 -
.../src/test/resources/readTest.rb | 30 -
.../src/test/resources/readWithParams.js | 32 -
.../src/test/resources/readWithParams.py | 32 -
.../src/test/resources/readWithParams.rb | 33 -
.../src/test/resources/routeTest.js | 41 -
.../src/test/resources/routeTest.py | 37 -
.../src/test/resources/routeTest.rb | 39 -
.../src/test/resources/simpleConverter.js | 45 -
.../src/test/resources/simpleConverter.py | 60 -
.../src/test/resources/simpleConverter.rb | 42 -
.../src/test/resources/writeTest.js | 26 -
.../src/test/resources/writeTest.py | 22 -
.../src/test/resources/writeTest.rb | 32 -
nar-bundles/execute-script-bundle/nar/pom.xml | 36 -
nar-bundles/execute-script-bundle/pom.xml | 81 -
.../framework/administration/.gitignore | 1 -
.../framework/administration/pom.xml | 116 -
.../nifi/admin/AuditDataSourceFactoryBean.java | 222 --
.../org/apache/nifi/admin/RepositoryUtils.java | 91 -
.../nifi/admin/UserDataSourceFactoryBean.java | 247 --
.../org/apache/nifi/admin/dao/ActionDAO.java | 74 -
.../org/apache/nifi/admin/dao/AuthorityDAO.java | 58 -
.../org/apache/nifi/admin/dao/DAOFactory.java | 29 -
.../nifi/admin/dao/DataAccessException.java | 39 -
.../java/org/apache/nifi/admin/dao/UserDAO.java | 127 -
.../nifi/admin/dao/impl/DAOFactoryImpl.java | 51 -
.../nifi/admin/dao/impl/StandardActionDAO.java | 1056 -----
.../admin/dao/impl/StandardAuthorityDAO.java | 172 -
.../nifi/admin/dao/impl/StandardUserDAO.java | 634 ---
.../admin/service/AccountDisabledException.java | 40 -
.../admin/service/AccountNotFoundException.java | 40 -
.../admin/service/AccountPendingException.java | 41 -
.../admin/service/AdministrationException.java | 39 -
.../apache/nifi/admin/service/AuditService.java | 76 -
.../apache/nifi/admin/service/UserService.java | 161 -
.../service/action/AbstractUserAction.java | 97 -
.../admin/service/action/AddActionsAction.java | 48 -
.../service/action/AdministrationAction.java | 38 -
.../service/action/AuthorizeDownloadAction.java | 54 -
.../service/action/AuthorizeUserAction.java | 175 -
.../admin/service/action/CreateUserAction.java | 53 -
.../admin/service/action/DeleteUserAction.java | 68 -
.../admin/service/action/DisableUserAction.java | 76 -
.../service/action/DisableUserGroupAction.java | 69 -
.../service/action/FindUserByDnAction.java | 49 -
.../service/action/FindUserByIdAction.java | 49 -
.../admin/service/action/GetActionAction.java | 41 -
.../admin/service/action/GetActionsAction.java | 48 -
.../admin/service/action/GetPreviousValues.java | 43 -
.../service/action/GetUserGroupAction.java | 50 -
.../admin/service/action/GetUsersAction.java | 39 -
.../service/action/HasPendingUserAccounts.java | 34 -
.../action/InvalidateUserAccountAction.java | 58 -
.../InvalidateUserGroupAccountsAction.java | 45 -
.../service/action/PurgeActionsAction.java | 51 -
.../action/RequestUserAccountAction.java | 67 -
.../service/action/SeedUserAccountsAction.java | 164 -
.../admin/service/action/UngroupUserAction.java | 69 -
.../service/action/UngroupUserGroupAction.java | 57 -
.../admin/service/action/UpdateUserAction.java | 124 -
.../UpdateUserAuthoritiesCacheAction.java | 73 -
.../service/action/UpdateUserCacheAction.java | 47 -
.../service/action/UpdateUserGroupAction.java | 171 -
.../service/impl/StandardAuditService.java | 230 --
.../admin/service/impl/StandardUserService.java | 663 ----
.../admin/service/transaction/Transaction.java | 49 -
.../service/transaction/TransactionBuilder.java | 25 -
.../transaction/TransactionException.java | 40 -
.../transaction/impl/StandardTransaction.java | 93 -
.../impl/StandardTransactionBuilder.java | 57 -
.../AuthorityProviderFactoryBean.java | 529 ---
...rdAuthorityProviderConfigurationContext.java | 50 -
...dAuthorityProviderInitializationContext.java | 42 -
.../java/org/apache/nifi/history/History.java | 56 -
.../org/apache/nifi/history/HistoryQuery.java | 99 -
.../org/apache/nifi/history/PreviousValue.java | 54 -
.../org/apache/nifi/user/AccountStatus.java | 47 -
.../java/org/apache/nifi/user/NiFiUser.java | 164 -
.../org/apache/nifi/user/NiFiUserGroup.java | 45 -
.../resources/nifi-administration-context.xml | 62 -
.../src/main/xsd/authority-providers.xsd | 49 -
.../service/action/AuthorizeUserActionTest.java | 433 ---
.../service/action/CreateUserActionTest.java | 144 -
.../service/action/DisableUserActionTest.java | 171 -
.../action/InvalidateUserAccountActionTest.java | 131 -
.../action/RequestUserAccountActionTest.java | 127 -
.../action/SeedUserAccountsActionTest.java | 263 --
.../action/SetUserAuthoritiesActionTest.java | 223 --
.../framework/client-dto/.gitignore | 6 -
.../framework/client-dto/pom.xml | 46 -
.../org/apache/nifi/web/api/dto/AboutDTO.java | 57 -
.../org/apache/nifi/web/api/dto/BannerDTO.java | 57 -
.../nifi/web/api/dto/BulletinBoardDTO.java | 63 -
.../apache/nifi/web/api/dto/BulletinDTO.java | 161 -
.../nifi/web/api/dto/BulletinQueryDTO.java | 113 -
.../org/apache/nifi/web/api/dto/ClusterDTO.java | 61 -
.../apache/nifi/web/api/dto/ConnectableDTO.java | 146 -
.../apache/nifi/web/api/dto/ConnectionDTO.java | 215 --
.../web/api/dto/ControllerConfigurationDTO.java | 158 -
.../apache/nifi/web/api/dto/ControllerDTO.java | 262 --
.../org/apache/nifi/web/api/dto/CounterDTO.java | 94 -
.../apache/nifi/web/api/dto/CountersDTO.java | 61 -
.../nifi/web/api/dto/DocumentedTypeDTO.java | 71 -
.../apache/nifi/web/api/dto/FlowSnippetDTO.java | 141 -
.../org/apache/nifi/web/api/dto/FunnelDTO.java | 29 -
.../org/apache/nifi/web/api/dto/LabelDTO.java | 94 -
.../nifi/web/api/dto/NiFiComponentDTO.java | 95 -
.../org/apache/nifi/web/api/dto/NodeDTO.java | 188 -
.../apache/nifi/web/api/dto/NodeEventDTO.java | 74 -
.../web/api/dto/NodeSystemDiagnosticsDTO.java | 56 -
.../org/apache/nifi/web/api/dto/PortDTO.java | 161 -
.../apache/nifi/web/api/dto/PositionDTO.java | 65 -
.../nifi/web/api/dto/PreviousValueDTO.java | 73 -
.../nifi/web/api/dto/ProcessGroupDTO.java | 219 --
.../nifi/web/api/dto/ProcessorConfigDTO.java | 486 ---
.../apache/nifi/web/api/dto/ProcessorDTO.java | 181 -
.../nifi/web/api/dto/ProcessorHistoryDTO.java | 56 -
.../nifi/web/api/dto/PropertyHistoryDTO.java | 43 -
.../nifi/web/api/dto/RelationshipDTO.java | 69 -
.../api/dto/RemoteProcessGroupContentsDTO.java | 56 -
.../nifi/web/api/dto/RemoteProcessGroupDTO.java | 279 --
.../web/api/dto/RemoteProcessGroupPortDTO.java | 192 -
.../apache/nifi/web/api/dto/RevisionDTO.java | 63 -
.../org/apache/nifi/web/api/dto/SnippetDTO.java | 239 --
.../nifi/web/api/dto/SystemDiagnosticsDTO.java | 461 ---
.../apache/nifi/web/api/dto/TemplateDTO.java | 117 -
.../org/apache/nifi/web/api/dto/UserDTO.java | 177 -
.../apache/nifi/web/api/dto/UserGroupDTO.java | 84 -
.../nifi/web/api/dto/action/ActionDTO.java | 176 -
.../nifi/web/api/dto/action/HistoryDTO.java | 74 -
.../web/api/dto/action/HistoryQueryDTO.java | 144 -
.../component/details/ComponentDetailsDTO.java | 32 -
.../component/details/ProcessorDetailsDTO.java | 41 -
.../details/RemoteProcessGroupDetailsDTO.java | 41 -
.../dto/action/details/ActionDetailsDTO.java | 34 -
.../dto/action/details/ConfigureDetailsDTO.java | 69 -
.../dto/action/details/ConnectDetailsDTO.java | 126 -
.../api/dto/action/details/MoveDetailsDTO.java | 83 -
.../api/dto/action/details/PurgeDetailsDTO.java | 45 -
.../web/api/dto/provenance/AttributeDTO.java | 69 -
.../web/api/dto/provenance/ProvenanceDTO.java | 165 -
.../api/dto/provenance/ProvenanceEventDTO.java | 630 ---
.../dto/provenance/ProvenanceOptionsDTO.java | 43 -
.../dto/provenance/ProvenanceRequestDTO.java | 118 -
.../dto/provenance/ProvenanceResultsDTO.java | 136 -
.../ProvenanceSearchableFieldDTO.java | 84 -
.../api/dto/provenance/lineage/LineageDTO.java | 161 -
.../provenance/lineage/LineageRequestDTO.java | 88 -
.../provenance/lineage/LineageResultsDTO.java | 73 -
.../provenance/lineage/ProvenanceLinkDTO.java | 101 -
.../provenance/lineage/ProvenanceNodeDTO.java | 162 -
.../dto/search/ComponentSearchResultDTO.java | 85 -
.../web/api/dto/search/NodeSearchResultDTO.java | 56 -
.../web/api/dto/search/SearchResultsDTO.java | 128 -
.../dto/search/UserGroupSearchResultDTO.java | 42 -
.../web/api/dto/search/UserSearchResultDTO.java | 56 -
.../dto/status/ClusterConnectionStatusDTO.java | 89 -
.../api/dto/status/ClusterPortStatusDTO.java | 89 -
.../dto/status/ClusterProcessorStatusDTO.java | 117 -
.../ClusterRemoteProcessGroupStatusDTO.java | 89 -
.../web/api/dto/status/ClusterStatusDTO.java | 44 -
.../api/dto/status/ClusterStatusHistoryDTO.java | 75 -
.../web/api/dto/status/ConnectionStatusDTO.java | 198 -
.../web/api/dto/status/ControllerStatusDTO.java | 187 -
.../api/dto/status/NodeConnectionStatusDTO.java | 57 -
.../web/api/dto/status/NodePortStatusDTO.java | 57 -
.../api/dto/status/NodeProcessorStatusDTO.java | 57 -
.../status/NodeRemoteProcessGroupStatusDTO.java | 57 -
.../nifi/web/api/dto/status/NodeStatusDTO.java | 57 -
.../api/dto/status/NodeStatusHistoryDTO.java | 57 -
.../nifi/web/api/dto/status/PortStatusDTO.java | 142 -
.../api/dto/status/ProcessGroupStatusDTO.java | 244 --
.../web/api/dto/status/ProcessorStatusDTO.java | 203 -
.../web/api/dto/status/RemotePortStatusDTO.java | 98 -
.../dto/status/RemoteProcessGroupStatusDTO.java | 159 -
.../nifi/web/api/dto/status/StatusDTO.java | 43 -
.../web/api/dto/status/StatusDescriptorDTO.java | 101 -
.../web/api/dto/status/StatusHistoryDTO.java | 92 -
.../api/dto/status/StatusHistoryDetailDTO.java | 56 -
.../web/api/dto/status/StatusSnapshotDTO.java | 58 -
.../nifi/web/api/dto/util/DateTimeAdapter.java | 47 -
.../nifi/web/api/dto/util/TimeAdapter.java | 47 -
.../nifi/web/api/dto/util/TimestampAdapter.java | 47 -
.../apache/nifi/web/api/entity/AboutEntity.java | 45 -
.../nifi/web/api/entity/ActionEntity.java | 45 -
.../nifi/web/api/entity/AuthorityEntity.java | 60 -
.../nifi/web/api/entity/BannerEntity.java | 46 -
.../web/api/entity/BulletinBoardEntity.java | 45 -
.../entity/ClusterConnectionStatusEntity.java | 45 -
.../nifi/web/api/entity/ClusterEntity.java | 45 -
.../web/api/entity/ClusterPortStatusEntity.java | 45 -
.../entity/ClusterProcessorStatusEntity.java | 45 -
.../ClusterRemoteProcessGroupStatusEntity.java | 45 -
.../api/entity/ClusterSearchResultsEntity.java | 46 -
.../web/api/entity/ClusterStatusEntity.java | 45 -
.../api/entity/ClusterStatusHistoryEntity.java | 45 -
.../nifi/web/api/entity/ConnectionEntity.java | 45 -
.../nifi/web/api/entity/ConnectionsEntity.java | 47 -
.../entity/ControllerConfigurationEntity.java | 45 -
.../nifi/web/api/entity/ControllerEntity.java | 45 -
.../web/api/entity/ControllerStatusEntity.java | 45 -
.../nifi/web/api/entity/CounterEntity.java | 45 -
.../nifi/web/api/entity/CountersEntity.java | 46 -
.../org/apache/nifi/web/api/entity/Entity.java | 43 -
.../nifi/web/api/entity/FlowSnippetEntity.java | 45 -
.../nifi/web/api/entity/FunnelEntity.java | 45 -
.../nifi/web/api/entity/FunnelsEntity.java | 47 -
.../nifi/web/api/entity/HistoryEntity.java | 45 -
.../nifi/web/api/entity/InputPortEntity.java | 45 -
.../nifi/web/api/entity/InputPortsEntity.java | 47 -
.../apache/nifi/web/api/entity/LabelEntity.java | 45 -
.../nifi/web/api/entity/LabelsEntity.java | 47 -
.../nifi/web/api/entity/LineageEntity.java | 45 -
.../apache/nifi/web/api/entity/NodeEntity.java | 45 -
.../nifi/web/api/entity/NodeStatusEntity.java | 45 -
.../api/entity/NodeSystemDiagnosticsEntity.java | 45 -
.../nifi/web/api/entity/OutputPortEntity.java | 45 -
.../nifi/web/api/entity/OutputPortsEntity.java | 47 -
.../web/api/entity/PrioritizerTypesEntity.java | 46 -
.../nifi/web/api/entity/ProcessGroupEntity.java | 45 -
.../api/entity/ProcessGroupStatusEntity.java | 45 -
.../web/api/entity/ProcessGroupsEntity.java | 46 -
.../nifi/web/api/entity/ProcessorEntity.java | 45 -
.../web/api/entity/ProcessorHistoryEntity.java | 45 -
.../web/api/entity/ProcessorTypesEntity.java | 46 -
.../nifi/web/api/entity/ProcessorsEntity.java | 47 -
.../nifi/web/api/entity/ProvenanceEntity.java | 40 -
.../web/api/entity/ProvenanceEventEntity.java | 45 -
.../web/api/entity/ProvenanceOptionsEntity.java | 46 -
.../api/entity/RemoteProcessGroupEntity.java | 45 -
.../entity/RemoteProcessGroupPortEntity.java | 45 -
.../api/entity/RemoteProcessGroupsEntity.java | 47 -
.../web/api/entity/SearchResultsEntity.java | 46 -
.../nifi/web/api/entity/SnippetEntity.java | 45 -
.../web/api/entity/StatusHistoryEntity.java | 45 -
.../web/api/entity/SystemDiagnosticsEntity.java | 45 -
.../nifi/web/api/entity/TemplateEntity.java | 45 -
.../nifi/web/api/entity/TemplatesEntity.java | 63 -
.../apache/nifi/web/api/entity/UserEntity.java | 45 -
.../nifi/web/api/entity/UserGroupEntity.java | 45 -
.../web/api/entity/UserSearchResultsEntity.java | 61 -
.../apache/nifi/web/api/entity/UsersEntity.java | 64 -
.../cluster-authorization-provider/.gitignore | 1 -
.../cluster-authorization-provider/pom.xml | 48 -
.../ClusterManagerAuthorizationProvider.java | 225 --
.../NodeAuthorizationProvider.java | 389 --
.../protocol/message/DoesDnExistMessage.java | 56 -
.../protocol/message/GetAuthoritiesMessage.java | 58 -
.../message/GetGroupForUserMessage.java | 55 -
.../protocol/message/ProtocolMessage.java | 57 -
.../message/jaxb/JaxbProtocolUtils.java | 42 -
.../protocol/message/jaxb/ObjectFactory.java | 45 -
....apache.nifi.authorization.AuthorityProvider | 16 -
.../framework/cluster-protocol/.gitignore | 1 -
.../framework/cluster-protocol/pom.xml | 69 -
.../protocol/ClusterManagerProtocolSender.java | 69 -
.../cluster/protocol/ConnectionRequest.java | 44 -
.../cluster/protocol/ConnectionResponse.java | 141 -
.../apache/nifi/cluster/protocol/Heartbeat.java | 68 -
.../nifi/cluster/protocol/NodeBulletins.java | 44 -
.../nifi/cluster/protocol/NodeIdentifier.java | 172 -
.../cluster/protocol/NodeProtocolSender.java | 73 -
.../nifi/cluster/protocol/ProtocolContext.java | 39 -
.../cluster/protocol/ProtocolException.java | 40 -
.../nifi/cluster/protocol/ProtocolHandler.java | 44 -
.../nifi/cluster/protocol/ProtocolListener.java | 72 -
.../protocol/ProtocolMessageMarshaller.java | 38 -
.../protocol/ProtocolMessageUnmarshaller.java | 38 -
.../nifi/cluster/protocol/StandardDataFlow.java | 105 -
.../UnknownServiceAddressException.java | 39 -
.../impl/ClusterManagerProtocolSenderImpl.java | 245 --
.../ClusterManagerProtocolSenderListener.java | 118 -
.../protocol/impl/ClusterServiceDiscovery.java | 181 -
.../protocol/impl/ClusterServiceLocator.java | 229 --
.../impl/ClusterServicesBroadcaster.java | 182 -
.../protocol/impl/CopyingInputStream.java | 77 -
.../impl/MulticastProtocolListener.java | 204 -
.../protocol/impl/NodeProtocolSenderImpl.java | 171 -
.../impl/NodeProtocolSenderListener.java | 115 -
.../protocol/impl/SocketProtocolListener.java | 205 -
.../protocol/jaxb/JaxbProtocolContext.java | 148 -
.../jaxb/message/AdaptedConnectionRequest.java | 40 -
.../jaxb/message/AdaptedConnectionResponse.java | 109 -
.../protocol/jaxb/message/AdaptedCounter.java | 56 -
.../protocol/jaxb/message/AdaptedDataFlow.java | 64 -
.../protocol/jaxb/message/AdaptedHeartbeat.java | 66 -
.../jaxb/message/AdaptedNodeBulletins.java | 50 -
.../jaxb/message/AdaptedNodeIdentifier.java | 76 -
.../jaxb/message/ConnectionRequestAdapter.java | 41 -
.../jaxb/message/ConnectionResponseAdapter.java | 55 -
.../protocol/jaxb/message/DataFlowAdapter.java | 50 -
.../protocol/jaxb/message/HeartbeatAdapter.java | 54 -
.../jaxb/message/JaxbProtocolUtils.java | 42 -
.../jaxb/message/NodeBulletinsAdapter.java | 48 -
.../jaxb/message/NodeIdentifierAdapter.java | 51 -
.../protocol/jaxb/message/ObjectFactory.java | 104 -
.../message/ConnectionRequestMessage.java | 46 -
.../message/ConnectionResponseMessage.java | 66 -
.../ControllerStartupFailureMessage.java | 49 -
.../protocol/message/DisconnectMessage.java | 55 -
.../protocol/message/ExceptionMessage.java | 44 -
.../protocol/message/FlowRequestMessage.java | 46 -
.../protocol/message/FlowResponseMessage.java | 44 -
.../protocol/message/HeartbeatMessage.java | 43 -
.../message/MulticastProtocolMessage.java | 66 -
.../protocol/message/NodeBulletinsMessage.java | 43 -
.../cluster/protocol/message/PingMessage.java | 55 -
.../message/PrimaryRoleAssignmentMessage.java | 56 -
.../protocol/message/ProtocolMessage.java | 61 -
.../message/ReconnectionFailureMessage.java | 45 -
.../message/ReconnectionRequestMessage.java | 94 -
.../message/ReconnectionResponseMessage.java | 32 -
.../message/ServiceBroadcastMessage.java | 64 -
.../MulticastConfigurationFactoryBean.java | 60 -
.../ServerSocketConfigurationFactoryBean.java | 65 -
.../spring/SocketConfigurationFactoryBean.java | 66 -
.../resources/nifi-cluster-protocol-context.xml | 110 -
.../ClusterManagerProtocolSenderImplTest.java | 134 -
.../impl/ClusterServiceDiscoveryTest.java | 135 -
.../impl/ClusterServiceLocatorTest.java | 121 -
.../impl/ClusterServicesBroadcasterTest.java | 133 -
.../impl/MulticastProtocolListenerTest.java | 171 -
.../impl/NodeProtocolSenderImplTest.java | 203 -
.../impl/testutils/DelayedProtocolHandler.java | 57 -
.../testutils/ReflexiveProtocolHandler.java | 47 -
.../framework/cluster-web/.gitignore | 1 -
.../framework/cluster-web/pom.xml | 50 -
.../nifi/cluster/context/ClusterContext.java | 59 -
.../cluster/context/ClusterContextImpl.java | 69 -
.../context/ClusterContextThreadLocal.java | 47 -
.../ClusterAwareOptimisticLockingManager.java | 96 -
.../framework/cluster/.gitignore | 1 -
.../framework-bundle/framework/cluster/pom.xml | 132 -
.../cluster/client/MulticastTestClient.java | 151 -
.../org/apache/nifi/cluster/event/Event.java | 122 -
.../apache/nifi/cluster/event/EventManager.java | 65 -
.../cluster/event/impl/EventManagerImpl.java | 143 -
.../cluster/firewall/ClusterNodeFirewall.java | 35 -
.../impl/FileBasedClusterNodeFirewall.java | 207 -
.../nifi/cluster/flow/ClusterDataFlow.java | 45 -
.../apache/nifi/cluster/flow/DaoException.java | 40 -
.../apache/nifi/cluster/flow/DataFlowDao.java | 62 -
.../cluster/flow/DataFlowManagementService.java | 115 -
.../nifi/cluster/flow/PersistedFlowState.java | 37 -
.../nifi/cluster/flow/StaleFlowException.java | 42 -
.../nifi/cluster/flow/impl/DataFlowDaoImpl.java | 600 ---
.../impl/DataFlowManagementServiceImpl.java | 356 --
.../nifi/cluster/manager/ClusterManager.java | 225 --
.../cluster/manager/HttpClusterManager.java | 169 -
.../cluster/manager/HttpRequestReplicator.java | 99 -
.../cluster/manager/HttpResponseMapper.java | 42 -
.../nifi/cluster/manager/NodeResponse.java | 329 --
.../exception/BlockedByFirewallException.java | 60 -
.../manager/exception/ClusterException.java | 40 -
.../ConnectingNodeMutableRequestException.java | 41 -
...DisconnectedNodeMutableRequestException.java | 41 -
.../exception/IllegalClusterStateException.java | 41 -
.../exception/IllegalNodeDeletionException.java | 41 -
.../IllegalNodeDisconnectionException.java | 42 -
.../IllegalNodeReconnectionException.java | 41 -
.../IneligiblePrimaryNodeException.java | 41 -
.../exception/MutableRequestException.java | 42 -
.../exception/NoConnectedNodesException.java | 41 -
.../exception/NoResponseFromNodesException.java | 42 -
.../exception/NodeDisconnectionException.java | 41 -
.../exception/NodeReconnectionException.java | 40 -
.../PrimaryRoleAssignmentException.java | 41 -
.../SafeModeMutableRequestException.java | 41 -
.../manager/exception/UnknownNodeException.java | 41 -
.../exception/UriConstructionException.java | 42 -
.../manager/impl/ClusteredEventAccess.java | 135 -
.../manager/impl/ClusteredReportingContext.java | 165 -
.../manager/impl/HttpRequestReplicatorImpl.java | 531 ---
.../manager/impl/HttpResponseMapperImpl.java | 85 -
.../cluster/manager/impl/WebClusterManager.java | 3620 ------------------
.../java/org/apache/nifi/cluster/node/Node.java | 252 --
...anagerProtocolServiceLocatorFactoryBean.java | 116 -
...FileBasedClusterNodeFirewallFactoryBean.java | 58 -
.../spring/WebClusterManagerFactoryBean.java | 139 -
.../reporting/ClusteredReportingTaskNode.java | 49 -
.../resources/nifi-cluster-manager-context.xml | 124 -
.../event/impl/EventManagerImplTest.java | 119 -
.../impl/FileBasedClusterNodeFirewallTest.java | 98 -
.../impl/DataFlowManagementServiceImplTest.java | 343 --
.../impl/HttpRequestReplicatorImplTest.java | 368 --
.../impl/HttpResponseMapperImplTest.java | 126 -
.../manager/impl/TestWebClusterManager.java | 54 -
.../cluster/manager/testutils/HttpRequest.java | 239 --
.../cluster/manager/testutils/HttpResponse.java | 93 -
.../manager/testutils/HttpResponseAction.java | 60 -
.../cluster/manager/testutils/HttpServer.java | 240 --
.../ClusterManagerProtocolSenderImplTest.java | 133 -
.../impl/ClusterServiceLocatorTest.java | 119 -
.../impl/ClusterServicesBroadcasterTest.java | 131 -
.../impl/MulticastProtocolListenerTest.java | 171 -
.../impl/NodeProtocolSenderImplTest.java | 201 -
.../impl/SocketProtocolListenerTest.java | 132 -
.../testutils/DelayedProtocolHandler.java | 57 -
.../testutils/ReflexiveProtocolHandler.java | 47 -
.../cluster/src/test/resources/logback-test.xml | 48 -
.../apache/nifi/cluster/firewall/impl/empty.txt | 0
.../apache/nifi/cluster/firewall/impl/ips.txt | 12 -
.../framework/core-api/.gitignore | 1 -
.../framework-bundle/framework/core-api/pom.xml | 56 -
.../nifi/cluster/AdaptedNodeInformation.java | 66 -
.../nifi/cluster/ClusterNodeInformation.java | 67 -
.../org/apache/nifi/cluster/NodeInformant.java | 22 -
.../apache/nifi/cluster/NodeInformation.java | 98 -
.../nifi/cluster/NodeInformationAdapter.java | 39 -
.../apache/nifi/cluster/protocol/DataFlow.java | 41 -
.../apache/nifi/connectable/Connectable.java | 291 --
.../nifi/connectable/ConnectableType.java | 44 -
.../org/apache/nifi/connectable/Connection.java | 78 -
.../org/apache/nifi/connectable/Funnel.java | 24 -
.../java/org/apache/nifi/connectable/Port.java | 31 -
.../org/apache/nifi/connectable/Position.java | 36 -
.../java/org/apache/nifi/connectable/Size.java | 36 -
.../controller/AbstractConfiguredComponent.java | 280 --
.../apache/nifi/controller/AbstractPort.java | 636 ---
.../apache/nifi/controller/Availability.java | 24 -
.../nifi/controller/ConfiguredComponent.java | 63 -
.../nifi/controller/ContentAvailability.java | 65 -
.../org/apache/nifi/controller/Counter.java | 32 -
.../nifi/controller/EventBasedWorker.java | 32 -
.../org/apache/nifi/controller/Heartbeater.java | 22 -
.../nifi/controller/ProcessScheduler.java | 146 -
.../apache/nifi/controller/ProcessorNode.java | 80 -
.../nifi/controller/ReportingTaskNode.java | 56 -
.../nifi/controller/StandardFlowFileQueue.java | 1096 ------
.../apache/nifi/controller/StandardFunnel.java | 541 ---
.../controller/ValidationContextFactory.java | 27 -
.../org/apache/nifi/controller/WorkerQueue.java | 36 -
.../exception/CommunicationsException.java | 40 -
...ControllerServiceAlreadyExistsException.java | 30 -
.../ControllerServiceNotFoundException.java | 51 -
.../ProcessorInstantiationException.java | 27 -
.../exception/ProcessorLifeCycleException.java | 30 -
.../org/apache/nifi/controller/label/Label.java | 48 -
.../ReportingTaskInstantiationException.java | 31 -
.../repository/ContentNotFoundException.java | 48 -
.../repository/CounterRepository.java | 34 -
.../controller/repository/FlowFileEvent.java | 54 -
.../repository/FlowFileEventRepository.java | 50 -
.../repository/RepositoryStatusReport.java | 28 -
.../service/ControllerServiceNode.java | 40 -
.../service/ControllerServiceProvider.java | 47 -
.../service/ControllerServiceReference.java | 50 -
.../org/apache/nifi/events/BulletinFactory.java | 52 -
.../nifi/events/BulletinProcessingStrategy.java | 27 -
.../apache/nifi/events/ComponentBulletin.java | 30 -
.../org/apache/nifi/events/SystemBulletin.java | 30 -
.../org/apache/nifi/groups/ProcessGroup.java | 723 ----
.../apache/nifi/groups/ProcessGroupCounts.java | 66 -
.../apache/nifi/groups/RemoteProcessGroup.java | 255 --
.../RemoteProcessGroupPortDescriptor.java | 92 -
.../org/apache/nifi/logging/LogMessage.java | 74 -
.../org/apache/nifi/logging/LogObserver.java | 22 -
.../org/apache/nifi/logging/LogRepository.java | 67 -
.../nifi/logging/LogRepositoryFactory.java | 61 -
.../java/org/apache/nifi/nar/NarCloseable.java | 44 -
.../nifi/nar/NarThreadContextClassLoader.java | 188 -
.../main/java/org/apache/nifi/remote/Peer.java | 107 -
.../java/org/apache/nifi/remote/PeerStatus.java | 72 -
.../nifi/remote/PortAuthorizationResult.java | 25 -
.../nifi/remote/RemoteAuthorizationState.java | 27 -
.../org/apache/nifi/remote/RemoteGroupPort.java | 35 -
.../org/apache/nifi/remote/RootGroupPort.java | 78 -
.../apache/nifi/remote/TransferDirection.java | 23 -
.../nifi/remote/VersionedRemoteResource.java | 24 -
.../apache/nifi/remote/codec/FlowFileCodec.java | 79 -
.../remote/exception/BadRequestException.java | 30 -
.../remote/exception/HandshakeException.java | 30 -
.../exception/NotAuthorizedException.java | 26 -
.../exception/PortNotRunningException.java | 26 -
.../remote/exception/ProtocolException.java | 34 -
.../exception/RequestExpiredException.java | 26 -
.../remote/exception/UnknownPortException.java | 26 -
.../nifi/remote/protocol/ClientProtocol.java | 78 -
.../remote/protocol/CommunicationsInput.java | 27 -
.../remote/protocol/CommunicationsOutput.java | 27 -
.../remote/protocol/CommunicationsSession.java | 64 -
.../nifi/remote/protocol/RequestType.java | 43 -
.../nifi/remote/protocol/ServerProtocol.java | 143 -
.../framework-bundle/framework/core/.gitignore | 1 -
.../framework-bundle/framework/core/pom.xml | 121 -
.../apache/nifi/cluster/BulletinsPayload.java | 95 -
.../nifi/cluster/ConnectionException.java | 42 -
.../nifi/cluster/DisconnectionException.java | 42 -
.../apache/nifi/cluster/HeartbeatPayload.java | 170 -
.../org/apache/nifi/connectable/LocalPort.java | 172 -
.../nifi/connectable/StandardConnection.java | 336 --
.../nifi/controller/EventDrivenWorkerQueue.java | 329 --
.../nifi/controller/FileSystemSwapManager.java | 768 ----
.../apache/nifi/controller/FlowController.java | 3579 -----------------
.../nifi/controller/FlowFromDOMFactory.java | 418 --
.../controller/FlowSerializationException.java | 48 -
.../apache/nifi/controller/FlowSerializer.java | 42 -
.../FlowSynchronizationException.java | 47 -
.../nifi/controller/FlowSynchronizer.java | 53 -
.../nifi/controller/FlowUnmarshaller.java | 78 -
.../apache/nifi/controller/SnippetManager.java | 96 -
.../apache/nifi/controller/StandardCounter.java | 108 -
.../nifi/controller/StandardFlowSerializer.java | 404 --
.../nifi/controller/StandardFlowService.java | 875 -----
.../controller/StandardFlowSynchronizer.java | 1026 -----
.../nifi/controller/StandardProcessorNode.java | 1243 ------
.../apache/nifi/controller/StandardSnippet.java | 186 -
.../org/apache/nifi/controller/Template.java | 37 -
.../apache/nifi/controller/TemplateManager.java | 507 ---
.../controller/UninheritableFlowException.java | 48 -
.../exception/FlowFileConsumptionException.java | 38 -
.../ReportingTaskInstantiationException.java | 31 -
.../exception/ValidationException.java | 47 -
.../nifi/controller/label/StandardLabel.java | 110 -
.../reporting/AbstractReportingTaskNode.java | 111 -
.../reporting/StandardReportingContext.java | 132 -
.../StandardReportingInitializationContext.java | 96 -
.../reporting/StandardReportingTaskNode.java | 40 -
.../repository/BatchingSessionFactory.java | 242 --
.../repository/ConnectionSwapInfo.java | 58 -
.../repository/FileSystemRepository.java | 1351 -------
.../controller/repository/ProcessContext.java | 291 --
.../repository/ProvenanceEventEnricher.java | 34 -
.../repository/RepositoryPurgeException.java | 59 -
.../repository/RingBufferEventRepository.java | 312 --
.../repository/StandardCounterRepository.java | 109 -
.../repository/StandardFlowFileEvent.java | 237 --
.../repository/StandardFlowFileRecord.java | 312 --
.../repository/StandardProcessSession.java | 2684 -------------
.../StandardProcessSessionFactory.java | 33 -
.../repository/StandardProvenanceReporter.java | 431 ---
.../repository/StandardRepositoryRecord.java | 196 -
.../StandardRepositoryStatusReport.java | 88 -
.../repository/VolatileContentRepository.java | 647 ----
.../repository/VolatileFlowFileRepository.java | 109 -
.../WriteAheadFlowFileRepository.java | 864 -----
.../repository/claim/ContentDirection.java | 35 -
.../repository/claim/StandardContentClaim.java | 154 -
.../claim/StandardContentClaimManager.java | 155 -
.../repository/io/ArrayManagedOutputStream.java | 245 --
.../repository/io/ByteCountingInputStream.java | 100 -
.../repository/io/ByteCountingOutputStream.java | 64 -
.../io/DisableOnCloseOutputStream.java | 73 -
.../io/FlowFileAccessInputStream.java | 168 -
.../io/FlowFileAccessOutputStream.java | 95 -
.../repository/io/LimitedInputStream.java | 107 -
.../controller/repository/io/LongHolder.java | 48 -
.../controller/repository/io/MemoryManager.java | 55 -
.../repository/io/SyncOnCloseOutputStream.java | 58 -
.../scheduling/ConnectableProcessContext.java | 194 -
.../scheduling/EventDrivenSchedulingAgent.java | 346 --
.../scheduling/ProcessContextFactory.java | 51 -
.../scheduling/QuartzSchedulingAgent.java | 220 --
.../controller/scheduling/ScheduleState.java | 90 -
.../controller/scheduling/SchedulingAgent.java | 45 -
.../scheduling/StandardProcessScheduler.java | 569 ---
.../scheduling/TimerDrivenSchedulingAgent.java | 139 -
.../service/ControllerServiceLoader.java | 156 -
.../service/StandardConfigurationContext.java | 64 -
...dControllerServiceInitializationContext.java | 64 -
.../service/StandardControllerServiceNode.java | 125 -
.../StandardControllerServiceProvider.java | 196 -
.../StandardControllerServiceReference.java | 100 -
.../history/StandardMetricDescriptor.java | 128 -
.../status/history/StandardStatusHistory.java | 54 -
.../status/history/StandardStatusSnapshot.java | 70 -
.../status/history/StatusHistoryUtil.java | 107 -
.../VolatileComponentStatusRepository.java | 649 ----
.../tasks/ContinuallyRunConnectableTask.java | 97 -
.../tasks/ContinuallyRunProcessorTask.java | 185 -
.../nifi/controller/tasks/ExpireFlowFiles.java | 114 -
.../controller/tasks/ReportingTaskWrapper.java | 63 -
.../nifi/diagnostics/DiagnosticUtils.java | 28 -
.../nifi/diagnostics/GarbageCollection.java | 60 -
.../apache/nifi/diagnostics/StorageUsage.java | 72 -
.../nifi/diagnostics/SystemDiagnostics.java | 212 -
.../diagnostics/SystemDiagnosticsFactory.java | 133 -
.../nifi/encrypt/EncryptionException.java | 43 -
.../apache/nifi/encrypt/StringEncryptor.java | 152 -
.../java/org/apache/nifi/engine/FlowEngine.java | 135 -
.../events/NodeBulletinProcessingStrategy.java | 66 -
.../nifi/events/VolatileBulletinRepository.java | 240 --
.../events/network/CommunicationsFailure.java | 36 -
.../nifi/events/network/NetworkTransfer.java | 48 -
.../nifi/fingerprint/FingerprintException.java | 41 -
.../nifi/fingerprint/FingerprintFactory.java | 985 -----
.../nifi/groups/StandardProcessGroup.java | 2019 ----------
.../org/apache/nifi/jaxb/AdaptedBulletin.java | 100 -
.../org/apache/nifi/jaxb/AdaptedCounter.java | 64 -
.../org/apache/nifi/jaxb/BulletinAdapter.java | 59 -
.../org/apache/nifi/jaxb/CounterAdapter.java | 51 -
.../org/apache/nifi/lifecycle/LifeCycle.java | 54 -
.../nifi/lifecycle/LifeCycleException.java | 48 -
.../nifi/lifecycle/LifeCycleStartException.java | 49 -
.../nifi/lifecycle/LifeCycleStopException.java | 48 -
.../nifi/logging/ProcessorLogObserver.java | 48 -
.../repository/StandardLogRepository.java | 173 -
.../nifi/persistence/FlowConfigurationDAO.java | 135 -
.../StandardSnippetDeserializer.java | 42 -
.../persistence/StandardSnippetSerializer.java | 47 -
.../StandardXMLFlowConfigurationDAO.java | 345 --
.../nifi/persistence/TemplateDeserializer.java | 42 -
.../nifi/persistence/TemplateSerializer.java | 47 -
.../nifi/processor/SimpleProcessLogger.java | 307 --
.../nifi/processor/StandardProcessContext.java | 173 -
.../StandardProcessorInitializationContext.java | 49 -
.../nifi/processor/StandardPropertyValue.java | 163 -
.../processor/StandardSchedulingContext.java | 113 -
.../processor/StandardValidationContext.java | 97 -
.../StandardValidationContextFactory.java | 39 -
.../nifi/remote/StandardRemoteProcessGroup.java | 1621 --------
...tandardRemoteProcessGroupPortDescriptor.java | 148 -
.../org/apache/nifi/services/FlowService.java | 139 -
.../nifi/spring/FlowControllerFactoryBean.java | 103 -
.../spring/RingBufferEventRepositoryBean.java | 45 -
.../spring/StandardFlowServiceFactoryBean.java | 98 -
.../apache/nifi/util/ComponentStatusReport.java | 139 -
.../java/org/apache/nifi/util/Connectables.java | 56 -
.../java/org/apache/nifi/util/DomUtils.java | 79 -
.../org/apache/nifi/util/ReflectionUtils.java | 157 -
.../java/org/apache/nifi/util/SnippetUtils.java | 241 --
.../ControllerServiceConfiguration.xsd | 61 -
.../src/main/resources/FlowConfiguration.xsd | 335 --
.../resources/ReportingTaskConfiguration.xsd | 87 -
.../core/src/main/resources/nifi-context.xml | 49 -
.../nifi/cluster/HeartbeatPayloadTest.java | 121 -
.../controller/StandardFlowServiceTest.java | 300 --
.../controller/TestFileSystemSwapManager.java | 104 -
.../repository/TestFileSystemRepository.java | 352 --
.../TestRingBufferEventRepository.java | 138 -
.../repository/TestStandardProcessSession.java | 1177 ------
.../TestStandardProvenanceReporter.java | 65 -
.../TestVolatileContentRepository.java | 183 -
.../TestWriteAheadFlowFileRepository.java | 135 -
.../repository/io/TestLimitedOutputStream.java | 76 -
.../controller/repository/util/DiskUtils.java | 77 -
.../fingerprint/FingerprintFactoryTest.java | 60 -
.../processor/TestStandardPropertyValue.java | 167 -
.../processors/DataGeneratorTestProcessor.java | 85 -
.../StubAttributeLoggerProcessor.java | 111 -
.../StubTerminationFileProcessor.java | 40 -
.../core/src/test/resources/conf/0bytes.xml | 0
.../test/resources/conf/all-flow-corrupt.xml | 201 -
.../resources/conf/all-flow-inheritable.xml | 196 -
.../resources/conf/all-flow-uninheritable.xml | 202 -
.../core/src/test/resources/conf/all-flow.xml | 198 -
.../src/test/resources/conf/nifi.properties | 127 -
.../test/resources/conf/only-termination.xml | 37 -
.../src/test/resources/conf/remote-flow.xml | 145 -
.../src/test/resources/conf/standard-flow.xml | 196 -
.../core/src/test/resources/conf/taskConfig.xml | 17 -
.../test/resources/conf/termination-only.xml | 0
.../framework/core/src/test/resources/hello.txt | 1 -
.../core/src/test/resources/logback-test.xml | 36 -
.../test/resources/nifi-with-remote.properties | 127 -
.../core/src/test/resources/nifi.properties | 127 -
.../test/resources/nifi/fingerprint/flow1a.xml | 160 -
.../test/resources/nifi/fingerprint/flow1b.xml | 160 -
.../test/resources/nifi/fingerprint/flow2.xml | 160 -
.../core/src/test/resources/old-swap-file.swap | Bin 1730054 -> 0 bytes
.../file-authorization-provider/pom.xml | 81 -
.../FileAuthorizationProvider.java | 584 ---
....apache.nifi.authorization.AuthorityProvider | 15 -
.../src/main/xsd/users.xsd | 64 -
.../FileAuthorizationProviderTest.java | 127 -
.../framework-bundle/framework/nar/.gitignore | 1 -
.../framework-bundle/framework/nar/pom.xml | 36 -
.../org/apache/nifi/nar/ExtensionManager.java | 188 -
.../org/apache/nifi/nar/ExtensionMapping.java | 73 -
.../org/apache/nifi/nar/NarClassLoader.java | 227 --
.../org/apache/nifi/nar/NarClassLoaders.java | 301 --
.../java/org/apache/nifi/nar/NarUnpacker.java | 351 --
.../java/org/apache/nifi/util/FileUtils.java | 187 -
nar-bundles/framework-bundle/framework/pom.xml | 51 -
.../framework/resources/pom.xml | 52 -
.../src/main/assembly/dependencies.xml | 36 -
.../src/main/resources/bin/dump-nifi.bat | 33 -
.../src/main/resources/bin/nifi-status.bat | 33 -
.../resources/src/main/resources/bin/nifi.sh | 181 -
.../src/main/resources/bin/run-nifi.bat | 33 -
.../src/main/resources/bin/start-nifi.bat | 33 -
.../src/main/resources/bin/stop-nifi.bat | 33 -
.../main/resources/conf/authority-providers.xml | 43 -
.../main/resources/conf/authorized-users.xml | 57 -
.../src/main/resources/conf/bootstrap.conf | 57 -
.../main/resources/conf/controller-services.xml | 18 -
.../src/main/resources/conf/logback.xml | 105 -
.../src/main/resources/conf/nifi.properties | 159 -
.../src/main/resources/conf/reporting-tasks.xml | 49 -
.../framework/runtime/.gitignore | 2 -
.../framework-bundle/framework/runtime/pom.xml | 42 -
.../java/org/apache/nifi/BootstrapListener.java | 378 --
.../src/main/java/org/apache/nifi/NiFi.java | 223 --
.../main/java/org/apache/nifi/NiFiServer.java | 31 -
.../apache/nifi/util/LimitingInputStream.java | 107 -
.../framework/security/.gitignore | 1 -
.../framework-bundle/framework/security/pom.xml | 42 -
.../util/SslContextCreationException.java | 40 -
.../security/util/SslContextFactory.java | 119 -
.../framework/security/util/SslException.java | 40 -
.../security/util/SslServerSocketFactory.java | 81 -
...SslServerSocketFactoryCreationException.java | 40 -
.../security/util/SslSocketFactory.java | 91 -
.../util/SslSocketFactoryCreationException.java | 40 -
.../security/util/SslContextFactoryTest.java | 69 -
.../src/test/resources/localhost-ks.jks | Bin 1298 -> 0 bytes
.../src/test/resources/localhost-ts.jks | Bin 891 -> 0 bytes
.../src/test/resources/log4j.properties | 21 -
.../framework/site-to-site/.gitignore | 1 -
.../framework/site-to-site/pom.xml | 74 -
.../util/RemoteProcessGroupUtils.java | 216 --
.../remote/AbstractCommunicationsSession.java | 54 -
.../nifi/remote/RemoteResourceFactory.java | 202 -
.../nifi/remote/RemoteResourceManager.java | 227 --
.../apache/nifi/remote/RemoteSiteListener.java | 33 -
.../nifi/remote/SocketRemoteSiteListener.java | 340 --
.../nifi/remote/StandardRemoteGroupPort.java | 764 ----
.../nifi/remote/StandardRootGroupPort.java | 629 ---
.../remote/codec/StandardFlowFileCodec.java | 169 -
.../exception/UnsupportedCodecException.java | 31 -
.../SocketChannelCommunicationsSession.java | 90 -
.../remote/io/socket/SocketChannelInput.java | 66 -
.../remote/io/socket/SocketChannelOutput.java | 58 -
.../SSLSocketChannelCommunicationsSession.java | 93 -
.../io/socket/ssl/SSLSocketChannelInput.java | 50 -
.../io/socket/ssl/SSLSocketChannelOutput.java | 44 -
.../socket/ClusterManagerServerProtocol.java | 208 -
.../protocol/socket/HandshakeProperty.java | 23 -
.../nifi/remote/protocol/socket/Response.java | 51 -
.../remote/protocol/socket/ResponseCode.java | 152 -
.../protocol/socket/SocketClientProtocol.java | 510 ---
.../socket/SocketFlowFileServerProtocol.java | 581 ---
.../org.apache.nifi.remote.codec.FlowFileCodec | 15 -
...g.apache.nifi.remote.protocol.ClientProtocol | 15 -
...g.apache.nifi.remote.protocol.ServerProtocol | 16 -
.../remote/TestStandardRemoteGroupPort.java | 97 -
.../remote/TestStandardSiteToSiteProtocol.java | 116 -
.../io/socket/TestSocketChannelStreams.java | 231 --
.../io/socket/ssl/TestSSLSocketChannel.java | 381 --
.../test/resources/dummy-certs/localhost-ks.jks | Bin 1298 -> 0 bytes
.../test/resources/dummy-certs/localhost-ts.jks | Bin 891 -> 0 bytes
.../src/test/resources/logback-test.xml | 30 -
.../src/test/resources/nifi.properties | 111 -
.../framework/user-actions/.gitignore | 1 -
.../framework/user-actions/pom.xml | 27 -
.../java/org/apache/nifi/action/Action.java | 121 -
.../java/org/apache/nifi/action/Component.java | 31 -
.../java/org/apache/nifi/action/Operation.java | 36 -
.../component/details/ComponentDetails.java | 26 -
.../component/details/ProcessorDetails.java | 34 -
.../details/RemoteProcessGroupDetails.java | 34 -
.../nifi/action/details/ActionDetails.java | 26 -
.../nifi/action/details/ConfigureDetails.java | 52 -
.../nifi/action/details/ConnectDetails.java | 90 -
.../apache/nifi/action/details/MoveDetails.java | 61 -
.../nifi/action/details/PurgeDetails.java | 41 -
.../web/custom-ui-utilities/.gitignore | 1 -
.../framework/web/custom-ui-utilities/pom.xml | 36 -
.../web/HttpServletRequestContextConfig.java | 118 -
.../framework/web/nifi-jetty/.gitignore | 1 -
.../framework/web/nifi-jetty/pom.xml | 166 -
.../src/main/assembly/distribution.xml | 42 -
.../org/apache/nifi/web/server/JettyServer.java | 686 ----
.../server/ServerConfigurationException.java | 36 -
.../org/apache/nifi/web/webdefault.xml | 556 ---
.../framework/web/nifi-web-api/.gitignore | 1 -
.../framework/web/nifi-web-api/pom.xml | 305 --
.../src/main/enunciate/enunciate.xml | 36 -
.../src/main/enunciate/images/home.png | Bin 144 -> 0 bytes
.../src/main/enunciate/override.css | 178 -
.../org/apache/nifi/aop/MethodProfiler.java | 45 -
.../apache/nifi/audit/ControllerAuditor.java | 257 --
.../org/apache/nifi/audit/FunnelAuditor.java | 135 -
.../java/org/apache/nifi/audit/NiFiAuditor.java | 119 -
.../java/org/apache/nifi/audit/PortAuditor.java | 334 --
.../apache/nifi/audit/ProcessGroupAuditor.java | 263 --
.../org/apache/nifi/audit/ProcessorAuditor.java | 408 --
.../apache/nifi/audit/RelationshipAuditor.java | 412 --
.../nifi/audit/RemoteProcessGroupAuditor.java | 376 --
.../org/apache/nifi/audit/SnippetAuditor.java | 519 ---
.../nifi/web/ClusterRequestException.java | 38 -
.../IllegalClusterResourceRequestException.java | 33 -
.../org/apache/nifi/web/NiFiCoreException.java | 32 -
.../org/apache/nifi/web/NiFiServiceFacade.java | 1263 ------
.../apache/nifi/web/NiFiServiceFacadeLock.java | 177 -
.../nifi/web/StandardNiFiServiceFacade.java | 2877 --------------
.../apache/nifi/web/StandardNiFiWebContext.java | 336 --
.../nifi/web/api/ApplicationResource.java | 399 --
.../nifi/web/api/BulletinBoardResource.java | 116 -
.../apache/nifi/web/api/ClusterResource.java | 724 ----
.../apache/nifi/web/api/ConnectionResource.java | 771 ----
.../apache/nifi/web/api/ControllerResource.java | 803 ----
.../org/apache/nifi/web/api/FunnelResource.java | 493 ---
.../apache/nifi/web/api/HistoryResource.java | 271 --
.../apache/nifi/web/api/InputPortResource.java | 531 ---
.../org/apache/nifi/web/api/LabelResource.java | 547 ---
.../org/apache/nifi/web/api/NodeResource.java | 302 --
.../apache/nifi/web/api/OutputPortResource.java | 532 ---
.../nifi/web/api/ProcessGroupResource.java | 1102 ------
.../apache/nifi/web/api/ProcessorResource.java | 693 ----
.../apache/nifi/web/api/ProvenanceResource.java | 1068 ------
.../web/api/RemoteProcessGroupResource.java | 912 -----
.../apache/nifi/web/api/SnippetResource.java | 601 ---
.../nifi/web/api/SystemDiagnosticsResource.java | 81 -
.../apache/nifi/web/api/TemplateResource.java | 400 --
.../apache/nifi/web/api/UserGroupResource.java | 383 --
.../org/apache/nifi/web/api/UserResource.java | 469 ---
.../api/config/AccessDeniedExceptionMapper.java | 54 -
.../config/AccountNotFoundExceptionMapper.java | 47 -
.../config/AdministrationExceptionMapper.java | 48 -
...ationCredentialsNotFoundExceptionMapper.java | 48 -
.../web/api/config/ClusterExceptionMapper.java | 47 -
...ectingNodeMutableRequestExceptionMapper.java | 49 -
...nectedNodeMutableRequestExceptionMapper.java | 49 -
.../config/IllegalArgumentExceptionMapper.java | 46 -
...alClusterResourceRequestExceptionMapper.java | 47 -
.../IllegalClusterStateExceptionMapper.java | 47 -
.../IllegalNodeDeletionExceptionMapper.java | 46 -
...IllegalNodeDisconnectionExceptionMapper.java | 46 -
.../IllegalNodeReconnectionExceptionMapper.java | 46 -
.../api/config/IllegalStateExceptionMapper.java | 46 -
.../IneligiblePrimaryNodeExceptionMapper.java | 47 -
.../config/InvalidRevisionExceptionMapper.java | 47 -
.../api/config/JsonMappingExceptionMapper.java | 46 -
.../api/config/JsonParseExceptionMapper.java | 46 -
.../config/MutableRequestExceptionMapper.java | 48 -
.../web/api/config/NiFiCoreExceptionMapper.java | 47 -
.../config/NoConnectedNodesExceptionMapper.java | 47 -
.../NoResponseFromNodesExceptionMapper.java | 49 -
.../NodeDisconnectionExceptionMapper.java | 47 -
.../config/NodeReconnectionExceptionMapper.java | 47 -
.../PrimaryRoleAssignmentExceptionMapper.java | 47 -
.../config/ResourceNotFoundExceptionMapper.java | 48 -
.../SafeModeMutableRequestExceptionMapper.java | 48 -
.../nifi/web/api/config/ThrowableMapper.java | 46 -
.../api/config/UnknownNodeExceptionMapper.java | 47 -
.../api/config/ValidationExceptionMapper.java | 49 -
.../config/WebApplicationExceptionMapper.java | 63 -
.../org/apache/nifi/web/api/dto/DtoFactory.java | 2142 -----------
.../org/apache/nifi/web/api/package-info.java | 58 -
.../request/BulletinBoardPatternParameter.java | 47 -
.../nifi/web/api/request/ClientIdParameter.java | 45 -
.../api/request/ConnectableTypeParameter.java | 41 -
.../nifi/web/api/request/DateTimeParameter.java | 63 -
.../nifi/web/api/request/DoubleParameter.java | 40 -
.../nifi/web/api/request/IntegerParameter.java | 40 -
.../nifi/web/api/request/LongParameter.java | 40 -
.../ApplicationStartupContextListener.java | 157 -
.../nifi/web/controller/ControllerFacade.java | 1392 -------
.../web/controller/StandardSearchContext.java | 64 -
.../org/apache/nifi/web/dao/ConnectionDAO.java | 112 -
.../java/org/apache/nifi/web/dao/FunnelDAO.java | 88 -
.../java/org/apache/nifi/web/dao/LabelDAO.java | 79 -
.../java/org/apache/nifi/web/dao/PortDAO.java | 96 -
.../apache/nifi/web/dao/ProcessGroupDAO.java | 89 -
.../org/apache/nifi/web/dao/ProcessorDAO.java | 97 -
.../nifi/web/dao/RemoteProcessGroupDAO.java | 135 -
.../org/apache/nifi/web/dao/SnippetDAO.java | 92 -
.../org/apache/nifi/web/dao/TemplateDAO.java | 77 -
.../apache/nifi/web/dao/impl/ComponentDAO.java | 72 -
.../web/dao/impl/StandardConnectionDAO.java | 519 ---
.../nifi/web/dao/impl/StandardFunnelDAO.java | 180 -
.../nifi/web/dao/impl/StandardInputPortDAO.java | 309 --
.../nifi/web/dao/impl/StandardLabelDAO.java | 184 -
.../web/dao/impl/StandardOutputPortDAO.java | 309 --
.../web/dao/impl/StandardProcessGroupDAO.java | 171 -
.../nifi/web/dao/impl/StandardProcessorDAO.java | 503 ---
.../dao/impl/StandardRemoteProcessGroupDAO.java | 402 --
.../nifi/web/dao/impl/StandardSnippetDAO.java | 344 --
.../nifi/web/dao/impl/StandardTemplateDAO.java | 141 -
.../nifi/web/filter/NodeRequestFilter.java | 130 -
.../apache/nifi/web/filter/RequestLogger.java | 78 -
.../nifi/web/filter/ThreadLocalFilter.java | 54 -
.../org/apache/nifi/web/filter/TimerFilter.java | 70 -
.../nifi/web/util/ClientResponseUtils.java | 48 -
.../nifi/web/util/DownloadableContent.java | 47 -
.../org/apache/nifi/web/util/SnippetUtils.java | 426 ---
.../src/main/resources/nifi-web-api-context.xml | 303 --
.../src/main/webapp/WEB-INF/web.xml | 105 -
.../apache/nifi/integration/NiFiWebApiTest.java | 274 --
.../accesscontrol/AdminAccessControlTest.java | 1090 ------
.../accesscontrol/DfmAccessControlTest.java | 1410 -------
.../ReadOnlyAccessControlTest.java | 1064 -----
.../util/NiFiTestAuthorizationProvider.java | 179 -
.../nifi/integration/util/NiFiTestServer.java | 235 --
.../nifi/integration/util/NiFiTestUser.java | 257 --
.../integration/util/SourceTestProcessor.java | 49 -
.../util/TerminationTestProcessor.java | 44 -
.../nifi/integration/util/TestPrioritizer.java | 31 -
....apache.nifi.authorization.AuthorityProvider | 15 -
...org.apache.nifi.flowfile.FlowFilePrioritizer | 15 -
.../org.apache.nifi.processor.Processor | 16 -
.../access-control/authority-providers.xml | 24 -
.../access-control/controller-services.xml | 18 -
.../resources/access-control/localhost-ks.jks | Bin 2246 -> 0 bytes
.../resources/access-control/localhost-ts.jks | Bin 957 -> 0 bytes
.../resources/access-control/nifi.properties | 136 -
.../access-control/reporting-tasks.xml | 17 -
.../src/test/resources/logback-test.xml | 36 -
.../framework/web/nifi-web-docs/.gitignore | 1 -
.../framework/web/nifi-web-docs/pom.xml | 57 -
.../nifi/web/docs/DocumentationController.java | 91 -
.../main/webapp/WEB-INF/jsp/documentation.jsp | 130 -
.../WEB-INF/jsp/no-documentation-found.jsp | 31 -
.../src/main/webapp/WEB-INF/web.xml | 33 -
.../src/main/webapp/css/component-usage.css | 127 -
.../nifi-web-docs/src/main/webapp/css/main.css | 214 --
.../src/main/webapp/images/bgBannerFoot.png | Bin 189 -> 0 bytes
.../src/main/webapp/images/bgHeader.png | Bin 1697 -> 0 bytes
.../src/main/webapp/images/bgTableHeader.png | Bin 232 -> 0 bytes
.../src/main/webapp/js/application.js | 282 --
.../framework/web/nifi-web-error/.gitignore | 1 -
.../framework/web/nifi-web-error/pom.xml | 48 -
.../apache/nifi/web/filter/CatchAllFilter.java | 45 -
.../src/main/webapp/WEB-INF/web.xml | 26 -
.../nifi-web-error/src/main/webapp/index.jsp | 37 -
.../framework/web/nifi-web-ui/.gitignore | 1 -
.../framework/web/nifi-web-ui/pom.xml | 655 ----
.../apache/nifi/web/filter/IeEdgeHeader.java | 57 -
.../org/apache/nifi/web/servlet/ConvertSvg.java | 117 -
.../filters/bulletin-board-min.properties | 18 -
.../resources/filters/bulletin-board.properties | 25 -
.../resources/filters/canvas-min.properties | 18 -
.../main/resources/filters/canvas.properties | 69 -
.../resources/filters/cluster-min.properties | 18 -
.../main/resources/filters/cluster.properties | 26 -
.../resources/filters/counters-min.properties | 18 -
.../main/resources/filters/counters.properties | 26 -
.../resources/filters/history-min.properties | 18 -
.../main/resources/filters/history.properties | 27 -
.../resources/filters/provenance-min.properties | 18 -
.../resources/filters/provenance.properties | 26 -
.../resources/filters/summary-min.properties | 18 -
.../main/resources/filters/summary.properties | 32 -
.../resources/filters/templates-min.properties | 18 -
.../main/resources/filters/templates.properties | 26 -
.../main/resources/filters/users-min.properties | 18 -
.../src/main/resources/filters/users.properties | 26 -
.../webapp/WEB-INF/pages/bulletin-board.jsp | 49 -
.../src/main/webapp/WEB-INF/pages/canvas.jsp | 117 -
.../src/main/webapp/WEB-INF/pages/cluster.jsp | 61 -
.../src/main/webapp/WEB-INF/pages/counters.jsp | 60 -
.../src/main/webapp/WEB-INF/pages/history.jsp | 64 -
.../main/webapp/WEB-INF/pages/message-page.jsp | 34 -
.../main/webapp/WEB-INF/pages/provenance.jsp | 67 -
.../src/main/webapp/WEB-INF/pages/summary.jsp | 79 -
.../src/main/webapp/WEB-INF/pages/templates.jsp | 62 -
.../src/main/webapp/WEB-INF/pages/users.jsp | 70 -
.../webapp/WEB-INF/partials/banners-main.jsp | 19 -
.../webapp/WEB-INF/partials/banners-utility.jsp | 19 -
.../bulletin-board/bulletin-board-content.jsp | 40 -
.../WEB-INF/partials/canvas/about-dialog.jsp | 29 -
.../WEB-INF/partials/canvas/canvas-header.jsp | 56 -
.../canvas/connection-configuration.jsp | 180 -
.../partials/canvas/connections-dialog.jsp | 35 -
.../partials/canvas/fill-color-dialog.jsp | 42 -
.../WEB-INF/partials/canvas/flow-status.jsp | 55 -
.../canvas/instantiate-template-dialog.jsp | 22 -
.../partials/canvas/label-configuration.jsp | 38 -
.../WEB-INF/partials/canvas/navigation.jsp | 57 -
.../WEB-INF/partials/canvas/new-port-dialog.jsp | 27 -
.../canvas/new-process-group-dialog.jsp | 27 -
.../partials/canvas/new-processor-dialog.jsp | 50 -
.../canvas/new-processor-property-dialog.jsp | 34 -
.../canvas/new-remote-process-group-dialog.jsp | 27 -
.../partials/canvas/new-template-dialog.jsp | 33 -
.../partials/canvas/port-configuration.jsp | 54 -
.../WEB-INF/partials/canvas/port-details.jsp | 39 -
.../canvas/process-group-configuration.jsp | 34 -
.../partials/canvas/process-group-details.jsp | 33 -
.../partials/canvas/processor-configuration.jsp | 213 --
.../WEB-INF/partials/canvas/registration.jsp | 44 -
.../canvas/remote-port-configuration.jsp | 42 -
.../remote-process-group-configuration.jsp | 50 -
.../canvas/remote-process-group-details.jsp | 48 -
.../canvas/remote-process-group-ports.jsp | 47 -
.../canvas/secure-port-configuration.jsp | 82 -
.../partials/canvas/secure-port-details.jsp | 67 -
.../partials/canvas/settings-content.jsp | 60 -
.../webapp/WEB-INF/partials/canvas/shell.jsp | 27 -
.../partials/cluster/cluster-content.jsp | 39 -
.../partials/cluster/node-details-dialog.jsp | 42 -
.../WEB-INF/partials/connection-details.jsp | 123 -
.../partials/counters/counters-content.jsp | 39 -
.../partials/history/action-details-dialog.jsp | 27 -
.../partials/history/history-content.jsp | 36 -
.../partials/history/history-filter-dialog.jsp | 73 -
.../partials/history/history-purge-dialog.jsp | 42 -
.../webapp/WEB-INF/partials/message-pane.jsp | 23 -
.../main/webapp/WEB-INF/partials/ok-dialog.jsp | 22 -
.../WEB-INF/partials/processor-details.jsp | 158 -
.../provenance/lineage-query-dialog.jsp | 26 -
.../partials/provenance/provenance-content.jsp | 73 -
.../provenance-event-details-dialog.jsp | 196 -
.../provenance/provenance-query-dialog.jsp | 26 -
.../provenance/provenance-search-dialog.jsp | 101 -
.../WEB-INF/partials/status-history-dialog.jsp | 37 -
.../cluster-connection-summary-dialog.jsp | 36 -
.../cluster-input-port-summary-dialog.jsp | 36 -
.../cluster-output-port-summary-dialog.jsp | 36 -
.../cluster-processor-summary-dialog.jsp | 36 -
...ster-remote-process-group-summary-dialog.jsp | 36 -
.../partials/summary/summary-content.jsp | 64 -
.../summary/system-diagnostics-dialog.jsp | 131 -
.../summary/view-single-node-dialog.jsp | 24 -
.../partials/templates/templates-content.jsp | 59 -
.../partials/users/group-revoke-dialog.jsp | 22 -
.../partials/users/group-roles-dialog.jsp | 52 -
.../partials/users/user-delete-dialog.jsp | 23 -
.../partials/users/user-details-dialog.jsp | 56 -
.../partials/users/user-group-dialog.jsp | 27 -
.../partials/users/user-revoke-dialog.jsp | 23 -
.../partials/users/user-roles-dialog.jsp | 60 -
.../WEB-INF/partials/users/users-content.jsp | 46 -
.../webapp/WEB-INF/partials/yes-no-dialog.jsp | 22 -
.../nifi-web-ui/src/main/webapp/WEB-INF/web.xml | 120 -
.../nifi-web-ui/src/main/webapp/css/about.css | 54 -
.../nifi-web-ui/src/main/webapp/css/banner.css | 80 -
.../src/main/webapp/css/bulletin-board.css | 238 --
.../nifi-web-ui/src/main/webapp/css/canvas.css | 41 -
.../nifi-web-ui/src/main/webapp/css/cluster.css | 152 -
.../webapp/css/connection-configuration.css | 168 -
.../src/main/webapp/css/connection-details.css | 77 -
.../src/main/webapp/css/counters.css | 134 -
.../nifi-web-ui/src/main/webapp/css/dialog.css | 351 --
.../src/main/webapp/css/flow-status.css | 110 -
.../nifi-web-ui/src/main/webapp/css/graph.css | 262 --
.../nifi-web-ui/src/main/webapp/css/header.css | 683 ----
.../nifi-web-ui/src/main/webapp/css/history.css | 216 --
.../src/main/webapp/css/label-configuration.css | 55 -
.../nifi-web-ui/src/main/webapp/css/main.css | 298 --
.../src/main/webapp/css/message-page.css | 30 -
.../src/main/webapp/css/message-pane.css | 44 -
.../src/main/webapp/css/navigation.css | 292 --
.../main/webapp/css/new-processor-dialog.css | 168 -
.../src/main/webapp/css/port-configuration.css | 192 -
.../src/main/webapp/css/port-details.css | 55 -
.../webapp/css/process-group-configuration.css | 40 -
.../main/webapp/css/process-group-details.css | 27 -
.../main/webapp/css/processor-configuration.css | 388 --
.../src/main/webapp/css/processor-details.css | 119 -
.../src/main/webapp/css/provenance.css | 654 ----
.../src/main/webapp/css/registration.css | 53 -
.../css/remote-process-group-configuration.css | 233 --
.../nifi-web-ui/src/main/webapp/css/reset.css | 43 -
.../src/main/webapp/css/settings.css | 66 -
.../nifi-web-ui/src/main/webapp/css/shell.css | 98 -
.../src/main/webapp/css/status-history.css | 199 -
.../nifi-web-ui/src/main/webapp/css/summary.css | 768 ----
.../src/main/webapp/css/templates.css | 229 --
.../nifi-web-ui/src/main/webapp/css/users.css | 254 --
.../src/main/webapp/images/addConnect.png | Bin 1996 -> 0 bytes
.../src/main/webapp/images/bg-error.png | Bin 63766 -> 0 bytes
.../src/main/webapp/images/bgBannerFoot.png | Bin 189 -> 0 bytes
.../src/main/webapp/images/bgBannerHead.png | Bin 422 -> 0 bytes
.../src/main/webapp/images/bgBreadcrumbs.png | Bin 2130 -> 0 bytes
.../src/main/webapp/images/bgButton.png | Bin 234 -> 0 bytes
.../src/main/webapp/images/bgButtonOver.png | Bin 234 -> 0 bytes
.../src/main/webapp/images/bgButtonSelected.png | Bin 166 -> 0 bytes
.../main/webapp/images/bgButtonSelectedOver.png | Bin 168 -> 0 bytes
.../src/main/webapp/images/bgContextMenu.png | Bin 746 -> 0 bytes
.../src/main/webapp/images/bgControlsInset.png | Bin 618 -> 0 bytes
.../src/main/webapp/images/bgHeader.png | Bin 1697 -> 0 bytes
.../src/main/webapp/images/bgInputText.png | Bin 139 -> 0 bytes
.../src/main/webapp/images/bgNifiLogo.png | Bin 4232 -> 0 bytes
.../src/main/webapp/images/bgNifiToolbar.png | Bin 2123 -> 0 bytes
.../main/webapp/images/bgNifiToolbarEdge.png | Bin 541 -> 0 bytes
.../src/main/webapp/images/bgOutline.png | Bin 585 -> 0 bytes
.../src/main/webapp/images/bgPanCenter.png | Bin 362 -> 0 bytes
.../src/main/webapp/images/bgPanZoom.png | Bin 832 -> 0 bytes
.../webapp/images/bgProcessGroupDetailsArea.png | Bin 6596 -> 0 bytes
.../main/webapp/images/bgProcessorStatArea.png | Bin 4062 -> 0 bytes
.../images/bgRemoteProcessGroupDetailsArea.png | Bin 4433 -> 0 bytes
.../src/main/webapp/images/bgSearchInput.png | Bin 221 -> 0 bytes
.../src/main/webapp/images/bgShellClose.png | Bin 169 -> 0 bytes
.../src/main/webapp/images/bgStatusPanel.png | Bin 229 -> 0 bytes
.../src/main/webapp/images/bgTabContainer.png | Bin 234 -> 0 bytes
.../src/main/webapp/images/bgTableHeader.png | Bin 232 -> 0 bytes
.../main/webapp/images/bgToolbarBtnBorder.png | Bin 227 -> 0 bytes
.../src/main/webapp/images/blueBtnBg-over.jpg | Bin 356 -> 0 bytes
.../src/main/webapp/images/blueBtnBg.jpg | Bin 356 -> 0 bytes
.../src/main/webapp/images/buttonColor.png | Bin 1679 -> 0 bytes
.../src/main/webapp/images/buttonCopy.png | Bin 1166 -> 0 bytes
.../src/main/webapp/images/buttonDelete.png | Bin 1837 -> 0 bytes
.../src/main/webapp/images/buttonDisable.png | Bin 1737 -> 0 bytes
.../src/main/webapp/images/buttonEnable.png | Bin 1375 -> 0 bytes
.../src/main/webapp/images/buttonGroup.png | Bin 880 -> 0 bytes
.../main/webapp/images/buttonNewProperty.png | Bin 590 -> 0 bytes
.../src/main/webapp/images/buttonOutline.png | Bin 495 -> 0 bytes
.../src/main/webapp/images/buttonPanDown.png | Bin 554 -> 0 bytes
.../src/main/webapp/images/buttonPanLeft.png | Bin 588 -> 0 bytes
.../src/main/webapp/images/buttonPanRight.png | Bin 530 -> 0 bytes
.../src/main/webapp/images/buttonPanUp.png | Bin 551 -> 0 bytes
.../src/main/webapp/images/buttonPaste.png | Bin 1372 -> 0 bytes
.../src/main/webapp/images/buttonRefresh.png | Bin 915 -> 0 bytes
.../src/main/webapp/images/buttonRun.png | Bin 1163 -> 0 bytes
.../src/main/webapp/images/buttonStop.png | Bin 869 -> 0 bytes
.../src/main/webapp/images/buttonTemplate.png | Bin 1290 -> 0 bytes
.../src/main/webapp/images/buttonZoom100.png | Bin 449 -> 0 bytes
.../src/main/webapp/images/buttonZoomFit.png | Bin 499 -> 0 bytes
.../src/main/webapp/images/buttonZoomIn.png | Bin 435 -> 0 bytes
.../src/main/webapp/images/buttonZoomOut.png | Bin 339 -> 0 bytes
.../src/main/webapp/images/cluster.png | Bin 326 -> 0 bytes
.../src/main/webapp/images/grayBtnBg-over.jpg | Bin 319 -> 0 bytes
.../src/main/webapp/images/grayBtnBg.jpg | Bin 317 -> 0 bytes
.../nifi-web-ui/src/main/webapp/images/grid.gif | Bin 135 -> 0 bytes
.../src/main/webapp/images/headerTabBg.gif | Bin 131 -> 0 bytes
.../src/main/webapp/images/iconAdminUser.png | Bin 1960 -> 0 bytes
.../src/main/webapp/images/iconAlert.png | Bin 1396 -> 0 bytes
.../src/main/webapp/images/iconAlertDialog.png | Bin 1241 -> 0 bytes
.../src/main/webapp/images/iconAutoRefresh.png | Bin 3102 -> 0 bytes
.../src/main/webapp/images/iconBulletin.png | Bin 1066 -> 0 bytes
.../main/webapp/images/iconBulletinBoard.png | Bin 2523 -> 0 bytes
.../src/main/webapp/images/iconCenterView.png | Bin 338 -> 0 bytes
.../src/main/webapp/images/iconChart.png | Bin 510 -> 0 bytes
.../src/main/webapp/images/iconClose.png | Bin 231 -> 0 bytes
.../src/main/webapp/images/iconCluster.png | Bin 2112 -> 0 bytes
.../src/main/webapp/images/iconClusterSmall.png | Bin 757 -> 0 bytes
.../main/webapp/images/iconCollapseChildren.png | Bin 571 -> 0 bytes
.../main/webapp/images/iconCollapseParents.png | Bin 645 -> 0 bytes
.../src/main/webapp/images/iconColor.png | Bin 738 -> 0 bytes
.../src/main/webapp/images/iconCommit.png | Bin 445 -> 0 bytes
.../src/main/webapp/images/iconConfigure.png | Bin 696 -> 0 bytes
.../src/main/webapp/images/iconConnect.png | Bin 589 -> 0 bytes
.../src/main/webapp/images/iconConnection.png | Bin 1517 -> 0 bytes
.../src/main/webapp/images/iconCopy.png | Bin 514 -> 0 bytes
.../src/main/webapp/images/iconCounter.png | Bin 1369 -> 0 bytes
.../src/main/webapp/images/iconDelete.png | Bin 670 -> 0 bytes
.../src/main/webapp/images/iconDetails.png | Bin 549 -> 0 bytes
.../src/main/webapp/images/iconDisable.png | Bin 764 -> 0 bytes
.../src/main/webapp/images/iconDisconnect.png | Bin 569 -> 0 bytes
.../src/main/webapp/images/iconEdit.png | Bin 493 -> 0 bytes
.../src/main/webapp/images/iconEditButton.png | Bin 915 -> 0 bytes
.../main/webapp/images/iconExpandChildren.png | Bin 521 -> 0 bytes
.../main/webapp/images/iconExpandParents.png | Bin 585 -> 0 bytes
.../src/main/webapp/images/iconExport.png | Bin 453 -> 0 bytes
.../main/webapp/images/iconExportLineage.png | Bin 405 -> 0 bytes
.../src/main/webapp/images/iconFlowSettings.png | Bin 2624 -> 0 bytes
.../src/main/webapp/images/iconFunnel.png | Bin 1223 -> 0 bytes
.../src/main/webapp/images/iconGoTo.png | Bin 448 -> 0 bytes
.../src/main/webapp/images/iconHistory.png | Bin 2347 -> 0 bytes
.../src/main/webapp/images/iconInfo.png | Bin 550 -> 0 bytes
.../src/main/webapp/images/iconInputPort.png | Bin 1842 -> 0 bytes
.../main/webapp/images/iconInputPortSmall.png | Bin 532 -> 0 bytes
.../webapp/images/iconIsolatedProcessor.png | Bin 1781 -> 0 bytes
.../src/main/webapp/images/iconLineage.png | Bin 432 -> 0 bytes
.../src/main/webapp/images/iconLoading.gif | Bin 2438 -> 0 bytes
.../src/main/webapp/images/iconNotSecure.png | Bin 137 -> 0 bytes
.../src/main/webapp/images/iconOutputPort.png | Bin 1658 -> 0 bytes
.../main/webapp/images/iconOutputPortSmall.png | Bin 459 -> 0 bytes
.../src/main/webapp/images/iconPaste.png | Bin 601 -> 0 bytes
.../src/main/webapp/images/iconPopOut.png | Bin 217 -> 0 bytes
.../webapp/images/iconPortNotTransmitting.png | Bin 231 -> 0 bytes
.../src/main/webapp/images/iconPortRemoved.png | Bin 456 -> 0 bytes
.../main/webapp/images/iconPortTransmitting.png | Bin 235 -> 0 bytes
.../src/main/webapp/images/iconPrimary.png | Bin 647 -> 0 bytes
.../src/main/webapp/images/iconProcessor.png | Bin 1446 -> 0 bytes
.../src/main/webapp/images/iconProvenance.png | Bin 1104 -> 0 bytes
.../src/main/webapp/images/iconRefresh.png | Bin 492 -> 0 bytes
.../src/main/webapp/images/iconRemotePorts.png | Bin 456 -> 0 bytes
.../webapp/images/iconRemoteProcessGroup.png | Bin 674 -> 0 bytes
.../src/main/webapp/images/iconReporting.png | Bin 803 -> 0 bytes
.../src/main/webapp/images/iconResetCounter.png | Bin 304 -> 0 bytes
.../src/main/webapp/images/iconResize.png | Bin 142 -> 0 bytes
.../src/main/webapp/images/iconRevoke.png | Bin 676 -> 0 bytes
.../src/main/webapp/images/iconRun.png | Bin 538 -> 0 bytes
.../src/main/webapp/images/iconSecure.png | Bin 133 -> 0 bytes
.../src/main/webapp/images/iconSmallFunnel.png | Bin 455 -> 0 bytes
.../main/webapp/images/iconSmallInputPort.png | Bin 216 -> 0 bytes
.../main/webapp/images/iconSmallOutputPort.png | Bin 264 -> 0 bytes
.../webapp/images/iconSmallProcessGroup.png | Bin 479 -> 0 bytes
.../main/webapp/images/iconSmallProcessor.png | Bin 647 -> 0 bytes
.../webapp/images/iconSmallRelationship.png | Bin 770 -> 0 bytes
.../images/iconSmallRemoteProcessGroup.png | Bin 402 -> 0 bytes
.../src/main/webapp/images/iconStop.png | Bin 402 -> 0 bytes
.../src/main/webapp/images/iconTemplates.png | Bin 2659 -> 0 bytes
.../src/main/webapp/images/iconToFront.png | Bin 475 -> 0 bytes
.../webapp/images/iconTransmissionActive.png | Bin 1330 -> 0 bytes
.../webapp/images/iconTransmissionInactive.png | Bin 1248 -> 0 bytes
.../src/main/webapp/images/iconTwistArrow.png | Bin 256 -> 0 bytes
.../src/main/webapp/images/iconUndo.png | Bin 642 -> 0 bytes
.../src/main/webapp/images/iconUsage.png | Bin 470 -> 0 bytes
.../src/main/webapp/images/inputCheckbox.png | Bin 389 -> 0 bytes
.../src/main/webapp/images/loadAnimation.gif | Bin 10789 -> 0 bytes
.../src/main/webapp/images/nifi16.ico | Bin 1150 -> 0 bytes
.../src/main/webapp/images/nifi_about.png | Bin 16323 -> 0 bytes
.../src/main/webapp/images/panelBg.jpg | Bin 312 -> 0 bytes
.../src/main/webapp/images/portRemoved.png | Bin 1090 -> 0 bytes
.../src/main/webapp/images/portRunning.png | Bin 337 -> 0 bytes
.../src/main/webapp/images/portStopped.png | Bin 192 -> 0 bytes
.../src/main/webapp/images/spacer.png | Bin 110 -> 0 bytes
.../src/main/webapp/images/starburst.png | Bin 190 -> 0 bytes
.../src/main/webapp/images/tabBg.jpg | Bin 320 -> 0 bytes
.../src/main/webapp/images/toolbarBg.gif | Bin 45 -> 0 bytes
.../main/webapp/images/toolboxIconFunnel.png | Bin 2409 -> 0 bytes
.../main/webapp/images/toolboxIconInputPort.png | Bin 2680 -> 0 bytes
.../src/main/webapp/images/toolboxIconLabel.png | Bin 2871 -> 0 bytes
.../webapp/images/toolboxIconOutputPort.png | Bin 2209 -> 0 bytes
.../webapp/images/toolboxIconProcessGroup.png | Bin 4176 -> 0 bytes
.../main/webapp/images/toolboxIconProcessor.png | Bin 4426 -> 0 bytes
.../images/toolboxIconRemoteProcessGroup.png | Bin 5653 -> 0 bytes
.../main/webapp/images/toolboxIconTemplate.png | Bin 2913 -> 0 bytes
.../images/transmissionSwitchDisabled.png | Bin 1240 -> 0 bytes
.../webapp/images/transmissionSwitchEnabled.png | Bin 1600 -> 0 bytes
.../src/main/webapp/images/ungroup.png | Bin 3409 -> 0 bytes
.../src/main/webapp/js/codemirror/LICENSE | 19 -
.../js/codemirror/addon/hint/show-hint.css | 38 -
.../js/codemirror/lib/codemirror-compressed.js | 16 -
.../webapp/js/codemirror/lib/codemirror.css | 318 --
.../nifi-web-ui/src/main/webapp/js/d3/LICENSE | 26 -
.../nifi-web-ui/src/main/webapp/js/d3/d3.min.js | 5 -
.../webapp/js/jquery/combo/jquery.combo.css | 82 -
.../main/webapp/js/jquery/combo/jquery.combo.js | 311 --
.../main/webapp/js/jquery/combo/menuArrow.png | Bin 251 -> 0 bytes
.../webapp/js/jquery/combo/menuArrowSmall.png | Bin 233 -> 0 bytes
.../main/webapp/js/jquery/jquery-2.1.1.min.js | 4 -
.../src/main/webapp/js/jquery/jquery.center.js | 35 -
.../src/main/webapp/js/jquery/jquery.count.js | 99 -
.../src/main/webapp/js/jquery/jquery.each.js | 31 -
.../main/webapp/js/jquery/jquery.ellipsis.js | 164 -
.../js/jquery/jquery.event.drag-2.2.min.js | 6 -
.../main/webapp/js/jquery/jquery.form.min.js | 11 -
.../src/main/webapp/js/jquery/jquery.tab.js | 69 -
.../js/jquery/minicolors/jquery.minicolors.css | 259 --
.../jquery/minicolors/jquery.minicolors.min.js | 9 -
.../js/jquery/minicolors/jquery.minicolors.png | Bin 77459 -> 0 bytes
.../webapp/js/jquery/modal/jquery.modal.css | 110 -
.../main/webapp/js/jquery/modal/jquery.modal.js | 266 --
.../js/jquery/nfeditor/jquery.nfeditor.css | 73 -
.../js/jquery/nfeditor/jquery.nfeditor.js | 315 --
.../js/jquery/nfeditor/languages/nfel.css | 51 -
.../webapp/js/jquery/nfeditor/languages/nfel.js | 831 ----
.../webapp/js/jquery/qtip2/jquery.qtip.min.css | 3 -
.../webapp/js/jquery/qtip2/jquery.qtip.min.js | 5 -
.../webapp/js/jquery/slickgrid/MIT-LICENSE.txt | 20 -
.../js/jquery/slickgrid/css/images/collapse.gif | Bin 846 -> 0 bytes
.../js/jquery/slickgrid/css/images/expand.gif | Bin 851 -> 0 bytes
.../js/jquery/slickgrid/css/images/sort-asc.gif | Bin 59 -> 0 bytes
.../jquery/slickgrid/css/images/sort-desc.gif | Bin 59 -> 0 bytes
.../slickgrid/css/slick-default-theme.css | 135 -
.../js/jquery/slickgrid/css/slick.grid.css | 158 -
.../slickgrid/plugins/slick.autotooltips.js | 83 -
.../plugins/slick.cellrangedecorator.js | 66 -
.../plugins/slick.cellrangeselector.js | 113 -
.../plugins/slick.cellselectionmodel.js | 154 -
.../plugins/slick.rowselectionmodel.js | 187 -
.../webapp/js/jquery/slickgrid/slick.core.js | 467 ---
.../js/jquery/slickgrid/slick.dataview.js | 1126 ------
.../webapp/js/jquery/slickgrid/slick.editors.js | 512 ---
.../js/jquery/slickgrid/slick.formatters.js | 59 -
.../webapp/js/jquery/slickgrid/slick.grid.js | 3422 -----------------
.../webapp/js/jquery/tabbs/jquery.tabbs.css | 48 -
.../main/webapp/js/jquery/tabbs/jquery.tabbs.js | 96 -
.../ui-smoothness/images/animated-overlay.gif | Bin 1738 -> 0 bytes
.../images/ui-bg_flat_0_aaaaaa_40x100.png | Bin 212 -> 0 bytes
.../images/ui-bg_flat_75_ffffff_40x100.png | Bin 208 -> 0 bytes
.../images/ui-bg_glass_55_fbf9ee_1x400.png | Bin 335 -> 0 bytes
.../images/ui-bg_glass_65_ffffff_1x400.png | Bin 207 -> 0 bytes
.../images/ui-bg_glass_75_dadada_1x400.png | Bin 262 -> 0 bytes
.../images/ui-bg_glass_75_e6e6e6_1x400.png | Bin 262 -> 0 bytes
.../images/ui-bg_glass_95_fef1ec_1x400.png | Bin 332 -> 0 bytes
.../ui-bg_highlight-soft_75_cccccc_1x100.png | Bin 280 -> 0 bytes
.../images/ui-icons_222222_256x240.png | Bin 6922 -> 0 bytes
.../images/ui-icons_2e83ff_256x240.png | Bin 4549 -> 0 bytes
.../images/ui-icons_454545_256x240.png | Bin 6992 -> 0 bytes
.../images/ui-icons_888888_256x240.png | Bin 6999 -> 0 bytes
.../images/ui-icons_cd0a0a_256x240.png | Bin 4549 -> 0 bytes
.../ui-smoothness/jquery-ui-1.10.4.min.css | 7 -
.../ui-smoothness/jquery-ui-1.10.4.min.js | 7 -
.../web/nifi-web-ui/src/main/webapp/js/json2.js | 475 ---
.../js/nf/bulletin-board/nf-bulletin-board.js | 422 --
.../src/main/webapp/js/nf/canvas/nf-actions.js | 1150 ------
.../src/main/webapp/js/nf/canvas/nf-birdseye.js | 340 --
.../webapp/js/nf/canvas/nf-canvas-header.js | 293 --
.../webapp/js/nf/canvas/nf-canvas-toolbar.js | 174 -
.../webapp/js/nf/canvas/nf-canvas-toolbox.js | 1211 ------
.../main/webapp/js/nf/canvas/nf-canvas-utils.js | 1307 -------
.../src/main/webapp/js/nf/canvas/nf-canvas.js | 1530 --------
.../main/webapp/js/nf/canvas/nf-clipboard.js | 96 -
.../main/webapp/js/nf/canvas/nf-connectable.js | 210 -
.../js/nf/canvas/nf-connection-configuration.js | 1381 -------
.../main/webapp/js/nf/canvas/nf-connection.js | 1533 --------
.../main/webapp/js/nf/canvas/nf-context-menu.js | 452 ---
.../js/nf/canvas/nf-custom-processor-ui.js | 43 -
.../main/webapp/js/nf/canvas/nf-draggable.js | 319 --
.../src/main/webapp/js/nf/canvas/nf-funnel.js | 279 --
.../src/main/webapp/js/nf/canvas/nf-go-to.js | 730 ----
.../webapp/js/nf/canvas/nf-graph-control.js | 137 -
.../src/main/webapp/js/nf/canvas/nf-graph.js | 194 -
.../js/nf/canvas/nf-label-configuration.js | 132 -
.../src/main/webapp/js/nf/canvas/nf-label.js | 559 ---
.../js/nf/canvas/nf-port-configuration.js | 170 -
.../main/webapp/js/nf/canvas/nf-port-details.js | 63 -
.../src/main/webapp/js/nf/canvas/nf-port.js | 632 ---
.../nf/canvas/nf-process-group-configuration.js | 107 -
.../js/nf/canvas/nf-process-group-details.js | 61 -
.../webapp/js/nf/canvas/nf-process-group.js | 1064 -----
.../js/nf/canvas/nf-processor-configuration.js | 734 ----
.../nf-processor-property-combo-editor.js | 175 -
.../canvas/nf-processor-property-nfel-editor.js | 207 -
.../js/nf/canvas/nf-processor-property-table.js | 567 ---
.../canvas/nf-processor-property-text-editor.js | 212 -
.../main/webapp/js/nf/canvas/nf-processor.js | 831 ----
.../main/webapp/js/nf/canvas/nf-registration.js | 68 -
.../nf-remote-process-group-configuration.js | 124 -
.../canvas/nf-remote-process-group-details.js | 67 -
.../nf/canvas/nf-remote-process-group-ports.js | 525 ---
.../js/nf/canvas/nf-remote-process-group.js | 1060 -----
.../src/main/webapp/js/nf/canvas/nf-search.js | 179 -
.../nf/canvas/nf-secure-port-configuration.js | 384 --
.../js/nf/canvas/nf-secure-port-details.js | 118 -
.../main/webapp/js/nf/canvas/nf-selectable.js | 58 -
.../src/main/webapp/js/nf/canvas/nf-settings.js | 146 -
.../src/main/webapp/js/nf/canvas/nf-snippet.js | 212 -
.../src/main/webapp/js/nf/canvas/nf-storage.js | 136 -
.../webapp/js/nf/canvas/nf-toolbar-action.js | 86 -
.../webapp/js/nf/cluster/nf-cluster-table.js | 620 ---
.../src/main/webapp/js/nf/cluster/nf-cluster.js | 154 -
.../webapp/js/nf/counters/nf-counters-table.js | 294 --
.../main/webapp/js/nf/counters/nf-counters.js | 153 -
.../webapp/js/nf/history/nf-history-model.js | 218 --
.../webapp/js/nf/history/nf-history-table.js | 429 ---
.../src/main/webapp/js/nf/history/nf-history.js | 154 -
.../src/main/webapp/js/nf/nf-client.js | 56 -
.../src/main/webapp/js/nf/nf-common.js | 899 -----
.../main/webapp/js/nf/nf-connection-details.js | 467 ---
.../src/main/webapp/js/nf/nf-dialog.js | 128 -
.../src/main/webapp/js/nf/nf-namespace.js | 20 -
.../main/webapp/js/nf/nf-processor-details.js | 586 ---
.../src/main/webapp/js/nf/nf-shell.js | 167 -
.../src/main/webapp/js/nf/nf-status-history.js | 1320 -------
.../js/nf/provenance/nf-provenance-lineage.js | 1413 -------
.../js/nf/provenance/nf-provenance-table.js | 1371 -------
.../webapp/js/nf/provenance/nf-provenance.js | 204 -
.../webapp/js/nf/summary/nf-cluster-search.js | 183 -
.../webapp/js/nf/summary/nf-summary-table.js | 2436 ------------
.../src/main/webapp/js/nf/summary/nf-summary.js | 174 -
.../js/nf/templates/nf-templates-table.js | 332 --
.../main/webapp/js/nf/templates/nf-templates.js | 235 --
.../main/webapp/js/nf/users/nf-users-table.js | 1090 ------
.../src/main/webapp/js/nf/users/nf-users.js | 149 -
.../framework-bundle/framework/web/pom.xml | 74 -
.../web/web-optimistic-locking/.gitignore | 1 -
.../web/web-optimistic-locking/pom.xml | 32 -
.../apache/nifi/web/ConfigurationSnapshot.java | 66 -
.../nifi/web/OptimisticLockingManager.java | 95 -
.../web/StandardOptimisticLockingManager.java | 77 -
.../framework/web/web-security/.gitignore | 1 -
.../framework/web/web-security/pom.xml | 81 -
.../org/apache/nifi/web/security/DnUtils.java | 87 -
.../web/security/UntrustedProxyException.java | 34 -
.../anonymous/NiFiAnonymousUserFilter.java | 101 -
.../NiFiAuthenticationEntryPoint.java | 68 -
.../authorization/NiFiAuthorizationService.java | 173 -
.../authorization/NodeAuthorizedUserFilter.java | 128 -
.../nifi/web/security/user/NiFiUserDetails.java | 110 -
.../nifi/web/security/user/NiFiUserUtils.java | 82 -
.../x509/SubjectDnX509PrincipalExtractor.java | 38 -
.../security/x509/X509AuthenticationFilter.java | 349 --
.../security/x509/X509CertificateExtractor.java | 52 -
.../x509/ocsp/CertificateStatusException.java | 42 -
.../x509/ocsp/OcspCertificateValidator.java | 444 ---
.../web/security/x509/ocsp/OcspRequest.java | 69 -
.../nifi/web/security/x509/ocsp/OcspStatus.java | 82 -
.../resources/nifi-web-security-context.xml | 83 -
.../NiFiAuthorizationServiceTest.java | 250 --
nar-bundles/framework-bundle/nar/.gitignore | 1 -
nar-bundles/framework-bundle/nar/pom.xml | 66 -
nar-bundles/framework-bundle/pom.xml | 114 -
.../hadoop-bundle/hdfs-processors/pom.xml | 65 -
.../hadoop/AbstractHadoopProcessor.java | 221 --
.../hadoop/CreateHadoopSequenceFile.java | 177 -
...lowFileStreamUnpackerSequenceFileWriter.java | 170 -
.../apache/nifi/processors/hadoop/GetHDFS.java | 563 ---
.../processors/hadoop/GetHDFSSequenceFile.java | 146 -
.../nifi/processors/hadoop/KeyValueReader.java | 132 -
.../apache/nifi/processors/hadoop/PutHDFS.java | 403 --
.../hadoop/SequenceFileWriterImpl.java | 119 -
.../hadoop/TarUnpackerSequenceFileWriter.java | 54 -
.../nifi/processors/hadoop/ValueReader.java | 116 -
.../hadoop/ZipUnpackerSequenceFileWriter.java | 57 -
.../hadoop/util/ByteFilteringOutputStream.java | 163 -
.../hadoop/util/InputStreamWritable.java | 61 -
.../hadoop/util/OutputStreamWritable.java | 85 -
.../hadoop/util/SequenceFileReader.java | 28 -
.../hadoop/util/SequenceFileWriter.java | 40 -
.../org.apache.nifi.processor.Processor | 18 -
.../index.html | 88 -
.../index.html | 162 -
.../index.html | 150 -
.../index.html | 159 -
.../processors/hadoop/AbstractHadoopTest.java | 90 -
.../nifi/processors/hadoop/GetHDFSTest.java | 123 -
.../nifi/processors/hadoop/PutHDFSTest.java | 158 -
.../hadoop/SimpleHadoopProcessor.java | 30 -
.../hadoop/TestCreateHadoopSequenceFile.java | 176 -
.../src/test/resources/core-site-broken.xml | 25 -
.../src/test/resources/core-site.xml | 25 -
.../resources/testdata/13545312236534130.tar | Bin 1505280 -> 0 bytes
.../resources/testdata/13545423550275052.zip | Bin 1500841 -> 0 bytes
.../resources/testdata/13545479542069498.pkg | Bin 1500429 -> 0 bytes
.../src/test/resources/testdata/randombytes-1 | Bin 500000 -> 0 bytes
.../src/test/resources/testdata/randombytes-2 | Bin 500000 -> 0 bytes
.../src/test/resources/testdata/randombytes-3 | Bin 500000 -> 0 bytes
nar-bundles/hadoop-bundle/nar/pom.xml | 41 -
nar-bundles/hadoop-bundle/pom.xml | 46 -
nar-bundles/hadoop-libraries-bundle/nar/pom.xml | 45 -
nar-bundles/hadoop-libraries-bundle/pom.xml | 36 -
nar-bundles/jetty-bundle/pom.xml | 82 -
nar-bundles/kafka-bundle/kafka-nar/pom.xml | 37 -
.../kafka-bundle/kafka-processors/pom.xml | 76 -
.../apache/nifi/processors/kafka/GetKafka.java | 330 --
.../apache/nifi/processors/kafka/PutKafka.java | 419 --
.../org.apache.nifi.processor.Processor | 16 -
.../index.html | 173 -
.../index.html | 189 -
.../nifi/processors/kafka/TestGetKafka.java | 162 -
.../nifi/processors/kafka/TestPutKafka.java | 236 --
nar-bundles/kafka-bundle/pom.xml | 35 -
.../monitor-threshold-bundle/nar/pom.xml | 39 -
nar-bundles/monitor-threshold-bundle/pom.xml | 51 -
.../monitor-threshold-bundle/processor/pom.xml | 45 -
.../processors/monitor/MonitorThreshold.java | 904 -----
.../processors/monitor/ThresholdsParser.java | 134 -
.../org.apache.nifi.processor.Processor | 15 -
.../index.html | 485 ---
.../monitor/TestMonitorThreshold.java | 1560 --------
.../processor/src/test/resources/testFile | 23 -
.../resources/threshold_settings_allow_0.xml | 21 -
.../resources/threshold_settings_allow_1.xml | 21 -
.../threshold_settings_allow_10_KB.xml | 21 -
.../threshold_settings_allow_10_KB_and_2_KB.xml | 26 -
.../threshold_settings_allow_1669_bytes.xml | 21 -
.../resources/threshold_settings_allow_2.xml | 21 -
.../threshold_settings_allow_2_default.xml | 21 -
...eshold_settings_allow_3336_bytes_default.xml | 21 -
...hold_settings_with_empty_attribute_value.xml | 21 -
...ettings_with_spaces_for_attribute_values.xml | 22 -
.../ui/nb-configuration.xml | 18 -
nar-bundles/monitor-threshold-bundle/ui/pom.xml | 134 -
.../thresholds/ui/AttributeComparator.java | 36 -
.../thresholds/ui/AttributeResource.java | 242 --
.../monitor/thresholds/ui/RuleComparator.java | 46 -
.../thresholds/ui/ThresholdResource.java | 227 --
.../ui/ThresholdSettingsResource.java | 191 -
.../thresholds/ui/ThresholdsConfigFile.java | 296 --
.../ui/src/main/webapp/META-INF/nifi-processor | 15 -
.../webapp/WEB-INF/jsp/attribute-add-dialog.jsp | 43 -
.../WEB-INF/jsp/attribute-confirm-dialog.jsp | 21 -
.../WEB-INF/jsp/attribute-edit-dialog.jsp | 27 -
.../WEB-INF/jsp/attribute-filter-dialog.jsp | 26 -
.../main/webapp/WEB-INF/jsp/error-dialog.jsp | 21 -
.../ui/src/main/webapp/WEB-INF/jsp/index.jsp | 101 -
.../webapp/WEB-INF/jsp/threshold-add-dialog.jsp | 45 -
.../WEB-INF/jsp/threshold-confirm-dialog.jsp | 21 -
.../webapp/WEB-INF/jsp/threshold-dialog.jsp | 48 -
.../WEB-INF/jsp/threshold-edit-dialog.jsp | 45 -
.../WEB-INF/jsp/threshold-filter-dialog.jsp | 40 -
.../ui/src/main/webapp/WEB-INF/web.xml | 48 -
.../ui/src/main/webapp/css/threshold_styles.css | 422 --
.../src/main/webapp/images/addWorksheetRow.png | Bin 323 -> 0 bytes
.../ui/src/main/webapp/images/clear.png | Bin 912 -> 0 bytes
.../ui/src/main/webapp/images/filter.gif | Bin 623 -> 0 bytes
.../ui/src/main/webapp/images/firstPage.gif | Bin 577 -> 0 bytes
.../main/webapp/images/firstPageDisabled.gif | Bin 365 -> 0 bytes
.../ui/src/main/webapp/images/lastPage.gif | Bin 579 -> 0 bytes
.../src/main/webapp/images/lastPageDisabled.gif | Bin 368 -> 0 bytes
.../ui/src/main/webapp/images/nextPage.gif | Bin 570 -> 0 bytes
.../src/main/webapp/images/nextPageDisabled.gif | Bin 362 -> 0 bytes
.../ui/src/main/webapp/images/prevPage.gif | Bin 567 -> 0 bytes
.../src/main/webapp/images/prevPageDisabled.gif | Bin 361 -> 0 bytes
.../main/webapp/images/removeWorksheetRow.png | Bin 655 -> 0 bytes
.../ui/src/main/webapp/images/separator.gif | Bin 63 -> 0 bytes
.../webapp/js/jquery/jqgrid/css/ui.jqgrid.css | 171 -
.../js/jquery/jqgrid/js/i18n/grid.locale-en.js | 170 -
.../js/jquery/jqgrid/js/jquery.jqGrid.min.js | 545 ---
.../ui/src/main/webapp/js/nf-common.js | 888 -----
.../ui/src/main/xsd/threshold_settings.xsd | 54 -
.../nar/pom.xml | 36 -
.../persistent-provenance-repository/pom.xml | 58 -
.../nifi/provenance/IndexConfiguration.java | 398 --
.../PersistentProvenanceRepository.java | 1892 ---------
.../provenance/RepositoryConfiguration.java | 299 --
.../nifi/provenance/StandardRecordReader.java | 305 --
.../nifi/provenance/StandardRecordWriter.java | 237 --
.../provenance/expiration/ExpirationAction.java | 35 -
.../expiration/FileRemovalAction.java | 49 -
.../provenance/lucene/DeleteIndexAction.java | 139 -
.../nifi/provenance/lucene/DocsReader.java | 133 -
.../nifi/provenance/lucene/FieldNames.java | 23 -
.../nifi/provenance/lucene/IndexSearch.java | 84 -
.../nifi/provenance/lucene/IndexingAction.java | 219 --
.../nifi/provenance/lucene/LineageQuery.java | 108 -
.../nifi/provenance/lucene/LuceneUtil.java | 141 -
.../provenance/rollover/CompressionAction.java | 59 -
.../provenance/rollover/RolloverAction.java | 35 -
.../provenance/serialization/RecordReader.java | 31 -
.../provenance/serialization/RecordReaders.java | 70 -
.../provenance/serialization/RecordWriter.java | 85 -
.../provenance/serialization/RecordWriters.java | 30 -
...he.nifi.provenance.ProvenanceEventRepository | 15 -
.../TestPersistentProvenanceRepository.java | 1118 ------
.../pom.xml | 42 -
nar-bundles/pom.xml | 134 -
.../standard-bundle/jms-processors/pom.xml | 55 -
.../apache/nifi/processors/jms/GetJMSQueue.java | 75 -
.../apache/nifi/processors/jms/GetJMSTopic.java | 359 --
.../apache/nifi/processors/jms/JmsConsumer.java | 208 -
.../org/apache/nifi/processors/jms/PutJMS.java | 374 --
.../nifi/processors/jms/util/JmsFactory.java | 435 ---
.../nifi/processors/jms/util/JmsProperties.java | 176 -
.../jms/util/WrappedMessageConsumer.java | 77 -
.../jms/util/WrappedMessageProducer.java | 77 -
.../org.apache.nifi.processor.Processor | 17 -
.../index.html | 118 -
.../index.html | 122 -
.../index.html | 152 -
.../nifi/processors/jms/GetJMSQueueTest.java | 141 -
nar-bundles/standard-bundle/nar/pom.xml | 57 -
nar-bundles/standard-bundle/pom.xml | 71 -
.../standard-ganglia-reporter/pom.xml | 52 -
.../ganglia/StandardGangliaReporter.java | 262 --
.../org.apache.nifi.reporting.ReportingTask | 15 -
.../index.html | 65 -
.../standard-prioritizers/pom.xml | 40 -
.../prioritizer/FirstInFirstOutPrioritizer.java | 37 -
.../NewestFlowFileFirstPrioritizer.java | 37 -
.../OldestFlowFileFirstPrioritizer.java | 37 -
.../PriorityAttributePrioritizer.java | 87 -
...org.apache.nifi.flowfile.FlowFilePrioritizer | 18 -
.../prioritizer/NewestFirstPrioritizerTest.java | 66 -
.../prioritizer/OldestFirstPrioritizerTest.java | 66 -
.../PriorityAttributePrioritizerTest.java | 117 -
.../standard-bundle/standard-processors/pom.xml | 154 -
.../standard/Base64EncodeContent.java | 142 -
.../processors/standard/CompressContent.java | 307 --
.../nifi/processors/standard/ControlRate.java | 381 --
.../standard/ConvertCharacterSet.java | 175 -
.../processors/standard/DetectDuplicate.java | 249 --
.../processors/standard/DistributeLoad.java | 498 ---
.../processors/standard/EncryptContent.java | 263 --
.../standard/EvaluateRegularExpression.java | 294 --
.../nifi/processors/standard/EvaluateXPath.java | 429 ---
.../processors/standard/EvaluateXQuery.java | 463 ---
.../standard/ExecuteStreamCommand.java | 358 --
.../processors/standard/GenerateFlowFile.java | 164 -
.../apache/nifi/processors/standard/GetFTP.java | 72 -
.../nifi/processors/standard/GetFile.java | 456 ---
.../processors/standard/GetFileTransfer.java | 301 --
.../nifi/processors/standard/GetHTTP.java | 480 ---
.../nifi/processors/standard/GetSFTP.java | 92 -
.../nifi/processors/standard/HashAttribute.java | 255 --
.../nifi/processors/standard/HashContent.java | 151 -
.../processors/standard/IdentifyMimeType.java | 474 ---
.../nifi/processors/standard/InvokeHTTP.java | 758 ----
.../nifi/processors/standard/ListenHTTP.java | 323 --
.../nifi/processors/standard/ListenUDP.java | 627 ---
.../nifi/processors/standard/LogAttribute.java | 262 --
.../nifi/processors/standard/MergeContent.java | 1015 -----
.../nifi/processors/standard/ModifyBytes.java | 134 -
.../processors/standard/MonitorActivity.java | 206 -
.../nifi/processors/standard/PostHTTP.java | 904 -----
.../nifi/processors/standard/PutEmail.java | 297 --
.../apache/nifi/processors/standard/PutFTP.java | 141 -
.../nifi/processors/standard/PutFile.java | 367 --
.../processors/standard/PutFileTransfer.java | 293 --
.../nifi/processors/standard/PutSFTP.java | 85 -
.../nifi/processors/standard/ReplaceText.java | 289 --
.../standard/ReplaceTextWithMapping.java | 383 --
.../processors/standard/RouteOnAttribute.java | 261 --
.../processors/standard/RouteOnContent.java | 232 --
.../nifi/processors/standard/ScanAttribute.java | 229 --
.../nifi/processors/standard/ScanContent.java | 292 --
.../processors/standard/SegmentContent.java | 163 -
.../nifi/processors/standard/SplitContent.java | 260 --
.../nifi/processors/standard/SplitText.java | 370 --
.../nifi/processors/standard/SplitXml.java | 300 --
.../nifi/processors/standard/TransformXml.java | 194 -
.../nifi/processors/standard/UnpackContent.java | 427 ---
.../nifi/processors/standard/ValidateXml.java | 147 -
.../servlets/ContentAcknowledgmentServlet.java | 136 -
.../standard/servlets/ListenHTTPServlet.java | 325 --
.../nifi/processors/standard/util/Bin.java | 168 -
.../processors/standard/util/BinManager.java | 241 --
.../standard/util/DocumentReaderCallback.java | 74 -
.../processors/standard/util/FTPTransfer.java | 541 ---
.../nifi/processors/standard/util/FTPUtils.java | 322 --
.../nifi/processors/standard/util/FileInfo.java | 167 -
.../processors/standard/util/FileTransfer.java | 244 --
.../standard/util/FlowFileSessionWrapper.java | 44 -
.../standard/util/NLKBufferedReader.java | 187 -
.../standard/util/SFTPConnection.java | 82 -
.../processors/standard/util/SFTPTransfer.java | 637 ---
.../processors/standard/util/SFTPUtils.java | 324 --
.../standard/util/UDPStreamConsumer.java | 214 --
.../util/ValidatingBase64InputStream.java | 76 -
.../standard/util/XmlElementNotifier.java | 22 -
.../standard/util/XmlSplitterSaxParser.java | 112 -
.../org.apache.nifi.processor.Processor | 58 -
.../index.html | 63 -
.../index.html | 166 -
.../index.html | 116 -
.../index.html | 65 -
.../index.html | 147 -
.../index.html | 106 -
.../index.html | 97 -
.../index.html | 160 -
.../index.html | 135 -
.../index.html | 311 --
.../index.html | 111 -
.../index.html | 64 -
.../index.html | 227 --
.../index.html | 186 -
.../index.html | 143 -
.../index.html | 250 --
.../index.html | 88 -
.../index.html | 89 -
.../index.html | 136 -
.../index.html | 181 -
.../index.html | 86 -
.../index.html | 144 -
.../index.html | 80 -
.../index.html | 347 --
.../index.html | 64 -
.../index.html | 143 -
.../index.html | 187 -
.../index.html | 114 -
.../index.html | 283 --
.../index.html | 109 -
.../index.html | 281 --
.../index.html | 91 -
.../index.html | 114 -
.../index.html | 110 -
.../index.html | 82 -
.../index.html | 85 -
.../index.html | 100 -
.../index.html | 123 -
.../index.html | 107 -
.../index.html | 121 -
.../index.html | 64 -
.../index.html | 63 -
.../index.html | 163 -
.../index.html | 56 -
.../src/test/java/TestIngestAndUpdate.java | 34 -
.../src/test/java/TestSuccess.java | 24 -
.../processors/standard/HelloWorldServlet.java | 41 -
.../standard/RESTServiceContentModified.java | 78 -
.../standard/TestBase64EncodeContent.java | 83 -
.../standard/TestCompressContent.java | 111 -
.../processors/standard/TestControlRate.java | 68 -
.../standard/TestConvertCharacterSet.java | 47 -
.../standard/TestDetectDuplicate.java | 206 -
.../processors/standard/TestDistributeLoad.java | 139 -
.../processors/standard/TestEncryptContent.java | 65 -
.../standard/TestEvaluateRegularExpression.java | 319 --
.../processors/standard/TestEvaluateXPath.java | 159 -
.../processors/standard/TestEvaluateXQuery.java | 651 ----
.../standard/TestExecuteStreamCommand.java | 188 -
.../nifi/processors/standard/TestGetFile.java | 186 -
.../nifi/processors/standard/TestGetHTTP.java | 354 --
.../processors/standard/TestHashAttribute.java | 99 -
.../processors/standard/TestHashContent.java | 67 -
.../standard/TestIdentifyMimeType.java | 131 -
.../processors/standard/TestInvokeHTTP.java | 593 ---
.../nifi/processors/standard/TestListenUDP.java | 214 --
.../processors/standard/TestMergeContent.java | 593 ---
.../processors/standard/TestModifyBytes.java | 190 -
.../standard/TestMonitorActivity.java | 193 -
.../processors/standard/TestReplaceText.java | 371 --
.../standard/TestReplaceTextLineByLine.java | 337 --
.../standard/TestReplaceTextWithMapping.java | 332 --
.../standard/TestRouteOnAttribute.java | 161 -
.../processors/standard/TestRouteOnContent.java | 73 -
.../processors/standard/TestScanAttribute.java | 149 -
.../processors/standard/TestScanContent.java | 85 -
.../processors/standard/TestSegmentContent.java | 65 -
.../nifi/processors/standard/TestServer.java | 177 -
.../processors/standard/TestSplitContent.java | 235 --
.../nifi/processors/standard/TestSplitText.java | 173 -
.../nifi/processors/standard/TestSplitXml.java | 103 -
.../processors/standard/TestTransformXml.java | 120 -
.../processors/standard/TestUnpackContent.java | 226 --
.../processors/standard/TestValidateXml.java | 42 -
.../standard/UserAgentTestingServlet.java | 42 -
.../CharacterSetConversionSamples/Converted.txt | 1 -
.../Converted2.txt | Bin 134580 -> 0 bytes
.../CharacterSetConversionSamples/Original.txt | 418 --
.../resources/CompressedData/SampleFile.txt | 418 --
.../resources/CompressedData/SampleFile.txt.bz2 | Bin 388 -> 0 bytes
.../resources/CompressedData/SampleFile.txt.gz | Bin 321 -> 0 bytes
.../CompressedData/SampleFile1.txt.bz2 | Bin 154 -> 0 bytes
.../resources/CompressedData/SampleFile1.txt.gz | Bin 321 -> 0 bytes
.../CompressedData/SampleFileConcat.txt | 835 ----
.../CompressedData/SampleFileConcat.txt.bz2 | Bin 542 -> 0 bytes
.../test/resources/ExecuteCommand/1000bytes.txt | 1 -
.../ExecuteCommand/TestIngestAndUpdate.jar | Bin 1170 -> 0 bytes
.../resources/ExecuteCommand/TestSuccess.jar | Bin 827 -> 0 bytes
.../src/test/resources/ExecuteCommand/test.txt | 2 -
.../dictionary-with-empty-new-lines | 5 -
.../ScanAttribute/dictionary-with-extra-info | 9 -
.../test/resources/ScanAttribute/dictionary1 | 5 -
.../test/resources/TestIdentifyMimeType/1.7z | Bin 133 -> 0 bytes
.../test/resources/TestIdentifyMimeType/1.jar | Bin 466 -> 0 bytes
.../test/resources/TestIdentifyMimeType/1.mdb | Bin 237568 -> 0 bytes
.../test/resources/TestIdentifyMimeType/1.pdf | Bin 277032 -> 0 bytes
.../test/resources/TestIdentifyMimeType/1.tar | Bin 2048 -> 0 bytes
.../resources/TestIdentifyMimeType/1.txt.bz2 | Bin 59 -> 0 bytes
.../resources/TestIdentifyMimeType/1.txt.gz | Bin 50 -> 0 bytes
.../test/resources/TestIdentifyMimeType/1.xml | 20 -
.../test/resources/TestIdentifyMimeType/1.zip | Bin 165 -> 0 bytes
.../TestIdentifyMimeType/bgBannerFoot.png | Bin 189 -> 0 bytes
.../TestIdentifyMimeType/blueBtnBg.jpg | Bin 356 -> 0 bytes
.../TestIdentifyMimeType/flowfilev1.tar | Bin 10240 -> 0 bytes
.../resources/TestIdentifyMimeType/flowfilev3 | Bin 40 -> 0 bytes
.../resources/TestIdentifyMimeType/grid.gif | Bin 135 -> 0 bytes
.../test/resources/TestMergeContent/demarcate | 1 -
.../src/test/resources/TestMergeContent/foot | 1 -
.../src/test/resources/TestMergeContent/head | 1 -
.../test/resources/TestModifyBytes/noFooter.txt | 10 -
.../TestModifyBytes/noFooter_noHeader.txt | 10 -
.../test/resources/TestModifyBytes/noHeader.txt | 11 -
.../test/resources/TestModifyBytes/testFile.txt | 11 -
.../TestReplaceTextLineByLine/$1$1.txt | 11 -
.../BRue_cRue_RiRey.txt | 11 -
.../TestReplaceTextLineByLine/Blu$2e_clu$2e.txt | 11 -
.../TestReplaceTextLineByLine/D$d_h$d.txt | 11 -
.../TestReplaceTextLineByLine/Good.txt | 1 -
.../TestReplaceTextLineByLine/Spider.txt | 11 -
.../TestReplaceTextLineByLine/[DODO].txt | 11 -
.../TestReplaceTextLineByLine/cu[$1]_Po[$1].txt | 11 -
.../TestReplaceTextLineByLine/cu_Po.txt | 11 -
.../TestReplaceTextLineByLine/food.txt | 11 -
.../TestReplaceTextLineByLine/testFile.txt | 11 -
.../color-fruit-backreference-mapping.txt | 7 -
.../color-fruit-blank-mapping.txt | 7 -
.../color-fruit-escaped-dollar-mapping.txt | 7 -
...t-excessive-backreference-mapping-simple.txt | 6 -
...or-fruit-excessive-backreference-mapping.txt | 6 -
...olor-fruit-invalid-backreference-mapping.txt | 7 -
.../color-fruit-mapping.txt | 7 -
.../color-fruit-no-match-mapping.txt | 7 -
.../color-fruit-space-mapping.txt | 6 -
.../colors-without-dashes.txt | 4 -
.../TestReplaceTextWithMapping/colors.txt | 4 -
.../test/resources/TestScanContent/helloWorld | 1 -
.../resources/TestScanContent/wellthengood-bye | 1 -
.../src/test/resources/TestSplitText/1.txt | 5 -
.../src/test/resources/TestSplitText/2.txt | 5 -
.../src/test/resources/TestSplitText/3.txt | 5 -
.../src/test/resources/TestSplitText/4.txt | 3 -
.../src/test/resources/TestSplitText/5.txt | 7 -
.../src/test/resources/TestSplitText/6.txt | 7 -
.../test/resources/TestSplitText/original.txt | 12 -
.../test/resources/TestTransformXml/math.html | 8 -
.../test/resources/TestTransformXml/math.xml | 21 -
.../test/resources/TestTransformXml/math.xsl | 36 -
.../test/resources/TestTransformXml/tokens.csv | 2 -
.../test/resources/TestTransformXml/tokens.xml | 17 -
.../test/resources/TestTransformXml/tokens.xsl | 103 -
.../resources/TestUnpackContent/data.flowfilev2 | Bin 255 -> 0 bytes
.../resources/TestUnpackContent/data.flowfilev3 | Bin 357 -> 0 bytes
.../test/resources/TestUnpackContent/data.tar | Bin 3584 -> 0 bytes
.../test/resources/TestUnpackContent/data.zip | Bin 359 -> 0 bytes
.../resources/TestUnpackContent/folder/cal.txt | 8 -
.../resources/TestUnpackContent/folder/date.txt | 1 -
.../src/test/resources/TestXml/XmlBundle.xsd | 34 -
.../src/test/resources/TestXml/fruit.xml | 47 -
.../src/test/resources/TestXml/subNode.xml | 21 -
.../src/test/resources/TestXml/xml-bundle-1 | 51 -
.../src/test/resources/TestXml/xml-snippet.xml | 25 -
.../src/test/resources/hello.txt | 1 -
.../src/test/resources/localhost-ks.jks | Bin 1298 -> 0 bytes
.../src/test/resources/localhost-ts.jks | Bin 891 -> 0 bytes
.../src/test/resources/localhost.cer | Bin 829 -> 0 bytes
.../src/test/resources/logback-test.xml | 49 -
.../standard-reporting-tasks/pom.xml | 60 -
.../ControllerStatusReportingTask.java | 347 --
.../nifi/controller/MonitorDiskUsage.java | 113 -
.../apache/nifi/controller/MonitorMemory.java | 260 --
.../org.apache.nifi.reporting.ReportingTask | 17 -
.../index.html | 85 -
.../index.html | 58 -
.../index.html | 77 -
.../nifi/controller/TestMonitorDiskUsage.java | 61 -
.../pom.xml | 36 -
.../distributed/cache/client/Deserializer.java | 41 -
.../cache/client/DistributedMapCacheClient.java | 114 -
.../cache/client/DistributedSetCacheClient.java | 74 -
.../distributed/cache/client/Serializer.java | 40 -
.../exception/DeserializationException.java | 33 -
.../exception/SerializationException.java | 33 -
.../distributed-cache-client-service/pom.xml | 56 -
.../distributed/cache/client/CommsSession.java | 46 -
.../DistributedMapCacheClientService.java | 305 --
.../DistributedSetCacheClientService.java | 215 --
.../cache/client/SSLCommsSession.java | 106 -
.../cache/client/StandardCommsSession.java | 124 -
...org.apache.nifi.controller.ControllerService | 16 -
.../index.html | 78 -
.../index.html | 51 -
.../distributed-cache-protocol/pom.xml | 39 -
.../cache/protocol/ProtocolHandshake.java | 119 -
.../protocol/exception/HandshakeException.java | 27 -
.../distributed-cache-server/pom.xml | 75 -
.../cache/server/AbstractCacheServer.java | 199 -
.../distributed/cache/server/CacheRecord.java | 57 -
.../distributed/cache/server/CacheServer.java | 26 -
.../cache/server/DistributedCacheServer.java | 107 -
.../cache/server/DistributedSetCacheServer.java | 68 -
.../cache/server/EvictionPolicy.java | 73 -
.../cache/server/SetCacheServer.java | 104 -
.../server/map/DistributedMapCacheServer.java | 71 -
.../distributed/cache/server/map/MapCache.java | 29 -
.../cache/server/map/MapCacheRecord.java | 58 -
.../cache/server/map/MapCacheServer.java | 145 -
.../cache/server/map/MapPutResult.java | 59 -
.../cache/server/map/PersistentMapCache.java | 210 -
.../cache/server/map/SimpleMapCache.java | 165 -
.../cache/server/set/PersistentSetCache.java | 194 -
.../distributed/cache/server/set/SetCache.java | 29 -
.../cache/server/set/SetCacheRecord.java | 55 -
.../cache/server/set/SetCacheResult.java | 43 -
.../cache/server/set/SimpleSetCache.java | 117 -
...org.apache.nifi.controller.ControllerService | 16 -
.../index.html | 82 -
.../cache/server/TestServerAndClient.java | 535 ---
.../src/test/resources/localhost-ks.jks | Bin 1298 -> 0 bytes
.../src/test/resources/localhost-ts.jks | Bin 891 -> 0 bytes
.../distributed-cache-services-nar/pom.xml | 47 -
.../distributed-cache-services-bundle/pom.xml | 35 -
.../load-distribution-service-api/pom.xml | 37 -
.../nifi/loading/LoadDistributionListener.java | 24 -
.../nifi/loading/LoadDistributionService.java | 33 -
nar-bundles/standard-services/pom.xml | 39 -
.../ssl-context-bundle/nar/pom.xml | 39 -
.../ssl-context-bundle/pom.xml | 34 -
.../ssl-context-service/pom.xml | 52 -
.../nifi/ssl/StandardSSLContextService.java | 354 --
...org.apache.nifi.controller.ControllerService | 15 -
.../index.html | 63 -
.../apache/nifi/ssl/SSLContextServiceTest.java | 197 -
.../java/org/apache/nifi/ssl/TestProcessor.java | 47 -
.../src/test/resources/localhost-ks.jks | Bin 1298 -> 0 bytes
.../src/test/resources/localhost-ts.jks | Bin 891 -> 0 bytes
.../ssl-context-service-api/pom.xml | 36 -
.../org/apache/nifi/ssl/SSLContextService.java | 55 -
.../standard-services-api-nar/pom.xml | 45 -
.../update-attribute-bundle/model/pom.xml | 27 -
.../apache/nifi/update/attributes/Action.java | 55 -
.../nifi/update/attributes/Condition.java | 49 -
.../apache/nifi/update/attributes/Criteria.java | 141 -
.../nifi/update/attributes/FlowFilePolicy.java | 34 -
.../org/apache/nifi/update/attributes/Rule.java | 65 -
.../update/attributes/serde/CriteriaSerDe.java | 128 -
nar-bundles/update-attribute-bundle/nar/pom.xml | 48 -
nar-bundles/update-attribute-bundle/pom.xml | 56 -
.../update-attribute-bundle/processor/pom.xml | 54 -
.../processors/attributes/UpdateAttribute.java | 508 ---
.../org.apache.nifi.processor.Processor | 15 -
.../index.html | 253 --
.../update/attributes/TestUpdateAttribute.java | 425 --
.../ui/nb-configuration.xml | 18 -
nar-bundles/update-attribute-bundle/ui/pom.xml | 91 -
.../attributes/UpdateAttributeModelFactory.java | 127 -
.../attributes/api/ObjectMapperResolver.java | 51 -
.../update/attributes/api/RuleResource.java | 684 ----
.../nifi/update/attributes/dto/ActionDTO.java | 68 -
.../update/attributes/dto/ConditionDTO.java | 59 -
.../nifi/update/attributes/dto/DtoFactory.java | 68 -
.../nifi/update/attributes/dto/RuleDTO.java | 65 -
.../update/attributes/entity/ActionEntity.java | 65 -
.../attributes/entity/ConditionEntity.java | 65 -
.../entity/EvaluationContextEntity.java | 74 -
.../update/attributes/entity/RuleEntity.java | 65 -
.../update/attributes/entity/RulesEntity.java | 66 -
.../ui/src/main/webapp/META-INF/nifi-processor | 15 -
.../src/main/webapp/WEB-INF/jsp/worksheet.jsp | 188 -
.../ui/src/main/webapp/WEB-INF/web.xml | 46 -
.../ui/src/main/webapp/css/main.css | 426 ---
.../ui/src/main/webapp/images/bgInputText.png | Bin 139 -> 0 bytes
.../ui/src/main/webapp/images/buttonNew.png | Bin 590 -> 0 bytes
.../ui/src/main/webapp/images/iconDelete.png | Bin 670 -> 0 bytes
.../ui/src/main/webapp/images/iconInfo.png | Bin 550 -> 0 bytes
.../ui/src/main/webapp/js/application.js | 1915 ---------
.../nar/pom.xml | 36 -
.../pom.xml | 44 -
.../volatile-provenance-repository/pom.xml | 47 -
.../VolatileProvenanceRepository.java | 745 ----
...he.nifi.provenance.ProvenanceEventRepository | 15 -
.../TestVolatileProvenanceRepository.java | 178 -
.../src/test/resources/nifi.properties | 18 -
nar-maven-plugin/pom.xml | 385 --
.../src/main/java/nifi/NarMojo.java | 613 ---
.../resources/META-INF/plexus/components.xml | 52 -
nifi-api/.gitignore | 2 -
nifi-api/pom.xml | 31 -
.../apache/nifi/authorization/Authority.java | 97 -
.../nifi/authorization/AuthorityProvider.java | 181 -
.../AuthorityProviderConfigurationContext.java | 52 -
.../AuthorityProviderInitializationContext.java | 27 -
.../authorization/AuthorityProviderLookup.java | 25 -
.../authorization/DownloadAuthorization.java | 86 -
.../annotation/AuthorityProviderContext.java | 36 -
.../exception/AuthorityAccessException.java | 33 -
.../IdentityAlreadyExistsException.java | 32 -
.../exception/ProviderCreationException.java | 40 -
.../exception/ProviderDestructionException.java | 40 -
.../exception/UnknownIdentityException.java | 32 -
.../AbstractConfigurableComponent.java | 221 --
.../apache/nifi/components/AllowableValue.java | 136 -
.../nifi/components/ConfigurableComponent.java | 77 -
.../nifi/components/PropertyDescriptor.java | 552 ---
.../apache/nifi/components/PropertyValue.java | 189 -
.../nifi/components/ValidationContext.java | 82 -
.../nifi/components/ValidationResult.java | 173 -
.../org/apache/nifi/components/Validator.java | 55 -
.../controller/AbstractControllerService.java | 91 -
.../nifi/controller/ConfigurationContext.java | 46 -
.../nifi/controller/ControllerService.java | 182 -
.../ControllerServiceInitializationContext.java | 36 -
.../controller/ControllerServiceLookup.java | 66 -
.../apache/nifi/controller/FlowFileQueue.java | 200 -
.../apache/nifi/controller/ScheduledState.java | 37 -
.../org/apache/nifi/controller/Snippet.java | 113 -
.../org/apache/nifi/controller/Triggerable.java | 116 -
.../controller/annotation/OnConfigured.java | 41 -
.../repository/ContentRepository.java | 314 --
.../controller/repository/FlowFileRecord.java | 54 -
.../repository/FlowFileRepository.java | 128 -
.../repository/FlowFileSwapManager.java | 63 -
.../controller/repository/QueueProvider.java | 35 -
.../controller/repository/RepositoryRecord.java | 100 -
.../repository/RepositoryRecordType.java | 26 -
.../repository/claim/ContentClaim.java | 55 -
.../repository/claim/ContentClaimManager.java | 143 -
.../controller/status/ConnectionStatus.java | 193 -
.../nifi/controller/status/PortStatus.java | 201 -
.../controller/status/ProcessGroupStatus.java | 547 ---
.../nifi/controller/status/ProcessorStatus.java | 275 --
.../status/RemoteProcessGroupStatus.java | 217 --
.../nifi/controller/status/RunStatus.java | 28 -
.../controller/status/TransmissionStatus.java | 23 -
.../history/ComponentStatusRepository.java | 167 -
.../status/history/MetricDescriptor.java | 75 -
.../status/history/StatusHistory.java | 50 -
.../status/history/StatusSnapshot.java | 48 -
.../controller/status/history/ValueMapper.java | 23 -
.../controller/status/history/ValueReducer.java | 25 -
.../org/apache/nifi/events/EventReporter.java | 27 -
.../nifi/expression/AttributeExpression.java | 79 -
.../expression/AttributeValueDecorator.java | 29 -
.../expression/ExpressionLanguageCompiler.java | 69 -
.../java/org/apache/nifi/flowfile/FlowFile.java | 109 -
.../nifi/flowfile/FlowFilePrioritizer.java | 30 -
.../java/org/apache/nifi/logging/LogLevel.java | 27 -
.../org/apache/nifi/logging/ProcessorLog.java | 73 -
.../nifi/processor/AbstractProcessor.java | 37 -
.../AbstractSessionFactoryProcessor.java | 122 -
.../org/apache/nifi/processor/DataUnit.java | 248 --
.../apache/nifi/processor/FlowFileFilter.java | 91 -
.../apache/nifi/processor/ProcessContext.java | 132 -
.../apache/nifi/processor/ProcessSession.java | 713 ----
.../nifi/processor/ProcessSessionFactory.java | 30 -
.../org/apache/nifi/processor/Processor.java | 91 -
.../ProcessorInitializationContext.java | 53 -
.../org/apache/nifi/processor/QueueSize.java | 49 -
.../org/apache/nifi/processor/Relationship.java | 127 -
.../nifi/processor/SchedulingContext.java | 70 -
.../annotation/CapabilityDescription.java | 39 -
.../nifi/processor/annotation/EventDriven.java | 49 -
.../nifi/processor/annotation/OnAdded.java | 41 -
.../nifi/processor/annotation/OnRemoved.java | 42 -
.../nifi/processor/annotation/OnScheduled.java | 44 -
.../nifi/processor/annotation/OnShutdown.java | 38 -
.../nifi/processor/annotation/OnStopped.java | 56 -
.../processor/annotation/OnUnscheduled.java | 46 -
.../processor/annotation/SideEffectFree.java | 45 -
.../processor/annotation/SupportsBatching.java | 51 -
.../apache/nifi/processor/annotation/Tags.java | 44 -
.../processor/annotation/TriggerSerially.java | 39 -
.../TriggerWhenAnyDestinationAvailable.java | 40 -
.../processor/annotation/TriggerWhenEmpty.java | 41 -
.../exception/FlowFileAccessException.java | 37 -
.../exception/FlowFileHandlingException.java | 39 -
.../exception/MissingFlowFileException.java | 37 -
.../processor/exception/ProcessException.java | 44 -
.../nifi/processor/io/InputStreamCallback.java | 38 -
.../nifi/processor/io/OutputStreamCallback.java | 39 -
.../nifi/processor/io/StreamCallback.java | 41 -
.../nifi/provenance/ProvenanceEventBuilder.java | 308 --
.../nifi/provenance/ProvenanceEventRecord.java | 296 --
.../provenance/ProvenanceEventRepository.java | 190 -
.../nifi/provenance/ProvenanceEventType.java | 92 -
.../nifi/provenance/ProvenanceReporter.java | 616 ---
.../lineage/ComputeLineageResult.java | 70 -
.../lineage/ComputeLineageSubmission.java | 82 -
.../apache/nifi/provenance/lineage/Lineage.java | 41 -
.../lineage/LineageComputationType.java | 27 -
.../nifi/provenance/lineage/LineageEdge.java | 26 -
.../nifi/provenance/lineage/LineageNode.java | 61 -
.../provenance/lineage/LineageNodeType.java | 23 -
.../lineage/ProvenanceEventLineageNode.java | 32 -
.../apache/nifi/provenance/search/Query.java | 91 -
.../nifi/provenance/search/QueryResult.java | 77 -
.../nifi/provenance/search/QuerySubmission.java | 63 -
.../nifi/provenance/search/SearchTerm.java | 24 -
.../nifi/provenance/search/SearchTerms.java | 39 -
.../nifi/provenance/search/SearchableField.java | 62 -
.../provenance/search/SearchableFieldType.java | 28 -
.../nifi/reporting/AbstractReportingTask.java | 94 -
.../org/apache/nifi/reporting/Bulletin.java | 120 -
.../apache/nifi/reporting/BulletinQuery.java | 109 -
.../nifi/reporting/BulletinRepository.java | 88 -
.../org/apache/nifi/reporting/EventAccess.java | 53 -
.../nifi/reporting/InitializationException.java | 32 -
.../apache/nifi/reporting/ReportingContext.java | 99 -
.../ReportingInitializationContext.java | 80 -
.../apache/nifi/reporting/ReportingTask.java | 75 -
.../org/apache/nifi/reporting/Severity.java | 24 -
.../nifi/scheduling/SchedulingStrategy.java | 86 -
.../org/apache/nifi/search/SearchContext.java | 57 -
.../org/apache/nifi/search/SearchResult.java | 82 -
.../java/org/apache/nifi/search/Searchable.java | 27 -
.../nifi/web/ClusterRequestException.java | 38 -
.../nifi/web/InvalidRevisionException.java | 33 -
.../org/apache/nifi/web/NiFiWebContext.java | 121 -
.../apache/nifi/web/NiFiWebContextConfig.java | 54 -
.../nifi/web/ProcessorConfigurationAction.java | 137 -
.../java/org/apache/nifi/web/ProcessorInfo.java | 110 -
.../nifi/web/ResourceNotFoundException.java | 32 -
.../main/java/org/apache/nifi/web/Revision.java | 110 -
.../nifi/components/TestPropertyDescriptor.java | 59 -
.../org/apache/nifi/processor/TestDataUnit.java | 44 -
nifi-api/src/test/resources/logback-test.xml | 27 -
nifi-bootstrap/pom.xml | 29 -
.../apache/nifi/bootstrap/BootstrapCodec.java | 92 -
.../org/apache/nifi/bootstrap/NiFiListener.java | 128 -
.../java/org/apache/nifi/bootstrap/RunNiFi.java | 941 -----
.../org/apache/nifi/bootstrap/ShutdownHook.java | 86 -
.../exception/InvalidCommandException.java | 37 -
.../bootstrap/util/LimitingInputStream.java | 107 -
nifi-docs/pom.xml | 131 -
.../src/main/asciidoc/administration-guide.adoc | 160 -
.../src/main/asciidoc/developer-guide.adoc | 45 -
.../asciidoc/expression-language-guide.adoc | 1727 ---------
.../images/add-processor-with-tag-cloud.png | Bin 22925 -> 0 bytes
.../src/main/asciidoc/images/add-processor.png | Bin 31524 -> 0 bytes
.../src/main/asciidoc/images/addConnect.png | Bin 1996 -> 0 bytes
.../src/main/asciidoc/images/comments-tab.png | Bin 6431 -> 0 bytes
.../src/main/asciidoc/images/components.png | Bin 8992 -> 0 bytes
.../asciidoc/images/connection-settings.png | Bin 15214 -> 0 bytes
.../main/asciidoc/images/create-connection.png | Bin 9430 -> 0 bytes
.../asciidoc/images/edit-property-dropdown.png | Bin 14291 -> 0 bytes
.../asciidoc/images/edit-property-textarea.png | Bin 17146 -> 0 bytes
.../main/asciidoc/images/event-attributes.png | Bin 94668 -> 0 bytes
.../src/main/asciidoc/images/event-content.png | Bin 88726 -> 0 bytes
.../src/main/asciidoc/images/event-details.png | Bin 139736 -> 0 bytes
.../src/main/asciidoc/images/expand-event.png | Bin 40476 -> 0 bytes
.../main/asciidoc/images/expanded-events.png | Bin 76082 -> 0 bytes
.../src/main/asciidoc/images/find-parents.png | Bin 35831 -> 0 bytes
.../src/main/asciidoc/images/iconAlert.png | Bin 1396 -> 0 bytes
.../src/main/asciidoc/images/iconConnection.png | Bin 1517 -> 0 bytes
.../src/main/asciidoc/images/iconDelete.png | Bin 670 -> 0 bytes
.../src/main/asciidoc/images/iconDisable.png | Bin 764 -> 0 bytes
nifi-docs/src/main/asciidoc/images/iconEdit.png | Bin 493 -> 0 bytes
.../src/main/asciidoc/images/iconEnable.png | Bin 667 -> 0 bytes
.../src/main/asciidoc/images/iconExport.png | Bin 453 -> 0 bytes
.../main/asciidoc/images/iconFlowHistory.png | Bin 2463 -> 0 bytes
.../src/main/asciidoc/images/iconFunnel.png | Bin 1223 -> 0 bytes
nifi-docs/src/main/asciidoc/images/iconInfo.png | Bin 550 -> 0 bytes
.../src/main/asciidoc/images/iconInputPort.png | Bin 1842 -> 0 bytes
.../main/asciidoc/images/iconInputPortSmall.png | Bin 532 -> 0 bytes
.../src/main/asciidoc/images/iconLabel.png | Bin 838 -> 0 bytes
.../src/main/asciidoc/images/iconLineage.png | Bin 2214 -> 0 bytes
.../main/asciidoc/images/iconNewTemplate.png | Bin 557 -> 0 bytes
.../src/main/asciidoc/images/iconNotSecure.png | Bin 221 -> 0 bytes
.../src/main/asciidoc/images/iconOutputPort.png | Bin 1658 -> 0 bytes
.../asciidoc/images/iconOutputPortSmall.png | Bin 459 -> 0 bytes
.../main/asciidoc/images/iconProcessGroup.png | Bin 1422 -> 0 bytes
.../src/main/asciidoc/images/iconProcessor.png | Bin 1446 -> 0 bytes
.../src/main/asciidoc/images/iconProvenance.png | Bin 2268 -> 0 bytes
.../asciidoc/images/iconRemoteProcessGroup.png | Bin 674 -> 0 bytes
.../src/main/asciidoc/images/iconResize.png | Bin 165 -> 0 bytes
nifi-docs/src/main/asciidoc/images/iconRun.png | Bin 538 -> 0 bytes
.../src/main/asciidoc/images/iconSecure.png | Bin 225 -> 0 bytes
.../src/main/asciidoc/images/iconSettings.png | Bin 2638 -> 0 bytes
nifi-docs/src/main/asciidoc/images/iconStop.png | Bin 402 -> 0 bytes
.../src/main/asciidoc/images/iconSummary.png | Bin 272 -> 0 bytes
.../src/main/asciidoc/images/iconTemplate.png | Bin 970 -> 0 bytes
.../asciidoc/images/iconTransmissionActive.png | Bin 1330 -> 0 bytes
.../images/iconTransmissionInactive.png | Bin 1248 -> 0 bytes
.../src/main/asciidoc/images/iconUsers.png | Bin 2272 -> 0 bytes
.../main/asciidoc/images/iconViewDetails.png | Bin 1788 -> 0 bytes
.../images/instantiate-template-description.png | Bin 6508 -> 0 bytes
.../asciidoc/images/instantiate-template.png | Bin 3216 -> 0 bytes
.../main/asciidoc/images/invalid-processor.png | Bin 8187 -> 0 bytes
.../main/asciidoc/images/lineage-flowfile.png | Bin 3855 -> 0 bytes
.../asciidoc/images/lineage-graph-annotated.png | Bin 170122 -> 0 bytes
nifi-docs/src/main/asciidoc/images/new-flow.png | Bin 266913 -> 0 bytes
.../main/asciidoc/images/nifi-arch-cluster.png | Bin 50384 -> 0 bytes
.../src/main/asciidoc/images/nifi-arch.png | Bin 43509 -> 0 bytes
.../main/asciidoc/images/nifi-navigation.png | Bin 339195 -> 0 bytes
.../asciidoc/images/nifi-toolbar-components.png | Bin 262314 -> 0 bytes
.../src/main/asciidoc/images/parent-found.png | Bin 42814 -> 0 bytes
.../asciidoc/images/process-group-anatomy.png | Bin 48554 -> 0 bytes
.../main/asciidoc/images/processor-anatomy.png | Bin 37035 -> 0 bytes
.../images/processor-connection-bubble.png | Bin 9936 -> 0 bytes
.../src/main/asciidoc/images/properties-tab.png | Bin 11847 -> 0 bytes
.../asciidoc/images/provenance-annotated.png | Bin 530929 -> 0 bytes
.../main/asciidoc/images/provenance-table.png | Bin 466599 -> 0 bytes
.../asciidoc/images/remote-group-anatomy.png | Bin 61383 -> 0 bytes
.../images/remote-group-ports-dialog.png | Bin 21662 -> 0 bytes
.../images/remote-port-connection-status.png | Bin 57332 -> 0 bytes
.../src/main/asciidoc/images/scheduling-tab.png | Bin 12288 -> 0 bytes
.../src/main/asciidoc/images/search-events.png | Bin 62626 -> 0 bytes
.../images/search-receive-event-abc.png | Bin 67817 -> 0 bytes
.../src/main/asciidoc/images/settings-tab.png | Bin 16225 -> 0 bytes
.../src/main/asciidoc/images/simple-flow.png | Bin 196808 -> 0 bytes
.../src/main/asciidoc/images/stats-history.png | Bin 33205 -> 0 bytes
.../src/main/asciidoc/images/status-bar.png | Bin 249077 -> 0 bytes
.../main/asciidoc/images/summary-annotated.png | Bin 111956 -> 0 bytes
.../src/main/asciidoc/images/summary-table.png | Bin 62114 -> 0 bytes
.../main/asciidoc/images/valid-processor.png | Bin 8090 -> 0 bytes
nifi-docs/src/main/asciidoc/overview.adoc | 296 --
nifi-docs/src/main/asciidoc/user-guide.adoc | 1286 -------
nifi-docs/src/main/assembly/dependencies.xml | 28 -
nifi-mock/pom.xml | 54 -
.../MockProvenanceEventRepository.java | 131 -
.../apache/nifi/reporting/BulletinFactory.java | 43 -
.../org/apache/nifi/reporting/MockBulletin.java | 24 -
.../util/ControllerServiceConfiguration.java | 74 -
.../nifi/util/MockBulletinRepository.java | 74 -
.../nifi/util/MockConfigurationContext.java | 50 -
...kControllerServiceInitializationContext.java | 41 -
.../nifi/util/MockControllerServiceLookup.java | 76 -
.../org/apache/nifi/util/MockEventAccess.java | 70 -
.../java/org/apache/nifi/util/MockFlowFile.java | 279 --
.../org/apache/nifi/util/MockFlowFileQueue.java | 85 -
.../apache/nifi/util/MockProcessContext.java | 284 --
.../apache/nifi/util/MockProcessSession.java | 1006 -----
.../MockProcessorInitializationContext.java | 74 -
.../org/apache/nifi/util/MockProcessorLog.java | 402 --
.../org/apache/nifi/util/MockPropertyValue.java | 185 -
.../nifi/util/MockProvenanceReporter.java | 202 -
.../apache/nifi/util/MockReportingContext.java | 117 -
.../MockReportingInitializationContext.java | 81 -
.../apache/nifi/util/MockSessionFactory.java | 46 -
.../apache/nifi/util/MockValidationContext.java | 93 -
.../org/apache/nifi/util/ReflectionUtils.java | 149 -
.../apache/nifi/util/SharedSessionState.java | 72 -
.../apache/nifi/util/SingleSessionFactory.java | 35 -
.../nifi/util/StandardProcessorTestRunner.java | 492 ---
.../java/org/apache/nifi/util/TestRunner.java | 542 ---
.../java/org/apache/nifi/util/TestRunners.java | 37 -
nifi/assembly/pom.xml | 458 +++
.../assembly/src/main/assembly/dependencies.xml | 140 +
nifi/commons/data-provenance-utils/.gitignore | 2 +
nifi/commons/data-provenance-utils/pom.xml | 40 +
.../nifi/provenance/AsyncLineageSubmission.java | 87 +
.../nifi/provenance/AsyncQuerySubmission.java | 81 +
.../nifi/provenance/NamedSearchableField.java | 95 +
.../nifi/provenance/SearchableFieldParser.java | 53 +
.../nifi/provenance/SearchableFields.java | 84 +
.../nifi/provenance/StandardLineageResult.java | 324 ++
.../StandardProvenanceEventRecord.java | 752 ++++
.../nifi/provenance/StandardQueryResult.java | 168 +
.../nifi/provenance/lineage/EdgeNode.java | 74 +
.../nifi/provenance/lineage/EventNode.java | 109 +
.../provenance/lineage/FlowFileLineage.java | 76 +
.../nifi/provenance/lineage/FlowFileNode.java | 83 +
nifi/commons/flowfile-packager/pom.xml | 41 +
.../org/apache/nifi/util/FlowFilePackager.java | 28 +
.../apache/nifi/util/FlowFilePackagerV1.java | 104 +
.../apache/nifi/util/FlowFilePackagerV2.java | 146 +
.../apache/nifi/util/FlowFilePackagerV3.java | 93 +
.../apache/nifi/util/FlowFileUnpackager.java | 30 +
.../apache/nifi/util/FlowFileUnpackagerV1.java | 155 +
.../apache/nifi/util/FlowFileUnpackagerV2.java | 143 +
.../apache/nifi/util/FlowFileUnpackagerV3.java | 161 +
.../java/org/apache/nifi/util/Unpackage.java | 119 +
.../nifi/util/TestPackageUnpackageV3.java | 56 +
nifi/commons/nifi-expression-language/pom.xml | 56 +
.../language/antlr/AttributeExpressionLexer.g | 210 +
.../language/antlr/AttributeExpressionParser.g | 139 +
.../output/AttributeExpressionLexer.tokens | 88 +
.../expression/language/EmptyPreparedQuery.java | 62 +
.../language/InvalidPreparedQuery.java | 71 +
.../expression/language/PreparedQuery.java | 39 +
.../attribute/expression/language/Query.java | 1186 ++++++
.../language/StandardAttributeExpression.java | 65 +
.../StandardExpressionLanguageCompiler.java | 58 +
.../language/StandardPreparedQuery.java | 83 +
.../language/evaluation/BooleanEvaluator.java | 32 +
.../language/evaluation/BooleanQueryResult.java | 43 +
.../language/evaluation/DateEvaluator.java | 34 +
.../language/evaluation/DateQueryResult.java | 45 +
.../language/evaluation/Evaluator.java | 32 +
.../language/evaluation/NumberEvaluator.java | 33 +
.../language/evaluation/NumberQueryResult.java | 43 +
.../language/evaluation/QueryResult.java | 26 +
.../language/evaluation/StringEvaluator.java | 32 +
.../language/evaluation/StringQueryResult.java | 43 +
.../evaluation/cast/BooleanCastEvaluator.java | 50 +
.../evaluation/cast/DateCastEvaluator.java | 117 +
.../evaluation/cast/NumberCastEvaluator.java | 72 +
.../evaluation/cast/StringCastEvaluator.java | 49 +
.../evaluation/functions/AndEvaluator.java | 60 +
.../evaluation/functions/AppendEvaluator.java | 50 +
.../functions/AttributeEvaluator.java | 45 +
.../evaluation/functions/ContainsEvaluator.java | 53 +
.../functions/DateToNumberEvaluator.java | 50 +
.../evaluation/functions/DivideEvaluator.java | 57 +
.../evaluation/functions/EndsWithEvaluator.java | 53 +
.../evaluation/functions/EqualsEvaluator.java | 89 +
.../functions/EqualsIgnoreCaseEvaluator.java | 60 +
.../evaluation/functions/FindEvaluator.java | 72 +
.../evaluation/functions/FormatEvaluator.java | 61 +
.../functions/GreaterThanEvaluator.java | 60 +
.../functions/GreaterThanOrEqualEvaluator.java | 60 +
.../evaluation/functions/HostnameEvaluator.java | 58 +
.../evaluation/functions/IPEvaluator.java | 46 +
.../evaluation/functions/IndexOfEvaluator.java | 53 +
.../evaluation/functions/IsEmptyEvaluator.java | 43 +
.../evaluation/functions/IsNullEvaluator.java | 45 +
.../functions/LastIndexOfEvaluator.java | 53 +
.../evaluation/functions/LengthEvaluator.java | 46 +
.../evaluation/functions/LessThanEvaluator.java | 60 +
.../functions/LessThanOrEqualEvaluator.java | 60 +
.../evaluation/functions/MatchesEvaluator.java | 71 +
.../evaluation/functions/MinusEvaluator.java | 57 +
.../evaluation/functions/ModEvaluator.java | 57 +
.../evaluation/functions/MultiplyEvaluator.java | 57 +
.../evaluation/functions/NotEvaluator.java | 49 +
.../evaluation/functions/NotNullEvaluator.java | 45 +
.../evaluation/functions/NowEvaluator.java | 39 +
.../functions/NumberToDateEvaluator.java | 52 +
.../functions/OneUpSequenceEvaluator.java | 41 +
.../evaluation/functions/OrEvaluator.java | 60 +
.../evaluation/functions/PlusEvaluator.java | 57 +
.../evaluation/functions/PrependEvaluator.java | 50 +
.../functions/ReplaceAllEvaluator.java | 55 +
.../functions/ReplaceEmptyEvaluator.java | 50 +
.../evaluation/functions/ReplaceEvaluator.java | 55 +
.../functions/ReplaceNullEvaluator.java | 47 +
.../functions/StartsWithEvaluator.java | 53 +
.../functions/StringToDateEvaluator.java | 65 +
.../functions/SubstringAfterEvaluator.java | 59 +
.../functions/SubstringAfterLastEvaluator.java | 55 +
.../functions/SubstringBeforeEvaluator.java | 58 +
.../functions/SubstringBeforeLastEvaluator.java | 55 +
.../functions/SubstringEvaluator.java | 65 +
.../evaluation/functions/ToLowerEvaluator.java | 45 +
.../evaluation/functions/ToNumberEvaluator.java | 46 +
.../evaluation/functions/ToRadixEvaluator.java | 77 +
.../evaluation/functions/ToStringEvaluator.java | 45 +
.../evaluation/functions/ToUpperEvaluator.java | 45 +
.../evaluation/functions/TrimEvaluator.java | 45 +
.../functions/UrlDecodeEvaluator.java | 55 +
.../functions/UrlEncodeEvaluator.java | 55 +
.../evaluation/functions/UuidEvaluator.java | 39 +
.../literals/BooleanLiteralEvaluator.java | 44 +
.../literals/NumberLiteralEvaluator.java | 44 +
.../literals/StringLiteralEvaluator.java | 77 +
.../evaluation/reduce/CountEvaluator.java | 56 +
.../evaluation/reduce/JoinEvaluator.java | 59 +
.../evaluation/reduce/ReduceEvaluator.java | 23 +
.../selection/AllAttributesEvaluator.java | 68 +
.../selection/AnyAttributeEvaluator.java | 68 +
.../AnyMatchingAttributeEvaluator.java | 21 +
.../selection/DelineatedAttributeEvaluator.java | 83 +
.../evaluation/selection/MappingEvaluator.java | 61 +
.../selection/MultiAttributeEvaluator.java | 24 +
.../selection/MultiMatchAttributeEvaluator.java | 82 +
.../selection/MultiNamedAttributeEvaluator.java | 64 +
.../AttributeExpressionLanguageException.java | 34 +
...ibuteExpressionLanguageParsingException.java | 34 +
.../exception/IllegalAttributeException.java | 29 +
.../expression/language/TestQuery.java | 1143 ++++++
.../language/TestStandardPreparedQuery.java | 92 +
nifi/commons/nifi-logging-utils/pom.xml | 36 +
.../java/org/apache/nifi/logging/NiFiLog.java | 367 ++
nifi/commons/nifi-properties/.gitignore | 3 +
nifi/commons/nifi-properties/pom.xml | 29 +
.../org/apache/nifi/util/NiFiProperties.java | 876 +++++
.../java/org/apache/nifi/util/StringUtils.java | 66 +
nifi/commons/nifi-security-utils/pom.xml | 40 +
.../nifi/security/util/CertificateUtils.java | 158 +
.../nifi/security/util/EncryptionMethod.java | 84 +
.../apache/nifi/security/util/KeystoreType.java | 26 +
.../nifi/security/util/SecurityStoreTypes.java | 144 +
.../nifi/security/util/SslContextFactory.java | 180 +
nifi/commons/nifi-socket-utils/pom.xml | 60 +
.../nifi/io/nio/AbstractChannelReader.java | 166 +
.../java/org/apache/nifi/io/nio/BufferPool.java | 114 +
.../apache/nifi/io/nio/ChannelDispatcher.java | 160 +
.../org/apache/nifi/io/nio/ChannelListener.java | 228 ++
.../nifi/io/nio/DatagramChannelReader.java | 59 +
.../apache/nifi/io/nio/SocketChannelReader.java | 55 +
.../io/nio/consumer/AbstractStreamConsumer.java | 132 +
.../nifi/io/nio/consumer/StreamConsumer.java | 80 +
.../io/nio/consumer/StreamConsumerFactory.java | 27 +
.../nifi/io/socket/SSLContextFactory.java | 102 +
.../io/socket/ServerSocketConfiguration.java | 83 +
.../nifi/io/socket/SocketConfiguration.java | 116 +
.../apache/nifi/io/socket/SocketListener.java | 211 +
.../org/apache/nifi/io/socket/SocketUtils.java | 169 +
.../socket/multicast/DiscoverableService.java | 43 +
.../multicast/DiscoverableServiceImpl.java | 78 +
.../multicast/MulticastConfiguration.java | 99 +
.../io/socket/multicast/MulticastListener.java | 193 +
.../multicast/MulticastServiceDiscovery.java | 34 +
.../multicast/MulticastServicesBroadcaster.java | 33 +
.../socket/multicast/MulticastTimeToLive.java | 50 +
.../io/socket/multicast/MulticastUtils.java | 109 +
.../io/socket/multicast/ServiceDiscovery.java | 31 +
.../socket/multicast/ServicesBroadcaster.java | 56 +
.../apache/nifi/io/nio/example/ServerMain.java | 141 +
.../apache/nifi/io/nio/example/TCPClient.java | 86 +
.../apache/nifi/io/nio/example/UDPClient.java | 51 +
.../io/nio/example/UselessStreamConsumer.java | 43 +
.../src/test/resources/log4j.xml | 36 +
nifi/commons/nifi-utils/.gitignore | 8 +
nifi/commons/nifi-utils/pom.xml | 33 +
.../flowfile/attributes/CoreAttributes.java | 72 +
.../attributes/FlowFileAttributeKey.java | 21 +
.../nifi/remote/StandardVersionNegotiator.java | 81 +
.../apache/nifi/remote/VersionNegotiator.java | 65 +
.../TransmissionDisabledException.java | 25 +
.../nifi/remote/io/CompressionInputStream.java | 184 +
.../nifi/remote/io/CompressionOutputStream.java | 147 +
.../remote/io/InterruptableInputStream.java | 117 +
.../remote/io/InterruptableOutputStream.java | 81 +
.../remote/io/socket/BufferStateManager.java | 111 +
.../io/socket/SocketChannelInputStream.java | 157 +
.../io/socket/SocketChannelOutputStream.java | 113 +
.../remote/io/socket/ssl/SSLSocketChannel.java | 602 +++
.../socket/ssl/SSLSocketChannelInputStream.java | 62 +
.../ssl/SSLSocketChannelOutputStream.java | 53 +
.../nifi/stream/io/BufferedInputStream.java | 37 +
.../nifi/stream/io/BufferedOutputStream.java | 140 +
.../nifi/stream/io/ByteArrayInputStream.java | 250 ++
.../nifi/stream/io/ByteArrayOutputStream.java | 250 ++
.../nifi/stream/io/ByteCountingInputStream.java | 104 +
.../stream/io/ByteCountingOutputStream.java | 63 +
.../apache/nifi/stream/io/DataOutputStream.java | 417 ++
.../apache/nifi/stream/io/GZIPOutputStream.java | 41 +
.../stream/io/LeakyBucketStreamThrottler.java | 331 ++
.../nifi/stream/io/NonCloseableInputStream.java | 56 +
.../stream/io/NonCloseableOutputStream.java | 51 +
.../apache/nifi/stream/io/NullOutputStream.java | 46 +
.../apache/nifi/stream/io/StreamThrottler.java | 33 +
.../org/apache/nifi/stream/io/StreamUtils.java | 257 ++
.../apache/nifi/stream/io/ZipOutputStream.java | 38 +
.../exception/BytePatternNotFoundException.java | 28 +
.../io/util/NonThreadSafeCircularBuffer.java | 69 +
.../org/apache/nifi/util/BooleanHolder.java | 25 +
.../java/org/apache/nifi/util/FormatUtils.java | 205 +
.../org/apache/nifi/util/IntegerHolder.java | 54 +
.../java/org/apache/nifi/util/LongHolder.java | 60 +
.../apache/nifi/util/NaiveSearchRingBuffer.java | 135 +
.../java/org/apache/nifi/util/ObjectHolder.java | 39 +
.../java/org/apache/nifi/util/RingBuffer.java | 292 ++
.../java/org/apache/nifi/util/StopWatch.java | 127 +
.../main/java/org/apache/nifi/util/Tuple.java | 83 +
.../concurrency/DebugDisabledTimedLock.java | 67 +
.../util/concurrency/DebugEnabledTimedLock.java | 136 +
.../util/concurrency/DebuggableTimedLock.java | 30 +
.../apache/nifi/util/concurrency/TimedLock.java | 59 +
.../org/apache/nifi/util/file/FileUtils.java | 623 +++
.../file/monitor/CompoundUpdateMonitor.java | 115 +
.../util/file/monitor/LastModifiedMonitor.java | 30 +
.../nifi/util/file/monitor/MD5SumMonitor.java | 51 +
.../file/monitor/SynchronousFileWatcher.java | 123 +
.../nifi/util/file/monitor/UpdateMonitor.java | 25 +
.../org/apache/nifi/util/search/Search.java | 57 +
.../org/apache/nifi/util/search/SearchTerm.java | 141 +
.../util/search/ahocorasick/AhoCorasick.java | 155 +
.../nifi/util/search/ahocorasick/Node.java | 72 +
.../util/search/ahocorasick/SearchState.java | 63 +
.../nifi/util/timebuffer/EntityAccess.java | 26 +
.../nifi/util/timebuffer/LongEntityAccess.java | 43 +
.../nifi/util/timebuffer/TimedBuffer.java | 114 +
.../nifi/util/timebuffer/TimestampedLong.java | 35 +
.../io/TestCompressionInputOutputStreams.java | 153 +
.../stream/io/TestLeakyBucketThrottler.java | 147 +
.../nifi/util/TestNaiveSearchRingBuffer.java | 72 +
.../file/monitor/TestCompoundUpdateMonitor.java | 71 +
.../monitor/TestSynchronousFileWatcher.java | 61 +
.../nifi/util/timebuffer/TestRingBuffer.java | 182 +
.../nifi/util/timebuffer/TestTimedBuffer.java | 106 +
.../src/test/resources/logback-test.xml | 32 +
nifi/commons/nifi-web-utils/pom.xml | 56 +
.../org/apache/nifi/web/util/ClientUtils.java | 132 +
.../nifi/web/util/ObjectMapperResolver.java | 48 +
.../java/org/apache/nifi/web/util/WebUtils.java | 198 +
nifi/commons/pom.xml | 44 +
nifi/commons/processor-utilities/pom.xml | 45 +
.../nifi/processor/util/FlowFileFilters.java | 65 +
.../nifi/processor/util/SSLProperties.java | 226 ++
.../nifi/processor/util/StandardValidators.java | 544 +++
.../apache/nifi/processor/TestFormatUtils.java | 40 +
.../processor/util/TestStandardValidators.java | 54 +
nifi/commons/wali/.gitignore | 2 +
nifi/commons/wali/pom.xml | 41 +
.../org/wali/MinimalLockingWriteAheadLog.java | 1008 +++++
.../wali/src/main/java/org/wali/SerDe.java | 128 +
.../src/main/java/org/wali/SyncListener.java | 62 +
.../wali/src/main/java/org/wali/UpdateType.java | 49 +
.../java/org/wali/WriteAheadRepository.java | 122 +
.../src/test/java/org/wali/DummyRecord.java | 61 +
.../test/java/org/wali/DummyRecordSerde.java | 107 +
.../wali/TestMinimalLockingWriteAheadLog.java | 298 ++
.../execute-script-processors/pom.xml | 81 +
.../nifi/processors/script/ExecuteScript.java | 566 +++
.../apache/nifi/scripting/ConverterScript.java | 131 +
.../nifi/scripting/JRubyScriptFactory.java | 46 +
.../nifi/scripting/JavaScriptScriptFactory.java | 56 +
.../nifi/scripting/JythonScriptFactory.java | 45 +
.../nifi/scripting/OutputStreamHandler.java | 24 +
.../org/apache/nifi/scripting/ReaderScript.java | 79 +
.../java/org/apache/nifi/scripting/Script.java | 303 ++
.../nifi/scripting/ScriptEngineFactory.java | 117 +
.../apache/nifi/scripting/ScriptFactory.java | 269 ++
.../org/apache/nifi/scripting/WriterScript.java | 67 +
.../org.apache.nifi.processor.Processor | 15 +
.../index.html | 264 ++
.../processors/script/TestExecuteScript.java | 939 +++++
.../src/test/resources/alwaysFail.js | 24 +
.../src/test/resources/alwaysFail.py | 19 +
.../src/test/resources/alwaysFail.rb | 21 +
.../src/test/resources/ffTest.js | 28 +
.../src/test/resources/ffTest.py | 22 +
.../src/test/resources/ffTest.rb | 30 +
.../src/test/resources/lib/Sub.py | 18 +
.../src/test/resources/lib/sub.js | 22 +
.../src/test/resources/lib/sub.rb | 17 +
.../src/test/resources/loadLocal.js | 30 +
.../src/test/resources/loadLocal.py | 26 +
.../src/test/resources/loadLocal.rb | 29 +
.../src/test/resources/log4j.xml | 54 +
.../src/test/resources/optionalValidators.js | 28 +
.../src/test/resources/optionalValidators.py | 22 +
.../src/test/resources/optionalValidators.rb | 39 +
.../src/test/resources/paramTest.js | 28 +
.../src/test/resources/paramTest.py | 26 +
.../src/test/resources/paramTest.rb | 31 +
.../src/test/resources/parseXml.js | 36 +
.../src/test/resources/readTest.js | 30 +
.../src/test/resources/readTest.py | 32 +
.../src/test/resources/readTest.rb | 30 +
.../src/test/resources/readWithParams.js | 32 +
.../src/test/resources/readWithParams.py | 32 +
.../src/test/resources/readWithParams.rb | 33 +
.../src/test/resources/routeTest.js | 41 +
.../src/test/resources/routeTest.py | 37 +
.../src/test/resources/routeTest.rb | 39 +
.../src/test/resources/simpleConverter.js | 45 +
.../src/test/resources/simpleConverter.py | 60 +
.../src/test/resources/simpleConverter.rb | 42 +
.../src/test/resources/writeTest.js | 26 +
.../src/test/resources/writeTest.py | 22 +
.../src/test/resources/writeTest.rb | 32 +
.../execute-script-bundle/nar/pom.xml | 36 +
nifi/nar-bundles/execute-script-bundle/pom.xml | 81 +
.../framework/administration/.gitignore | 1 +
.../framework/administration/pom.xml | 116 +
.../nifi/admin/AuditDataSourceFactoryBean.java | 222 ++
.../org/apache/nifi/admin/RepositoryUtils.java | 91 +
.../nifi/admin/UserDataSourceFactoryBean.java | 247 ++
.../org/apache/nifi/admin/dao/ActionDAO.java | 74 +
.../org/apache/nifi/admin/dao/AuthorityDAO.java | 58 +
.../org/apache/nifi/admin/dao/DAOFactory.java | 29 +
.../nifi/admin/dao/DataAccessException.java | 39 +
.../java/org/apache/nifi/admin/dao/UserDAO.java | 127 +
.../nifi/admin/dao/impl/DAOFactoryImpl.java | 51 +
.../nifi/admin/dao/impl/StandardActionDAO.java | 1056 +++++
.../admin/dao/impl/StandardAuthorityDAO.java | 172 +
.../nifi/admin/dao/impl/StandardUserDAO.java | 634 +++
.../admin/service/AccountDisabledException.java | 40 +
.../admin/service/AccountNotFoundException.java | 40 +
.../admin/service/AccountPendingException.java | 41 +
.../admin/service/AdministrationException.java | 39 +
.../apache/nifi/admin/service/AuditService.java | 76 +
.../apache/nifi/admin/service/UserService.java | 161 +
.../service/action/AbstractUserAction.java | 97 +
.../admin/service/action/AddActionsAction.java | 48 +
.../service/action/AdministrationAction.java | 38 +
.../service/action/AuthorizeDownloadAction.java | 54 +
.../service/action/AuthorizeUserAction.java | 175 +
.../admin/service/action/CreateUserAction.java | 53 +
.../admin/service/action/DeleteUserAction.java | 68 +
.../admin/service/action/DisableUserAction.java | 76 +
.../service/action/DisableUserGroupAction.java | 69 +
.../service/action/FindUserByDnAction.java | 49 +
.../service/action/FindUserByIdAction.java | 49 +
.../admin/service/action/GetActionAction.java | 41 +
.../admin/service/action/GetActionsAction.java | 48 +
.../admin/service/action/GetPreviousValues.java | 43 +
.../service/action/GetUserGroupAction.java | 50 +
.../admin/service/action/GetUsersAction.java | 39 +
.../service/action/HasPendingUserAccounts.java | 34 +
.../action/InvalidateUserAccountAction.java | 58 +
.../InvalidateUserGroupAccountsAction.java | 45 +
.../service/action/PurgeActionsAction.java | 51 +
.../action/RequestUserAccountAction.java | 67 +
.../service/action/SeedUserAccountsAction.java | 164 +
.../admin/service/action/UngroupUserAction.java | 69 +
.../service/action/UngroupUserGroupAction.java | 57 +
.../admin/service/action/UpdateUserAction.java | 124 +
.../UpdateUserAuthoritiesCacheAction.java | 73 +
.../service/action/UpdateUserCacheAction.java | 47 +
.../service/action/UpdateUserGroupAction.java | 171 +
.../service/impl/StandardAuditService.java | 230 ++
.../admin/service/impl/StandardUserService.java | 663 ++++
.../admin/service/transaction/Transaction.java | 49 +
.../service/transaction/TransactionBuilder.java | 25 +
.../transaction/TransactionException.java | 40 +
.../transaction/impl/StandardTransaction.java | 93 +
.../impl/StandardTransactionBuilder.java | 57 +
.../AuthorityProviderFactoryBean.java | 529 +++
...rdAuthorityProviderConfigurationContext.java | 50 +
...dAuthorityProviderInitializationContext.java | 42 +
.../java/org/apache/nifi/history/History.java | 56 +
.../org/apache/nifi/history/HistoryQuery.java | 99 +
.../org/apache/nifi/history/PreviousValue.java | 54 +
.../org/apache/nifi/user/AccountStatus.java | 47 +
.../java/org/apache/nifi/user/NiFiUser.java | 164 +
.../org/apache/nifi/user/NiFiUserGroup.java | 45 +
.../resources/nifi-administration-context.xml | 62 +
.../src/main/xsd/authority-providers.xsd | 49 +
.../service/action/AuthorizeUserActionTest.java | 433 +++
.../service/action/CreateUserActionTest.java | 144 +
.../service/action/DisableUserActionTest.java | 171 +
.../action/InvalidateUserAccountActionTest.java | 131 +
.../action/RequestUserAccountActionTest.java | 127 +
.../action/SeedUserAccountsActionTest.java | 263 ++
.../action/SetUserAuthoritiesActionTest.java | 223 ++
.../framework/client-dto/.gitignore | 6 +
.../framework/client-dto/pom.xml | 46 +
.../org/apache/nifi/web/api/dto/AboutDTO.java | 57 +
.../org/apache/nifi/web/api/dto/BannerDTO.java | 57 +
.../nifi/web/api/dto/BulletinBoardDTO.java | 63 +
.../apache/nifi/web/api/dto/BulletinDTO.java | 161 +
.../nifi/web/api/dto/BulletinQueryDTO.java | 113 +
.../org/apache/nifi/web/api/dto/ClusterDTO.java | 61 +
.../apache/nifi/web/api/dto/ConnectableDTO.java | 146 +
.../apache/nifi/web/api/dto/ConnectionDTO.java | 215 ++
.../web/api/dto/ControllerConfigurationDTO.java | 158 +
.../apache/nifi/web/api/dto/ControllerDTO.java | 262 ++
.../org/apache/nifi/web/api/dto/CounterDTO.java | 94 +
.../apache/nifi/web/api/dto/CountersDTO.java | 61 +
.../nifi/web/api/dto/DocumentedTypeDTO.java | 71 +
.../apache/nifi/web/api/dto/FlowSnippetDTO.java | 141 +
.../org/apache/nifi/web/api/dto/FunnelDTO.java | 29 +
.../org/apache/nifi/web/api/dto/LabelDTO.java | 94 +
.../nifi/web/api/dto/NiFiComponentDTO.java | 95 +
.../org/apache/nifi/web/api/dto/NodeDTO.java | 188 +
.../apache/nifi/web/api/dto/NodeEventDTO.java | 74 +
.../web/api/dto/NodeSystemDiagnosticsDTO.java | 56 +
.../org/apache/nifi/web/api/dto/PortDTO.java | 161 +
.../apache/nifi/web/api/dto/PositionDTO.java | 65 +
.../nifi/web/api/dto/PreviousValueDTO.java | 73 +
.../nifi/web/api/dto/ProcessGroupDTO.java | 219 ++
.../nifi/web/api/dto/ProcessorConfigDTO.java | 486 +++
.../apache/nifi/web/api/dto/ProcessorDTO.java | 181 +
.../nifi/web/api/dto/ProcessorHistoryDTO.java | 56 +
.../nifi/web/api/dto/PropertyHistoryDTO.java | 43 +
.../nifi/web/api/dto/RelationshipDTO.java | 69 +
.../api/dto/RemoteProcessGroupContentsDTO.java | 56 +
.../nifi/web/api/dto/RemoteProcessGroupDTO.java | 279 ++
.../web/api/dto/RemoteProcessGroupPortDTO.java | 192 +
.../apache/nifi/web/api/dto/RevisionDTO.java | 63 +
.../org/apache/nifi/web/api/dto/SnippetDTO.java | 239 ++
.../nifi/web/api/dto/SystemDiagnosticsDTO.java | 461 +++
.../apache/nifi/web/api/dto/TemplateDTO.java | 117 +
.../org/apache/nifi/web/api/dto/UserDTO.java | 177 +
.../apache/nifi/web/api/dto/UserGroupDTO.java | 84 +
.../nifi/web/api/dto/action/ActionDTO.java | 176 +
.../nifi/web/api/dto/action/HistoryDTO.java | 74 +
.../web/api/dto/action/HistoryQueryDTO.java | 144 +
.../component/details/ComponentDetailsDTO.java | 32 +
.../component/details/ProcessorDetailsDTO.java | 41 +
.../details/RemoteProcessGroupDetailsDTO.java | 41 +
.../dto/action/details/ActionDetailsDTO.java | 34 +
.../dto/action/details/ConfigureDetailsDTO.java | 69 +
.../dto/action/details/ConnectDetailsDTO.java | 126 +
.../api/dto/action/details/MoveDetailsDTO.java | 83 +
.../api/dto/action/details/PurgeDetailsDTO.java | 45 +
.../web/api/dto/provenance/AttributeDTO.java | 69 +
.../web/api/dto/provenance/ProvenanceDTO.java | 165 +
.../api/dto/provenance/ProvenanceEventDTO.java | 630 +++
.../dto/provenance/ProvenanceOptionsDTO.java | 43 +
.../dto/provenance/ProvenanceRequestDTO.java | 118 +
.../dto/provenance/ProvenanceResultsDTO.java | 136 +
.../ProvenanceSearchableFieldDTO.java | 84 +
.../api/dto/provenance/lineage/LineageDTO.java | 161 +
.../provenance/lineage/LineageRequestDTO.java | 88 +
.../provenance/lineage/LineageResultsDTO.java | 73 +
.../provenance/lineage/ProvenanceLinkDTO.java | 101 +
.../provenance/lineage/ProvenanceNodeDTO.java | 162 +
.../dto/search/ComponentSearchResultDTO.java | 85 +
.../web/api/dto/search/NodeSearchResultDTO.java | 56 +
.../web/api/dto/search/SearchResultsDTO.java | 128 +
.../dto/search/UserGroupSearchResultDTO.java | 42 +
.../web/api/dto/search/UserSearchResultDTO.java | 56 +
.../dto/status/ClusterConnectionStatusDTO.java | 89 +
.../api/dto/status/ClusterPortStatusDTO.java | 89 +
.../dto/status/ClusterProcessorStatusDTO.java | 117 +
.../ClusterRemoteProcessGroupStatusDTO.java | 89 +
.../web/api/dto/status/ClusterStatusDTO.java | 44 +
.../api/dto/status/ClusterStatusHistoryDTO.java | 75 +
.../web/api/dto/status/ConnectionStatusDTO.java | 198 +
.../web/api/dto/status/ControllerStatusDTO.java | 187 +
.../api/dto/status/NodeConnectionStatusDTO.java | 57 +
.../web/api/dto/status/NodePortStatusDTO.java | 57 +
.../api/dto/status/NodeProcessorStatusDTO.java | 57 +
.../status/NodeRemoteProcessGroupStatusDTO.java | 57 +
.../nifi/web/api/dto/status/NodeStatusDTO.java | 57 +
.../api/dto/status/NodeStatusHistoryDTO.java | 57 +
.../nifi/web/api/dto/status/PortStatusDTO.java | 142 +
.../api/dto/status/ProcessGroupStatusDTO.java | 244 ++
.../web/api/dto/status/ProcessorStatusDTO.java | 203 +
.../web/api/dto/status/RemotePortStatusDTO.java | 98 +
.../dto/status/RemoteProcessGroupStatusDTO.java | 159 +
.../nifi/web/api/dto/status/StatusDTO.java | 43 +
.../web/api/dto/status/StatusDescriptorDTO.java | 101 +
.../web/api/dto/status/StatusHistoryDTO.java | 92 +
.../api/dto/status/StatusHistoryDetailDTO.java | 56 +
.../web/api/dto/status/StatusSnapshotDTO.java | 58 +
.../nifi/web/api/dto/util/DateTimeAdapter.java | 47 +
.../nifi/web/api/dto/util/TimeAdapter.java | 47 +
.../nifi/web/api/dto/util/TimestampAdapter.java | 47 +
.../apache/nifi/web/api/entity/AboutEntity.java | 45 +
.../nifi/web/api/entity/ActionEntity.java | 45 +
.../nifi/web/api/entity/AuthorityEntity.java | 60 +
.../nifi/web/api/entity/BannerEntity.java | 46 +
.../web/api/entity/BulletinBoardEntity.java | 45 +
.../entity/ClusterConnectionStatusEntity.java | 45 +
.../nifi/web/api/entity/ClusterEntity.java | 45 +
.../web/api/entity/ClusterPortStatusEntity.java | 45 +
.../entity/ClusterProcessorStatusEntity.java | 45 +
.../ClusterRemoteProcessGroupStatusEntity.java | 45 +
.../api/entity/ClusterSearchResultsEntity.java | 46 +
.../web/api/entity/ClusterStatusEntity.java | 45 +
.../api/entity/ClusterStatusHistoryEntity.java | 45 +
.../nifi/web/api/entity/ConnectionEntity.java | 45 +
.../nifi/web/api/entity/ConnectionsEntity.java | 47 +
.../entity/ControllerConfigurationEntity.java | 45 +
.../nifi/web/api/entity/ControllerEntity.java | 45 +
.../web/api/entity/ControllerStatusEntity.java | 45 +
.../nifi/web/api/entity/CounterEntity.java | 45 +
.../nifi/web/api/entity/CountersEntity.java | 46 +
.../org/apache/nifi/web/api/entity/Entity.java | 43 +
.../nifi/web/api/entity/FlowSnippetEntity.java | 45 +
.../nifi/web/api/entity/FunnelEntity.java | 45 +
.../nifi/web/api/entity/FunnelsEntity.java | 47 +
.../nifi/web/api/entity/HistoryEntity.java | 45 +
.../nifi/web/api/entity/InputPortEntity.java | 45 +
.../nifi/web/api/entity/InputPortsEntity.java | 47 +
.../apache/nifi/web/api/entity/LabelEntity.java | 45 +
.../nifi/web/api/entity/LabelsEntity.java | 47 +
.../nifi/web/api/entity/LineageEntity.java | 45 +
.../apache/nifi/web/api/entity/NodeEntity.java | 45 +
.../nifi/web/api/entity/NodeStatusEntity.java | 45 +
.../api/entity/NodeSystemDiagnosticsEntity.java | 45 +
.../nifi/web/api/entity/OutputPortEntity.java | 45 +
.../nifi/web/api/entity/OutputPortsEntity.java | 47 +
.../web/api/entity/PrioritizerTypesEntity.java | 46 +
.../nifi/web/api/entity/ProcessGroupEntity.java | 45 +
.../api/entity/ProcessGroupStatusEntity.java | 45 +
.../web/api/entity/ProcessGroupsEntity.java | 46 +
.../nifi/web/api/entity/ProcessorEntity.java | 45 +
.../web/api/entity/ProcessorHistoryEntity.java | 45 +
.../web/api/entity/ProcessorTypesEntity.java | 46 +
.../nifi/web/api/entity/ProcessorsEntity.java | 47 +
.../nifi/web/api/entity/ProvenanceEntity.java | 40 +
.../web/api/entity/ProvenanceEventEntity.java | 45 +
.../web/api/entity/ProvenanceOptionsEntity.java | 46 +
.../api/entity/RemoteProcessGroupEntity.java | 45 +
.../entity/RemoteProcessGroupPortEntity.java | 45 +
.../api/entity/RemoteProcessGroupsEntity.java | 47 +
.../web/api/entity/SearchResultsEntity.java | 46 +
.../nifi/web/api/entity/SnippetEntity.java | 45 +
.../web/api/entity/StatusHistoryEntity.java | 45 +
.../web/api/entity/SystemDiagnosticsEntity.java | 45 +
.../nifi/web/api/entity/TemplateEntity.java | 45 +
.../nifi/web/api/entity/TemplatesEntity.java | 63 +
.../apache/nifi/web/api/entity/UserEntity.java | 45 +
.../nifi/web/api/entity/UserGroupEntity.java | 45 +
.../web/api/entity/UserSearchResultsEntity.java | 61 +
.../apache/nifi/web/api/entity/UsersEntity.java | 64 +
.../cluster-authorization-provider/.gitignore | 1 +
.../cluster-authorization-provider/pom.xml | 48 +
.../ClusterManagerAuthorizationProvider.java | 225 ++
.../NodeAuthorizationProvider.java | 389 ++
.../protocol/message/DoesDnExistMessage.java | 56 +
.../protocol/message/GetAuthoritiesMessage.java | 58 +
.../message/GetGroupForUserMessage.java | 55 +
.../protocol/message/ProtocolMessage.java | 57 +
.../message/jaxb/JaxbProtocolUtils.java | 42 +
.../protocol/message/jaxb/ObjectFactory.java | 45 +
....apache.nifi.authorization.AuthorityProvider | 16 +
.../framework/cluster-protocol/.gitignore | 1 +
.../framework/cluster-protocol/pom.xml | 69 +
.../protocol/ClusterManagerProtocolSender.java | 69 +
.../cluster/protocol/ConnectionRequest.java | 44 +
.../cluster/protocol/ConnectionResponse.java | 141 +
.../apache/nifi/cluster/protocol/Heartbeat.java | 68 +
.../nifi/cluster/protocol/NodeBulletins.java | 44 +
.../nifi/cluster/protocol/NodeIdentifier.java | 172 +
.../cluster/protocol/NodeProtocolSender.java | 73 +
.../nifi/cluster/protocol/ProtocolContext.java | 39 +
.../cluster/protocol/ProtocolException.java | 40 +
.../nifi/cluster/protocol/ProtocolHandler.java | 44 +
.../nifi/cluster/protocol/ProtocolListener.java | 72 +
.../protocol/ProtocolMessageMarshaller.java | 38 +
.../protocol/ProtocolMessageUnmarshaller.java | 38 +
.../nifi/cluster/protocol/StandardDataFlow.java | 105 +
.../UnknownServiceAddressException.java | 39 +
.../impl/ClusterManagerProtocolSenderImpl.java | 245 ++
.../ClusterManagerProtocolSenderListener.java | 118 +
.../protocol/impl/ClusterServiceDiscovery.java | 181 +
.../protocol/impl/ClusterServiceLocator.java | 229 ++
.../impl/ClusterServicesBroadcaster.java | 182 +
.../protocol/impl/CopyingInputStream.java | 77 +
.../impl/MulticastProtocolListener.java | 204 +
.../protocol/impl/NodeProtocolSenderImpl.java | 171 +
.../impl/NodeProtocolSenderListener.java | 115 +
.../protocol/impl/SocketProtocolListener.java | 205 +
.../protocol/jaxb/JaxbProtocolContext.java | 148 +
.../jaxb/message/AdaptedConnectionRequest.java | 40 +
.../jaxb/message/AdaptedConnectionResponse.java | 109 +
.../protocol/jaxb/message/AdaptedCounter.java | 56 +
.../protocol/jaxb/message/AdaptedDataFlow.java | 64 +
.../protocol/jaxb/message/AdaptedHeartbeat.java | 66 +
.../jaxb/message/AdaptedNodeBulletins.java | 50 +
.../jaxb/message/AdaptedNodeIdentifier.java | 76 +
.../jaxb/message/ConnectionRequestAdapter.java | 41 +
.../jaxb/message/ConnectionResponseAdapter.java | 55 +
.../protocol/jaxb/message/DataFlowAdapter.java | 50 +
.../protocol/jaxb/message/HeartbeatAdapter.java | 54 +
.../jaxb/message/JaxbProtocolUtils.java | 42 +
.../jaxb/message/NodeBulletinsAdapter.java | 48 +
.../jaxb/message/NodeIdentifierAdapter.java | 51 +
.../protocol/jaxb/message/ObjectFactory.java | 104 +
.../message/ConnectionRequestMessage.java | 46 +
.../message/ConnectionResponseMessage.java | 66 +
.../ControllerStartupFailureMessage.java | 49 +
.../protocol/message/DisconnectMessage.java | 55 +
.../protocol/message/ExceptionMessage.java | 44 +
.../protocol/message/FlowRequestMessage.java | 46 +
.../protocol/message/FlowResponseMessage.java | 44 +
.../protocol/message/HeartbeatMessage.java | 43 +
.../message/MulticastProtocolMessage.java | 66 +
.../protocol/message/NodeBulletinsMessage.java | 43 +
.../cluster/protocol/message/PingMessage.java | 55 +
.../message/PrimaryRoleAssignmentMessage.java | 56 +
.../protocol/message/ProtocolMessage.java | 61 +
.../message/ReconnectionFailureMessage.java | 45 +
.../message/ReconnectionRequestMessage.java | 94 +
.../message/ReconnectionResponseMessage.java | 32 +
.../message/ServiceBroadcastMessage.java | 64 +
.../MulticastConfigurationFactoryBean.java | 60 +
.../ServerSocketConfigurationFactoryBean.java | 65 +
.../spring/SocketConfigurationFactoryBean.java | 66 +
.../resources/nifi-cluster-protocol-context.xml | 110 +
.../ClusterManagerProtocolSenderImplTest.java | 134 +
.../impl/ClusterServiceDiscoveryTest.java | 135 +
.../impl/ClusterServiceLocatorTest.java | 121 +
.../impl/ClusterServicesBroadcasterTest.java | 133 +
.../impl/MulticastProtocolListenerTest.java | 171 +
.../impl/NodeProtocolSenderImplTest.java | 203 +
.../impl/testutils/DelayedProtocolHandler.java | 57 +
.../testutils/ReflexiveProtocolHandler.java | 47 +
.../framework/cluster-web/.gitignore | 1 +
.../framework/cluster-web/pom.xml | 50 +
.../nifi/cluster/context/ClusterContext.java | 59 +
.../cluster/context/ClusterContextImpl.java | 69 +
.../context/ClusterContextThreadLocal.java | 47 +
.../ClusterAwareOptimisticLockingManager.java | 96 +
.../framework/cluster/.gitignore | 1 +
.../framework-bundle/framework/cluster/pom.xml | 132 +
.../cluster/client/MulticastTestClient.java | 151 +
.../org/apache/nifi/cluster/event/Event.java | 122 +
.../apache/nifi/cluster/event/EventManager.java | 65 +
.../cluster/event/impl/EventManagerImpl.java | 143 +
.../cluster/firewall/ClusterNodeFirewall.java | 35 +
.../impl/FileBasedClusterNodeFirewall.java | 207 +
.../nifi/cluster/flow/ClusterDataFlow.java | 45 +
.../apache/nifi/cluster/flow/DaoException.java | 40 +
.../apache/nifi/cluster/flow/DataFlowDao.java | 62 +
.../cluster/flow/DataFlowManagementService.java | 115 +
.../nifi/cluster/flow/PersistedFlowState.java | 37 +
.../nifi/cluster/flow/StaleFlowException.java | 42 +
.../nifi/cluster/flow/impl/DataFlowDaoImpl.java | 600 +++
.../impl/DataFlowManagementServiceImpl.java | 356 ++
.../nifi/cluster/manager/ClusterManager.java | 225 ++
.../cluster/manager/HttpClusterManager.java | 169 +
.../cluster/manager/HttpRequestReplicator.java | 99 +
.../cluster/manager/HttpResponseMapper.java | 42 +
.../nifi/cluster/manager/NodeResponse.java | 329 ++
.../exception/BlockedByFirewallException.java | 60 +
.../manager/exception/ClusterException.java | 40 +
.../ConnectingNodeMutableRequestException.java | 41 +
...DisconnectedNodeMutableRequestException.java | 41 +
.../exception/IllegalClusterStateException.java | 41 +
.../exception/IllegalNodeDeletionException.java | 41 +
.../IllegalNodeDisconnectionException.java | 42 +
.../IllegalNodeReconnectionException.java | 41 +
.../IneligiblePrimaryNodeException.java | 41 +
.../exception/MutableRequestException.java | 42 +
.../exception/NoConnectedNodesException.java | 41 +
.../exception/NoResponseFromNodesException.java | 42 +
.../exception/NodeDisconnectionException.java | 41 +
.../exception/NodeReconnectionException.java | 40 +
.../PrimaryRoleAssignmentException.java | 41 +
.../SafeModeMutableRequestException.java | 41 +
.../manager/exception/UnknownNodeException.java | 41 +
.../exception/UriConstructionException.java | 42 +
.../manager/impl/ClusteredEventAccess.java | 135 +
.../manager/impl/ClusteredReportingContext.java | 165 +
.../manager/impl/HttpRequestReplicatorImpl.java | 531 +++
.../manager/impl/HttpResponseMapperImpl.java | 85 +
.../cluster/manager/impl/WebClusterManager.java | 3620 ++++++++++++++++++
.../java/org/apache/nifi/cluster/node/Node.java | 252 ++
...anagerProtocolServiceLocatorFactoryBean.java | 116 +
...FileBasedClusterNodeFirewallFactoryBean.java | 58 +
.../spring/WebClusterManagerFactoryBean.java | 139 +
.../reporting/ClusteredReportingTaskNode.java | 49 +
.../resources/nifi-cluster-manager-context.xml | 124 +
.../event/impl/EventManagerImplTest.java | 119 +
.../impl/FileBasedClusterNodeFirewallTest.java | 98 +
.../impl/DataFlowManagementServiceImplTest.java | 343 ++
.../impl/HttpRequestReplicatorImplTest.java | 368 ++
.../impl/HttpResponseMapperImplTest.java | 126 +
.../manager/impl/TestWebClusterManager.java | 54 +
.../cluster/manager/testutils/HttpRequest.java | 239 ++
.../cluster/manager/testutils/HttpResponse.java | 93 +
.../manager/testutils/HttpResponseAction.java | 60 +
.../cluster/manager/testutils/HttpServer.java | 240 ++
.../ClusterManagerProtocolSenderImplTest.java | 133 +
.../impl/ClusterServiceLocatorTest.java | 119 +
.../impl/ClusterServicesBroadcasterTest.java | 131 +
.../impl/MulticastProtocolListenerTest.java | 171 +
.../impl/NodeProtocolSenderImplTest.java | 201 +
.../impl/SocketProtocolListenerTest.java | 132 +
.../testutils/DelayedProtocolHandler.java | 57 +
.../testutils/ReflexiveProtocolHandler.java | 47 +
.../cluster/src/test/resources/logback-test.xml | 48 +
.../apache/nifi/cluster/firewall/impl/empty.txt | 0
.../apache/nifi/cluster/firewall/impl/ips.txt | 12 +
.../framework/core-api/.gitignore | 1 +
.../framework-bundle/framework/core-api/pom.xml | 56 +
.../nifi/cluster/AdaptedNodeInformation.java | 66 +
.../nifi/cluster/ClusterNodeInformation.java | 67 +
.../org/apache/nifi/cluster/NodeInformant.java | 22 +
.../apache/nifi/cluster/NodeInformation.java | 98 +
.../nifi/cluster/NodeInformationAdapter.java | 39 +
.../apache/nifi/cluster/protocol/DataFlow.java | 41 +
.../apache/nifi/connectable/Connectable.java | 291 ++
.../nifi/connectable/ConnectableType.java | 44 +
.../org/apache/nifi/connectable/Connection.java | 78 +
.../org/apache/nifi/connectable/Funnel.java | 24 +
.../java/org/apache/nifi/connectable/Port.java | 31 +
.../org/apache/nifi/connectable/Position.java | 36 +
.../java/org/apache/nifi/connectable/Size.java | 36 +
.../controller/AbstractConfiguredComponent.java | 280 ++
.../apache/nifi/controller/AbstractPort.java | 636 +++
.../apache/nifi/controller/Availability.java | 24 +
.../nifi/controller/ConfiguredComponent.java | 63 +
.../nifi/controller/ContentAvailability.java | 65 +
.../org/apache/nifi/controller/Counter.java | 32 +
.../nifi/controller/EventBasedWorker.java | 32 +
.../org/apache/nifi/controller/Heartbeater.java | 22 +
.../nifi/controller/ProcessScheduler.java | 146 +
.../apache/nifi/controller/ProcessorNode.java | 80 +
.../nifi/controller/ReportingTaskNode.java | 56 +
.../nifi/controller/StandardFlowFileQueue.java | 1096 ++++++
.../apache/nifi/controller/StandardFunnel.java | 541 +++
.../controller/ValidationContextFactory.java | 27 +
.../org/apache/nifi/controller/WorkerQueue.java | 36 +
.../exception/CommunicationsException.java | 40 +
...ControllerServiceAlreadyExistsException.java | 30 +
.../ControllerServiceNotFoundException.java | 51 +
.../ProcessorInstantiationException.java | 27 +
.../exception/ProcessorLifeCycleException.java | 30 +
.../org/apache/nifi/controller/label/Label.java | 48 +
.../ReportingTaskInstantiationException.java | 31 +
.../repository/ContentNotFoundException.java | 48 +
.../repository/CounterRepository.java | 34 +
.../controller/repository/FlowFileEvent.java | 54 +
.../repository/FlowFileEventRepository.java | 50 +
.../repository/RepositoryStatusReport.java | 28 +
.../service/ControllerServiceNode.java | 40 +
.../service/ControllerServiceProvider.java | 47 +
.../service/ControllerServiceReference.java | 50 +
.../org/apache/nifi/events/BulletinFactory.java | 52 +
.../nifi/events/BulletinProcessingStrategy.java | 27 +
.../apache/nifi/events/ComponentBulletin.java | 30 +
.../org/apache/nifi/events/SystemBulletin.java | 30 +
.../org/apache/nifi/groups/ProcessGroup.java | 723 ++++
.../apache/nifi/groups/ProcessGroupCounts.java | 66 +
.../apache/nifi/groups/RemoteProcessGroup.java | 255 ++
.../RemoteProcessGroupPortDescriptor.java | 92 +
.../org/apache/nifi/logging/LogMessage.java | 74 +
.../org/apache/nifi/logging/LogObserver.java | 22 +
.../org/apache/nifi/logging/LogRepository.java | 67 +
.../nifi/logging/LogRepositoryFactory.java | 61 +
.../java/org/apache/nifi/nar/NarCloseable.java | 44 +
.../nifi/nar/NarThreadContextClassLoader.java | 188 +
.../main/java/org/apache/nifi/remote/Peer.java | 107 +
.../java/org/apache/nifi/remote/PeerStatus.java | 72 +
.../nifi/remote/PortAuthorizationResult.java | 25 +
.../nifi/remote/RemoteAuthorizationState.java | 27 +
.../org/apache/nifi/remote/RemoteGroupPort.java | 35 +
.../org/apache/nifi/remote/RootGroupPort.java | 78 +
.../apache/nifi/remote/TransferDirection.java | 23 +
.../nifi/remote/VersionedRemoteResource.java | 24 +
.../apache/nifi/remote/codec/FlowFileCodec.java | 79 +
.../remote/exception/BadRequestException.java | 30 +
.../remote/exception/HandshakeException.java | 30 +
.../exception/NotAuthorizedException.java | 26 +
.../exception/PortNotRunningException.java | 26 +
.../remote/exception/ProtocolException.java | 34 +
.../exception/RequestExpiredException.java | 26 +
.../remote/exception/UnknownPortException.java | 26 +
.../nifi/remote/protocol/ClientProtocol.java | 78 +
.../remote/protocol/CommunicationsInput.java | 27 +
.../remote/protocol/CommunicationsOutput.java | 27 +
.../remote/protocol/CommunicationsSession.java | 64 +
.../nifi/remote/protocol/RequestType.java | 43 +
.../nifi/remote/protocol/ServerProtocol.java | 143 +
.../framework-bundle/framework/core/.gitignore | 1 +
.../framework-bundle/framework/core/pom.xml | 121 +
.../apache/nifi/cluster/BulletinsPayload.java | 95 +
.../nifi/cluster/ConnectionException.java | 42 +
.../nifi/cluster/DisconnectionException.java | 42 +
.../apache/nifi/cluster/HeartbeatPayload.java | 170 +
.../org/apache/nifi/connectable/LocalPort.java | 172 +
.../nifi/connectable/StandardConnection.java | 336 ++
.../nifi/controller/EventDrivenWorkerQueue.java | 329 ++
.../nifi/controller/FileSystemSwapManager.java | 768 ++++
.../apache/nifi/controller/FlowController.java | 3579 +++++++++++++++++
.../nifi/controller/FlowFromDOMFactory.java | 418 ++
.../controller/FlowSerializationException.java | 48 +
.../apache/nifi/controller/FlowSerializer.java | 42 +
.../FlowSynchronizationException.java | 47 +
.../nifi/controller/FlowSynchronizer.java | 53 +
.../nifi/controller/FlowUnmarshaller.java | 78 +
.../apache/nifi/controller/SnippetManager.java | 96 +
.../apache/nifi/controller/StandardCounter.java | 108 +
.../nifi/controller/StandardFlowSerializer.java | 404 ++
.../nifi/controller/StandardFlowService.java | 875 +++++
.../controller/StandardFlowSynchronizer.java | 1026 +++++
.../nifi/controller/StandardProcessorNode.java | 1243 ++++++
.../apache/nifi/controller/StandardSnippet.java | 186 +
.../org/apache/nifi/controller/Template.java | 37 +
.../apache/nifi/controller/TemplateManager.java | 507 +++
.../controller/UninheritableFlowException.java | 48 +
.../exception/FlowFileConsumptionException.java | 38 +
.../ReportingTaskInstantiationException.java | 31 +
.../exception/ValidationException.java | 47 +
.../nifi/controller/label/StandardLabel.java | 110 +
.../reporting/AbstractReportingTaskNode.java | 111 +
.../reporting/StandardReportingContext.java | 132 +
.../StandardReportingInitializationContext.java | 96 +
.../reporting/StandardReportingTaskNode.java | 40 +
.../repository/BatchingSessionFactory.java | 242 ++
.../repository/ConnectionSwapInfo.java | 58 +
.../repository/FileSystemRepository.java | 1351 +++++++
.../controller/repository/ProcessContext.java | 291 ++
.../repository/ProvenanceEventEnricher.java | 34 +
.../repository/RepositoryPurgeException.java | 59 +
.../repository/RingBufferEventRepository.java | 312 ++
.../repository/StandardCounterRepository.java | 109 +
.../repository/StandardFlowFileEvent.java | 237 ++
.../repository/StandardFlowFileRecord.java | 312 ++
.../repository/StandardProcessSession.java | 2684 +++++++++++++
.../StandardProcessSessionFactory.java | 33 +
.../repository/StandardProvenanceReporter.java | 431 +++
.../repository/StandardRepositoryRecord.java | 196 +
.../StandardRepositoryStatusReport.java | 88 +
.../repository/VolatileContentRepository.java | 647 ++++
.../repository/VolatileFlowFileRepository.java | 109 +
.../WriteAheadFlowFileRepository.java | 864 +++++
.../repository/claim/ContentDirection.java | 35 +
.../repository/claim/StandardContentClaim.java | 154 +
.../claim/StandardContentClaimManager.java | 155 +
.../repository/io/ArrayManagedOutputStream.java | 245 ++
.../repository/io/ByteCountingInputStream.java | 100 +
.../repository/io/ByteCountingOutputStream.java | 64 +
.../io/DisableOnCloseOutputStream.java | 73 +
.../io/FlowFileAccessInputStream.java | 168 +
.../io/FlowFileAccessOutputStream.java | 95 +
.../repository/io/LimitedInputStream.java | 107 +
.../controller/repository/io/LongHolder.java | 48 +
.../controller/repository/io/MemoryManager.java | 55 +
.../repository/io/SyncOnCloseOutputStream.java | 58 +
.../scheduling/ConnectableProcessContext.java | 194 +
.../scheduling/EventDrivenSchedulingAgent.java | 346 ++
.../scheduling/ProcessContextFactory.java | 51 +
.../scheduling/QuartzSchedulingAgent.java | 220 ++
.../controller/scheduling/ScheduleState.java | 90 +
.../controller/scheduling/SchedulingAgent.java | 45 +
.../scheduling/StandardProcessScheduler.java | 569 +++
.../scheduling/TimerDrivenSchedulingAgent.java | 139 +
.../service/ControllerServiceLoader.java | 156 +
.../service/StandardConfigurationContext.java | 64 +
...dControllerServiceInitializationContext.java | 64 +
.../service/StandardControllerServiceNode.java | 125 +
.../StandardControllerServiceProvider.java | 196 +
.../StandardControllerServiceReference.java | 100 +
.../history/StandardMetricDescriptor.java | 128 +
.../status/history/StandardStatusHistory.java | 54 +
.../status/history/StandardStatusSnapshot.java | 70 +
.../status/history/StatusHistoryUtil.java | 107 +
.../VolatileComponentStatusRepository.java | 649 ++++
.../tasks/ContinuallyRunConnectableTask.java | 97 +
.../tasks/ContinuallyRunProcessorTask.java | 185 +
.../nifi/controller/tasks/ExpireFlowFiles.java | 114 +
.../controller/tasks/ReportingTaskWrapper.java | 63 +
.../nifi/diagnostics/DiagnosticUtils.java | 28 +
.../nifi/diagnostics/GarbageCollection.java | 60 +
.../apache/nifi/diagnostics/StorageUsage.java | 72 +
.../nifi/diagnostics/SystemDiagnostics.java | 212 +
.../diagnostics/SystemDiagnosticsFactory.java | 133 +
.../nifi/encrypt/EncryptionException.java | 43 +
.../apache/nifi/encrypt/StringEncryptor.java | 152 +
.../java/org/apache/nifi/engine/FlowEngine.java | 135 +
.../events/NodeBulletinProcessingStrategy.java | 66 +
.../nifi/events/VolatileBulletinRepository.java | 240 ++
.../events/network/CommunicationsFailure.java | 36 +
.../nifi/events/network/NetworkTransfer.java | 48 +
.../nifi/fingerprint/FingerprintException.java | 41 +
.../nifi/fingerprint/FingerprintFactory.java | 985 +++++
.../nifi/groups/StandardProcessGroup.java | 2019 ++++++++++
.../org/apache/nifi/jaxb/AdaptedBulletin.java | 100 +
.../org/apache/nifi/jaxb/AdaptedCounter.java | 64 +
.../org/apache/nifi/jaxb/BulletinAdapter.java | 59 +
.../org/apache/nifi/jaxb/CounterAdapter.java | 51 +
.../org/apache/nifi/lifecycle/LifeCycle.java | 54 +
.../nifi/lifecycle/LifeCycleException.java | 48 +
.../nifi/lifecycle/LifeCycleStartException.java | 49 +
.../nifi/lifecycle/LifeCycleStopException.java | 48 +
.../nifi/logging/ProcessorLogObserver.java | 48 +
.../repository/StandardLogRepository.java | 173 +
.../nifi/persistence/FlowConfigurationDAO.java | 135 +
.../StandardSnippetDeserializer.java | 42 +
.../persistence/StandardSnippetSerializer.java | 47 +
.../StandardXMLFlowConfigurationDAO.java | 345 ++
.../nifi/persistence/TemplateDeserializer.java | 42 +
.../nifi/persistence/TemplateSerializer.java | 47 +
.../nifi/processor/SimpleProcessLogger.java | 307 ++
.../nifi/processor/StandardProcessContext.java | 173 +
.../StandardProcessorInitializationContext.java | 49 +
.../nifi/processor/StandardPropertyValue.java | 163 +
.../processor/StandardSchedulingContext.java | 113 +
.../processor/StandardValidationContext.java | 97 +
.../StandardValidationContextFactory.java | 39 +
.../nifi/remote/StandardRemoteProcessGroup.java | 1621 ++++++++
...tandardRemoteProcessGroupPortDescriptor.java | 148 +
.../org/apache/nifi/services/FlowService.java | 139 +
.../nifi/spring/FlowControllerFactoryBean.java | 103 +
.../spring/RingBufferEventRepositoryBean.java | 45 +
.../spring/StandardFlowServiceFactoryBean.java | 98 +
.../apache/nifi/util/ComponentStatusReport.java | 139 +
.../java/org/apache/nifi/util/Connectables.java | 56 +
.../java/org/apache/nifi/util/DomUtils.java | 79 +
.../org/apache/nifi/util/ReflectionUtils.java | 157 +
.../java/org/apache/nifi/util/SnippetUtils.java | 241 ++
.../ControllerServiceConfiguration.xsd | 61 +
.../src/main/resources/FlowConfiguration.xsd | 335 ++
.../resources/ReportingTaskConfiguration.xsd | 87 +
.../core/src/main/resources/nifi-context.xml | 49 +
.../nifi/cluster/HeartbeatPayloadTest.java | 121 +
.../controller/StandardFlowServiceTest.java | 300 ++
.../controller/TestFileSystemSwapManager.java | 104 +
.../repository/TestFileSystemRepository.java | 352 ++
.../TestRingBufferEventRepository.java | 138 +
.../repository/TestStandardProcessSession.java | 1177 ++++++
.../TestStandardProvenanceReporter.java | 65 +
.../TestVolatileContentRepository.java | 183 +
.../TestWriteAheadFlowFileRepository.java | 135 +
.../repository/io/TestLimitedOutputStream.java | 76 +
.../controller/repository/util/DiskUtils.java | 77 +
.../fingerprint/FingerprintFactoryTest.java | 60 +
.../processor/TestStandardPropertyValue.java | 167 +
.../processors/DataGeneratorTestProcessor.java | 85 +
.../StubAttributeLoggerProcessor.java | 111 +
.../StubTerminationFileProcessor.java | 40 +
.../core/src/test/resources/conf/0bytes.xml | 0
.../test/resources/conf/all-flow-corrupt.xml | 201 +
.../resources/conf/all-flow-inheritable.xml | 196 +
.../resources/conf/all-flow-uninheritable.xml | 202 +
.../core/src/test/resources/conf/all-flow.xml | 198 +
.../src/test/resources/conf/nifi.properties | 127 +
.../test/resources/conf/only-termination.xml | 37 +
.../src/test/resources/conf/remote-flow.xml | 145 +
.../src/test/resources/conf/standard-flow.xml | 196 +
.../core/src/test/resources/conf/taskConfig.xml | 17 +
.../test/resources/conf/termination-only.xml | 0
.../framework/core/src/test/resources/hello.txt | 1 +
.../core/src/test/resources/logback-test.xml | 36 +
.../test/resources/nifi-with-remote.properties | 127 +
.../core/src/test/resources/nifi.properties | 127 +
.../test/resources/nifi/fingerprint/flow1a.xml | 160 +
.../test/resources/nifi/fingerprint/flow1b.xml | 160 +
.../test/resources/nifi/fingerprint/flow2.xml | 160 +
.../core/src/test/resources/old-swap-file.swap | Bin 0 -> 1730054 bytes
.../file-authorization-provider/pom.xml | 81 +
.../FileAuthorizationProvider.java | 584 +++
....apache.nifi.authorization.AuthorityProvider | 15 +
.../src/main/xsd/users.xsd | 64 +
.../FileAuthorizationProviderTest.java | 127 +
.../framework-bundle/framework/nar/.gitignore | 1 +
.../framework-bundle/framework/nar/pom.xml | 36 +
.../org/apache/nifi/nar/ExtensionManager.java | 188 +
.../org/apache/nifi/nar/ExtensionMapping.java | 73 +
.../org/apache/nifi/nar/NarClassLoader.java | 227 ++
.../org/apache/nifi/nar/NarClassLoaders.java | 301 ++
.../java/org/apache/nifi/nar/NarUnpacker.java | 351 ++
.../java/org/apache/nifi/util/FileUtils.java | 187 +
.../framework-bundle/framework/pom.xml | 51 +
.../framework/resources/pom.xml | 52 +
.../src/main/assembly/dependencies.xml | 36 +
.../src/main/resources/bin/dump-nifi.bat | 33 +
.../src/main/resources/bin/nifi-status.bat | 33 +
.../resources/src/main/resources/bin/nifi.sh | 181 +
.../src/main/resources/bin/run-nifi.bat | 33 +
.../src/main/resources/bin/start-nifi.bat | 33 +
.../src/main/resources/bin/stop-nifi.bat | 33 +
.../main/resources/conf/authority-providers.xml | 43 +
.../main/resources/conf/authorized-users.xml | 57 +
.../src/main/resources/conf/bootstrap.conf | 57 +
.../main/resources/conf/controller-services.xml | 18 +
.../src/main/resources/conf/logback.xml | 105 +
.../src/main/resources/conf/nifi.properties | 159 +
.../src/main/resources/conf/reporting-tasks.xml | 49 +
.../framework/runtime/.gitignore | 2 +
.../framework-bundle/framework/runtime/pom.xml | 42 +
.../java/org/apache/nifi/BootstrapListener.java | 378 ++
.../src/main/java/org/apache/nifi/NiFi.java | 223 ++
.../main/java/org/apache/nifi/NiFiServer.java | 31 +
.../apache/nifi/util/LimitingInputStream.java | 107 +
.../framework/security/.gitignore | 1 +
.../framework-bundle/framework/security/pom.xml | 42 +
.../util/SslContextCreationException.java | 40 +
.../security/util/SslContextFactory.java | 119 +
.../framework/security/util/SslException.java | 40 +
.../security/util/SslServerSocketFactory.java | 81 +
...SslServerSocketFactoryCreationException.java | 40 +
.../security/util/SslSocketFactory.java | 91 +
.../util/SslSocketFactoryCreationException.java | 40 +
.../security/util/SslContextFactoryTest.java | 69 +
.../src/test/resources/localhost-ks.jks | Bin 0 -> 1298 bytes
.../src/test/resources/localhost-ts.jks | Bin 0 -> 891 bytes
.../src/test/resources/log4j.properties | 21 +
.../framework/site-to-site/.gitignore | 1 +
.../framework/site-to-site/pom.xml | 74 +
.../util/RemoteProcessGroupUtils.java | 216 ++
.../remote/AbstractCommunicationsSession.java | 54 +
.../nifi/remote/RemoteResourceFactory.java | 202 +
.../nifi/remote/RemoteResourceManager.java | 227 ++
.../apache/nifi/remote/RemoteSiteListener.java | 33 +
.../nifi/remote/SocketRemoteSiteListener.java | 340 ++
.../nifi/remote/StandardRemoteGroupPort.java | 764 ++++
.../nifi/remote/StandardRootGroupPort.java | 629 +++
.../remote/codec/StandardFlowFileCodec.java | 169 +
.../exception/UnsupportedCodecException.java | 31 +
.../SocketChannelCommunicationsSession.java | 90 +
.../remote/io/socket/SocketChannelInput.java | 66 +
.../remote/io/socket/SocketChannelOutput.java | 58 +
.../SSLSocketChannelCommunicationsSession.java | 93 +
.../io/socket/ssl/SSLSocketChannelInput.java | 50 +
.../io/socket/ssl/SSLSocketChannelOutput.java | 44 +
.../socket/ClusterManagerServerProtocol.java | 208 +
.../protocol/socket/HandshakeProperty.java | 23 +
.../nifi/remote/protocol/socket/Response.java | 51 +
.../remote/protocol/socket/ResponseCode.java | 152 +
.../protocol/socket/SocketClientProtocol.java | 510 +++
.../socket/SocketFlowFileServerProtocol.java | 581 +++
.../org.apache.nifi.remote.codec.FlowFileCodec | 15 +
...g.apache.nifi.remote.protocol.ClientProtocol | 15 +
...g.apache.nifi.remote.protocol.ServerProtocol | 16 +
.../remote/TestStandardRemoteGroupPort.java | 97 +
.../remote/TestStandardSiteToSiteProtocol.java | 116 +
.../io/socket/TestSocketChannelStreams.java | 231 ++
.../io/socket/ssl/TestSSLSocketChannel.java | 381 ++
.../test/resources/dummy-certs/localhost-ks.jks | Bin 0 -> 1298 bytes
.../test/resources/dummy-certs/localhost-ts.jks | Bin 0 -> 891 bytes
.../src/test/resources/logback-test.xml | 30 +
.../src/test/resources/nifi.properties | 111 +
.../framework/user-actions/.gitignore | 1 +
.../framework/user-actions/pom.xml | 27 +
.../java/org/apache/nifi/action/Action.java | 121 +
.../java/org/apache/nifi/action/Component.java | 31 +
.../java/org/apache/nifi/action/Operation.java | 36 +
.../component/details/ComponentDetails.java | 26 +
.../component/details/ProcessorDetails.java | 34 +
.../details/RemoteProcessGroupDetails.java | 34 +
.../nifi/action/details/ActionDetails.java | 26 +
.../nifi/action/details/ConfigureDetails.java | 52 +
.../nifi/action/details/ConnectDetails.java | 90 +
.../apache/nifi/action/details/MoveDetails.java | 61 +
.../nifi/action/details/PurgeDetails.java | 41 +
.../web/custom-ui-utilities/.gitignore | 1 +
.../framework/web/custom-ui-utilities/pom.xml | 36 +
.../web/HttpServletRequestContextConfig.java | 118 +
.../framework/web/nifi-jetty/.gitignore | 1 +
.../framework/web/nifi-jetty/pom.xml | 166 +
.../src/main/assembly/distribution.xml | 42 +
.../org/apache/nifi/web/server/JettyServer.java | 686 ++++
.../server/ServerConfigurationException.java | 36 +
.../org/apache/nifi/web/webdefault.xml | 556 +++
.../framework/web/nifi-web-api/.gitignore | 1 +
.../framework/web/nifi-web-api/pom.xml | 305 ++
.../src/main/enunciate/enunciate.xml | 36 +
.../src/main/enunciate/images/home.png | Bin 0 -> 144 bytes
.../src/main/enunciate/override.css | 178 +
.../org/apache/nifi/aop/MethodProfiler.java | 45 +
.../apache/nifi/audit/ControllerAuditor.java | 257 ++
.../org/apache/nifi/audit/FunnelAuditor.java | 135 +
.../java/org/apache/nifi/audit/NiFiAuditor.java | 119 +
.../java/org/apache/nifi/audit/PortAuditor.java | 334 ++
.../apache/nifi/audit/ProcessGroupAuditor.java | 263 ++
.../org/apache/nifi/audit/ProcessorAuditor.java | 408 ++
.../apache/nifi/audit/RelationshipAuditor.java | 412 ++
.../nifi/audit/RemoteProcessGroupAuditor.java | 376 ++
.../org/apache/nifi/audit/SnippetAuditor.java | 519 +++
.../nifi/web/ClusterRequestException.java | 38 +
.../IllegalClusterResourceRequestException.java | 33 +
.../org/apache/nifi/web/NiFiCoreException.java | 32 +
.../org/apache/nifi/web/NiFiServiceFacade.java | 1263 ++++++
.../apache/nifi/web/NiFiServiceFacadeLock.java | 177 +
.../nifi/web/StandardNiFiServiceFacade.java | 2877 ++++++++++++++
.../apache/nifi/web/StandardNiFiWebContext.java | 336 ++
.../nifi/web/api/ApplicationResource.java | 399 ++
.../nifi/web/api/BulletinBoardResource.java | 116 +
.../apache/nifi/web/api/ClusterResource.java | 724 ++++
.../apache/nifi/web/api/ConnectionResource.java | 771 ++++
.../apache/nifi/web/api/ControllerResource.java | 803 ++++
.../org/apache/nifi/web/api/FunnelResource.java | 493 +++
.../apache/nifi/web/api/HistoryResource.java | 271 ++
.../apache/nifi/web/api/InputPortResource.java | 531 +++
.../org/apache/nifi/web/api/LabelResource.java | 547 +++
.../org/apache/nifi/web/api/NodeResource.java | 302 ++
.../apache/nifi/web/api/OutputPortResource.java | 532 +++
.../nifi/web/api/ProcessGroupResource.java | 1102 ++++++
.../apache/nifi/web/api/ProcessorResource.java | 693 ++++
.../apache/nifi/web/api/ProvenanceResource.java | 1068 ++++++
.../web/api/RemoteProcessGroupResource.java | 912 +++++
.../apache/nifi/web/api/SnippetResource.java | 601 +++
.../nifi/web/api/SystemDiagnosticsResource.java | 81 +
.../apache/nifi/web/api/TemplateResource.java | 400 ++
.../apache/nifi/web/api/UserGroupResource.java | 383 ++
.../org/apache/nifi/web/api/UserResource.java | 469 +++
.../api/config/AccessDeniedExceptionMapper.java | 54 +
.../config/AccountNotFoundExceptionMapper.java | 47 +
.../config/AdministrationExceptionMapper.java | 48 +
...ationCredentialsNotFoundExceptionMapper.java | 48 +
.../web/api/config/ClusterExceptionMapper.java | 47 +
...ectingNodeMutableRequestExceptionMapper.java | 49 +
...nectedNodeMutableRequestExceptionMapper.java | 49 +
.../config/IllegalArgumentExceptionMapper.java | 46 +
...alClusterResourceRequestExceptionMapper.java | 47 +
.../IllegalClusterStateExceptionMapper.java | 47 +
.../IllegalNodeDeletionExceptionMapper.java | 46 +
...IllegalNodeDisconnectionExceptionMapper.java | 46 +
.../IllegalNodeReconnectionExceptionMapper.java | 46 +
.../api/config/IllegalStateExceptionMapper.java | 46 +
.../IneligiblePrimaryNodeExceptionMapper.java | 47 +
.../config/InvalidRevisionExceptionMapper.java | 47 +
.../api/config/JsonMappingExceptionMapper.java | 46 +
.../api/config/JsonParseExceptionMapper.java | 46 +
.../config/MutableRequestExceptionMapper.java | 48 +
.../web/api/config/NiFiCoreExceptionMapper.java | 47 +
.../config/NoConnectedNodesExceptionMapper.java | 47 +
.../NoResponseFromNodesExceptionMapper.java | 49 +
.../NodeDisconnectionExceptionMapper.java | 47 +
.../config/NodeReconnectionExceptionMapper.java | 47 +
.../PrimaryRoleAssignmentExceptionMapper.java | 47 +
.../config/ResourceNotFoundExceptionMapper.java | 48 +
.../SafeModeMutableRequestExceptionMapper.java | 48 +
.../nifi/web/api/config/ThrowableMapper.java | 46 +
.../api/config/UnknownNodeExceptionMapper.java | 47 +
.../api/config/ValidationExceptionMapper.java | 49 +
.../config/WebApplicationExceptionMapper.java | 63 +
.../org/apache/nifi/web/api/dto/DtoFactory.java | 2142 +++++++++++
.../org/apache/nifi/web/api/package-info.java | 58 +
.../request/BulletinBoardPatternParameter.java | 47 +
.../nifi/web/api/request/ClientIdParameter.java | 45 +
.../api/request/ConnectableTypeParameter.java | 41 +
.../nifi/web/api/request/DateTimeParameter.java | 63 +
.../nifi/web/api/request/DoubleParameter.java | 40 +
.../nifi/web/api/request/IntegerParameter.java | 40 +
.../nifi/web/api/request/LongParameter.java | 40 +
.../ApplicationStartupContextListener.java | 157 +
.../nifi/web/controller/ControllerFacade.java | 1392 +++++++
.../web/controller/StandardSearchContext.java | 64 +
.../org/apache/nifi/web/dao/ConnectionDAO.java | 112 +
.../java/org/apache/nifi/web/dao/FunnelDAO.java | 88 +
.../java/org/apache/nifi/web/dao/LabelDAO.java | 79 +
.../java/org/apache/nifi/web/dao/PortDAO.java | 96 +
.../apache/nifi/web/dao/ProcessGroupDAO.java | 89 +
.../org/apache/nifi/web/dao/ProcessorDAO.java | 97 +
.../nifi/web/dao/RemoteProcessGroupDAO.java | 135 +
.../org/apache/nifi/web/dao/SnippetDAO.java | 92 +
.../org/apache/nifi/web/dao/TemplateDAO.java | 77 +
.../apache/nifi/web/dao/impl/ComponentDAO.java | 72 +
.../web/dao/impl/StandardConnectionDAO.java | 519 +++
.../nifi/web/dao/impl/StandardFunnelDAO.java | 180 +
.../nifi/web/dao/impl/StandardInputPortDAO.java | 309 ++
.../nifi/web/dao/impl/StandardLabelDAO.java | 184 +
.../web/dao/impl/StandardOutputPortDAO.java | 309 ++
.../web/dao/impl/StandardProcessGroupDAO.java | 171 +
.../nifi/web/dao/impl/StandardProcessorDAO.java | 503 +++
.../dao/impl/StandardRemoteProcessGroupDAO.java | 402 ++
.../nifi/web/dao/impl/StandardSnippetDAO.java | 344 ++
.../nifi/web/dao/impl/StandardTemplateDAO.java | 141 +
.../nifi/web/filter/NodeRequestFilter.java | 130 +
.../apache/nifi/web/filter/RequestLogger.java | 78 +
.../nifi/web/filter/ThreadLocalFilter.java | 54 +
.../org/apache/nifi/web/filter/TimerFilter.java | 70 +
.../nifi/web/util/ClientResponseUtils.java | 48 +
.../nifi/web/util/DownloadableContent.java | 47 +
.../org/apache/nifi/web/util/SnippetUtils.java | 426 +++
.../src/main/resources/nifi-web-api-context.xml | 303 ++
.../src/main/webapp/WEB-INF/web.xml | 105 +
.../apache/nifi/integration/NiFiWebApiTest.java | 274 ++
.../accesscontrol/AdminAccessControlTest.java | 1090 ++++++
.../accesscontrol/DfmAccessControlTest.java | 1410 +++++++
.../ReadOnlyAccessControlTest.java | 1064 +++++
.../util/NiFiTestAuthorizationProvider.java | 179 +
.../nifi/integration/util/NiFiTestServer.java | 235 ++
.../nifi/integration/util/NiFiTestUser.java | 257 ++
.../integration/util/SourceTestProcessor.java | 49 +
.../util/TerminationTestProcessor.java | 44 +
.../nifi/integration/util/TestPrioritizer.java | 31 +
....apache.nifi.authorization.AuthorityProvider | 15 +
...org.apache.nifi.flowfile.FlowFilePrioritizer | 15 +
.../org.apache.nifi.processor.Processor | 16 +
.../access-control/authority-providers.xml | 24 +
.../access-control/controller-services.xml | 18 +
.../resources/access-control/localhost-ks.jks | Bin 0 -> 2246 bytes
.../resources/access-control/localhost-ts.jks | Bin 0 -> 957 bytes
.../resources/access-control/nifi.properties | 136 +
.../access-control/reporting-tasks.xml | 17 +
.../src/test/resources/logback-test.xml | 36 +
.../framework/web/nifi-web-docs/.gitignore | 1 +
.../framework/web/nifi-web-docs/pom.xml | 57 +
.../nifi/web/docs/DocumentationController.java | 91 +
.../main/webapp/WEB-INF/jsp/documentation.jsp | 130 +
.../WEB-INF/jsp/no-documentation-found.jsp | 31 +
.../src/main/webapp/WEB-INF/web.xml | 33 +
.../src/main/webapp/css/component-usage.css | 127 +
.../nifi-web-docs/src/main/webapp/css/main.css | 214 ++
.../src/main/webapp/images/bgBannerFoot.png | Bin 0 -> 189 bytes
.../src/main/webapp/images/bgHeader.png | Bin 0 -> 1697 bytes
.../src/main/webapp/images/bgTableHeader.png | Bin 0 -> 232 bytes
.../src/main/webapp/js/application.js | 282 ++
.../framework/web/nifi-web-error/.gitignore | 1 +
.../framework/web/nifi-web-error/pom.xml | 48 +
.../apache/nifi/web/filter/CatchAllFilter.java | 45 +
.../src/main/webapp/WEB-INF/web.xml | 26 +
.../nifi-web-error/src/main/webapp/index.jsp | 37 +
.../framework/web/nifi-web-ui/.gitignore | 1 +
.../framework/web/nifi-web-ui/pom.xml | 648 ++++
.../apache/nifi/web/filter/IeEdgeHeader.java | 57 +
.../org/apache/nifi/web/servlet/ConvertSvg.java | 117 +
.../filters/bulletin-board-min.properties | 18 +
.../resources/filters/bulletin-board.properties | 25 +
.../resources/filters/canvas-min.properties | 18 +
.../main/resources/filters/canvas.properties | 69 +
.../resources/filters/cluster-min.properties | 18 +
.../main/resources/filters/cluster.properties | 26 +
.../resources/filters/counters-min.properties | 18 +
.../main/resources/filters/counters.properties | 26 +
.../resources/filters/history-min.properties | 18 +
.../main/resources/filters/history.properties | 27 +
.../resources/filters/provenance-min.properties | 18 +
.../resources/filters/provenance.properties | 26 +
.../resources/filters/summary-min.properties | 18 +
.../main/resources/filters/summary.properties | 32 +
.../resources/filters/templates-min.properties | 18 +
.../main/resources/filters/templates.properties | 26 +
.../main/resources/filters/users-min.properties | 18 +
.../src/main/resources/filters/users.properties | 26 +
.../webapp/WEB-INF/pages/bulletin-board.jsp | 50 +
.../src/main/webapp/WEB-INF/pages/canvas.jsp | 118 +
.../src/main/webapp/WEB-INF/pages/cluster.jsp | 62 +
.../src/main/webapp/WEB-INF/pages/counters.jsp | 61 +
.../src/main/webapp/WEB-INF/pages/history.jsp | 65 +
.../main/webapp/WEB-INF/pages/message-page.jsp | 34 +
.../main/webapp/WEB-INF/pages/provenance.jsp | 68 +
.../src/main/webapp/WEB-INF/pages/summary.jsp | 80 +
.../src/main/webapp/WEB-INF/pages/templates.jsp | 63 +
.../src/main/webapp/WEB-INF/pages/users.jsp | 71 +
.../webapp/WEB-INF/partials/banners-main.jsp | 19 +
.../webapp/WEB-INF/partials/banners-utility.jsp | 19 +
.../bulletin-board/bulletin-board-content.jsp | 40 +
.../WEB-INF/partials/canvas/about-dialog.jsp | 29 +
.../WEB-INF/partials/canvas/canvas-header.jsp | 56 +
.../canvas/connection-configuration.jsp | 180 +
.../partials/canvas/connections-dialog.jsp | 35 +
.../partials/canvas/fill-color-dialog.jsp | 42 +
.../WEB-INF/partials/canvas/flow-status.jsp | 55 +
.../canvas/instantiate-template-dialog.jsp | 22 +
.../partials/canvas/label-configuration.jsp | 38 +
.../WEB-INF/partials/canvas/navigation.jsp | 57 +
.../WEB-INF/partials/canvas/new-port-dialog.jsp | 27 +
.../canvas/new-process-group-dialog.jsp | 27 +
.../partials/canvas/new-processor-dialog.jsp | 50 +
.../canvas/new-processor-property-dialog.jsp | 34 +
.../canvas/new-remote-process-group-dialog.jsp | 27 +
.../partials/canvas/new-template-dialog.jsp | 33 +
.../partials/canvas/port-configuration.jsp | 54 +
.../WEB-INF/partials/canvas/port-details.jsp | 39 +
.../canvas/process-group-configuration.jsp | 34 +
.../partials/canvas/process-group-details.jsp | 33 +
.../partials/canvas/processor-configuration.jsp | 213 ++
.../WEB-INF/partials/canvas/registration.jsp | 44 +
.../canvas/remote-port-configuration.jsp | 42 +
.../remote-process-group-configuration.jsp | 50 +
.../canvas/remote-process-group-details.jsp | 48 +
.../canvas/remote-process-group-ports.jsp | 47 +
.../canvas/secure-port-configuration.jsp | 82 +
.../partials/canvas/secure-port-details.jsp | 67 +
.../partials/canvas/settings-content.jsp | 60 +
.../webapp/WEB-INF/partials/canvas/shell.jsp | 27 +
.../partials/cluster/cluster-content.jsp | 39 +
.../partials/cluster/node-details-dialog.jsp | 42 +
.../WEB-INF/partials/connection-details.jsp | 123 +
.../partials/counters/counters-content.jsp | 39 +
.../partials/history/action-details-dialog.jsp | 27 +
.../partials/history/history-content.jsp | 36 +
.../partials/history/history-filter-dialog.jsp | 73 +
.../partials/history/history-purge-dialog.jsp | 42 +
.../webapp/WEB-INF/partials/message-pane.jsp | 23 +
.../main/webapp/WEB-INF/partials/ok-dialog.jsp | 22 +
.../WEB-INF/partials/processor-details.jsp | 158 +
.../provenance/lineage-query-dialog.jsp | 26 +
.../partials/provenance/provenance-content.jsp | 73 +
.../provenance-event-details-dialog.jsp | 196 +
.../provenance/provenance-query-dialog.jsp | 26 +
.../provenance/provenance-search-dialog.jsp | 101 +
.../WEB-INF/partials/status-history-dialog.jsp | 37 +
.../cluster-connection-summary-dialog.jsp | 36 +
.../cluster-input-port-summary-dialog.jsp | 36 +
.../cluster-output-port-summary-dialog.jsp | 36 +
.../cluster-processor-summary-dialog.jsp | 36 +
...ster-remote-process-group-summary-dialog.jsp | 36 +
.../partials/summary/summary-content.jsp | 64 +
.../summary/system-diagnostics-dialog.jsp | 131 +
.../summary/view-single-node-dialog.jsp | 24 +
.../partials/templates/templates-content.jsp | 59 +
.../partials/users/group-revoke-dialog.jsp | 22 +
.../partials/users/group-roles-dialog.jsp | 52 +
.../partials/users/user-delete-dialog.jsp | 23 +
.../partials/users/user-details-dialog.jsp | 56 +
.../partials/users/user-group-dialog.jsp | 27 +
.../partials/users/user-revoke-dialog.jsp | 23 +
.../partials/users/user-roles-dialog.jsp | 60 +
.../WEB-INF/partials/users/users-content.jsp | 46 +
.../webapp/WEB-INF/partials/yes-no-dialog.jsp | 22 +
.../nifi-web-ui/src/main/webapp/WEB-INF/web.xml | 120 +
.../nifi-web-ui/src/main/webapp/css/about.css | 54 +
.../nifi-web-ui/src/main/webapp/css/banner.css | 80 +
.../src/main/webapp/css/bulletin-board.css | 238 ++
.../nifi-web-ui/src/main/webapp/css/canvas.css | 41 +
.../nifi-web-ui/src/main/webapp/css/cluster.css | 152 +
.../webapp/css/connection-configuration.css | 168 +
.../src/main/webapp/css/connection-details.css | 77 +
.../src/main/webapp/css/counters.css | 134 +
.../nifi-web-ui/src/main/webapp/css/dialog.css | 351 ++
.../src/main/webapp/css/flow-status.css | 110 +
.../nifi-web-ui/src/main/webapp/css/graph.css | 262 ++
.../nifi-web-ui/src/main/webapp/css/header.css | 683 ++++
.../nifi-web-ui/src/main/webapp/css/history.css | 216 ++
.../src/main/webapp/css/label-configuration.css | 55 +
.../nifi-web-ui/src/main/webapp/css/main.css | 298 ++
.../src/main/webapp/css/message-page.css | 30 +
.../src/main/webapp/css/message-pane.css | 44 +
.../src/main/webapp/css/navigation.css | 292 ++
.../main/webapp/css/new-processor-dialog.css | 168 +
.../src/main/webapp/css/port-configuration.css | 192 +
.../src/main/webapp/css/port-details.css | 55 +
.../webapp/css/process-group-configuration.css | 40 +
.../main/webapp/css/process-group-details.css | 27 +
.../main/webapp/css/processor-configuration.css | 388 ++
.../src/main/webapp/css/processor-details.css | 119 +
.../src/main/webapp/css/provenance.css | 654 ++++
.../src/main/webapp/css/registration.css | 53 +
.../css/remote-process-group-configuration.css | 233 ++
.../nifi-web-ui/src/main/webapp/css/reset.css | 43 +
.../src/main/webapp/css/settings.css | 66 +
.../nifi-web-ui/src/main/webapp/css/shell.css | 98 +
.../src/main/webapp/css/status-history.css | 199 +
.../nifi-web-ui/src/main/webapp/css/summary.css | 768 ++++
.../src/main/webapp/css/templates.css | 229 ++
.../nifi-web-ui/src/main/webapp/css/users.css | 254 ++
.../src/main/webapp/images/addConnect.png | Bin 0 -> 1996 bytes
.../src/main/webapp/images/bg-error.png | Bin 0 -> 63766 bytes
.../src/main/webapp/images/bgBannerFoot.png | Bin 0 -> 189 bytes
.../src/main/webapp/images/bgBannerHead.png | Bin 0 -> 422 bytes
.../src/main/webapp/images/bgBreadcrumbs.png | Bin 0 -> 2130 bytes
.../src/main/webapp/images/bgButton.png | Bin 0 -> 234 bytes
.../src/main/webapp/images/bgButtonOver.png | Bin 0 -> 234 bytes
.../src/main/webapp/images/bgButtonSelected.png | Bin 0 -> 166 bytes
.../main/webapp/images/bgButtonSelectedOver.png | Bin 0 -> 168 bytes
.../src/main/webapp/images/bgContextMenu.png | Bin 0 -> 746 bytes
.../src/main/webapp/images/bgControlsInset.png | Bin 0 -> 618 bytes
.../src/main/webapp/images/bgHeader.png | Bin 0 -> 1697 bytes
.../src/main/webapp/images/bgInputText.png | Bin 0 -> 139 bytes
.../src/main/webapp/images/bgNifiLogo.png | Bin 0 -> 4232 bytes
.../src/main/webapp/images/bgNifiToolbar.png | Bin 0 -> 2123 bytes
.../main/webapp/images/bgNifiToolbarEdge.png | Bin 0 -> 541 bytes
.../src/main/webapp/images/bgOutline.png | Bin 0 -> 585 bytes
.../src/main/webapp/images/bgPanCenter.png | Bin 0 -> 362 bytes
.../src/main/webapp/images/bgPanZoom.png | Bin 0 -> 832 bytes
.../webapp/images/bgProcessGroupDetailsArea.png | Bin 0 -> 6596 bytes
.../main/webapp/images/bgProcessorStatArea.png | Bin 0 -> 4062 bytes
.../images/bgRemoteProcessGroupDetailsArea.png | Bin 0 -> 4433 bytes
.../src/main/webapp/images/bgSearchInput.png | Bin 0 -> 221 bytes
.../src/main/webapp/images/bgShellClose.png | Bin 0 -> 169 bytes
.../src/main/webapp/images/bgStatusPanel.png | Bin 0 -> 229 bytes
.../src/main/webapp/images/bgTabContainer.png | Bin 0 -> 234 bytes
.../src/main/webapp/images/bgTableHeader.png | Bin 0 -> 232 bytes
.../main/webapp/images/bgToolbarBtnBorder.png | Bin 0 -> 227 bytes
.../src/main/webapp/images/blueBtnBg-over.jpg | Bin 0 -> 356 bytes
.../src/main/webapp/images/blueBtnBg.jpg | Bin 0 -> 356 bytes
.../src/main/webapp/images/buttonColor.png | Bin 0 -> 1679 bytes
.../src/main/webapp/images/buttonCopy.png | Bin 0 -> 1166 bytes
.../src/main/webapp/images/buttonDelete.png | Bin 0 -> 1837 bytes
.../src/main/webapp/images/buttonDisable.png | Bin 0 -> 1737 bytes
.../src/main/webapp/images/buttonEnable.png | Bin 0 -> 1375 bytes
.../src/main/webapp/images/buttonGroup.png | Bin 0 -> 880 bytes
.../main/webapp/images/buttonNewProperty.png | Bin 0 -> 590 bytes
.../src/main/webapp/images/buttonOutline.png | Bin 0 -> 495 bytes
.../src/main/webapp/images/buttonPanDown.png | Bin 0 -> 554 bytes
.../src/main/webapp/images/buttonPanLeft.png | Bin 0 -> 588 bytes
.../src/main/webapp/images/buttonPanRight.png | Bin 0 -> 530 bytes
.../src/main/webapp/images/buttonPanUp.png | Bin 0 -> 551 bytes
.../src/main/webapp/images/buttonPaste.png | Bin 0 -> 1372 bytes
.../src/main/webapp/images/buttonRefresh.png | Bin 0 -> 915 bytes
.../src/main/webapp/images/buttonRun.png | Bin 0 -> 1163 bytes
.../src/main/webapp/images/buttonStop.png | Bin 0 -> 869 bytes
.../src/main/webapp/images/buttonTemplate.png | Bin 0 -> 1290 bytes
.../src/main/webapp/images/buttonZoom100.png | Bin 0 -> 449 bytes
.../src/main/webapp/images/buttonZoomFit.png | Bin 0 -> 499 bytes
.../src/main/webapp/images/buttonZoomIn.png | Bin 0 -> 435 bytes
.../src/main/webapp/images/buttonZoomOut.png | Bin 0 -> 339 bytes
.../src/main/webapp/images/cluster.png | Bin 0 -> 326 bytes
.../src/main/webapp/images/grayBtnBg-over.jpg | Bin 0 -> 319 bytes
.../src/main/webapp/images/grayBtnBg.jpg | Bin 0 -> 317 bytes
.../nifi-web-ui/src/main/webapp/images/grid.gif | Bin 0 -> 135 bytes
.../src/main/webapp/images/headerTabBg.gif | Bin 0 -> 131 bytes
.../src/main/webapp/images/iconAdminUser.png | Bin 0 -> 1960 bytes
.../src/main/webapp/images/iconAlert.png | Bin 0 -> 1396 bytes
.../src/main/webapp/images/iconAlertDialog.png | Bin 0 -> 1241 bytes
.../src/main/webapp/images/iconAutoRefresh.png | Bin 0 -> 3102 bytes
.../src/main/webapp/images/iconBulletin.png | Bin 0 -> 1066 bytes
.../main/webapp/images/iconBulletinBoard.png | Bin 0 -> 2523 bytes
.../src/main/webapp/images/iconCenterView.png | Bin 0 -> 338 bytes
.../src/main/webapp/images/iconChart.png | Bin 0 -> 510 bytes
.../src/main/webapp/images/iconClose.png | Bin 0 -> 231 bytes
.../src/main/webapp/images/iconCluster.png | Bin 0 -> 2112 bytes
.../src/main/webapp/images/iconClusterSmall.png | Bin 0 -> 757 bytes
.../main/webapp/images/iconCollapseChildren.png | Bin 0 -> 571 bytes
.../main/webapp/images/iconCollapseParents.png | Bin 0 -> 645 bytes
.../src/main/webapp/images/iconColor.png | Bin 0 -> 738 bytes
.../src/main/webapp/images/iconCommit.png | Bin 0 -> 445 bytes
.../src/main/webapp/images/iconConfigure.png | Bin 0 -> 696 bytes
.../src/main/webapp/images/iconConnect.png | Bin 0 -> 589 bytes
.../src/main/webapp/images/iconConnection.png | Bin 0 -> 1517 bytes
.../src/main/webapp/images/iconCopy.png | Bin 0 -> 514 bytes
.../src/main/webapp/images/iconCounter.png | Bin 0 -> 1369 bytes
.../src/main/webapp/images/iconDelete.png | Bin 0 -> 670 bytes
.../src/main/webapp/images/iconDetails.png | Bin 0 -> 549 bytes
.../src/main/webapp/images/iconDisable.png | Bin 0 -> 764 bytes
.../src/main/webapp/images/iconDisconnect.png | Bin 0 -> 569 bytes
.../src/main/webapp/images/iconEdit.png | Bin 0 -> 493 bytes
.../src/main/webapp/images/iconEditButton.png | Bin 0 -> 915 bytes
.../main/webapp/images/iconExpandChildren.png | Bin 0 -> 521 bytes
.../main/webapp/images/iconExpandParents.png | Bin 0 -> 585 bytes
.../src/main/webapp/images/iconExport.png | Bin 0 -> 453 bytes
.../main/webapp/images/iconExportLineage.png | Bin 0 -> 405 bytes
.../src/main/webapp/images/iconFlowSettings.png | Bin 0 -> 2624 bytes
.../src/main/webapp/images/iconFunnel.png | Bin 0 -> 1223 bytes
.../src/main/webapp/images/iconGoTo.png | Bin 0 -> 448 bytes
.../src/main/webapp/images/iconHistory.png | Bin 0 -> 2347 bytes
.../src/main/webapp/images/iconInfo.png | Bin 0 -> 550 bytes
.../src/main/webapp/images/iconInputPort.png | Bin 0 -> 1842 bytes
.../main/webapp/images/iconInputPortSmall.png | Bin 0 -> 532 bytes
.../webapp/images/iconIsolatedProcessor.png | Bin 0 -> 1781 bytes
.../src/main/webapp/images/iconLineage.png | Bin 0 -> 432 bytes
.../src/main/webapp/images/iconLoading.gif | Bin 0 -> 2438 bytes
.../src/main/webapp/images/iconNotSecure.png | Bin 0 -> 137 bytes
.../src/main/webapp/images/iconOutputPort.png | Bin 0 -> 1658 bytes
.../main/webapp/images/iconOutputPortSmall.png | Bin 0 -> 459 bytes
.../src/main/webapp/images/iconPaste.png | Bin 0 -> 601 bytes
.../src/main/webapp/images/iconPopOut.png | Bin 0 -> 217 bytes
.../webapp/images/iconPortNotTransmitting.png | Bin 0 -> 231 bytes
.../src/main/webapp/images/iconPortRemoved.png | Bin 0 -> 456 bytes
.../main/webapp/images/iconPortTransmitting.png | Bin 0 -> 235 bytes
.../src/main/webapp/images/iconPrimary.png | Bin 0 -> 647 bytes
.../src/main/webapp/images/iconProcessor.png | Bin 0 -> 1446 bytes
.../src/main/webapp/images/iconProvenance.png | Bin 0 -> 1104 bytes
.../src/main/webapp/images/iconRefresh.png | Bin 0 -> 492 bytes
.../src/main/webapp/images/iconRemotePorts.png | Bin 0 -> 456 bytes
.../webapp/images/iconRemoteProcessGroup.png | Bin 0 -> 674 bytes
.../src/main/webapp/images/iconReporting.png | Bin 0 -> 803 bytes
.../src/main/webapp/images/iconResetCounter.png | Bin 0 -> 304 bytes
.../src/main/webapp/images/iconResize.png | Bin 0 -> 142 bytes
.../src/main/webapp/images/iconRevoke.png | Bin 0 -> 676 bytes
.../src/main/webapp/images/iconRun.png | Bin 0 -> 538 bytes
.../src/main/webapp/images/iconSecure.png | Bin 0 -> 133 bytes
.../src/main/webapp/images/iconSmallFunnel.png | Bin 0 -> 455 bytes
.../main/webapp/images/iconSmallInputPort.png | Bin 0 -> 216 bytes
.../main/webapp/images/iconSmallOutputPort.png | Bin 0 -> 264 bytes
.../webapp/images/iconSmallProcessGroup.png | Bin 0 -> 479 bytes
.../main/webapp/images/iconSmallProcessor.png | Bin 0 -> 647 bytes
.../webapp/images/iconSmallRelationship.png | Bin 0 -> 770 bytes
.../images/iconSmallRemoteProcessGroup.png | Bin 0 -> 402 bytes
.../src/main/webapp/images/iconStop.png | Bin 0 -> 402 bytes
.../src/main/webapp/images/iconTemplates.png | Bin 0 -> 2659 bytes
.../src/main/webapp/images/iconToFront.png | Bin 0 -> 475 bytes
.../webapp/images/iconTransmissionActive.png | Bin 0 -> 1330 bytes
.../webapp/images/iconTransmissionInactive.png | Bin 0 -> 1248 bytes
.../src/main/webapp/images/iconTwistArrow.png | Bin 0 -> 256 bytes
.../src/main/webapp/images/iconUndo.png | Bin 0 -> 642 bytes
.../src/main/webapp/images/iconUsage.png | Bin 0 -> 470 bytes
.../src/main/webapp/images/inputCheckbox.png | Bin 0 -> 389 bytes
.../src/main/webapp/images/loadAnimation.gif | Bin 0 -> 10789 bytes
.../src/main/webapp/images/nifi16.ico | Bin 0 -> 1150 bytes
.../src/main/webapp/images/nifi_about.png | Bin 0 -> 16323 bytes
.../src/main/webapp/images/panelBg.jpg | Bin 0 -> 312 bytes
.../src/main/webapp/images/portRemoved.png | Bin 0 -> 1090 bytes
.../src/main/webapp/images/portRunning.png | Bin 0 -> 337 bytes
.../src/main/webapp/images/portStopped.png | Bin 0 -> 192 bytes
.../src/main/webapp/images/spacer.png | Bin 0 -> 110 bytes
.../src/main/webapp/images/starburst.png | Bin 0 -> 190 bytes
.../src/main/webapp/images/tabBg.jpg | Bin 0 -> 320 bytes
.../src/main/webapp/images/toolbarBg.gif | Bin 0 -> 45 bytes
.../main/webapp/images/toolboxIconFunnel.png | Bin 0 -> 2409 bytes
.../main/webapp/images/toolboxIconInputPort.png | Bin 0 -> 2680 bytes
.../src/main/webapp/images/toolboxIconLabel.png | Bin 0 -> 2871 bytes
.../webapp/images/toolboxIconOutputPort.png | Bin 0 -> 2209 bytes
.../webapp/images/toolboxIconProcessGroup.png | Bin 0 -> 4176 bytes
.../main/webapp/images/toolboxIconProcessor.png | Bin 0 -> 4426 bytes
.../images/toolboxIconRemoteProcessGroup.png | Bin 0 -> 5653 bytes
.../main/webapp/images/toolboxIconTemplate.png | Bin 0 -> 2913 bytes
.../images/transmissionSwitchDisabled.png | Bin 0 -> 1240 bytes
.../webapp/images/transmissionSwitchEnabled.png | Bin 0 -> 1600 bytes
.../src/main/webapp/images/ungroup.png | Bin 0 -> 3409 bytes
.../src/main/webapp/js/codemirror/LICENSE | 19 +
.../js/codemirror/addon/hint/show-hint.css | 38 +
.../js/codemirror/lib/codemirror-compressed.js | 16 +
.../webapp/js/codemirror/lib/codemirror.css | 318 ++
.../nifi-web-ui/src/main/webapp/js/d3/LICENSE | 26 +
.../nifi-web-ui/src/main/webapp/js/d3/d3.min.js | 5 +
.../webapp/js/jquery/combo/jquery.combo.css | 88 +
.../main/webapp/js/jquery/combo/jquery.combo.js | 311 ++
.../main/webapp/js/jquery/combo/menuArrow.png | Bin 0 -> 251 bytes
.../webapp/js/jquery/combo/menuArrowSmall.png | Bin 0 -> 233 bytes
.../main/webapp/js/jquery/jquery-2.1.1.min.js | 4 +
.../src/main/webapp/js/jquery/jquery.center.js | 35 +
.../src/main/webapp/js/jquery/jquery.count.js | 99 +
.../src/main/webapp/js/jquery/jquery.each.js | 31 +
.../main/webapp/js/jquery/jquery.ellipsis.js | 164 +
.../js/jquery/jquery.event.drag-2.2.min.js | 6 +
.../main/webapp/js/jquery/jquery.form.min.js | 11 +
.../src/main/webapp/js/jquery/jquery.tab.js | 69 +
.../js/jquery/minicolors/jquery.minicolors.css | 259 ++
.../jquery/minicolors/jquery.minicolors.min.js | 9 +
.../js/jquery/minicolors/jquery.minicolors.png | Bin 0 -> 77459 bytes
.../webapp/js/jquery/modal/jquery.modal.css | 110 +
.../main/webapp/js/jquery/modal/jquery.modal.js | 266 ++
.../js/jquery/nfeditor/jquery.nfeditor.css | 73 +
.../js/jquery/nfeditor/jquery.nfeditor.js | 315 ++
.../js/jquery/nfeditor/languages/nfel.css | 51 +
.../webapp/js/jquery/nfeditor/languages/nfel.js | 831 ++++
.../webapp/js/jquery/qtip2/jquery.qtip.min.css | 3 +
.../webapp/js/jquery/qtip2/jquery.qtip.min.js | 5 +
.../webapp/js/jquery/slickgrid/MIT-LICENSE.txt | 20 +
.../js/jquery/slickgrid/css/images/collapse.gif | Bin 0 -> 846 bytes
.../js/jquery/slickgrid/css/images/expand.gif | Bin 0 -> 851 bytes
.../js/jquery/slickgrid/css/images/sort-asc.gif | Bin 0 -> 59 bytes
.../jquery/slickgrid/css/images/sort-desc.gif | Bin 0 -> 59 bytes
.../slickgrid/css/slick-default-theme.css | 135 +
.../js/jquery/slickgrid/css/slick.grid.css | 158 +
.../slickgrid/plugins/slick.autotooltips.js | 83 +
.../plugins/slick.cellrangedecorator.js | 66 +
.../plugins/slick.cellrangeselector.js | 113 +
.../plugins/slick.cellselectionmodel.js | 154 +
.../plugins/slick.rowselectionmodel.js | 187 +
.../webapp/js/jquery/slickgrid/slick.core.js | 467 +++
.../js/jquery/slickgrid/slick.dataview.js | 1126 ++++++
.../webapp/js/jquery/slickgrid/slick.editors.js | 512 +++
.../js/jquery/slickgrid/slick.formatters.js | 59 +
.../webapp/js/jquery/slickgrid/slick.grid.js | 3422 +++++++++++++++++
.../webapp/js/jquery/tabbs/jquery.tabbs.css | 48 +
.../main/webapp/js/jquery/tabbs/jquery.tabbs.js | 96 +
.../ui-smoothness/images/animated-overlay.gif | Bin 0 -> 1738 bytes
.../images/ui-bg_flat_0_aaaaaa_40x100.png | Bin 0 -> 212 bytes
.../images/ui-bg_flat_75_ffffff_40x100.png | Bin 0 -> 208 bytes
.../images/ui-bg_glass_55_fbf9ee_1x400.png | Bin 0 -> 335 bytes
.../images/ui-bg_glass_65_ffffff_1x400.png | Bin 0 -> 207 bytes
.../images/ui-bg_glass_75_dadada_1x400.png | Bin 0 -> 262 bytes
.../images/ui-bg_glass_75_e6e6e6_1x400.png | Bin 0 -> 262 bytes
.../images/ui-bg_glass_95_fef1ec_1x400.png | Bin 0 -> 332 bytes
.../ui-bg_highlight-soft_75_cccccc_1x100.png | Bin 0 -> 280 bytes
.../images/ui-icons_222222_256x240.png | Bin 0 -> 6922 bytes
.../images/ui-icons_2e83ff_256x240.png | Bin 0 -> 4549 bytes
.../images/ui-icons_454545_256x240.png | Bin 0 -> 6992 bytes
.../images/ui-icons_888888_256x240.png | Bin 0 -> 6999 bytes
.../images/ui-icons_cd0a0a_256x240.png | Bin 0 -> 4549 bytes
.../ui-smoothness/jquery-ui-1.10.4.min.css | 7 +
.../ui-smoothness/jquery-ui-1.10.4.min.js | 7 +
.../web/nifi-web-ui/src/main/webapp/js/json2.js | 475 +++
.../js/nf/bulletin-board/nf-bulletin-board.js | 422 ++
.../src/main/webapp/js/nf/canvas/nf-actions.js | 1150 ++++++
.../src/main/webapp/js/nf/canvas/nf-birdseye.js | 340 ++
.../webapp/js/nf/canvas/nf-canvas-header.js | 293 ++
.../webapp/js/nf/canvas/nf-canvas-toolbar.js | 174 +
.../webapp/js/nf/canvas/nf-canvas-toolbox.js | 1211 ++++++
.../main/webapp/js/nf/canvas/nf-canvas-utils.js | 1307 +++++++
.../src/main/webapp/js/nf/canvas/nf-canvas.js | 1530 ++++++++
.../main/webapp/js/nf/canvas/nf-clipboard.js | 96 +
.../main/webapp/js/nf/canvas/nf-connectable.js | 210 +
.../js/nf/canvas/nf-connection-configuration.js | 1381 +++++++
.../main/webapp/js/nf/canvas/nf-connection.js | 1533 ++++++++
.../main/webapp/js/nf/canvas/nf-context-menu.js | 452 +++
.../js/nf/canvas/nf-custom-processor-ui.js | 43 +
.../main/webapp/js/nf/canvas/nf-draggable.js | 319 ++
.../src/main/webapp/js/nf/canvas/nf-funnel.js | 279 ++
.../src/main/webapp/js/nf/canvas/nf-go-to.js | 730 ++++
.../webapp/js/nf/canvas/nf-graph-control.js | 137 +
.../src/main/webapp/js/nf/canvas/nf-graph.js | 194 +
.../js/nf/canvas/nf-label-configuration.js | 132 +
.../src/main/webapp/js/nf/canvas/nf-label.js | 559 +++
.../js/nf/canvas/nf-port-configuration.js | 170 +
.../main/webapp/js/nf/canvas/nf-port-details.js | 63 +
.../src/main/webapp/js/nf/canvas/nf-port.js | 632 +++
.../nf/canvas/nf-process-group-configuration.js | 107 +
.../js/nf/canvas/nf-process-group-details.js | 61 +
.../webapp/js/nf/canvas/nf-process-group.js | 1064 +++++
.../js/nf/canvas/nf-processor-configuration.js | 734 ++++
.../nf-processor-property-combo-editor.js | 175 +
.../canvas/nf-processor-property-nfel-editor.js | 207 +
.../js/nf/canvas/nf-processor-property-table.js | 567 +++
.../canvas/nf-processor-property-text-editor.js | 212 +
.../main/webapp/js/nf/canvas/nf-processor.js | 831 ++++
.../main/webapp/js/nf/canvas/nf-registration.js | 68 +
.../nf-remote-process-group-configuration.js | 124 +
.../canvas/nf-remote-process-group-details.js | 67 +
.../nf/canvas/nf-remote-process-group-ports.js | 525 +++
.../js/nf/canvas/nf-remote-process-group.js | 1060 +++++
.../src/main/webapp/js/nf/canvas/nf-search.js | 179 +
.../nf/canvas/nf-secure-port-configuration.js | 384 ++
.../js/nf/canvas/nf-secure-port-details.js | 118 +
.../main/webapp/js/nf/canvas/nf-selectable.js | 58 +
.../src/main/webapp/js/nf/canvas/nf-settings.js | 146 +
.../src/main/webapp/js/nf/canvas/nf-snippet.js | 212 +
.../src/main/webapp/js/nf/canvas/nf-storage.js | 136 +
.../webapp/js/nf/canvas/nf-toolbar-action.js | 86 +
.../webapp/js/nf/cluster/nf-cluster-table.js | 620 +++
.../src/main/webapp/js/nf/cluster/nf-cluster.js | 154 +
.../webapp/js/nf/counters/nf-counters-table.js | 294 ++
.../main/webapp/js/nf/counters/nf-counters.js | 153 +
.../webapp/js/nf/history/nf-history-model.js | 218 ++
.../webapp/js/nf/history/nf-history-table.js | 429 +++
.../src/main/webapp/js/nf/history/nf-history.js | 154 +
.../src/main/webapp/js/nf/nf-client.js | 56 +
.../src/main/webapp/js/nf/nf-common.js | 899 +++++
.../main/webapp/js/nf/nf-connection-details.js | 467 +++
.../src/main/webapp/js/nf/nf-dialog.js | 128 +
.../src/main/webapp/js/nf/nf-namespace.js | 20 +
.../main/webapp/js/nf/nf-processor-details.js | 586 +++
.../src/main/webapp/js/nf/nf-shell.js | 167 +
.../src/main/webapp/js/nf/nf-status-history.js | 1320 +++++++
.../js/nf/provenance/nf-provenance-lineage.js | 1413 +++++++
.../js/nf/provenance/nf-provenance-table.js | 1371 +++++++
.../webapp/js/nf/provenance/nf-provenance.js | 204 +
.../webapp/js/nf/summary/nf-cluster-search.js | 183 +
.../webapp/js/nf/summary/nf-summary-table.js | 2436 ++++++++++++
.../src/main/webapp/js/nf/summary/nf-summary.js | 174 +
.../js/nf/templates/nf-templates-table.js | 332 ++
.../main/webapp/js/nf/templates/nf-templates.js | 235 ++
.../main/webapp/js/nf/users/nf-users-table.js | 1090 ++++++
.../src/main/webapp/js/nf/users/nf-users.js | 149 +
.../framework-bundle/framework/web/pom.xml | 74 +
.../web/web-optimistic-locking/.gitignore | 1 +
.../web/web-optimistic-locking/pom.xml | 32 +
.../apache/nifi/web/ConfigurationSnapshot.java | 66 +
.../nifi/web/OptimisticLockingManager.java | 95 +
.../web/StandardOptimisticLockingManager.java | 77 +
.../framework/web/web-security/.gitignore | 1 +
.../framework/web/web-security/pom.xml | 81 +
.../org/apache/nifi/web/security/DnUtils.java | 87 +
.../web/security/UntrustedProxyException.java | 34 +
.../anonymous/NiFiAnonymousUserFilter.java | 101 +
.../NiFiAuthenticationEntryPoint.java | 68 +
.../authorization/NiFiAuthorizationService.java | 173 +
.../authorization/NodeAuthorizedUserFilter.java | 128 +
.../nifi/web/security/user/NiFiUserDetails.java | 110 +
.../nifi/web/security/user/NiFiUserUtils.java | 82 +
.../x509/SubjectDnX509PrincipalExtractor.java | 38 +
.../security/x509/X509AuthenticationFilter.java | 349 ++
.../security/x509/X509CertificateExtractor.java | 52 +
.../x509/ocsp/CertificateStatusException.java | 42 +
.../x509/ocsp/OcspCertificateValidator.java | 444 +++
.../web/security/x509/ocsp/OcspRequest.java | 69 +
.../nifi/web/security/x509/ocsp/OcspStatus.java | 82 +
.../resources/nifi-web-security-context.xml | 83 +
.../NiFiAuthorizationServiceTest.java | 250 ++
.../nar-bundles/framework-bundle/nar/.gitignore | 1 +
nifi/nar-bundles/framework-bundle/nar/pom.xml | 66 +
nifi/nar-bundles/framework-bundle/pom.xml | 114 +
.../hadoop-bundle/hdfs-processors/pom.xml | 65 +
.../hadoop/AbstractHadoopProcessor.java | 221 ++
.../hadoop/CreateHadoopSequenceFile.java | 177 +
...lowFileStreamUnpackerSequenceFileWriter.java | 170 +
.../apache/nifi/processors/hadoop/GetHDFS.java | 563 +++
.../processors/hadoop/GetHDFSSequenceFile.java | 146 +
.../nifi/processors/hadoop/KeyValueReader.java | 132 +
.../apache/nifi/processors/hadoop/PutHDFS.java | 403 ++
.../hadoop/SequenceFileWriterImpl.java | 119 +
.../hadoop/TarUnpackerSequenceFileWriter.java | 54 +
.../nifi/processors/hadoop/ValueReader.java | 116 +
.../hadoop/ZipUnpackerSequenceFileWriter.java | 57 +
.../hadoop/util/ByteFilteringOutputStream.java | 163 +
.../hadoop/util/InputStreamWritable.java | 61 +
.../hadoop/util/OutputStreamWritable.java | 85 +
.../hadoop/util/SequenceFileReader.java | 28 +
.../hadoop/util/SequenceFileWriter.java | 40 +
.../org.apache.nifi.processor.Processor | 18 +
.../index.html | 88 +
.../index.html | 162 +
.../index.html | 150 +
.../index.html | 159 +
.../processors/hadoop/AbstractHadoopTest.java | 90 +
.../nifi/processors/hadoop/GetHDFSTest.java | 123 +
.../nifi/processors/hadoop/PutHDFSTest.java | 158 +
.../hadoop/SimpleHadoopProcessor.java | 30 +
.../hadoop/TestCreateHadoopSequenceFile.java | 176 +
.../src/test/resources/core-site-broken.xml | 25 +
.../src/test/resources/core-site.xml | 25 +
.../resources/testdata/13545312236534130.tar | Bin 0 -> 1505280 bytes
.../resources/testdata/13545423550275052.zip | Bin 0 -> 1500841 bytes
.../resources/testdata/13545479542069498.pkg | Bin 0 -> 1500429 bytes
.../src/test/resources/testdata/randombytes-1 | Bin 0 -> 500000 bytes
.../src/test/resources/testdata/randombytes-2 | Bin 0 -> 500000 bytes
.../src/test/resources/testdata/randombytes-3 | Bin 0 -> 500000 bytes
nifi/nar-bundles/hadoop-bundle/nar/pom.xml | 41 +
nifi/nar-bundles/hadoop-bundle/pom.xml | 46 +
.../hadoop-libraries-bundle/nar/pom.xml | 45 +
.../nar-bundles/hadoop-libraries-bundle/pom.xml | 36 +
nifi/nar-bundles/jetty-bundle/pom.xml | 82 +
nifi/nar-bundles/kafka-bundle/kafka-nar/pom.xml | 37 +
.../kafka-bundle/kafka-processors/pom.xml | 76 +
.../apache/nifi/processors/kafka/GetKafka.java | 330 ++
.../apache/nifi/processors/kafka/PutKafka.java | 419 ++
.../org.apache.nifi.processor.Processor | 16 +
.../index.html | 173 +
.../index.html | 189 +
.../nifi/processors/kafka/TestGetKafka.java | 162 +
.../nifi/processors/kafka/TestPutKafka.java | 236 ++
nifi/nar-bundles/kafka-bundle/pom.xml | 35 +
.../monitor-threshold-bundle/nar/pom.xml | 39 +
.../monitor-threshold-bundle/pom.xml | 51 +
.../monitor-threshold-bundle/processor/pom.xml | 45 +
.../processors/monitor/MonitorThreshold.java | 904 +++++
.../processors/monitor/ThresholdsParser.java | 134 +
.../org.apache.nifi.processor.Processor | 15 +
.../index.html | 485 +++
.../monitor/TestMonitorThreshold.java | 1560 ++++++++
.../processor/src/test/resources/testFile | 23 +
.../resources/threshold_settings_allow_0.xml | 21 +
.../resources/threshold_settings_allow_1.xml | 21 +
.../threshold_settings_allow_10_KB.xml | 21 +
.../threshold_settings_allow_10_KB_and_2_KB.xml | 26 +
.../threshold_settings_allow_1669_bytes.xml | 21 +
.../resources/threshold_settings_allow_2.xml | 21 +
.../threshold_settings_allow_2_default.xml | 21 +
...eshold_settings_allow_3336_bytes_default.xml | 21 +
...hold_settings_with_empty_attribute_value.xml | 21 +
...ettings_with_spaces_for_attribute_values.xml | 22 +
.../ui/nb-configuration.xml | 18 +
.../monitor-threshold-bundle/ui/pom.xml | 134 +
.../thresholds/ui/AttributeComparator.java | 36 +
.../thresholds/ui/AttributeResource.java | 242 ++
.../monitor/thresholds/ui/RuleComparator.java | 46 +
.../thresholds/ui/ThresholdResource.java | 227 ++
.../ui/ThresholdSettingsResource.java | 191 +
.../thresholds/ui/ThresholdsConfigFile.java | 296 ++
.../ui/src/main/webapp/META-INF/nifi-processor | 15 +
.../webapp/WEB-INF/jsp/attribute-add-dialog.jsp | 43 +
.../WEB-INF/jsp/attribute-confirm-dialog.jsp | 21 +
.../WEB-INF/jsp/attribute-edit-dialog.jsp | 27 +
.../WEB-INF/jsp/attribute-filter-dialog.jsp | 26 +
.../main/webapp/WEB-INF/jsp/error-dialog.jsp | 21 +
.../ui/src/main/webapp/WEB-INF/jsp/index.jsp | 101 +
.../webapp/WEB-INF/jsp/threshold-add-dialog.jsp | 45 +
.../WEB-INF/jsp/threshold-confirm-dialog.jsp | 21 +
.../webapp/WEB-INF/jsp/threshold-dialog.jsp | 48 +
.../WEB-INF/jsp/threshold-edit-dialog.jsp | 45 +
.../WEB-INF/jsp/threshold-filter-dialog.jsp | 40 +
.../ui/src/main/webapp/WEB-INF/web.xml | 48 +
.../ui/src/main/webapp/css/threshold_styles.css | 422 ++
.../src/main/webapp/images/addWorksheetRow.png | Bin 0 -> 323 bytes
.../ui/src/main/webapp/images/clear.png | Bin 0 -> 912 bytes
.../ui/src/main/webapp/images/filter.gif | Bin 0 -> 623 bytes
.../ui/src/main/webapp/images/firstPage.gif | Bin 0 -> 577 bytes
.../main/webapp/images/firstPageDisabled.gif | Bin 0 -> 365 bytes
.../ui/src/main/webapp/images/lastPage.gif | Bin 0 -> 579 bytes
.../src/main/webapp/images/lastPageDisabled.gif | Bin 0 -> 368 bytes
.../ui/src/main/webapp/images/nextPage.gif | Bin 0 -> 570 bytes
.../src/main/webapp/images/nextPageDisabled.gif | Bin 0 -> 362 bytes
.../ui/src/main/webapp/images/prevPage.gif | Bin 0 -> 567 bytes
.../src/main/webapp/images/prevPageDisabled.gif | Bin 0 -> 361 bytes
.../main/webapp/images/removeWorksheetRow.png | Bin 0 -> 655 bytes
.../ui/src/main/webapp/images/separator.gif | Bin 0 -> 63 bytes
.../webapp/js/jquery/jqgrid/css/ui.jqgrid.css | 171 +
.../js/jquery/jqgrid/js/i18n/grid.locale-en.js | 170 +
.../js/jquery/jqgrid/js/jquery.jqGrid.min.js | 545 +++
.../ui/src/main/webapp/js/nf-common.js | 888 +++++
.../ui/src/main/xsd/threshold_settings.xsd | 54 +
.../nar/pom.xml | 36 +
.../persistent-provenance-repository/pom.xml | 58 +
.../nifi/provenance/IndexConfiguration.java | 398 ++
.../PersistentProvenanceRepository.java | 1892 +++++++++
.../provenance/RepositoryConfiguration.java | 299 ++
.../nifi/provenance/StandardRecordReader.java | 305 ++
.../nifi/provenance/StandardRecordWriter.java | 237 ++
.../provenance/expiration/ExpirationAction.java | 35 +
.../expiration/FileRemovalAction.java | 49 +
.../provenance/lucene/DeleteIndexAction.java | 139 +
.../nifi/provenance/lucene/DocsReader.java | 133 +
.../nifi/provenance/lucene/FieldNames.java | 23 +
.../nifi/provenance/lucene/IndexSearch.java | 84 +
.../nifi/provenance/lucene/IndexingAction.java | 219 ++
.../nifi/provenance/lucene/LineageQuery.java | 108 +
.../nifi/provenance/lucene/LuceneUtil.java | 141 +
.../provenance/rollover/CompressionAction.java | 59 +
.../provenance/rollover/RolloverAction.java | 35 +
.../provenance/serialization/RecordReader.java | 31 +
.../provenance/serialization/RecordReaders.java | 70 +
.../provenance/serialization/RecordWriter.java | 85 +
.../provenance/serialization/RecordWriters.java | 30 +
...he.nifi.provenance.ProvenanceEventRepository | 15 +
.../TestPersistentProvenanceRepository.java | 1118 ++++++
.../pom.xml | 42 +
nifi/nar-bundles/pom.xml | 134 +
.../standard-bundle/jms-processors/pom.xml | 55 +
.../apache/nifi/processors/jms/GetJMSQueue.java | 75 +
.../apache/nifi/processors/jms/GetJMSTopic.java | 359 ++
.../apache/nifi/processors/jms/JmsConsumer.java | 208 +
.../org/apache/nifi/processors/jms/PutJMS.java | 374 ++
.../nifi/processors/jms/util/JmsFactory.java | 435 +++
.../nifi/processors/jms/util/JmsProperties.java | 176 +
.../jms/util/WrappedMessageConsumer.java | 77 +
.../jms/util/WrappedMessageProducer.java | 77 +
.../org.apache.nifi.processor.Processor | 17 +
.../index.html | 118 +
.../index.html | 122 +
.../index.html | 152 +
.../nifi/processors/jms/GetJMSQueueTest.java | 141 +
nifi/nar-bundles/standard-bundle/nar/pom.xml | 57 +
nifi/nar-bundles/standard-bundle/pom.xml | 71 +
.../standard-ganglia-reporter/pom.xml | 52 +
.../ganglia/StandardGangliaReporter.java | 262 ++
.../org.apache.nifi.reporting.ReportingTask | 15 +
.../index.html | 65 +
.../standard-prioritizers/pom.xml | 40 +
.../prioritizer/FirstInFirstOutPrioritizer.java | 37 +
.../NewestFlowFileFirstPrioritizer.java | 37 +
.../OldestFlowFileFirstPrioritizer.java | 37 +
.../PriorityAttributePrioritizer.java | 87 +
...org.apache.nifi.flowfile.FlowFilePrioritizer | 18 +
.../prioritizer/NewestFirstPrioritizerTest.java | 66 +
.../prioritizer/OldestFirstPrioritizerTest.java | 66 +
.../PriorityAttributePrioritizerTest.java | 117 +
.../standard-bundle/standard-processors/pom.xml | 154 +
.../standard/Base64EncodeContent.java | 142 +
.../processors/standard/CompressContent.java | 307 ++
.../nifi/processors/standard/ControlRate.java | 381 ++
.../standard/ConvertCharacterSet.java | 175 +
.../processors/standard/DetectDuplicate.java | 249 ++
.../processors/standard/DistributeLoad.java | 498 +++
.../processors/standard/EncryptContent.java | 263 ++
.../standard/EvaluateRegularExpression.java | 294 ++
.../nifi/processors/standard/EvaluateXPath.java | 429 +++
.../processors/standard/EvaluateXQuery.java | 463 +++
.../standard/ExecuteStreamCommand.java | 358 ++
.../processors/standard/GenerateFlowFile.java | 164 +
.../apache/nifi/processors/standard/GetFTP.java | 72 +
.../nifi/processors/standard/GetFile.java | 456 +++
.../processors/standard/GetFileTransfer.java | 301 ++
.../nifi/processors/standard/GetHTTP.java | 480 +++
.../nifi/processors/standard/GetSFTP.java | 92 +
.../nifi/processors/standard/HashAttribute.java | 255 ++
.../nifi/processors/standard/HashContent.java | 151 +
.../processors/standard/IdentifyMimeType.java | 474 +++
.../nifi/processors/standard/InvokeHTTP.java | 758 ++++
.../nifi/processors/standard/ListenHTTP.java | 323 ++
.../nifi/processors/standard/ListenUDP.java | 627 +++
.../nifi/processors/standard/LogAttribute.java | 262 ++
.../nifi/processors/standard/MergeContent.java | 1015 +++++
.../nifi/processors/standard/ModifyBytes.java | 134 +
.../processors/standard/MonitorActivity.java | 206 +
.../nifi/processors/standard/PostHTTP.java | 904 +++++
.../nifi/processors/standard/PutEmail.java | 297 ++
.../apache/nifi/processors/standard/PutFTP.java | 141 +
.../nifi/processors/standard/PutFile.java | 367 ++
.../processors/standard/PutFileTransfer.java | 293 ++
.../nifi/processors/standard/PutSFTP.java | 85 +
.../nifi/processors/standard/ReplaceText.java | 289 ++
.../standard/ReplaceTextWithMapping.java | 383 ++
.../processors/standard/RouteOnAttribute.java | 261 ++
.../processors/standard/RouteOnContent.java | 232 ++
.../nifi/processors/standard/ScanAttribute.java | 229 ++
.../nifi/processors/standard/ScanContent.java | 292 ++
.../processors/standard/SegmentContent.java | 163 +
.../nifi/processors/standard/SplitContent.java | 260 ++
.../nifi/processors/standard/SplitText.java | 370 ++
.../nifi/processors/standard/SplitXml.java | 300 ++
.../nifi/processors/standard/TransformXml.java | 194 +
.../nifi/processors/standard/UnpackContent.java | 427 +++
.../nifi/processors/standard/ValidateXml.java | 147 +
.../servlets/ContentAcknowledgmentServlet.java | 136 +
.../standard/servlets/ListenHTTPServlet.java | 325 ++
.../nifi/processors/standard/util/Bin.java | 168 +
.../processors/standard/util/BinManager.java | 241 ++
.../standard/util/DocumentReaderCallback.java | 74 +
.../processors/standard/util/FTPTransfer.java | 541 +++
.../nifi/processors/standard/util/FTPUtils.java | 322 ++
.../nifi/processors/standard/util/FileInfo.java | 167 +
.../processors/standard/util/FileTransfer.java | 244 ++
.../standard/util/FlowFileSessionWrapper.java | 44 +
.../standard/util/NLKBufferedReader.java | 187 +
.../standard/util/SFTPConnection.java | 82 +
.../processors/standard/util/SFTPTransfer.java | 637 +++
.../processors/standard/util/SFTPUtils.java | 324 ++
.../standard/util/UDPStreamConsumer.java | 214 ++
.../util/ValidatingBase64InputStream.java | 76 +
.../standard/util/XmlElementNotifier.java | 22 +
.../standard/util/XmlSplitterSaxParser.java | 112 +
.../org.apache.nifi.processor.Processor | 58 +
.../index.html | 63 +
.../index.html | 166 +
.../index.html | 116 +
.../index.html | 65 +
.../index.html | 147 +
.../index.html | 106 +
.../index.html | 97 +
.../index.html | 160 +
.../index.html | 135 +
.../index.html | 311 ++
.../index.html | 111 +
.../index.html | 64 +
.../index.html | 227 ++
.../index.html | 186 +
.../index.html | 143 +
.../index.html | 250 ++
.../index.html | 88 +
.../index.html | 89 +
.../index.html | 136 +
.../index.html | 181 +
.../index.html | 86 +
.../index.html | 144 +
.../index.html | 80 +
.../index.html | 347 ++
.../index.html | 64 +
.../index.html | 143 +
.../index.html | 187 +
.../index.html | 114 +
.../index.html | 283 ++
.../index.html | 109 +
.../index.html | 281 ++
.../index.html | 91 +
.../index.html | 114 +
.../index.html | 110 +
.../index.html | 82 +
.../index.html | 85 +
.../index.html | 100 +
.../index.html | 123 +
.../index.html | 107 +
.../index.html | 121 +
.../index.html | 64 +
.../index.html | 63 +
.../index.html | 163 +
.../index.html | 56 +
.../src/test/java/TestIngestAndUpdate.java | 34 +
.../src/test/java/TestSuccess.java | 24 +
.../processors/standard/HelloWorldServlet.java | 41 +
.../standard/RESTServiceContentModified.java | 78 +
.../standard/TestBase64EncodeContent.java | 83 +
.../standard/TestCompressContent.java | 111 +
.../processors/standard/TestControlRate.java | 68 +
.../standard/TestConvertCharacterSet.java | 47 +
.../standard/TestDetectDuplicate.java | 206 +
.../processors/standard/TestDistributeLoad.java | 139 +
.../processors/standard/TestEncryptContent.java | 65 +
.../standard/TestEvaluateRegularExpression.java | 319 ++
.../processors/standard/TestEvaluateXPath.java | 159 +
.../processors/standard/TestEvaluateXQuery.java | 651 ++++
.../standard/TestExecuteStreamCommand.java | 188 +
.../nifi/processors/standard/TestGetFile.java | 186 +
.../nifi/processors/standard/TestGetHTTP.java | 354 ++
.../processors/standard/TestHashAttribute.java | 99 +
.../processors/standard/TestHashContent.java | 67 +
.../standard/TestIdentifyMimeType.java | 131 +
.../processors/standard/TestInvokeHTTP.java | 593 +++
.../nifi/processors/standard/TestListenUDP.java | 214 ++
.../processors/standard/TestMergeContent.java | 593 +++
.../processors/standard/TestModifyBytes.java | 190 +
.../standard/TestMonitorActivity.java | 193 +
.../processors/standard/TestReplaceText.java | 371 ++
.../standard/TestReplaceTextLineByLine.java | 337 ++
.../standard/TestReplaceTextWithMapping.java | 332 ++
.../standard/TestRouteOnAttribute.java | 161 +
.../processors/standard/TestRouteOnContent.java | 73 +
.../processors/standard/TestScanAttribute.java | 149 +
.../processors/standard/TestScanContent.java | 85 +
.../processors/standard/TestSegmentContent.java | 65 +
.../nifi/processors/standard/TestServer.java | 177 +
.../processors/standard/TestSplitContent.java | 235 ++
.../nifi/processors/standard/TestSplitText.java | 173 +
.../nifi/processors/standard/TestSplitXml.java | 103 +
.../processors/standard/TestTransformXml.java | 120 +
.../processors/standard/TestUnpackContent.java | 226 ++
.../processors/standard/TestValidateXml.java | 42 +
.../standard/UserAgentTestingServlet.java | 42 +
.../CharacterSetConversionSamples/Converted.txt | 1 +
.../Converted2.txt | Bin 0 -> 134580 bytes
.../CharacterSetConversionSamples/Original.txt | 418 ++
.../resources/CompressedData/SampleFile.txt | 418 ++
.../resources/CompressedData/SampleFile.txt.bz2 | Bin 0 -> 388 bytes
.../resources/CompressedData/SampleFile.txt.gz | Bin 0 -> 321 bytes
.../CompressedData/SampleFile1.txt.bz2 | Bin 0 -> 154 bytes
.../resources/CompressedData/SampleFile1.txt.gz | Bin 0 -> 321 bytes
.../CompressedData/SampleFileConcat.txt | 835 ++++
.../CompressedData/SampleFileConcat.txt.bz2 | Bin 0 -> 542 bytes
.../test/resources/ExecuteCommand/1000bytes.txt | 1 +
.../ExecuteCommand/TestIngestAndUpdate.jar | Bin 0 -> 1170 bytes
.../resources/ExecuteCommand/TestSuccess.jar | Bin 0 -> 827 bytes
.../src/test/resources/ExecuteCommand/test.txt | 2 +
.../dictionary-with-empty-new-lines | 5 +
.../ScanAttribute/dictionary-with-extra-info | 9 +
.../test/resources/ScanAttribute/dictionary1 | 5 +
.../test/resources/TestIdentifyMimeType/1.7z | Bin 0 -> 133 bytes
.../test/resources/TestIdentifyMimeType/1.jar | Bin 0 -> 466 bytes
.../test/resources/TestIdentifyMimeType/1.mdb | Bin 0 -> 237568 bytes
.../test/resources/TestIdentifyMimeType/1.pdf | Bin 0 -> 277032 bytes
.../test/resources/TestIdentifyMimeType/1.tar | Bin 0 -> 2048 bytes
.../resources/TestIdentifyMimeType/1.txt.bz2 | Bin 0 -> 59 bytes
.../resources/TestIdentifyMimeType/1.txt.gz | Bin 0 -> 50 bytes
.../test/resources/TestIdentifyMimeType/1.xml | 20 +
.../test/resources/TestIdentifyMimeType/1.zip | Bin 0 -> 165 bytes
.../TestIdentifyMimeType/bgBannerFoot.png | Bin 0 -> 189 bytes
.../TestIdentifyMimeType/blueBtnBg.jpg | Bin 0 -> 356 bytes
.../TestIdentifyMimeType/flowfilev1.tar | Bin 0 -> 10240 bytes
.../resources/TestIdentifyMimeType/flowfilev3 | Bin 0 -> 40 bytes
.../resources/TestIdentifyMimeType/grid.gif | Bin 0 -> 135 bytes
.../test/resources/TestMergeContent/demarcate | 1 +
.../src/test/resources/TestMergeContent/foot | 1 +
.../src/test/resources/TestMergeContent/head | 1 +
.../test/resources/TestModifyBytes/noFooter.txt | 10 +
.../TestModifyBytes/noFooter_noHeader.txt | 10 +
.../test/resources/TestModifyBytes/noHeader.txt | 11 +
.../test/resources/TestModifyBytes/testFile.txt | 11 +
.../TestReplaceTextLineByLine/$1$1.txt | 11 +
.../BRue_cRue_RiRey.txt | 11 +
.../TestReplaceTextLineByLine/Blu$2e_clu$2e.txt | 11 +
.../TestReplaceTextLineByLine/D$d_h$d.txt | 11 +
.../TestReplaceTextLineByLine/Good.txt | 1 +
.../TestReplaceTextLineByLine/Spider.txt | 11 +
.../TestReplaceTextLineByLine/[DODO].txt | 11 +
.../TestReplaceTextLineByLine/cu[$1]_Po[$1].txt | 11 +
.../TestReplaceTextLineByLine/cu_Po.txt | 11 +
.../TestReplaceTextLineByLine/food.txt | 11 +
.../TestReplaceTextLineByLine/testFile.txt | 11 +
.../color-fruit-backreference-mapping.txt | 7 +
.../color-fruit-blank-mapping.txt | 7 +
.../color-fruit-escaped-dollar-mapping.txt | 7 +
...t-excessive-backreference-mapping-simple.txt | 6 +
...or-fruit-excessive-backreference-mapping.txt | 6 +
...olor-fruit-invalid-backreference-mapping.txt | 7 +
.../color-fruit-mapping.txt | 7 +
.../color-fruit-no-match-mapping.txt | 7 +
.../color-fruit-space-mapping.txt | 6 +
.../colors-without-dashes.txt | 4 +
.../TestReplaceTextWithMapping/colors.txt | 4 +
.../test/resources/TestScanContent/helloWorld | 1 +
.../resources/TestScanContent/wellthengood-bye | 1 +
.../src/test/resources/TestSplitText/1.txt | 5 +
.../src/test/resources/TestSplitText/2.txt | 5 +
.../src/test/resources/TestSplitText/3.txt | 5 +
.../src/test/resources/TestSplitText/4.txt | 3 +
.../src/test/resources/TestSplitText/5.txt | 7 +
.../src/test/resources/TestSplitText/6.txt | 7 +
.../test/resources/TestSplitText/original.txt | 12 +
.../test/resources/TestTransformXml/math.html | 8 +
.../test/resources/TestTransformXml/math.xml | 21 +
.../test/resources/TestTransformXml/math.xsl | 36 +
.../test/resources/TestTransformXml/tokens.csv | 2 +
.../test/resources/TestTransformXml/tokens.xml | 17 +
.../test/resources/TestTransformXml/tokens.xsl | 103 +
.../resources/TestUnpackContent/data.flowfilev2 | Bin 0 -> 255 bytes
.../resources/TestUnpackContent/data.flowfilev3 | Bin 0 -> 357 bytes
.../test/resources/TestUnpackContent/data.tar | Bin 0 -> 3584 bytes
.../test/resources/TestUnpackContent/data.zip | Bin 0 -> 359 bytes
.../resources/TestUnpackContent/folder/cal.txt | 8 +
.../resources/TestUnpackContent/folder/date.txt | 1 +
.../src/test/resources/TestXml/XmlBundle.xsd | 34 +
.../src/test/resources/TestXml/fruit.xml | 47 +
.../src/test/resources/TestXml/subNode.xml | 21 +
.../src/test/resources/TestXml/xml-bundle-1 | 51 +
.../src/test/resources/TestXml/xml-snippet.xml | 25 +
.../src/test/resources/hello.txt | 1 +
.../src/test/resources/localhost-ks.jks | Bin 0 -> 1298 bytes
.../src/test/resources/localhost-ts.jks | Bin 0 -> 891 bytes
.../src/test/resources/localhost.cer | Bin 0 -> 829 bytes
.../src/test/resources/logback-test.xml | 49 +
.../standard-reporting-tasks/pom.xml | 60 +
.../ControllerStatusReportingTask.java | 347 ++
.../nifi/controller/MonitorDiskUsage.java | 113 +
.../apache/nifi/controller/MonitorMemory.java | 260 ++
.../org.apache.nifi.reporting.ReportingTask | 17 +
.../index.html | 85 +
.../index.html | 58 +
.../index.html | 77 +
.../nifi/controller/TestMonitorDiskUsage.java | 61 +
.../pom.xml | 36 +
.../distributed/cache/client/Deserializer.java | 41 +
.../cache/client/DistributedMapCacheClient.java | 114 +
.../cache/client/DistributedSetCacheClient.java | 74 +
.../distributed/cache/client/Serializer.java | 40 +
.../exception/DeserializationException.java | 33 +
.../exception/SerializationException.java | 33 +
.../distributed-cache-client-service/pom.xml | 56 +
.../distributed/cache/client/CommsSession.java | 46 +
.../DistributedMapCacheClientService.java | 305 ++
.../DistributedSetCacheClientService.java | 215 ++
.../cache/client/SSLCommsSession.java | 106 +
.../cache/client/StandardCommsSession.java | 124 +
...org.apache.nifi.controller.ControllerService | 16 +
.../index.html | 78 +
.../index.html | 51 +
.../distributed-cache-protocol/pom.xml | 39 +
.../cache/protocol/ProtocolHandshake.java | 119 +
.../protocol/exception/HandshakeException.java | 27 +
.../distributed-cache-server/pom.xml | 75 +
.../cache/server/AbstractCacheServer.java | 199 +
.../distributed/cache/server/CacheRecord.java | 57 +
.../distributed/cache/server/CacheServer.java | 26 +
.../cache/server/DistributedCacheServer.java | 107 +
.../cache/server/DistributedSetCacheServer.java | 68 +
.../cache/server/EvictionPolicy.java | 73 +
.../cache/server/SetCacheServer.java | 104 +
.../server/map/DistributedMapCacheServer.java | 71 +
.../distributed/cache/server/map/MapCache.java | 29 +
.../cache/server/map/MapCacheRecord.java | 58 +
.../cache/server/map/MapCacheServer.java | 145 +
.../cache/server/map/MapPutResult.java | 59 +
.../cache/server/map/PersistentMapCache.java | 210 +
.../cache/server/map/SimpleMapCache.java | 165 +
.../cache/server/set/PersistentSetCache.java | 194 +
.../distributed/cache/server/set/SetCache.java | 29 +
.../cache/server/set/SetCacheRecord.java | 55 +
.../cache/server/set/SetCacheResult.java | 43 +
.../cache/server/set/SimpleSetCache.java | 117 +
...org.apache.nifi.controller.ControllerService | 16 +
.../index.html | 82 +
.../cache/server/TestServerAndClient.java | 535 +++
.../src/test/resources/localhost-ks.jks | Bin 0 -> 1298 bytes
.../src/test/resources/localhost-ts.jks | Bin 0 -> 891 bytes
.../distributed-cache-services-nar/pom.xml | 47 +
.../distributed-cache-services-bundle/pom.xml | 35 +
.../load-distribution-service-api/pom.xml | 37 +
.../nifi/loading/LoadDistributionListener.java | 24 +
.../nifi/loading/LoadDistributionService.java | 33 +
nifi/nar-bundles/standard-services/pom.xml | 39 +
.../ssl-context-bundle/nar/pom.xml | 39 +
.../ssl-context-bundle/pom.xml | 34 +
.../ssl-context-service/pom.xml | 52 +
.../nifi/ssl/StandardSSLContextService.java | 354 ++
...org.apache.nifi.controller.ControllerService | 15 +
.../index.html | 63 +
.../apache/nifi/ssl/SSLContextServiceTest.java | 197 +
.../java/org/apache/nifi/ssl/TestProcessor.java | 47 +
.../src/test/resources/localhost-ks.jks | Bin 0 -> 1298 bytes
.../src/test/resources/localhost-ts.jks | Bin 0 -> 891 bytes
.../ssl-context-service-api/pom.xml | 36 +
.../org/apache/nifi/ssl/SSLContextService.java | 55 +
.../standard-services-api-nar/pom.xml | 45 +
.../update-attribute-bundle/model/pom.xml | 27 +
.../apache/nifi/update/attributes/Action.java | 55 +
.../nifi/update/attributes/Condition.java | 49 +
.../apache/nifi/update/attributes/Criteria.java | 141 +
.../nifi/update/attributes/FlowFilePolicy.java | 34 +
.../org/apache/nifi/update/attributes/Rule.java | 65 +
.../update/attributes/serde/CriteriaSerDe.java | 128 +
.../update-attribute-bundle/nar/pom.xml | 48 +
.../nar-bundles/update-attribute-bundle/pom.xml | 56 +
.../update-attribute-bundle/processor/pom.xml | 54 +
.../processors/attributes/UpdateAttribute.java | 508 +++
.../org.apache.nifi.processor.Processor | 15 +
.../index.html | 253 ++
.../update/attributes/TestUpdateAttribute.java | 425 ++
.../ui/nb-configuration.xml | 18 +
.../update-attribute-bundle/ui/pom.xml | 91 +
.../attributes/UpdateAttributeModelFactory.java | 127 +
.../attributes/api/ObjectMapperResolver.java | 51 +
.../update/attributes/api/RuleResource.java | 684 ++++
.../nifi/update/attributes/dto/ActionDTO.java | 68 +
.../update/attributes/dto/ConditionDTO.java | 59 +
.../nifi/update/attributes/dto/DtoFactory.java | 68 +
.../nifi/update/attributes/dto/RuleDTO.java | 65 +
.../update/attributes/entity/ActionEntity.java | 65 +
.../attributes/entity/ConditionEntity.java | 65 +
.../entity/EvaluationContextEntity.java | 74 +
.../update/attributes/entity/RuleEntity.java | 65 +
.../update/attributes/entity/RulesEntity.java | 66 +
.../ui/src/main/webapp/META-INF/nifi-processor | 15 +
.../src/main/webapp/WEB-INF/jsp/worksheet.jsp | 188 +
.../ui/src/main/webapp/WEB-INF/web.xml | 46 +
.../ui/src/main/webapp/css/main.css | 426 +++
.../ui/src/main/webapp/images/bgInputText.png | Bin 0 -> 139 bytes
.../ui/src/main/webapp/images/buttonNew.png | Bin 0 -> 590 bytes
.../ui/src/main/webapp/images/iconDelete.png | Bin 0 -> 670 bytes
.../ui/src/main/webapp/images/iconInfo.png | Bin 0 -> 550 bytes
.../ui/src/main/webapp/js/application.js | 1915 +++++++++
.../nar/pom.xml | 36 +
.../pom.xml | 44 +
.../volatile-provenance-repository/pom.xml | 47 +
.../VolatileProvenanceRepository.java | 745 ++++
...he.nifi.provenance.ProvenanceEventRepository | 15 +
.../TestVolatileProvenanceRepository.java | 178 +
.../src/test/resources/nifi.properties | 18 +
nifi/nifi-api/.gitignore | 2 +
nifi/nifi-api/pom.xml | 31 +
.../apache/nifi/authorization/Authority.java | 97 +
.../nifi/authorization/AuthorityProvider.java | 181 +
.../AuthorityProviderConfigurationContext.java | 52 +
.../AuthorityProviderInitializationContext.java | 27 +
.../authorization/AuthorityProviderLookup.java | 25 +
.../authorization/DownloadAuthorization.java | 86 +
.../annotation/AuthorityProviderContext.java | 36 +
.../exception/AuthorityAccessException.java | 33 +
.../IdentityAlreadyExistsException.java | 32 +
.../exception/ProviderCreationException.java | 40 +
.../exception/ProviderDestructionException.java | 40 +
.../exception/UnknownIdentityException.java | 32 +
.../AbstractConfigurableComponent.java | 221 ++
.../apache/nifi/components/AllowableValue.java | 136 +
.../nifi/components/ConfigurableComponent.java | 77 +
.../nifi/components/PropertyDescriptor.java | 552 +++
.../apache/nifi/components/PropertyValue.java | 189 +
.../nifi/components/ValidationContext.java | 82 +
.../nifi/components/ValidationResult.java | 173 +
.../org/apache/nifi/components/Validator.java | 55 +
.../controller/AbstractControllerService.java | 91 +
.../nifi/controller/ConfigurationContext.java | 46 +
.../nifi/controller/ControllerService.java | 182 +
.../ControllerServiceInitializationContext.java | 36 +
.../controller/ControllerServiceLookup.java | 66 +
.../apache/nifi/controller/FlowFileQueue.java | 200 +
.../apache/nifi/controller/ScheduledState.java | 37 +
.../org/apache/nifi/controller/Snippet.java | 113 +
.../org/apache/nifi/controller/Triggerable.java | 116 +
.../controller/annotation/OnConfigured.java | 41 +
.../repository/ContentRepository.java | 314 ++
.../controller/repository/FlowFileRecord.java | 54 +
.../repository/FlowFileRepository.java | 128 +
.../repository/FlowFileSwapManager.java | 63 +
.../controller/repository/QueueProvider.java | 35 +
.../controller/repository/RepositoryRecord.java | 100 +
.../repository/RepositoryRecordType.java | 26 +
.../repository/claim/ContentClaim.java | 55 +
.../repository/claim/ContentClaimManager.java | 143 +
.../controller/status/ConnectionStatus.java | 193 +
.../nifi/controller/status/PortStatus.java | 201 +
.../controller/status/ProcessGroupStatus.java | 547 +++
.../nifi/controller/status/ProcessorStatus.java | 275 ++
.../status/RemoteProcessGroupStatus.java | 217 ++
.../nifi/controller/status/RunStatus.java | 28 +
.../controller/status/TransmissionStatus.java | 23 +
.../history/ComponentStatusRepository.java | 167 +
.../status/history/MetricDescriptor.java | 75 +
.../status/history/StatusHistory.java | 50 +
.../status/history/StatusSnapshot.java | 48 +
.../controller/status/history/ValueMapper.java | 23 +
.../controller/status/history/ValueReducer.java | 25 +
.../org/apache/nifi/events/EventReporter.java | 27 +
.../nifi/expression/AttributeExpression.java | 79 +
.../expression/AttributeValueDecorator.java | 29 +
.../expression/ExpressionLanguageCompiler.java | 69 +
.../java/org/apache/nifi/flowfile/FlowFile.java | 109 +
.../nifi/flowfile/FlowFilePrioritizer.java | 30 +
.../java/org/apache/nifi/logging/LogLevel.java | 27 +
.../org/apache/nifi/logging/ProcessorLog.java | 73 +
.../nifi/processor/AbstractProcessor.java | 37 +
.../AbstractSessionFactoryProcessor.java | 122 +
.../org/apache/nifi/processor/DataUnit.java | 248 ++
.../apache/nifi/processor/FlowFileFilter.java | 91 +
.../apache/nifi/processor/ProcessContext.java | 132 +
.../apache/nifi/processor/ProcessSession.java | 713 ++++
.../nifi/processor/ProcessSessionFactory.java | 30 +
.../org/apache/nifi/processor/Processor.java | 91 +
.../ProcessorInitializationContext.java | 53 +
.../org/apache/nifi/processor/QueueSize.java | 49 +
.../org/apache/nifi/processor/Relationship.java | 127 +
.../nifi/processor/SchedulingContext.java | 70 +
.../annotation/CapabilityDescription.java | 39 +
.../nifi/processor/annotation/EventDriven.java | 49 +
.../nifi/processor/annotation/OnAdded.java | 41 +
.../nifi/processor/annotation/OnRemoved.java | 42 +
.../nifi/processor/annotation/OnScheduled.java | 44 +
.../nifi/processor/annotation/OnShutdown.java | 38 +
.../nifi/processor/annotation/OnStopped.java | 56 +
.../processor/annotation/OnUnscheduled.java | 46 +
.../processor/annotation/SideEffectFree.java | 45 +
.../processor/annotation/SupportsBatching.java | 51 +
.../apache/nifi/processor/annotation/Tags.java | 44 +
.../processor/annotation/TriggerSerially.java | 39 +
.../TriggerWhenAnyDestinationAvailable.java | 40 +
.../processor/annotation/TriggerWhenEmpty.java | 41 +
.../exception/FlowFileAccessException.java | 37 +
.../exception/FlowFileHandlingException.java | 39 +
.../exception/MissingFlowFileException.java | 37 +
.../processor/exception/ProcessException.java | 44 +
.../nifi/processor/io/InputStreamCallback.java | 38 +
.../nifi/processor/io/OutputStreamCallback.java | 39 +
.../nifi/processor/io/StreamCallback.java | 41 +
.../nifi/provenance/ProvenanceEventBuilder.java | 308 ++
.../nifi/provenance/ProvenanceEventRecord.java | 296 ++
.../provenance/ProvenanceEventRepository.java | 190 +
.../nifi/provenance/ProvenanceEventType.java | 92 +
.../nifi/provenance/ProvenanceReporter.java | 616 +++
.../lineage/ComputeLineageResult.java | 70 +
.../lineage/ComputeLineageSubmission.java | 82 +
.../apache/nifi/provenance/lineage/Lineage.java | 41 +
.../lineage/LineageComputationType.java | 27 +
.../nifi/provenance/lineage/LineageEdge.java | 26 +
.../nifi/provenance/lineage/LineageNode.java | 61 +
.../provenance/lineage/LineageNodeType.java | 23 +
.../lineage/ProvenanceEventLineageNode.java | 32 +
.../apache/nifi/provenance/search/Query.java | 91 +
.../nifi/provenance/search/QueryResult.java | 77 +
.../nifi/provenance/search/QuerySubmission.java | 63 +
.../nifi/provenance/search/SearchTerm.java | 24 +
.../nifi/provenance/search/SearchTerms.java | 39 +
.../nifi/provenance/search/SearchableField.java | 62 +
.../provenance/search/SearchableFieldType.java | 28 +
.../nifi/reporting/AbstractReportingTask.java | 94 +
.../org/apache/nifi/reporting/Bulletin.java | 120 +
.../apache/nifi/reporting/BulletinQuery.java | 109 +
.../nifi/reporting/BulletinRepository.java | 88 +
.../org/apache/nifi/reporting/EventAccess.java | 53 +
.../nifi/reporting/InitializationException.java | 32 +
.../apache/nifi/reporting/ReportingContext.java | 99 +
.../ReportingInitializationContext.java | 80 +
.../apache/nifi/reporting/ReportingTask.java | 75 +
.../org/apache/nifi/reporting/Severity.java | 24 +
.../nifi/scheduling/SchedulingStrategy.java | 86 +
.../org/apache/nifi/search/SearchContext.java | 57 +
.../org/apache/nifi/search/SearchResult.java | 82 +
.../java/org/apache/nifi/search/Searchable.java | 27 +
.../nifi/web/ClusterRequestException.java | 38 +
.../nifi/web/InvalidRevisionException.java | 33 +
.../org/apache/nifi/web/NiFiWebContext.java | 121 +
.../apache/nifi/web/NiFiWebContextConfig.java | 54 +
.../nifi/web/ProcessorConfigurationAction.java | 137 +
.../java/org/apache/nifi/web/ProcessorInfo.java | 110 +
.../nifi/web/ResourceNotFoundException.java | 32 +
.../main/java/org/apache/nifi/web/Revision.java | 110 +
.../nifi/components/TestPropertyDescriptor.java | 59 +
.../org/apache/nifi/processor/TestDataUnit.java | 44 +
.../src/test/resources/logback-test.xml | 27 +
nifi/nifi-bootstrap/pom.xml | 29 +
.../apache/nifi/bootstrap/BootstrapCodec.java | 92 +
.../org/apache/nifi/bootstrap/NiFiListener.java | 128 +
.../java/org/apache/nifi/bootstrap/RunNiFi.java | 941 +++++
.../org/apache/nifi/bootstrap/ShutdownHook.java | 86 +
.../exception/InvalidCommandException.java | 37 +
.../bootstrap/util/LimitingInputStream.java | 107 +
nifi/nifi-docs/pom.xml | 131 +
.../src/main/asciidoc/administration-guide.adoc | 160 +
.../src/main/asciidoc/developer-guide.adoc | 45 +
.../asciidoc/expression-language-guide.adoc | 1727 +++++++++
.../images/add-processor-with-tag-cloud.png | Bin 0 -> 22925 bytes
.../src/main/asciidoc/images/add-processor.png | Bin 0 -> 31524 bytes
.../src/main/asciidoc/images/addConnect.png | Bin 0 -> 1996 bytes
.../src/main/asciidoc/images/comments-tab.png | Bin 0 -> 6431 bytes
.../src/main/asciidoc/images/components.png | Bin 0 -> 8992 bytes
.../asciidoc/images/connection-settings.png | Bin 0 -> 15214 bytes
.../main/asciidoc/images/create-connection.png | Bin 0 -> 9430 bytes
.../asciidoc/images/edit-property-dropdown.png | Bin 0 -> 14291 bytes
.../asciidoc/images/edit-property-textarea.png | Bin 0 -> 17146 bytes
.../main/asciidoc/images/event-attributes.png | Bin 0 -> 94668 bytes
.../src/main/asciidoc/images/event-content.png | Bin 0 -> 88726 bytes
.../src/main/asciidoc/images/event-details.png | Bin 0 -> 139736 bytes
.../src/main/asciidoc/images/expand-event.png | Bin 0 -> 40476 bytes
.../main/asciidoc/images/expanded-events.png | Bin 0 -> 76082 bytes
.../src/main/asciidoc/images/find-parents.png | Bin 0 -> 35831 bytes
.../src/main/asciidoc/images/iconAlert.png | Bin 0 -> 1396 bytes
.../src/main/asciidoc/images/iconConnection.png | Bin 0 -> 1517 bytes
.../src/main/asciidoc/images/iconDelete.png | Bin 0 -> 670 bytes
.../src/main/asciidoc/images/iconDisable.png | Bin 0 -> 764 bytes
.../src/main/asciidoc/images/iconEdit.png | Bin 0 -> 493 bytes
.../src/main/asciidoc/images/iconEnable.png | Bin 0 -> 667 bytes
.../src/main/asciidoc/images/iconExport.png | Bin 0 -> 453 bytes
.../main/asciidoc/images/iconFlowHistory.png | Bin 0 -> 2463 bytes
.../src/main/asciidoc/images/iconFunnel.png | Bin 0 -> 1223 bytes
.../src/main/asciidoc/images/iconInfo.png | Bin 0 -> 550 bytes
.../src/main/asciidoc/images/iconInputPort.png | Bin 0 -> 1842 bytes
.../main/asciidoc/images/iconInputPortSmall.png | Bin 0 -> 532 bytes
.../src/main/asciidoc/images/iconLabel.png | Bin 0 -> 838 bytes
.../src/main/asciidoc/images/iconLineage.png | Bin 0 -> 2214 bytes
.../main/asciidoc/images/iconNewTemplate.png | Bin 0 -> 557 bytes
.../src/main/asciidoc/images/iconNotSecure.png | Bin 0 -> 221 bytes
.../src/main/asciidoc/images/iconOutputPort.png | Bin 0 -> 1658 bytes
.../asciidoc/images/iconOutputPortSmall.png | Bin 0 -> 459 bytes
.../main/asciidoc/images/iconProcessGroup.png | Bin 0 -> 1422 bytes
.../src/main/asciidoc/images/iconProcessor.png | Bin 0 -> 1446 bytes
.../src/main/asciidoc/images/iconProvenance.png | Bin 0 -> 2268 bytes
.../asciidoc/images/iconRemoteProcessGroup.png | Bin 0 -> 674 bytes
.../src/main/asciidoc/images/iconResize.png | Bin 0 -> 165 bytes
.../src/main/asciidoc/images/iconRun.png | Bin 0 -> 538 bytes
.../src/main/asciidoc/images/iconSecure.png | Bin 0 -> 225 bytes
.../src/main/asciidoc/images/iconSettings.png | Bin 0 -> 2638 bytes
.../src/main/asciidoc/images/iconStop.png | Bin 0 -> 402 bytes
.../src/main/asciidoc/images/iconSummary.png | Bin 0 -> 272 bytes
.../src/main/asciidoc/images/iconTemplate.png | Bin 0 -> 970 bytes
.../asciidoc/images/iconTransmissionActive.png | Bin 0 -> 1330 bytes
.../images/iconTransmissionInactive.png | Bin 0 -> 1248 bytes
.../src/main/asciidoc/images/iconUsers.png | Bin 0 -> 2272 bytes
.../main/asciidoc/images/iconViewDetails.png | Bin 0 -> 1788 bytes
.../images/instantiate-template-description.png | Bin 0 -> 6508 bytes
.../asciidoc/images/instantiate-template.png | Bin 0 -> 3216 bytes
.../main/asciidoc/images/invalid-processor.png | Bin 0 -> 8187 bytes
.../main/asciidoc/images/lineage-flowfile.png | Bin 0 -> 3855 bytes
.../asciidoc/images/lineage-graph-annotated.png | Bin 0 -> 170122 bytes
.../src/main/asciidoc/images/new-flow.png | Bin 0 -> 266913 bytes
.../main/asciidoc/images/nifi-arch-cluster.png | Bin 0 -> 50384 bytes
.../src/main/asciidoc/images/nifi-arch.png | Bin 0 -> 43509 bytes
.../main/asciidoc/images/nifi-navigation.png | Bin 0 -> 339195 bytes
.../asciidoc/images/nifi-toolbar-components.png | Bin 0 -> 262314 bytes
.../src/main/asciidoc/images/parent-found.png | Bin 0 -> 42814 bytes
.../asciidoc/images/process-group-anatomy.png | Bin 0 -> 48554 bytes
.../main/asciidoc/images/processor-anatomy.png | Bin 0 -> 37035 bytes
.../images/processor-connection-bubble.png | Bin 0 -> 9936 bytes
.../src/main/asciidoc/images/properties-tab.png | Bin 0 -> 11847 bytes
.../asciidoc/images/provenance-annotated.png | Bin 0 -> 530929 bytes
.../main/asciidoc/images/provenance-table.png | Bin 0 -> 466599 bytes
.../asciidoc/images/remote-group-anatomy.png | Bin 0 -> 61383 bytes
.../images/remote-group-ports-dialog.png | Bin 0 -> 21662 bytes
.../images/remote-port-connection-status.png | Bin 0 -> 57332 bytes
.../src/main/asciidoc/images/scheduling-tab.png | Bin 0 -> 12288 bytes
.../src/main/asciidoc/images/search-events.png | Bin 0 -> 62626 bytes
.../images/search-receive-event-abc.png | Bin 0 -> 67817 bytes
.../src/main/asciidoc/images/settings-tab.png | Bin 0 -> 16225 bytes
.../src/main/asciidoc/images/simple-flow.png | Bin 0 -> 196808 bytes
.../src/main/asciidoc/images/stats-history.png | Bin 0 -> 33205 bytes
.../src/main/asciidoc/images/status-bar.png | Bin 0 -> 249077 bytes
.../main/asciidoc/images/summary-annotated.png | Bin 0 -> 111956 bytes
.../src/main/asciidoc/images/summary-table.png | Bin 0 -> 62114 bytes
.../main/asciidoc/images/valid-processor.png | Bin 0 -> 8090 bytes
nifi/nifi-docs/src/main/asciidoc/overview.adoc | 296 ++
.../nifi-docs/src/main/asciidoc/user-guide.adoc | 1286 +++++++
.../src/main/assembly/dependencies.xml | 28 +
nifi/nifi-mock/pom.xml | 54 +
.../MockProvenanceEventRepository.java | 131 +
.../apache/nifi/reporting/BulletinFactory.java | 43 +
.../org/apache/nifi/reporting/MockBulletin.java | 24 +
.../util/ControllerServiceConfiguration.java | 74 +
.../nifi/util/MockBulletinRepository.java | 74 +
.../nifi/util/MockConfigurationContext.java | 50 +
...kControllerServiceInitializationContext.java | 41 +
.../nifi/util/MockControllerServiceLookup.java | 76 +
.../org/apache/nifi/util/MockEventAccess.java | 70 +
.../java/org/apache/nifi/util/MockFlowFile.java | 279 ++
.../org/apache/nifi/util/MockFlowFileQueue.java | 85 +
.../apache/nifi/util/MockProcessContext.java | 284 ++
.../apache/nifi/util/MockProcessSession.java | 1006 +++++
.../MockProcessorInitializationContext.java | 74 +
.../org/apache/nifi/util/MockProcessorLog.java | 402 ++
.../org/apache/nifi/util/MockPropertyValue.java | 185 +
.../nifi/util/MockProvenanceReporter.java | 202 +
.../apache/nifi/util/MockReportingContext.java | 117 +
.../MockReportingInitializationContext.java | 81 +
.../apache/nifi/util/MockSessionFactory.java | 46 +
.../apache/nifi/util/MockValidationContext.java | 93 +
.../org/apache/nifi/util/ReflectionUtils.java | 149 +
.../apache/nifi/util/SharedSessionState.java | 72 +
.../apache/nifi/util/SingleSessionFactory.java | 35 +
.../nifi/util/StandardProcessorTestRunner.java | 492 +++
.../java/org/apache/nifi/util/TestRunner.java | 542 +++
.../java/org/apache/nifi/util/TestRunners.java | 37 +
nifi/pom.xml | 1085 ++++++
pom.xml | 1092 ------
4901 files changed, 324985 insertions(+), 324948 deletions(-)
----------------------------------------------------------------------
[4/8] incubator-nifi git commit: Merge branch 'develop' of
https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
index 0000000,a755b1a..3ac55d2
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java
@@@ -1,0 -1,498 +1,498 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processors.standard;
+
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.LinkedHashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.TreeSet;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicLong;
+ import java.util.concurrent.atomic.AtomicReference;
+
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.components.ValidationContext;
+ import org.apache.nifi.components.ValidationResult;
+ import org.apache.nifi.components.Validator;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.loading.LoadDistributionListener;
+ import org.apache.nifi.loading.LoadDistributionService;
+ import org.apache.nifi.processor.AbstractProcessor;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessorInitializationContext;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.annotation.CapabilityDescription;
+ import org.apache.nifi.processor.annotation.EventDriven;
+ import org.apache.nifi.processor.annotation.OnScheduled;
+ import org.apache.nifi.processor.annotation.SideEffectFree;
+ import org.apache.nifi.processor.annotation.SupportsBatching;
+ import org.apache.nifi.processor.annotation.Tags;
+ import org.apache.nifi.processor.annotation.TriggerWhenAnyDestinationAvailable;
+ import org.apache.nifi.processor.util.StandardValidators;
+
+ import org.apache.commons.lang3.StringUtils;
+
+ @EventDriven
+ @SideEffectFree
+ @SupportsBatching
+ @TriggerWhenAnyDestinationAvailable
+ @Tags({"distribute", "load balance", "route", "round robin", "weighted"})
+ @CapabilityDescription("Distributes FlowFiles to downstream processors based on a Distribution Strategy. If using the Round Robin "
+ + "strategy, the default is to assign each destination a weighting of 1 (evenly distributed). However, optional properties"
+ + "can be added to the change this; adding a property with the name '5' and value '10' means that the relationship with name "
+ + "'5' will be receive 10 FlowFiles in each iteration instead of 1.")
+ public class DistributeLoad extends AbstractProcessor {
+
+ public static final String STRATEGY_ROUND_ROBIN = "round robin";
+ public static final String STRATEGY_NEXT_AVAILABLE = "next available";
+ public static final String STRATEGY_LOAD_DISTRIBUTION_SERVICE = "load distribution service";
+
+ public static final PropertyDescriptor NUM_RELATIONSHIPS = new PropertyDescriptor.Builder()
+ .name("Number of Relationships")
+ .description("Determines the number of Relationships to which the load should be distributed")
+ .required(true)
+ .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+ .defaultValue("1")
+ .build();
+ public static final PropertyDescriptor DISTRIBUTION_STRATEGY = new PropertyDescriptor.Builder()
+ .name("Distribution Strategy")
+ .description(
+ "Determines how the load will be distributed. If using Round Robin, will not distribute any FlowFiles unless all destinations can accept FlowFiles; when using Next Available, will distribute FlowFiles as long as at least 1 destination can accept FlowFiles.")
+ .required(true)
+ .allowableValues(STRATEGY_ROUND_ROBIN, STRATEGY_NEXT_AVAILABLE, STRATEGY_LOAD_DISTRIBUTION_SERVICE)
+ .defaultValue(STRATEGY_ROUND_ROBIN)
+ .build();
+
+ public static final PropertyDescriptor HOSTNAMES = new PropertyDescriptor.Builder()
+ .name("Hostnames")
+ .description("List of remote servers to distribute across. Each server must be FQDN and use either ',', ';', or [space] as a delimiter")
+ .required(true)
+ .addValidator(new Validator() {
+
+ @Override
+ public ValidationResult validate(String subject, String input, ValidationContext context) {
+ ValidationResult result = new ValidationResult.Builder()
+ .subject(subject)
+ .valid(true)
+ .input(input)
+ .explanation("Good FQDNs")
+ .build();
+ if (null == input) {
+ result = new ValidationResult.Builder()
+ .subject(subject)
+ .input(input)
+ .valid(false)
+ .explanation("Need to specify delimited list of FQDNs")
+ .build();
+ return result;
+ }
+ String[] hostNames = input.split("(?:,+|;+|\\s+)");
+ for (String hostName : hostNames) {
+ if (StringUtils.isNotBlank(hostName) && !hostName.contains(".")) {
+ result = new ValidationResult.Builder()
+ .subject(subject)
+ .input(input)
+ .valid(false)
+ .explanation("Need a FQDN rather than a simple host name.")
+ .build();
+ return result;
+ }
+ }
+ return result;
+ }
+ })
+ .build();
+ public static final PropertyDescriptor LOAD_DISTRIBUTION_SERVICE_TEMPLATE = new PropertyDescriptor.Builder()
+ .name("Load Distribution Service ID")
+ .description("The identifier of the Load Distribution Service")
+ .required(true)
+ .identifiesControllerService(LoadDistributionService.class)
+ .build();
+
+ private List<PropertyDescriptor> properties;
+ private final AtomicReference<Set<Relationship>> relationshipsRef = new AtomicReference<>();
+ private final AtomicReference<DistributionStrategy> strategyRef = new AtomicReference<DistributionStrategy>(new RoundRobinStrategy());
+ private final AtomicReference<List<Relationship>> weightedRelationshipListRef = new AtomicReference<>();
+ private final AtomicBoolean doCustomValidate = new AtomicBoolean(false);
+ private volatile LoadDistributionListener myListener;
+ private final AtomicBoolean doSetProps = new AtomicBoolean(true);
+
+ @Override
+ protected void init(final ProcessorInitializationContext context) {
+ final Set<Relationship> relationships = new HashSet<>();
+ relationships.add(createRelationship(1));
+ relationshipsRef.set(Collections.unmodifiableSet(relationships));
+
+ final List<PropertyDescriptor> properties = new ArrayList<>();
+ properties.add(NUM_RELATIONSHIPS);
+ properties.add(DISTRIBUTION_STRATEGY);
+ this.properties = Collections.unmodifiableList(properties);
+ }
+
+ private static Relationship createRelationship(final int num) {
+ return new Relationship.Builder().name(String.valueOf(num)).build();
+ }
+
+ @Override
+ public Set<Relationship> getRelationships() {
+ return relationshipsRef.get();
+ }
+
+ @Override
+ public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
+ if (descriptor.equals(NUM_RELATIONSHIPS)) {
+ final Set<Relationship> relationships = new HashSet<>();
+ for (int i = 1; i <= Integer.parseInt(newValue); i++) {
+ relationships.add(createRelationship(i));
+ }
+ this.relationshipsRef.set(Collections.unmodifiableSet(relationships));
+ } else if (descriptor.equals(DISTRIBUTION_STRATEGY)) {
+ switch (newValue.toLowerCase()) {
+ case STRATEGY_ROUND_ROBIN:
+ strategyRef.set(new RoundRobinStrategy());
+ break;
+ case STRATEGY_NEXT_AVAILABLE:
+ strategyRef.set(new NextAvailableStrategy());
+ break;
+ case STRATEGY_LOAD_DISTRIBUTION_SERVICE:
+ strategyRef.set(new LoadDistributionStrategy());
+ }
+ doSetProps.set(true);
+ doCustomValidate.set(true);
+ }
+ }
+
+ @Override
+ protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+ if (strategyRef.get() instanceof LoadDistributionStrategy && doSetProps.getAndSet(false)) {
+ final List<PropertyDescriptor> props = new ArrayList<>(properties);
+ props.add(LOAD_DISTRIBUTION_SERVICE_TEMPLATE);
+ props.add(HOSTNAMES);
+ this.properties = Collections.unmodifiableList(props);
+ } else if (doSetProps.getAndSet(false)) {
+ final List<PropertyDescriptor> props = new ArrayList<>();
+ props.add(NUM_RELATIONSHIPS);
+ props.add(DISTRIBUTION_STRATEGY);
+ this.properties = Collections.unmodifiableList(props);
+ }
+ return properties;
+ }
+
+ @Override
+ protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+ // validate that the property name is valid.
+ final int numRelationships = this.relationshipsRef.get().size();
+ try {
+ final int value = Integer.parseInt(propertyDescriptorName);
+ if (value <= 0 || value > numRelationships) {
+ return new PropertyDescriptor.Builder().addValidator(new InvalidPropertyNameValidator(propertyDescriptorName))
+ .name(propertyDescriptorName).build();
+ }
+ } catch (final NumberFormatException e) {
+ return new PropertyDescriptor.Builder().addValidator(new InvalidPropertyNameValidator(propertyDescriptorName))
+ .name(propertyDescriptorName).build();
+ }
+
+ // validate that the property value is valid
+ return new PropertyDescriptor.Builder().addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+ .name(propertyDescriptorName).dynamic(true).build();
+ }
+
+ @Override
+ protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+ Collection<ValidationResult> results = new ArrayList<>();
+ if (doCustomValidate.getAndSet(false)) {
+ String distStrat = validationContext.getProperty(DISTRIBUTION_STRATEGY).getValue();
+ if (distStrat.equals(STRATEGY_LOAD_DISTRIBUTION_SERVICE)) {
+ // make sure Hostnames and Controller service are set
+ PropertyValue propDesc = validationContext.getProperty(HOSTNAMES);
+ if (null == propDesc || null == propDesc.getValue() || propDesc.getValue().isEmpty()) {
+ results.add(new ValidationResult.Builder()
+ .subject(HOSTNAMES.getName())
+ .explanation("Must specify Hostnames when using 'Load Distribution Strategy'")
+ .valid(false)
+ .build());
+ }
+ propDesc = validationContext.getProperty(LOAD_DISTRIBUTION_SERVICE_TEMPLATE);
+ if (null == propDesc || null == propDesc.getValue() || propDesc.getValue().isEmpty()) {
+ results.add(new ValidationResult.Builder()
+ .subject(LOAD_DISTRIBUTION_SERVICE_TEMPLATE.getName())
+ .explanation("Must specify 'Load Distribution Service ID' when using 'Load Distribution Strategy'")
+ .valid(false)
+ .build());
+ }
+ if (results.isEmpty()) {
+ int numRels = validationContext.getProperty(NUM_RELATIONSHIPS).asInteger();
+ String hostNamesValue = validationContext.getProperty(HOSTNAMES).getValue();
+ String[] hostNames = hostNamesValue.split("(?:,+|;+|\\s+)");
+ int numHosts = 0;
+ for (String hostName : hostNames) {
+ if (StringUtils.isNotBlank(hostName)) {
+ hostNames[numHosts++] = hostName;
+ }
+ }
+ if (numHosts > numRels) {
+ results.add(new ValidationResult.Builder()
+ .subject("Number of Relationships and Hostnames")
+ .explanation("Number of Relationships must be equal to, or greater than, the number of host names")
+ .valid(false)
+ .build());
+ } else {
+ // create new relationships with descriptions of hostname
+ Set<Relationship> relsWithDesc = new TreeSet<>();
+ for (int i = 0; i < numHosts; i++) {
+ relsWithDesc.add(new Relationship.Builder().name(String.valueOf(i + 1)).description(hostNames[i]).build());
+ }
+ // add add'l rels if configuration requires it...it probably shouldn't
+ for (int i = numHosts + 1; i <= numRels; i++) {
+ relsWithDesc.add(createRelationship(i));
+ }
+ relationshipsRef.set(Collections.unmodifiableSet(relsWithDesc));
+ }
+ }
+ }
+ }
+ return results;
+ }
+
+ @OnScheduled
+ public void createWeightedList(final ProcessContext context) {
+ final Map<Integer, Integer> weightings = new LinkedHashMap<>();
+
+ String distStrat = context.getProperty(DISTRIBUTION_STRATEGY).getValue();
+ if (distStrat.equals(STRATEGY_LOAD_DISTRIBUTION_SERVICE)) {
+ String hostNamesValue = context.getProperty(HOSTNAMES).getValue();
+ String[] hostNames = hostNamesValue.split("(?:,+|;+|\\s+)");
+ Set<String> hostNameSet = new HashSet<>();
+ for (String hostName : hostNames) {
+ if (StringUtils.isNotBlank(hostName)) {
+ hostNameSet.add(hostName);
+ }
+ }
+ LoadDistributionService svc = context.getProperty(LOAD_DISTRIBUTION_SERVICE_TEMPLATE).asControllerService(LoadDistributionService.class);
+ myListener = new LoadDistributionListener() {
+
+ @Override
+ public void update(Map<String, Integer> loadInfo) {
+ for (Relationship rel : relationshipsRef.get()) {
+ String hostname = rel.getDescription();
+ Integer weight = 1;
+ if (loadInfo.containsKey(hostname)) {
+ weight = loadInfo.get(hostname);
+ }
+ weightings.put(Integer.decode(rel.getName()), weight);
+ }
+ updateWeightedRelationships(weightings);
+ }
+ };
+
+ Map<String, Integer> loadInfo = svc.getLoadDistribution(hostNameSet, myListener);
+ for (Relationship rel : relationshipsRef.get()) {
+ String hostname = rel.getDescription();
+ Integer weight = 1;
+ if (loadInfo.containsKey(hostname)) {
+ weight = loadInfo.get(hostname);
+ }
+ weightings.put(Integer.decode(rel.getName()), weight);
+ }
+
+ } else {
+ final int numRelationships = context.getProperty(NUM_RELATIONSHIPS).asInteger();
+ for (int i = 1; i <= numRelationships; i++) {
+ weightings.put(i, 1);
+ }
+ for (final PropertyDescriptor propDesc : context.getProperties().keySet()) {
+ if (!this.properties.contains(propDesc)) {
+ final int relationship = Integer.parseInt(propDesc.getName());
+ final int weighting = context.getProperty(propDesc).asInteger();
+ weightings.put(relationship, weighting);
+ }
+ }
+ }
+ updateWeightedRelationships(weightings);
+ }
+
+ private void updateWeightedRelationships(final Map<Integer, Integer> weightings) {
+ final List<Relationship> relationshipList = new ArrayList<>();
+ for (final Map.Entry<Integer, Integer> entry : weightings.entrySet()) {
+ final String relationshipName = String.valueOf(entry.getKey());
+ final Relationship relationship = new Relationship.Builder().name(relationshipName).build();
+ for (int i = 0; i < entry.getValue(); i++) {
+ relationshipList.add(relationship);
+ }
+ }
+
+ this.weightedRelationshipListRef.set(Collections.unmodifiableList(relationshipList));
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context, final ProcessSession session) {
+ final FlowFile flowFile = session.get();
+ if (flowFile == null) {
+ return;
+ }
+
+ final DistributionStrategy strategy = strategyRef.get();
- final Set<Relationship> available = session.getAvailableRelationships();
++ final Set<Relationship> available = context.getAvailableRelationships();
+ final int numRelationships = context.getProperty(NUM_RELATIONSHIPS).asInteger();
+ final boolean allDestinationsAvailable = (available.size() == numRelationships);
+ if (!allDestinationsAvailable && strategy.requiresAllDestinationsAvailable()) {
+ return;
+ }
+
- final Relationship relationship = strategy.mapToRelationship(session, flowFile);
++ final Relationship relationship = strategy.mapToRelationship(context, flowFile);
+ if (relationship == null) {
+ // can't transfer the FlowFiles. Roll back and yield
+ session.rollback();
+ context.yield();
+ return;
+ }
+
+ session.transfer(flowFile, relationship);
+ session.getProvenanceReporter().route(flowFile, relationship);
+ }
+
+ private static class InvalidPropertyNameValidator implements Validator {
+
+ private final String propertyName;
+
+ public InvalidPropertyNameValidator(final String propertyName) {
+ this.propertyName = propertyName;
+ }
+
+ @Override
+ public ValidationResult validate(final String subject, final String input, final ValidationContext validationContext) {
+ return new ValidationResult.Builder().subject("Property Name").input(propertyName)
+ .explanation("Property Name must be a positive integer between 1 and the number of relationships (inclusive)")
+ .valid(false)
+ .build();
+ }
+ }
+
+ /**
+ * Implementations must be thread-safe.
+ */
+ private static interface DistributionStrategy {
+
+ /**
+ * Returns a mapping of FlowFile to Relationship or <code>null</code> if
+ * the needed relationships are not available to accept files.
+ *
+ * @param session
+ * @param flowFiles
+ * @return
+ */
- Relationship mapToRelationship(ProcessSession session, FlowFile flowFile);
++ Relationship mapToRelationship(ProcessContext context, FlowFile flowFile);
+
+ boolean requiresAllDestinationsAvailable();
+ }
+
+ private class LoadDistributionStrategy implements DistributionStrategy {
+
+ private final AtomicLong counter = new AtomicLong(0L);
+
+ @Override
- public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
++ public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
+ final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
+ final int numRelationships = relationshipList.size();
+
+ // create a HashSet that contains all of the available relationships, as calling #contains on HashSet
+ // is much faster than calling it on a List
+ boolean foundFreeRelationship = false;
+ Relationship relationship = null;
+
+ int attempts = 0;
+ while (!foundFreeRelationship) {
+ final long counterValue = counter.getAndIncrement();
+ final int idx = (int) (counterValue % numRelationships);
+ relationship = relationshipList.get(idx);
- foundFreeRelationship = session.getAvailableRelationships().contains(relationship);
++ foundFreeRelationship = context.getAvailableRelationships().contains(relationship);
+ if (++attempts % numRelationships == 0 && !foundFreeRelationship) {
+ return null;
+ }
+ }
+
+ return relationship;
+ }
+
+ @Override
+ public boolean requiresAllDestinationsAvailable() {
+ return false;
+ }
+
+ }
+
+ private class RoundRobinStrategy implements DistributionStrategy {
+
+ private final AtomicLong counter = new AtomicLong(0L);
+
+ @Override
- public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
++ public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
+ final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
+ final long counterValue = counter.getAndIncrement();
+ final int idx = (int) (counterValue % relationshipList.size());
+ final Relationship relationship = relationshipList.get(idx);
+ return relationship;
+ }
+
+ @Override
+ public boolean requiresAllDestinationsAvailable() {
+ return true;
+ }
+ }
+
+ private class NextAvailableStrategy implements DistributionStrategy {
+
+ private final AtomicLong counter = new AtomicLong(0L);
+
+ @Override
- public Relationship mapToRelationship(final ProcessSession session, final FlowFile flowFile) {
++ public Relationship mapToRelationship(final ProcessContext context, final FlowFile flowFile) {
+ final List<Relationship> relationshipList = DistributeLoad.this.weightedRelationshipListRef.get();
+ final int numRelationships = relationshipList.size();
+
+ // create a HashSet that contains all of the available relationships, as calling #contains on HashSet
+ // is much faster than calling it on a List
+ boolean foundFreeRelationship = false;
+ Relationship relationship = null;
+
+ int attempts = 0;
+ while (!foundFreeRelationship) {
+ final long counterValue = counter.getAndIncrement();
+ final int idx = (int) (counterValue % numRelationships);
+ relationship = relationshipList.get(idx);
- foundFreeRelationship = session.getAvailableRelationships().contains(relationship);
++ foundFreeRelationship = context.getAvailableRelationships().contains(relationship);
+ if (++attempts % numRelationships == 0 && !foundFreeRelationship) {
+ return null;
+ }
+ }
+
+ return relationship;
+ }
+
+ @Override
+ public boolean requiresAllDestinationsAvailable() {
+ return false;
+ }
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
index 0000000,2b0b437..b7fe97a
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenHTTP.java
@@@ -1,0 -1,321 +1,323 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processors.standard;
+
+ import java.util.ArrayList;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.concurrent.ConcurrentMap;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.regex.Pattern;
+
+ import javax.servlet.Servlet;
+ import javax.ws.rs.Path;
+
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.stream.io.LeakyBucketStreamThrottler;
+ import org.apache.nifi.stream.io.StreamThrottler;
+ import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+ import org.apache.nifi.processor.DataUnit;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.ProcessorInitializationContext;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.annotation.CapabilityDescription;
+ import org.apache.nifi.processor.annotation.OnScheduled;
+ import org.apache.nifi.processor.annotation.OnStopped;
+ import org.apache.nifi.processor.annotation.Tags;
+ import org.apache.nifi.processor.util.StandardValidators;
+ import org.apache.nifi.processors.standard.servlets.ContentAcknowledgmentServlet;
+ import org.apache.nifi.processors.standard.servlets.ListenHTTPServlet;
+ import org.apache.nifi.ssl.SSLContextService;
+
+ import org.eclipse.jetty.server.Connector;
+ import org.eclipse.jetty.server.HttpConfiguration;
+ import org.eclipse.jetty.server.HttpConnectionFactory;
+ import org.eclipse.jetty.server.SecureRequestCustomizer;
+ import org.eclipse.jetty.server.Server;
+ import org.eclipse.jetty.server.ServerConnector;
+ import org.eclipse.jetty.server.SslConnectionFactory;
+ import org.eclipse.jetty.servlet.ServletContextHandler;
+ import org.eclipse.jetty.util.ssl.SslContextFactory;
+ import org.eclipse.jetty.util.thread.QueuedThreadPool;
+
+ @Tags({"ingest", "http", "https", "rest", "listen"})
+ @CapabilityDescription("Starts an HTTP Server that is used to receive FlowFiles from remote sources. The URL of the Service will be http://{hostname}:{port}/contentListener")
+ public class ListenHTTP extends AbstractSessionFactoryProcessor {
+
+ private Set<Relationship> relationships;
+ private List<PropertyDescriptor> properties;
+
+ public static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+ .name("success")
+ .description("Relationship for successfully received FlowFiles")
+ .build();
+
+ public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+ .name("Listening Port")
+ .description("The Port to listen on for incoming connections")
+ .required(true)
+ .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor AUTHORIZED_DN_PATTERN = new PropertyDescriptor.Builder()
+ .name("Authorized DN Pattern")
+ .description("A Regular Expression to apply against the Distinguished Name of incoming connections. If the Pattern does not match the DN, the connection will be refused.")
+ .required(true)
+ .defaultValue(".*")
+ .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor MAX_UNCONFIRMED_TIME = new PropertyDescriptor.Builder()
+ .name("Max Unconfirmed Flowfile Time")
+ .description("The maximum amount of time to wait for a FlowFile to be confirmed before it is removed from the cache")
+ .required(true)
+ .defaultValue("60 secs")
+ .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor MAX_DATA_RATE = new PropertyDescriptor.Builder()
+ .name("Max Data to Receive per Second")
+ .description("The maximum amount of data to receive per second; this allows the bandwidth to be throttled to a specified data rate; if not specified, the data rate is not throttled")
+ .required(false)
+ .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+ .name("SSL Context Service")
+ .description("The Controller Service to use in order to obtain an SSL Context")
+ .required(false)
+ .identifiesControllerService(SSLContextService.class)
+ .build();
+ public static final PropertyDescriptor HEADERS_AS_ATTRIBUTES_REGEX = new PropertyDescriptor.Builder()
+ .name("HTTP Headers to receive as Attributes (Regex)")
+ .description("Specifies the Regular Expression that determines the names of HTTP Headers that should be passed along as FlowFile attributes")
+ .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+ .required(false)
+ .build();
+
+ public static final String URI = "/contentListener";
+ public static final String CONTEXT_ATTRIBUTE_PROCESSOR = "processor";
+ public static final String CONTEXT_ATTRIBUTE_LOGGER = "logger";
+ public static final String CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER = "sessionFactoryHolder";
++ public static final String CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER = "processContextHolder";
+ public static final String CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN = "authorityPattern";
+ public static final String CONTEXT_ATTRIBUTE_HEADER_PATTERN = "headerPattern";
+ public static final String CONTEXT_ATTRIBUTE_FLOWFILE_MAP = "flowFileMap";
+ public static final String CONTEXT_ATTRIBUTE_STREAM_THROTTLER = "streamThrottler";
+
+ private volatile Server server = null;
+ private final ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap = new ConcurrentHashMap<String, FlowFileEntryTimeWrapper>();
+ private final AtomicReference<ProcessSessionFactory> sessionFactoryReference = new AtomicReference<>();
+
+ @Override
+ protected void init(final ProcessorInitializationContext context) {
+ final Set<Relationship> relationships = new HashSet<>();
+ relationships.add(RELATIONSHIP_SUCCESS);
+ this.relationships = Collections.unmodifiableSet(relationships);
+
+ final List<PropertyDescriptor> descriptors = new ArrayList<>();
+ descriptors.add(PORT);
+ descriptors.add(MAX_DATA_RATE);
+ descriptors.add(SSL_CONTEXT_SERVICE);
+ descriptors.add(AUTHORIZED_DN_PATTERN);
+ descriptors.add(MAX_UNCONFIRMED_TIME);
+ descriptors.add(HEADERS_AS_ATTRIBUTES_REGEX);
+ this.properties = Collections.unmodifiableList(descriptors);
+ }
+
+ @Override
+ public Set<Relationship> getRelationships() {
+ return relationships;
+ }
+
+ @Override
+ protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+ return properties;
+ }
+
+ @OnStopped
+ public void shutdownHttpServer() {
+ final Server toShutdown = this.server;
+ if (toShutdown == null) {
+ return;
+ }
+
+ try {
+ toShutdown.stop();
+ toShutdown.destroy();
+ } catch (final Exception ex) {
+ getLogger().warn("unable to cleanly shutdown embedded server due to {}", new Object[]{ex});
+ this.server = null;
+ }
+ }
+
+ private void createHttpServerFromService(final ProcessContext context) throws Exception {
+ final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+ final Double maxBytesPerSecond = context.getProperty(MAX_DATA_RATE).asDataSize(DataUnit.B);
+ final StreamThrottler streamThrottler = (maxBytesPerSecond == null) ? null : new LeakyBucketStreamThrottler(maxBytesPerSecond.intValue());
+
+ final boolean needClientAuth = sslContextService == null ? false : sslContextService.getTrustStoreFile() != null;
+
+ final SslContextFactory contextFactory = new SslContextFactory();
+ contextFactory.setNeedClientAuth(needClientAuth);
+
+ if (needClientAuth) {
+ contextFactory.setTrustStorePath(sslContextService.getTrustStoreFile());
+ contextFactory.setTrustStoreType(sslContextService.getTrustStoreType());
+ contextFactory.setTrustStorePassword(sslContextService.getTrustStorePassword());
+ }
+
+ final String keystorePath = sslContextService == null ? null : sslContextService.getKeyStoreFile();
+ if (keystorePath != null) {
+ final String keystorePassword = sslContextService.getKeyStorePassword();
+ final String keyStoreType = sslContextService.getKeyStoreType();
+
+ contextFactory.setKeyStorePath(keystorePath);
+ contextFactory.setKeyManagerPassword(keystorePassword);
+ contextFactory.setKeyStorePassword(keystorePassword);
+ contextFactory.setKeyStoreType(keyStoreType);
+ }
+
+ // thread pool for the jetty instance
+ final QueuedThreadPool threadPool = new QueuedThreadPool();
+ threadPool.setName(String.format("%s (%s) Web Server", getClass().getSimpleName(), getIdentifier()));
+
+ // create the server instance
+ final Server server = new Server(threadPool);
+
+ // get the configured port
+ final int port = context.getProperty(PORT).asInteger();
+
+ final ServerConnector connector;
+ final HttpConfiguration httpConfiguration = new HttpConfiguration();
+ if (keystorePath == null) {
+ // create the connector
+ connector = new ServerConnector(server, new HttpConnectionFactory(httpConfiguration));
+ } else {
+ // configure the ssl connector
+ httpConfiguration.setSecureScheme("https");
+ httpConfiguration.setSecurePort(port);
+ httpConfiguration.addCustomizer(new SecureRequestCustomizer());
+
+ // build the connector
+ connector = new ServerConnector(server,
+ new SslConnectionFactory(contextFactory, "http/1.1"),
+ new HttpConnectionFactory(httpConfiguration));
+ }
+
+ // configure the port
+ connector.setPort(port);
+
+ // add the connector to the server
+ server.setConnectors(new Connector[]{connector});
+
+ final ServletContextHandler contextHandler = new ServletContextHandler(server, "/", true, (keystorePath != null));
+ for (final Class<? extends Servlet> cls : getServerClasses()) {
+ final Path path = cls.getAnnotation(Path.class);
+ if (path == null) {
+ contextHandler.addServlet(cls, "/*");
+ } else {
+ contextHandler.addServlet(cls, path.value());
+ }
+ }
+ contextHandler.setAttribute(CONTEXT_ATTRIBUTE_PROCESSOR, this);
+ contextHandler.setAttribute(CONTEXT_ATTRIBUTE_LOGGER, getLogger());
+ contextHandler.setAttribute(CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER, sessionFactoryReference);
++ contextHandler.setAttribute(CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER, context);
+ contextHandler.setAttribute(CONTEXT_ATTRIBUTE_FLOWFILE_MAP, flowFileMap);
+ contextHandler.setAttribute(CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN, Pattern.compile(context.getProperty(AUTHORIZED_DN_PATTERN).getValue()));
+ contextHandler.setAttribute(CONTEXT_ATTRIBUTE_STREAM_THROTTLER, streamThrottler);
+
+ if (context.getProperty(HEADERS_AS_ATTRIBUTES_REGEX).isSet()) {
+ contextHandler.setAttribute(CONTEXT_ATTRIBUTE_HEADER_PATTERN, Pattern.compile(context.getProperty(HEADERS_AS_ATTRIBUTES_REGEX).getValue()));
+ }
+ server.start();
+
+ this.server = server;
+ }
+
+ @OnScheduled
+ public void createHttpServer(final ProcessContext context) throws Exception {
+ createHttpServerFromService(context);
+ }
+
+ protected Set<Class<? extends Servlet>> getServerClasses() {
+ final Set<Class<? extends Servlet>> s = new HashSet<>();
+ s.add(ListenHTTPServlet.class);
+ s.add(ContentAcknowledgmentServlet.class);
+ return s;
+ }
+
+ private Set<String> findOldFlowFileIds(final ProcessContext ctx) {
+ final Set<String> old = new HashSet<>();
+
+ final long expiryMillis = ctx.getProperty(MAX_UNCONFIRMED_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
+ final long cutoffTime = System.currentTimeMillis() - expiryMillis;
+ for (final Map.Entry<String, FlowFileEntryTimeWrapper> entry : flowFileMap.entrySet()) {
+ final FlowFileEntryTimeWrapper wrapper = entry.getValue();
+ if (wrapper != null && wrapper.getEntryTime() < cutoffTime) {
+ old.add(entry.getKey());
+ }
+ }
+
+ return old;
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) {
+ sessionFactoryReference.compareAndSet(null, sessionFactory);
+
+ for (final String id : findOldFlowFileIds(context)) {
+ final FlowFileEntryTimeWrapper wrapper = flowFileMap.remove(id);
+ if (wrapper != null) {
+ getLogger().warn("failed to received acknowledgment for HOLD with ID {}; rolling back session", new Object[]{id});
+ wrapper.session.rollback();
+ }
+ }
+
+ context.yield();
+ }
+
+ public static class FlowFileEntryTimeWrapper {
+
+ private final Set<FlowFile> flowFiles;
+ private final long entryTime;
+ private final ProcessSession session;
+
+ public FlowFileEntryTimeWrapper(final ProcessSession session, final Set<FlowFile> flowFiles, final long entryTime) {
+ this.flowFiles = flowFiles;
+ this.entryTime = entryTime;
+ this.session = session;
+ }
+
+ public Set<FlowFile> getFlowFiles() {
+ return flowFiles;
+ }
+
+ public long getEntryTime() {
+ return entryTime;
+ }
+
+ public ProcessSession getSession() {
+ return session;
+ }
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
index 0000000,65b3c66..43d8395
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/ListenUDP.java
@@@ -1,0 -1,627 +1,627 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processors.standard;
+
+ import java.io.IOException;
+ import java.net.InetAddress;
+ import java.net.NetworkInterface;
+ import java.net.SocketException;
+ import java.net.UnknownHostException;
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.Enumeration;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Set;
+ import java.util.concurrent.BlockingQueue;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.ExecutionException;
+ import java.util.concurrent.ExecutorService;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.LinkedBlockingQueue;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.concurrent.locks.Lock;
+ import java.util.concurrent.locks.ReentrantLock;
+
+ 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.flowfile.FlowFile;
+ import org.apache.nifi.io.nio.BufferPool;
+ import org.apache.nifi.io.nio.ChannelListener;
+ import org.apache.nifi.io.nio.consumer.StreamConsumer;
+ import org.apache.nifi.io.nio.consumer.StreamConsumerFactory;
+ import org.apache.nifi.logging.ProcessorLog;
+ import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+ import org.apache.nifi.processor.DataUnit;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.annotation.CapabilityDescription;
+ import org.apache.nifi.processor.annotation.OnScheduled;
+ import org.apache.nifi.processor.annotation.OnStopped;
+ import org.apache.nifi.processor.annotation.OnUnscheduled;
+ import org.apache.nifi.processor.annotation.Tags;
+ import org.apache.nifi.processor.annotation.TriggerWhenEmpty;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.processor.util.StandardValidators;
+ import org.apache.nifi.processors.standard.util.UDPStreamConsumer;
+ import org.apache.nifi.util.Tuple;
+
+ import org.apache.commons.lang3.StringUtils;
+
+ /**
+ * <p>
+ * This processor listens for Datagram Packets on a given port and concatenates
+ * the contents of those packets together generating flow files roughly as often
+ * as the internal buffer fills up or until no more data is currently available.
+ * </p>
+ *
+ * <p>
+ * This processor has the following required properties:
+ * <ul>
+ * <li><b>Port</b> - The port to listen on for data packets. Must be known by
+ * senders of Datagrams.</li>
+ * <li><b>Receive Timeout</b> - The time out period when waiting to receive data
+ * from the socket. Specify units. Default is 5 secs.</li>
+ * <li><b>Max Buffer Size</b> - Determines the size each receive buffer may be.
+ * Specify units. Default is 1 MB.</li>
+ * <li><b>FlowFile Size Trigger</b> - Determines the (almost) upper bound size
+ * at which a flow file would be generated. A flow file will get made even if
+ * this value isn't reached if there is no more data streaming in and this value
+ * may be exceeded by the size of a single packet. Specify units. Default is 1
+ * MB.</li>
+ * <li><b>Max size of UDP Buffer</b> - The maximum UDP buffer size that should
+ * be used. This is a suggestion to the Operating System to indicate how big the
+ * udp socket buffer should be. Specify units. Default is 1 MB.")</li>
+ * <li><b>Receive Buffer Count</b> - Number of receiving buffers to be used to
+ * accept data from the socket. Higher numbers means more ram is allocated but
+ * can allow better throughput. Default is 4.</li>
+ * <li><b>Channel Reader Interval</b> - Scheduling interval for each read
+ * channel. Specify units. Default is 50 millisecs.</li>
+ * <li><b>FlowFiles Per Session</b> - The number of flow files per session.
+ * Higher number is more efficient, but will lose more data if a problem occurs
+ * that causes a rollback of a session. Default is 10</li>
+ * </ul>
+ * </p>
+ *
+ * This processor has the following optional properties:
+ * <ul>
+ * <li><b>Sending Host</b> - IP, or name, of a remote host. Only Datagrams from
+ * the specified Sending Host Port and this host will be accepted. Improves
+ * Performance. May be a system property or an environment variable.</li>
+ * <li><b>Sending Host Port</b> - Port being used by remote host to send
+ * Datagrams. Only Datagrams from the specified Sending Host and this port will
+ * be accepted. Improves Performance. May be a system property or an environment
+ * variable.</li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ * The following relationships are required:
+ * <ul>
+ * <li><b>success</b> - Where to route newly created flow files.</li>
+ * </ul>
+ * </p>
+ *
+ */
+ @TriggerWhenEmpty
+ @Tags({"ingest", "udp", "listen", "source"})
+ @CapabilityDescription("Listens for Datagram Packets on a given port and concatenates the contents of those packets "
+ + "together generating flow files")
+ public class ListenUDP extends AbstractSessionFactoryProcessor {
+
+ private static final Set<Relationship> relationships;
+ private static final List<PropertyDescriptor> properties;
+
+ // relationships.
+ public static final Relationship RELATIONSHIP_SUCCESS = new Relationship.Builder()
+ .name("success")
+ .description("Connection which contains concatenated Datagram Packets")
+ .build();
+
+ static {
+ Set<Relationship> rels = new HashSet<>();
+ rels.add(RELATIONSHIP_SUCCESS);
+ relationships = Collections.unmodifiableSet(rels);
+ }
+ // required properties.
+ public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+ .name("Port")
+ .description("Port to listen on. Must be known by senders of Datagrams.")
+ .addValidator(StandardValidators.PORT_VALIDATOR)
+ .required(true)
+ .build();
+
+ public static final PropertyDescriptor RECV_TIMEOUT = new PropertyDescriptor.Builder()
+ .name("Receive Timeout")
+ .description("The time out period when waiting to receive data from the socket. Specify units.")
+ .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+ .defaultValue("5 secs")
+ .required(true)
+ .build();
+
+ public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
+ .name("Max Buffer Size")
+ .description("Determines the size each receive buffer may be")
+ .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+ .defaultValue("1 MB")
+ .required(true)
+ .build();
+
+ public static final PropertyDescriptor FLOW_FILE_SIZE_TRIGGER = new PropertyDescriptor.Builder()
+ .name("FlowFile Size Trigger")
+ .description("Determines the (almost) upper bound size at which a flow file would be generated.")
+ .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+ .defaultValue("1 MB")
+ .required(true)
+ .build();
+
+ public static final PropertyDescriptor MAX_UDP_BUFFER = new PropertyDescriptor.Builder()
+ .name("Max size of UDP Buffer")
+ .description("The maximum UDP buffer size that should be used. This is a suggestion to the Operating System "
+ + "to indicate how big the udp socket buffer should be.")
+ .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+ .defaultValue("1 MB")
+ .required(true)
+ .build();
+
+ public static final PropertyDescriptor RECV_BUFFER_COUNT = new PropertyDescriptor.Builder()
+ .name("Receive Buffer Count")
+ .description("Number of receiving buffers to be used to accept data from the socket. Higher numbers "
+ + "means more ram is allocated but can allow better throughput.")
+ .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+ .defaultValue("4")
+ .required(true)
+ .build();
+
+ public static final PropertyDescriptor CHANNEL_READER_PERIOD = new PropertyDescriptor.Builder()
+ .name("Channel Reader Interval")
+ .description("Scheduling interval for each read channel.")
+ .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+ .defaultValue("50 ms")
+ .required(true)
+ .build();
+
+ public static final PropertyDescriptor FLOW_FILES_PER_SESSION = new PropertyDescriptor.Builder()
+ .name("FlowFiles Per Session")
+ .description("The number of flow files per session.")
+ .required(true)
+ .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+ .defaultValue("10")
+ .build();
+
+ // optional properties.
+ public static final PropertyDescriptor SENDING_HOST = new PropertyDescriptor.Builder()
+ .name("Sending Host")
+ .description("IP, or name, of a remote host. Only Datagrams from the specified Sending Host Port and this host will "
+ + "be accepted. Improves Performance. May be a system property or an environment variable.")
+ .addValidator(new HostValidator())
+ .expressionLanguageSupported(true)
+ .build();
+
+ public static final PropertyDescriptor SENDING_HOST_PORT = new PropertyDescriptor.Builder()
+ .name("Sending Host Port")
+ .description("Port being used by remote host to send Datagrams. Only Datagrams from the specified Sending Host and "
+ + "this port will be accepted. Improves Performance. May be a system property or an environment variable.")
+ .addValidator(StandardValidators.PORT_VALIDATOR)
+ .expressionLanguageSupported(true)
+ .build();
+
+ private static final Set<String> interfaceSet = new HashSet<>();
+
+ static {
+ try {
+ final Enumeration<NetworkInterface> interfaceEnum
+ = NetworkInterface.getNetworkInterfaces();
+ while (interfaceEnum.hasMoreElements()) {
+ final NetworkInterface ifc = interfaceEnum.nextElement();
+ interfaceSet.add(ifc.getName());
+ }
+ } catch (SocketException e) {
+ }
+ }
+ 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 for UDP Datagrams to a specific LAN."
+ + "May be a system property or an environment variable.")
+ .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;
+ try {
+ AttributeExpression ae = context.newExpressionLanguageCompiler().compile(input);
+ String realValue = ae.evaluate();
+ if (interfaceSet.contains(realValue.toLowerCase())) {
+ return result;
+ }
+
+ message = realValue + " is not a valid network name. Valid names are " + interfaceSet.toString();
+
+ } 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(true)
+ .build();
+
+ static {
+ List<PropertyDescriptor> props = new ArrayList<>();
+ props.add(SENDING_HOST);
+ props.add(SENDING_HOST_PORT);
+ props.add(NETWORK_INTF_NAME);
+ props.add(CHANNEL_READER_PERIOD);
+ props.add(FLOW_FILE_SIZE_TRIGGER);
+ props.add(MAX_BUFFER_SIZE);
+ props.add(MAX_UDP_BUFFER);
+ props.add(PORT);
+ props.add(RECV_BUFFER_COUNT);
+ props.add(FLOW_FILES_PER_SESSION);
+ props.add(RECV_TIMEOUT);
+ properties = Collections.unmodifiableList(props);
+ }
+ // defaults
+ public static final int DEFAULT_LISTENING_THREADS = 2;
+ // lock used to protect channelListener
+ private final Lock lock = new ReentrantLock();
+ private volatile ChannelListener channelListener = null;
+ private final BlockingQueue<Tuple<ProcessSession, List<FlowFile>>> flowFilesPerSessionQueue = new LinkedBlockingQueue<>();
+ private final List<FlowFile> newFlowFiles = new ArrayList<>();
+ private final AtomicReference<UDPStreamConsumer> consumerRef = new AtomicReference<>();
+ private final AtomicBoolean stopping = new AtomicBoolean(false);
+ private final AtomicReference<ProcessSessionFactory> sessionFactoryRef = new AtomicReference<>();
+ private final ExecutorService consumerExecutorService = Executors.newSingleThreadExecutor();
+ private final AtomicReference<Future<Tuple<ProcessSession, List<FlowFile>>>> consumerFutureRef = new AtomicReference<>();
+ private final AtomicBoolean resetChannelListener = new AtomicBoolean(false);
+ // instance attribute for provenance receive event generation
+ private volatile String sendingHost;
+
+ @Override
+ public Set<Relationship> getRelationships() {
+ return relationships;
+ }
+
+ @Override
+ protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+ return properties;
+ }
+
+ /**
+ * Create the ChannelListener and a thread that causes the Consumer to
+ * create flow files.
+ *
+ * @param context
+ * @throws IOException
+ */
+ @OnScheduled
+ public void initializeChannelListenerAndConsumerProcessing(final ProcessContext context) throws IOException {
+ getChannelListener(context);
+ stopping.set(false);
+ Future<Tuple<ProcessSession, List<FlowFile>>> consumerFuture = consumerExecutorService
+ .submit(new Callable<Tuple<ProcessSession, List<FlowFile>>>() {
+
+ @Override
+ public Tuple<ProcessSession, List<FlowFile>> call() {
+ final int maxFlowFilesPerSession = context.getProperty(FLOW_FILES_PER_SESSION).asInteger();
+ final long channelReaderIntervalMSecs = context.getProperty(CHANNEL_READER_PERIOD).asTimePeriod(TimeUnit.MILLISECONDS);
+ // number of waits in 5 secs, or 1
+ final int maxWaits = (int) (channelReaderIntervalMSecs <= 1000 ? 5000 / channelReaderIntervalMSecs : 1);
+ final ProcessorLog logger = getLogger();
+ int flowFileCount = maxFlowFilesPerSession;
+ ProcessSession session = null;
+ int numWaits = 0;
+ while (!stopping.get()) {
+ UDPStreamConsumer consumer = consumerRef.get();
+ if (consumer == null || sessionFactoryRef.get() == null) {
+ try {
+ Thread.sleep(100L);
+ } catch (InterruptedException swallow) {
+ }
+ } else {
+ try {
+ // first time through, flowFileCount is maxFlowFilesPerSession so that a session
+ // is created and the consumer is updated with it.
+ if (flowFileCount == maxFlowFilesPerSession || numWaits == maxWaits) {
+ logger.debug("Have waited {} times", new Object[]{numWaits});
+ numWaits = 0;
+ if (session != null) {
+ Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = new Tuple<ProcessSession, List<FlowFile>>(
+ session,
+ new ArrayList<>(newFlowFiles));
+ newFlowFiles.clear();
+ flowFilesPerSessionQueue.add(flowFilesPerSession);
+ }
+ session = sessionFactoryRef.get().createSession();
+ consumer.setSession(session);
+ flowFileCount = 0;
+ }
+ // this will throttle the processing of the received datagrams. If there are no more
+ // buffers to read into because none have been returned to the pool via consumer.process(),
+ // then the desired back pressure on the channel is created.
- if (session.getAvailableRelationships().size() > 0) {
++ if (context.getAvailableRelationships().size() > 0) {
+ consumer.process();
+ if (flowFileCount == newFlowFiles.size()) {
+ // no new datagrams received, need to throttle this thread back so it does
+ // not consume all cpu...but don't want to cause back pressure on the channel
+ // so the sleep time is same as the reader interval
+ // If have done this for approx. 5 secs, assume datagram sender is down. So, push
+ // out the remaining flow files (see numWaits == maxWaits above)
+ Thread.sleep(channelReaderIntervalMSecs);
+ if (flowFileCount > 0) {
+ numWaits++;
+ }
+ } else {
+ flowFileCount = newFlowFiles.size();
+ }
+ } else {
+ logger.debug("Creating back pressure...no available destinations");
+ Thread.sleep(1000L);
+ }
+ } catch (final IOException ioe) {
+ logger.error("Unable to fully process consumer {}", new Object[]{consumer}, ioe);
+ } catch (InterruptedException e) {
+ // don't care
+ } finally {
+ if (consumer.isConsumerFinished()) {
+ logger.info("Consumer {} was closed and is finished", new Object[]{consumer});
+ consumerRef.set(null);
+ disconnect();
+ if (!stopping.get()) {
+ resetChannelListener.set(true);
+ }
+ }
+ }
+ }
+ }
+ // when shutting down, need consumer to drain rest of cached buffers and clean up.
+ // prior to getting here, the channelListener was shutdown
+ UDPStreamConsumer consumer;
+ while ((consumer = consumerRef.get()) != null && !consumer.isConsumerFinished()) {
+ try {
+ consumer.process();
+ } catch (IOException swallow) {
+ // if this is blown...consumer.isConsumerFinished will be true
+ }
+ }
+ Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = new Tuple<ProcessSession, List<FlowFile>>(session,
+ new ArrayList<>(newFlowFiles));
+ return flowFilesPerSession;
+ }
+ });
+ consumerFutureRef.set(consumerFuture);
+ }
+
+ private void disconnect() {
+ if (lock.tryLock()) {
+ try {
+ if (channelListener != null) {
+ getLogger().debug("Shutting down channel listener {}", new Object[]{channelListener});
+ channelListener.shutdown(500L, TimeUnit.MILLISECONDS);
+ channelListener = null;
+ }
+ } finally {
+ lock.unlock();
+ }
+ }
+ }
+
+ private void getChannelListener(final ProcessContext context) throws IOException {
+ if (lock.tryLock()) {
+ try {
+ ProcessorLog logger = getLogger();
+ logger.debug("Instantiating a new channel listener");
+ final int port = context.getProperty(PORT).asInteger();
+ final int bufferCount = context.getProperty(RECV_BUFFER_COUNT).asInteger();
+ final Double bufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B);
+ final Double rcvBufferSize = context.getProperty(MAX_UDP_BUFFER).asDataSize(DataUnit.B);
+ sendingHost = context.getProperty(SENDING_HOST).evaluateAttributeExpressions().getValue();
+ final Integer sendingHostPort = context.getProperty(SENDING_HOST_PORT).evaluateAttributeExpressions().asInteger();
+ final String nicIPAddressStr = context.getProperty(NETWORK_INTF_NAME).evaluateAttributeExpressions().getValue();
+ final Double flowFileSizeTrigger = context.getProperty(FLOW_FILE_SIZE_TRIGGER).asDataSize(DataUnit.B);
+ final int recvTimeoutMS = context.getProperty(RECV_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
+ final StreamConsumerFactory consumerFactory = new StreamConsumerFactory() {
+
+ @Override
+ public StreamConsumer newInstance(final String streamId) {
+ final UDPStreamConsumer consumer = new UDPStreamConsumer(streamId, newFlowFiles, flowFileSizeTrigger.intValue(), getLogger());
+ consumerRef.set(consumer);
+ return consumer;
+ }
+ };
+ final int readerMilliseconds = context.getProperty(CHANNEL_READER_PERIOD).asTimePeriod(TimeUnit.MILLISECONDS).intValue();
+ final BufferPool bufferPool = new BufferPool(bufferCount, bufferSize.intValue(), false, Integer.MAX_VALUE);
+ channelListener = new ChannelListener(DEFAULT_LISTENING_THREADS, consumerFactory, bufferPool, recvTimeoutMS, TimeUnit.MILLISECONDS);
+ // specifying a sufficiently low number for each stream to be fast enough though very efficient
+ channelListener.setChannelReaderSchedulingPeriod(readerMilliseconds, TimeUnit.MILLISECONDS);
+ InetAddress nicIPAddress = null;
+ if (null != nicIPAddressStr) {
+ NetworkInterface netIF = NetworkInterface.getByName(nicIPAddressStr);
+ nicIPAddress = netIF.getInetAddresses().nextElement();
+ }
+ channelListener.addDatagramChannel(nicIPAddress, port, rcvBufferSize.intValue(), sendingHost, sendingHostPort);
+ logger.info("Registered service and initialized UDP socket listener. Now listening on port " + port + "...");
+ } finally {
+ lock.unlock();
+ }
+ }
+ }
+
+ @Override
+ protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+ Collection<ValidationResult> result = new ArrayList<>();
+ String sendingHost = validationContext.getProperty(SENDING_HOST).getValue();
+ String sendingPort = validationContext.getProperty(SENDING_HOST_PORT).getValue();
+ if (StringUtils.isBlank(sendingHost) && StringUtils.isNotBlank(sendingPort)) {
+ result.add(new ValidationResult.Builder()
+ .subject(SENDING_HOST.getName())
+ .valid(false)
+ .explanation("Must specify Sending Host when specifying Sending Host Port")
+ .build());
+ } else if (StringUtils.isBlank(sendingPort) && StringUtils.isNotBlank(sendingHost)) {
+ result.add(new ValidationResult.Builder()
+ .subject(SENDING_HOST_PORT.getName())
+ .valid(false)
+ .explanation("Must specify Sending Host Port when specifying Sending Host")
+ .build());
+ }
+ return result;
+ }
+
+ @Override
+ public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
+ final ProcessorLog logger = getLogger();
+ sessionFactoryRef.compareAndSet(null, sessionFactory);
+ if (resetChannelListener.getAndSet(false) && !stopping.get()) {
+ try {
+ getChannelListener(context);
+ } catch (IOException e) {
+ logger.error("Tried to reset Channel Listener and failed due to:", e);
+ resetChannelListener.set(true);
+ }
+ }
+
+ transferFlowFiles();
+ }
+
+ private boolean transferFlowFiles() {
+ final ProcessorLog logger = getLogger();
+ ProcessSession session;
+ Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = null;
+ boolean transferred = false;
+ try {
+ flowFilesPerSession = flowFilesPerSessionQueue.poll(100L, TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ }
+ if (flowFilesPerSession != null) {
+ session = flowFilesPerSession.getKey();
+ List<FlowFile> flowFiles = flowFilesPerSession.getValue();
+ String sourceSystem = sendingHost == null ? "Unknown" : sendingHost;
+ try {
+ for (FlowFile flowFile : flowFiles) {
+ session.getProvenanceReporter().receive(flowFile, sourceSystem);
+ session.transfer(flowFile, RELATIONSHIP_SUCCESS);
+ }
+ logger.info("Transferred flow files {} to success", new Object[]{flowFiles});
+ transferred = true;
+
+ // need to check for erroneous flow files in input queue
+ List<FlowFile> existingFlowFiles = session.get(10);
+ for (FlowFile existingFlowFile : existingFlowFiles) {
+ if (existingFlowFile != null && existingFlowFile.getSize() > 0) {
+ session.transfer(existingFlowFile, RELATIONSHIP_SUCCESS);
+ logger.warn("Found flow file in input queue (shouldn't have). Transferred flow file {} to success",
+ new Object[]{existingFlowFile});
+ } else if (existingFlowFile != null) {
+ session.remove(existingFlowFile);
+ logger.warn("Found empty flow file in input queue (shouldn't have). Removed flow file {}", new Object[]{existingFlowFile});
+ }
+ }
+ session.commit();
+ } catch (Throwable t) {
+ session.rollback();
+ logger.error("Failed to transfer flow files or commit session...rolled back", t);
+ throw t;
+ }
+ }
+ return transferred;
+ }
+
+ @OnUnscheduled
+ public void stopping() {
+ getLogger().debug("Stopping Processor");
+ disconnect();
+ stopping.set(true);
+ Future<Tuple<ProcessSession, List<FlowFile>>> future;
+ Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession;
+ if ((future = consumerFutureRef.getAndSet(null)) != null) {
+ try {
+ flowFilesPerSession = future.get();
+ if (flowFilesPerSession.getValue().size() > 0) {
+ getLogger().debug("Draining remaining flow Files when stopping");
+ flowFilesPerSessionQueue.add(flowFilesPerSession);
+ } else {
+ // need to close out the session that has no flow files
+ flowFilesPerSession.getKey().commit();
+ }
+ } catch (InterruptedException | ExecutionException e) {
+ getLogger().error("Failure in cleaning up!", e);
+ }
+ boolean moreFiles = true;
+ while (moreFiles) {
+ try {
+ moreFiles = transferFlowFiles();
+ } catch (Throwable t) {
+ getLogger().error("Problem transferring cached flowfiles", t);
+ }
+ }
+ }
+ }
+
+ @OnStopped
+ public void stopped() {
+ sessionFactoryRef.set(null);
+ }
+
+ public static class HostValidator implements Validator {
+
+ @Override
+ public ValidationResult validate(String subject, String input, ValidationContext context) {
+ try {
+ InetAddress.getByName(input);
+ return new ValidationResult.Builder()
+ .subject(subject)
+ .valid(true)
+ .input(input)
+ .build();
+ } catch (final UnknownHostException e) {
+ return new ValidationResult.Builder()
+ .subject(subject)
+ .valid(false)
+ .input(input)
+ .explanation("Unknown host: " + e)
+ .build();
+ }
+ }
+
+ }
+
+ }
[6/8] incubator-nifi git commit: Merge branch 'develop' of
https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
index 0000000,3d3e854..dcb461c
mode 000000,100644..100644
--- a/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
+++ b/nifi/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java
@@@ -1,0 -1,2689 +1,2684 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.controller.repository;
+
+ import java.io.ByteArrayInputStream;
+ import java.io.EOFException;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.nio.file.Files;
+ import java.nio.file.Path;
+ import java.util.ArrayList;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.LinkedHashSet;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.NoSuchElementException;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicLong;
+ import java.util.regex.Pattern;
+
+ import org.apache.nifi.connectable.Connectable;
+ import org.apache.nifi.connectable.Connection;
+ import org.apache.nifi.controller.FlowFileQueue;
+ import org.apache.nifi.controller.ProcessorNode;
+ import org.apache.nifi.controller.repository.claim.ContentClaim;
+ import org.apache.nifi.controller.repository.io.ByteCountingInputStream;
+ import org.apache.nifi.controller.repository.io.ByteCountingOutputStream;
+ import org.apache.nifi.controller.repository.io.DisableOnCloseOutputStream;
+ import org.apache.nifi.controller.repository.io.FlowFileAccessInputStream;
+ import org.apache.nifi.controller.repository.io.FlowFileAccessOutputStream;
+ import org.apache.nifi.controller.repository.io.LimitedInputStream;
+ import org.apache.nifi.controller.repository.io.LongHolder;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.stream.io.BufferedOutputStream;
+ import org.apache.nifi.stream.io.NonCloseableInputStream;
+ import org.apache.nifi.stream.io.StreamUtils;
+ import org.apache.nifi.processor.DataUnit;
+ import org.apache.nifi.processor.FlowFileFilter;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.QueueSize;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.exception.FlowFileAccessException;
+ import org.apache.nifi.processor.exception.FlowFileHandlingException;
+ import org.apache.nifi.processor.exception.MissingFlowFileException;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.processor.io.InputStreamCallback;
+ import org.apache.nifi.processor.io.OutputStreamCallback;
+ import org.apache.nifi.processor.io.StreamCallback;
+ import org.apache.nifi.provenance.ProvenanceEventBuilder;
+ import org.apache.nifi.provenance.ProvenanceEventRecord;
+ import org.apache.nifi.provenance.ProvenanceEventRepository;
+ import org.apache.nifi.provenance.ProvenanceEventType;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+ import org.apache.nifi.provenance.StandardProvenanceEventRecord;
+ import org.apache.nifi.util.NiFiProperties;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+
+ /**
+ * <p>
+ * Provides a ProcessSession that ensures all accesses, changes and transfers
+ * occur in an atomic manner for all FlowFiles including their contents and
+ * attributes</p>
+ * <p>
+ * NOT THREAD SAFE</p>
+ * <p/>
+ * @author none
+ */
+ public final class StandardProcessSession implements ProcessSession, ProvenanceEventEnricher {
+
+ private static final AtomicLong idGenerator = new AtomicLong(0L);
+
+ // determines how many things must be transferred, removed, modified in order to avoid logging the FlowFile ID's on commit/rollback
+ public static final int VERBOSE_LOG_THRESHOLD = 10;
+ private static final long MAX_APPENDABLE_CLAIM_SIZE = DataUnit.parseDataSize(
+ NiFiProperties.getInstance().getMaxAppendableClaimSize(), DataUnit.B).longValue();
+ private static final int MAX_FLOWFILES_PER_CLAIM = NiFiProperties.getInstance().getMaxFlowFilesPerClaim();
+
+ public static final String DEFAULT_FLOWFILE_PATH = "./";
+
+ private static final Logger LOG = LoggerFactory.getLogger(StandardProcessSession.class);
+ private static final Logger claimLog = LoggerFactory.getLogger(StandardProcessSession.class.getSimpleName() + ".claims");
+
+ private final Map<FlowFileRecord, StandardRepositoryRecord> records = new HashMap<>();
+ private final Map<Connection, StandardFlowFileEvent> connectionCounts = new HashMap<>();
+ private final Map<Connection, Set<FlowFileRecord>> unacknowledgedFlowFiles = new HashMap<>();
+ private final Map<String, Long> localCounters = new HashMap<>();
+ private final Map<String, Long> globalCounters = new HashMap<>();
+ private final Map<ContentClaim, ByteCountingOutputStream> appendableStreams = new HashMap<>();
+ private final ProcessContext context;
+ private final Set<FlowFile> recursionSet = new HashSet<>();//set used to track what is currently being operated on to prevent logic failures if recursive calls occurring
+ private final Set<Path> deleteOnCommit = new HashSet<>();
+ private final long sessionId;
+ private final String connectableDescription;
+
+ private final Set<String> removedFlowFiles = new HashSet<>();
+ private final Set<String> createdFlowFiles = new HashSet<>();
+
+ private final StandardProvenanceReporter provenanceReporter;
+
+ private int removedCount = 0; // number of flowfiles removed in this session
+ private long removedBytes = 0L; // size of all flowfiles removed in this session
+ private LongHolder bytesRead = new LongHolder(0L);
+ private LongHolder bytesWritten = new LongHolder(0L);
+ private int flowFilesIn = 0, flowFilesOut = 0;
+ private long contentSizeIn = 0L, contentSizeOut = 0L;
+ private int writeRecursionLevel = 0;
+
+ private ContentClaim currentWriteClaim = null;
+ private OutputStream currentWriteClaimStream = null;
+ private long currentWriteClaimSize = 0L;
+ private int currentWriteClaimFlowFileCount = 0;
+
+ private ContentClaim currentReadClaim = null;
+ private ByteCountingInputStream currentReadClaimStream = null;
+ private long processingStartTime;
+
+ // maps a FlowFile to all Provenance Events that were generated for that FlowFile.
+ // we do this so that if we generate a Fork event, for example, and then remove the event in the same
+ // Session, we will not send that event to the Provenance Repository
+ private Map<FlowFile, List<ProvenanceEventRecord>> generatedProvenanceEvents = new HashMap<>();
+
+ // when Forks are generated for a single parent, we add the Fork event to this map, with the Key being the parent
+ // so that we are able to aggregate many into a single Fork Event.
+ private Map<FlowFile, ProvenanceEventBuilder> forkEventBuilders = new HashMap<>();
+
+ private Checkpoint checkpoint = new Checkpoint();
+
+ public StandardProcessSession(final ProcessContext context) {
+ this.context = context;
+
+ final Connectable connectable = context.getConnectable();
+ final String componentType;
+
+ String description = connectable.toString();
+ switch (connectable.getConnectableType()) {
+ case PROCESSOR:
+ final ProcessorNode procNode = (ProcessorNode) connectable;
+ componentType = procNode.getProcessor().getClass().getSimpleName();
+ description = procNode.getProcessor().toString();
+ break;
+ case INPUT_PORT:
+ componentType = "Input Port";
+ break;
+ case OUTPUT_PORT:
+ componentType = "Output Port";
+ break;
+ case REMOTE_INPUT_PORT:
+ componentType = "Remote Input Port";
+ break;
+ case REMOTE_OUTPUT_PORT:
+ componentType = "Remote Output Port";
+ break;
+ case FUNNEL:
+ componentType = "Funnel";
+ break;
+ default:
+ throw new AssertionError("Connectable type is " + connectable.getConnectableType());
+ }
+
+ this.provenanceReporter = new StandardProvenanceReporter(connectable.getIdentifier(), componentType, context.getProvenanceRepository(), this);
+ this.sessionId = idGenerator.getAndIncrement();
+ this.connectableDescription = description;
+
+ LOG.trace("Session {} created for {}", this, connectableDescription);
+ processingStartTime = System.nanoTime();
+ }
+
+ public void checkpoint() {
+ if (!recursionSet.isEmpty()) {
+ throw new IllegalStateException();
+ }
+
+ if (this.checkpoint == null) {
+ this.checkpoint = new Checkpoint();
+ }
+
+ if (records.isEmpty()) {
+ LOG.trace("{} checkpointed, but no events were performed by this ProcessSession", this);
+ return;
+ }
+
+ // any drop event that is the result of an auto-terminate should happen at the very end, so we keep the
+ // records in a separate List so that they can be persisted to the Provenance Repo after all of the
+ // Processor-reported events.
+ List<ProvenanceEventRecord> autoTerminatedEvents = null;
+
+ //validate that all records have a transfer relationship for them and if so determine the destination node and clone as necessary
+ final Map<FlowFileRecord, StandardRepositoryRecord> toAdd = new HashMap<>();
+ for (final StandardRepositoryRecord record : records.values()) {
+ if (record.isMarkedForDelete()) {
+ continue;
+ }
+ final Relationship relationship = record.getTransferRelationship();
+ if (relationship == null) {
+ rollback();
+ throw new FlowFileHandlingException(record.getCurrent() + " transfer relationship not specified");
+ }
+ final List<Connection> destinations = new ArrayList<>(context.getConnections(relationship));
+ if (destinations.isEmpty() && !context.getConnectable().isAutoTerminated(relationship)) {
+ if (relationship != Relationship.SELF) {
+ rollback();
+ throw new FlowFileHandlingException(relationship + " does not have any destinations for " + context.getConnectable());
+ }
+ }
+
+ if (destinations.isEmpty() && relationship == Relationship.SELF) {
+ record.setDestination(record.getOriginalQueue());
+ } else if (destinations.isEmpty()) {
+ record.markForDelete();
+
+ if (autoTerminatedEvents == null) {
+ autoTerminatedEvents = new ArrayList<>();
+ }
+
+ final ProvenanceEventRecord dropEvent;
+ try {
+ dropEvent = provenanceReporter.generateDropEvent(record.getCurrent(), "Auto-Terminated by " + relationship.getName() + " Relationship");
+ autoTerminatedEvents.add(dropEvent);
+ } catch (final Exception e) {
+ LOG.warn("Unable to generate Provenance Event for {} on behalf of {} due to {}", record.getCurrent(), connectableDescription, e);
+ if (LOG.isDebugEnabled()) {
+ LOG.warn("", e);
+ }
+ }
+ } else {
+ final Connection finalDestination = destinations.remove(destinations.size() - 1); //remove last element
+ record.setDestination(finalDestination.getFlowFileQueue());
+ incrementConnectionInputCounts(finalDestination, record);
+
+ for (final Connection destination : destinations) { //iterate over remaining destinations and "clone" as needed
+ incrementConnectionInputCounts(destination, record);
+ final FlowFileRecord currRec = record.getCurrent();
+ final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder().fromFlowFile(currRec);
+ builder.id(context.getNextFlowFileSequence());
+
+ final String newUuid = UUID.randomUUID().toString();
+ builder.addAttribute(CoreAttributes.UUID.key(), newUuid);
+
+ final FlowFileRecord clone = builder.build();
+ final StandardRepositoryRecord newRecord = new StandardRepositoryRecord(destination.getFlowFileQueue());
+ getProvenanceReporter().clone(currRec, clone);
+
+ final ContentClaim claim = clone.getContentClaim();
+ if (claim != null) {
+ context.getContentRepository().incrementClaimaintCount(claim);
+ }
+ newRecord.setWorking(clone, CoreAttributes.UUID.key(), newUuid);
+
+ newRecord.setDestination(destination.getFlowFileQueue());
+ newRecord.setTransferRelationship(record.getTransferRelationship());
+ // put the mapping into toAdd because adding to records now will cause a ConcurrentModificationException
+ toAdd.put(clone, newRecord);
+ }
+ }
+ }
+
+ records.putAll(toAdd);
+ toAdd.clear();
+
+ checkpoint.checkpoint(this, autoTerminatedEvents);
+ resetState();
+ }
+
+ @Override
+ public void commit() {
+ checkpoint();
+ commit(this.checkpoint);
+ this.checkpoint = null;
+ }
+
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ private void commit(final Checkpoint checkpoint) {
+ final long commitStartNanos = System.nanoTime();
+
+ resetWriteClaims();
+ resetReadClaim();
+
+ final long updateProvenanceStart = System.nanoTime();
+ updateProvenanceRepo(checkpoint);
+
+ final long claimRemovalStart = System.nanoTime();
+ final long updateProvenanceNanos = claimRemovalStart - updateProvenanceStart;
+
+ // Figure out which content claims can be released.
+ // At this point, we will decrement the Claimant Count for the claims via the Content Repository.
+ // We do not actually destroy the content because otherwise, we could remove the
+ // Original Claim and crash/restart before the FlowFileRepository is updated. This will result in the FlowFile being restored such that
+ // the content claim points to the Original Claim -- which has already been removed!
+ for (final Map.Entry<FlowFileRecord, StandardRepositoryRecord> entry : checkpoint.records.entrySet()) {
+ final FlowFile flowFile = entry.getKey();
+ final StandardRepositoryRecord record = entry.getValue();
+
+ if (record.isMarkedForDelete()) {
+ // if the working claim is not the same as the original claim, we can immediately destroy the working claim
+ // because it was created in this session and is to be deleted. We don't need to wait for the FlowFile Repo to sync.
+ removeContent(record.getWorkingClaim());
+
+ if (record.getOriginalClaim() != null && !record.getOriginalClaim().equals(record.getWorkingClaim())) {
+ // if working & original claim are same, don't remove twice; we only want to remove the original
+ // if it's different from the working. Otherwise, we remove two claimant counts. This causes
+ // an issue if we only updated the FlowFile attributes.
+ removeContent(record.getOriginalClaim());
+ }
+ final long flowFileLife = System.currentTimeMillis() - flowFile.getEntryDate();
+ final Connectable connectable = context.getConnectable();
+ final Object terminator = (connectable instanceof ProcessorNode) ? ((ProcessorNode) connectable).getProcessor() : connectable;
+ LOG.info("{} terminated by {}; life of FlowFile = {} ms", new Object[]{flowFile, terminator, flowFileLife});
+ } else if (record.isWorking() && record.getWorkingClaim() != record.getOriginalClaim()) {
+ //records which have been updated - remove original if exists
+ removeContent(record.getOriginalClaim());
+ }
+ }
+
+ final long claimRemovalFinishNanos = System.nanoTime();
+ final long claimRemovalNanos = claimRemovalFinishNanos - claimRemovalStart;
+
+ // Update the FlowFile Repository
+ try {
+ final Collection<StandardRepositoryRecord> repoRecords = checkpoint.records.values();
+ context.getFlowFileRepository().updateRepository((Collection) repoRecords);
+ } catch (final IOException ioe) {
+ rollback();
+ throw new ProcessException("FlowFile Repository failed to update", ioe);
+ }
+ final long flowFileRepoUpdateFinishNanos = System.nanoTime();
+ final long flowFileRepoUpdateNanos = flowFileRepoUpdateFinishNanos - claimRemovalFinishNanos;
+
+ updateEventRepository(checkpoint);
+
+ final long updateEventRepositoryFinishNanos = System.nanoTime();
+ final long updateEventRepositoryNanos = updateEventRepositoryFinishNanos - claimRemovalFinishNanos;
+
+ // transfer the flowfiles to the connections' queues.
+ final Map<FlowFileQueue, Collection<FlowFileRecord>> recordMap = new HashMap<>();
+ for (final StandardRepositoryRecord record : checkpoint.records.values()) {
+ if (record.isMarkedForAbort() || record.isMarkedForDelete()) {
+ continue; //these don't need to be transferred
+ }
+ // record.getCurrent() will return null if this record was created in this session --
+ // in this case, we just ignore it, and it will be cleaned up by clearing the records map.
+ if (record.getCurrent() != null) {
+ Collection<FlowFileRecord> collection = recordMap.get(record.getDestination());
+ if (collection == null) {
+ collection = new ArrayList<>();
+ recordMap.put(record.getDestination(), collection);
+ }
+ collection.add(record.getCurrent());
+ }
+ }
+
+ for (final Map.Entry<FlowFileQueue, Collection<FlowFileRecord>> entry : recordMap.entrySet()) {
+ entry.getKey().putAll(entry.getValue());
+ }
+
+ final long enqueueFlowFileFinishNanos = System.nanoTime();
+ final long enqueueFlowFileNanos = enqueueFlowFileFinishNanos - updateEventRepositoryFinishNanos;
+
+ // Delete any files from disk that need to be removed.
+ for (final Path path : checkpoint.deleteOnCommit) {
+ try {
+ Files.deleteIfExists(path);
+ } catch (final IOException e) {
+ throw new FlowFileAccessException("Unable to delete " + path.toFile().getAbsolutePath(), e);
+ }
+ }
+ checkpoint.deleteOnCommit.clear();
+
+ if (LOG.isInfoEnabled()) {
+ final String sessionSummary = summarizeEvents(checkpoint);
+ if (!sessionSummary.isEmpty()) {
+ LOG.info("{} for {}, committed the following events: {}", new Object[]{this, connectableDescription, sessionSummary});
+ }
+ }
+
+ for (final Map.Entry<String, Long> entry : checkpoint.localCounters.entrySet()) {
+ adjustCounter(entry.getKey(), entry.getValue(), true);
+ }
+
+ for (final Map.Entry<String, Long> entry : checkpoint.globalCounters.entrySet()) {
+ adjustCounter(entry.getKey(), entry.getValue(), true);
+ }
+
+ acknowledgeRecords();
+ resetState();
+
+ if (LOG.isDebugEnabled()) {
+ final StringBuilder timingInfo = new StringBuilder();
+ timingInfo.append("Session commit for ").append(this).append(" [").append(connectableDescription).append("]").append(" took ");
+
+ final long commitNanos = System.nanoTime() - commitStartNanos;
+ formatNanos(commitNanos, timingInfo);
+ timingInfo.append("; FlowFile Repository Update took ");
+ formatNanos(flowFileRepoUpdateNanos, timingInfo);
+ timingInfo.append("; Claim Removal took ");
+ formatNanos(claimRemovalNanos, timingInfo);
+ timingInfo.append("; FlowFile Event Update took ");
+ formatNanos(updateEventRepositoryNanos, timingInfo);
+ timingInfo.append("; Enqueuing FlowFiles took ");
+ formatNanos(enqueueFlowFileNanos, timingInfo);
+ timingInfo.append("; Updating Provenance Event Repository took ");
+ formatNanos(updateProvenanceNanos, timingInfo);
+
+ LOG.debug(timingInfo.toString());
+ }
+ }
+
+ private void updateEventRepository(final Checkpoint checkpoint) {
+ int flowFilesReceived = 0;
+ int flowFilesSent = 0;
+ long bytesReceived = 0L;
+ long bytesSent = 0L;
+
+ for (final ProvenanceEventRecord event : checkpoint.reportedEvents) {
+ if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) {
+ continue;
+ }
+
+ switch (event.getEventType()) {
+ case SEND:
+ flowFilesSent++;
+ bytesSent += event.getFileSize();
+ break;
+ case RECEIVE:
+ flowFilesReceived++;
+ bytesReceived += event.getFileSize();
+ break;
+ default:
+ break;
+ }
+ }
+
+ try {
+ // update event repository
+ final Connectable connectable = context.getConnectable();
+ final StandardFlowFileEvent flowFileEvent = new StandardFlowFileEvent(connectable.getIdentifier());
+ flowFileEvent.setBytesRead(checkpoint.bytesRead);
+ flowFileEvent.setBytesWritten(checkpoint.bytesWritten);
+ flowFileEvent.setContentSizeIn(checkpoint.contentSizeIn);
+ flowFileEvent.setContentSizeOut(checkpoint.contentSizeOut);
+ flowFileEvent.setContentSizeRemoved(checkpoint.removedBytes);
+ flowFileEvent.setFlowFilesIn(checkpoint.flowFilesIn);
+ flowFileEvent.setFlowFilesOut(checkpoint.flowFilesOut);
+ flowFileEvent.setFlowFilesRemoved(checkpoint.removedCount);
+ flowFileEvent.setFlowFilesReceived(flowFilesReceived);
+ flowFileEvent.setBytesReceived(bytesReceived);
+ flowFileEvent.setFlowFilesSent(flowFilesSent);
+ flowFileEvent.setBytesSent(bytesSent);
+
+ long lineageMillis = 0L;
+ for (final Map.Entry<FlowFileRecord, StandardRepositoryRecord> entry : checkpoint.records.entrySet()) {
+ final FlowFile flowFile = entry.getKey();
+ final long lineageDuration = System.currentTimeMillis() - flowFile.getLineageStartDate();
+ lineageMillis += lineageDuration;
+ }
+ flowFileEvent.setAggregateLineageMillis(lineageMillis);
+
+ context.getFlowFileEventRepository().updateRepository(flowFileEvent);
+
+ for (final FlowFileEvent connectionEvent : checkpoint.connectionCounts.values()) {
+ context.getFlowFileEventRepository().updateRepository(connectionEvent);
+ }
+ } catch (final IOException ioe) {
+ LOG.error("FlowFile Event Repository failed to update", ioe);
+ }
+ }
+
+ private void addEventType(final Map<String, Set<ProvenanceEventType>> map, final String id, final ProvenanceEventType eventType) {
+ Set<ProvenanceEventType> eventTypes = map.get(id);
+ if ( eventTypes == null ) {
+ eventTypes = new HashSet<>();
+ map.put(id, eventTypes);
+ }
+
+ eventTypes.add(eventType);
+ }
+
+ private void updateProvenanceRepo(final Checkpoint checkpoint) {
+ // Update Provenance Repository
+ final ProvenanceEventRepository provenanceRepo = context.getProvenanceRepository();
+
+ // We need to de-dupe the events that we've created and those reported to the provenance reporter,
+ // in case the Processor developer submitted the same events to the reporter. So we use a LinkedHashSet
+ // for this, so that we are able to ensure that the events are submitted in the proper order.
+ final Set<ProvenanceEventRecord> recordsToSubmit = new LinkedHashSet<>();
+ final Map<String, Set<ProvenanceEventType>> eventTypesPerFlowFileId = new HashMap<>();
+
+ final Set<ProvenanceEventRecord> processorGenerated = checkpoint.reportedEvents;
+
+ // We first want to submit FORK events because if the Processor is going to create events against
+ // a FlowFile, that FlowFile needs to be shown to be created first.
+ // However, if the Processor has generated a FORK event, we don't want to use the Framework-created one --
+ // we prefer to use the event generated by the Processor. We can determine this by checking if the Set of events genereated
+ // by the Processor contains any of the FORK events that we generated
+ for (final Map.Entry<FlowFile, ProvenanceEventBuilder> entry : checkpoint.forkEventBuilders.entrySet()) {
+ final ProvenanceEventBuilder builder = entry.getValue();
+ final FlowFile flowFile = entry.getKey();
+
+ updateEventContentClaims(builder, flowFile, checkpoint.records.get(flowFile));
+ final ProvenanceEventRecord event = builder.build();
+
+ if (!event.getChildUuids().isEmpty() && !isSpuriousForkEvent(event, checkpoint.removedFlowFiles) && !processorGenerated.contains(event)) {
+ recordsToSubmit.add(event);
+
+ for ( final String childUuid : event.getChildUuids() ) {
+ addEventType(eventTypesPerFlowFileId, childUuid, event.getEventType());
+ }
+ for ( final String parentUuid : event.getParentUuids() ) {
+ addEventType(eventTypesPerFlowFileId, parentUuid, event.getEventType());
+ }
+ }
+ }
+
+ // Now add any Processor-reported events.
+ for (final ProvenanceEventRecord event : processorGenerated) {
+ if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) {
+ continue;
+ }
+ if ( isSpuriousRouteEvent(event, checkpoint.records) ) {
+ continue;
+ }
+
+ // Check if the event indicates that the FlowFile was routed to the same
+ // connection from which it was pulled (and only this connection). If so, discard the event.
+ isSpuriousRouteEvent(event, checkpoint.records);
+
+ recordsToSubmit.add(event);
+ addEventType(eventTypesPerFlowFileId, event.getFlowFileUuid(), event.getEventType());
+ }
+
+ // Finally, add any other events that we may have generated.
+ for (final List<ProvenanceEventRecord> eventList : checkpoint.generatedProvenanceEvents.values()) {
+ for (final ProvenanceEventRecord event : eventList) {
+ if (isSpuriousForkEvent(event, checkpoint.removedFlowFiles)) {
+ continue;
+ }
+
+ recordsToSubmit.add(event);
+ addEventType(eventTypesPerFlowFileId, event.getFlowFileUuid(), event.getEventType());
+ }
+ }
+
+ // Check if content or attributes changed. If so, register the appropriate events.
+ for (final StandardRepositoryRecord repoRecord : checkpoint.records.values() ) {
+ final ContentClaim original = repoRecord.getOriginalClaim();
+ final ContentClaim current = repoRecord.getCurrentClaim();
+
+ boolean contentChanged = false;
+ if ( original == null && current != null ) {
+ contentChanged = true;
+ }
+ if ( original != null && current == null ) {
+ contentChanged = true;
+ }
+ if ( original != null && current != null && !original.equals(current) ) {
+ contentChanged = true;
+ }
+
+ final FlowFileRecord curFlowFile = repoRecord.getCurrent();
+ final String flowFileId = curFlowFile.getAttribute(CoreAttributes.UUID.key());
+ boolean eventAdded = false;
+
+ if (checkpoint.removedFlowFiles.contains(flowFileId)) {
+ continue;
+ }
+
+ final boolean newFlowFile = repoRecord.getOriginal() == null;
+ if ( contentChanged && !newFlowFile ) {
+ recordsToSubmit.add(provenanceReporter.build(curFlowFile, ProvenanceEventType.CONTENT_MODIFIED).build());
+ addEventType(eventTypesPerFlowFileId, flowFileId, ProvenanceEventType.CONTENT_MODIFIED);
+ eventAdded = true;
+ }
+
+ if ( checkpoint.createdFlowFiles.contains(flowFileId) ) {
+ final Set<ProvenanceEventType> registeredTypes = eventTypesPerFlowFileId.get(flowFileId);
+ boolean creationEventRegistered = false;
+ if ( registeredTypes != null ) {
+ if ( registeredTypes.contains(ProvenanceEventType.CREATE) ||
+ registeredTypes.contains(ProvenanceEventType.FORK) ||
+ registeredTypes.contains(ProvenanceEventType.JOIN) ||
+ registeredTypes.contains(ProvenanceEventType.RECEIVE) ) {
+ creationEventRegistered = true;
+ }
+ }
+
+ if ( !creationEventRegistered ) {
+ recordsToSubmit.add(provenanceReporter.build(curFlowFile, ProvenanceEventType.CREATE).build());
+ eventAdded = true;
+ }
+ }
+
+ if ( !eventAdded && !repoRecord.getUpdatedAttributes().isEmpty() ) {
+ // We generate an ATTRIBUTES_MODIFIED event only if no other event has been
+ // created for the FlowFile. We do this because all events contain both the
+ // newest and the original attributes, so generating an ATTRIBUTES_MODIFIED
+ // event is redundant if another already exists.
+ if ( !eventTypesPerFlowFileId.containsKey(flowFileId) ) {
+ recordsToSubmit.add(provenanceReporter.build(curFlowFile, ProvenanceEventType.ATTRIBUTES_MODIFIED).build());
+ addEventType(eventTypesPerFlowFileId, flowFileId, ProvenanceEventType.ATTRIBUTES_MODIFIED);
+ }
+ }
+ }
+
+ // We want to submit the 'recordsToSubmit' collection, followed by the auto-terminated events to the Provenance Repository.
+ // We want to do this with a single call to ProvenanceEventRepository#registerEvents because it may be much more efficient
+ // to do so.
+ // However, we want to modify the events in 'recordsToSubmit' to obtain the data from the most recent version of the FlowFiles
+ // (except for SEND events); see note below as to why this is
+ // Therefore, we create an Iterable that can iterate over each of these events, modifying them as needed, and returning them
+ // in the appropriate order. This prevents an unnecessary step of creating an intermediate List and adding all of those values
+ // to the List.
+ // This is done in a similar veign to how Java 8's streams work, iterating over the events and returning a processed version
+ // one-at-a-time as opposed to iterating over the entire Collection and putting the results in another Collection. However,
+ // we don't want to change the Framework to require Java 8 at this time, because it's not yet as prevalent as we would desire
+ final Map<String, FlowFileRecord> flowFileRecordMap = new HashMap<>();
+ for (final StandardRepositoryRecord repoRecord : checkpoint.records.values()) {
+ final FlowFileRecord flowFile = repoRecord.getCurrent();
+ flowFileRecordMap.put(flowFile.getAttribute(CoreAttributes.UUID.key()), flowFile);
+ }
+
+ final List<ProvenanceEventRecord> autoTermEvents = checkpoint.autoTerminatedEvents;
+ final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() {
+ final Iterator<ProvenanceEventRecord> recordsToSubmitIterator = recordsToSubmit.iterator();
+ final Iterator<ProvenanceEventRecord> autoTermIterator = autoTermEvents == null ? null : autoTermEvents.iterator();
+
+ @Override
+ public Iterator<ProvenanceEventRecord> iterator() {
+ return new Iterator<ProvenanceEventRecord>() {
+ @Override
+ public boolean hasNext() {
+ return recordsToSubmitIterator.hasNext() || (autoTermIterator != null && autoTermIterator.hasNext());
+ }
+
+ @Override
+ public ProvenanceEventRecord next() {
+ if (recordsToSubmitIterator.hasNext()) {
+ final ProvenanceEventRecord rawEvent = recordsToSubmitIterator.next();
+
+ // Update the Provenance Event Record with all of the info that we know about the event.
+ // For SEND events, we do not want to update the FlowFile info on the Event, because the event should
+ // reflect the FlowFile as it was sent to the remote system. However, for other events, we want to use
+ // the representation of the FlowFile as it is committed, as this is the only way in which it really
+ // exists in our system -- all other representations are volatile representations that have not been
+ // exposed.
+ return enrich(rawEvent, flowFileRecordMap, checkpoint.records, rawEvent.getEventType() != ProvenanceEventType.SEND);
+ } else if (autoTermIterator != null && autoTermIterator.hasNext()) {
+ return enrich(autoTermIterator.next(), flowFileRecordMap, checkpoint.records, true);
+ }
+
+ throw new NoSuchElementException();
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+ };
+
+ provenanceRepo.registerEvents(iterable);
+ }
+
+ private void updateEventContentClaims(final ProvenanceEventBuilder builder, final FlowFile flowFile, final StandardRepositoryRecord repoRecord) {
+ final ContentClaim originalClaim = repoRecord.getOriginalClaim();
+ if (originalClaim == null) {
+ builder.setCurrentContentClaim(null, null, null, null, 0L);
+ } else {
+ builder.setCurrentContentClaim(originalClaim.getContainer(), originalClaim.getSection(), originalClaim.getId(), repoRecord.getOriginal().getContentClaimOffset(), repoRecord.getOriginal().getSize());
+ builder.setPreviousContentClaim(originalClaim.getContainer(), originalClaim.getSection(), originalClaim.getId(), repoRecord.getOriginal().getContentClaimOffset(), repoRecord.getOriginal().getSize());
+ }
+ }
+
+ @Override
+ public StandardProvenanceEventRecord enrich(final ProvenanceEventRecord rawEvent, final FlowFile flowFile) {
+ final StandardRepositoryRecord repoRecord = records.get(flowFile);
+ if (repoRecord == null) {
+ throw new FlowFileHandlingException(flowFile + " is not known in this session (" + toString() + ")");
+ }
+
+ final StandardProvenanceEventRecord.Builder recordBuilder = new StandardProvenanceEventRecord.Builder().fromEvent(rawEvent);
+ if (repoRecord.getCurrent() != null && repoRecord.getCurrentClaim() != null) {
+ final ContentClaim currentClaim = repoRecord.getCurrentClaim();
+ final long currentOffset = repoRecord.getCurrentClaimOffset();
+ final long size = flowFile.getSize();
+ recordBuilder.setCurrentContentClaim(currentClaim.getContainer(), currentClaim.getSection(), currentClaim.getId(), currentOffset, size);
+ }
+
+ if (repoRecord.getOriginal() != null && repoRecord.getOriginalClaim() != null) {
+ final ContentClaim originalClaim = repoRecord.getOriginalClaim();
+ final long originalOffset = repoRecord.getOriginal().getContentClaimOffset();
+ final long originalSize = repoRecord.getOriginal().getSize();
+ recordBuilder.setPreviousContentClaim(originalClaim.getContainer(), originalClaim.getSection(), originalClaim.getId(), originalOffset, originalSize);
+ }
+
+ final FlowFileQueue originalQueue = repoRecord.getOriginalQueue();
+ if (originalQueue != null) {
+ recordBuilder.setSourceQueueIdentifier(originalQueue.getIdentifier());
+ }
+
+ recordBuilder.setAttributes(repoRecord.getOriginalAttributes(), repoRecord.getUpdatedAttributes());
+ return recordBuilder.build();
+ }
+
+ private StandardProvenanceEventRecord enrich(final ProvenanceEventRecord rawEvent, final Map<String, FlowFileRecord> flowFileRecordMap, final Map<FlowFileRecord, StandardRepositoryRecord> records, final boolean updateAttributes) {
+ final StandardProvenanceEventRecord.Builder recordBuilder = new StandardProvenanceEventRecord.Builder().fromEvent(rawEvent);
+ final FlowFileRecord eventFlowFile = flowFileRecordMap.get(rawEvent.getFlowFileUuid());
+ if (eventFlowFile != null) {
+ final StandardRepositoryRecord repoRecord = records.get(eventFlowFile);
+
+ if (repoRecord.getCurrent() != null && repoRecord.getCurrentClaim() != null) {
+ final ContentClaim currentClaim = repoRecord.getCurrentClaim();
+ final long currentOffset = repoRecord.getCurrentClaimOffset();
+ final long size = eventFlowFile.getSize();
+ recordBuilder.setCurrentContentClaim(currentClaim.getContainer(), currentClaim.getSection(), currentClaim.getId(), currentOffset, size);
+ }
+
+ if (repoRecord.getOriginal() != null && repoRecord.getOriginalClaim() != null) {
+ final ContentClaim originalClaim = repoRecord.getOriginalClaim();
+ final long originalOffset = repoRecord.getOriginal().getContentClaimOffset();
+ final long originalSize = repoRecord.getOriginal().getSize();
+ recordBuilder.setPreviousContentClaim(originalClaim.getContainer(), originalClaim.getSection(), originalClaim.getId(), originalOffset, originalSize);
+ }
+
+ final FlowFileQueue originalQueue = repoRecord.getOriginalQueue();
+ if (originalQueue != null) {
+ recordBuilder.setSourceQueueIdentifier(originalQueue.getIdentifier());
+ }
+ }
+
+ if (updateAttributes) {
+ final FlowFileRecord flowFileRecord = flowFileRecordMap.get(rawEvent.getFlowFileUuid());
+ if (flowFileRecord != null) {
+ final StandardRepositoryRecord record = records.get(flowFileRecord);
+ if (record != null) {
+ recordBuilder.setAttributes(record.getOriginalAttributes(), record.getUpdatedAttributes());
+ }
+ }
+ }
+
+ return recordBuilder.build();
+ }
+
+ /**
+ * Checks if the given event is a spurious FORK, meaning that the FORK has a
+ * single child and that child was removed in this session. This happens
+ * when a Processor calls #create(FlowFile) and then removes the created
+ * FlowFile.
+ *
+ * @param event
+ * @return
+ */
+ private boolean isSpuriousForkEvent(final ProvenanceEventRecord event, final Set<String> removedFlowFiles) {
+ if (event.getEventType() == ProvenanceEventType.FORK) {
+ final List<String> childUuids = event.getChildUuids();
+ if (childUuids != null && childUuids.size() == 1 && removedFlowFiles.contains(childUuids.get(0))) {
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+
+ /**
+ * Checks if the given event is a spurious ROUTE, meaning that the ROUTE indicates that a FlowFile
+ * was routed to a relationship with only 1 connection and that Connection is the Connection from which
+ * the FlowFile was pulled. I.e., the FlowFile was really routed nowhere.
+ *
+ * @param event
+ * @param records
+ * @return
+ */
+ private boolean isSpuriousRouteEvent(final ProvenanceEventRecord event, final Map<FlowFileRecord, StandardRepositoryRecord> records) {
+ if ( event.getEventType() == ProvenanceEventType.ROUTE ) {
+ final String relationshipName = event.getRelationship();
+ final Relationship relationship = new Relationship.Builder().name(relationshipName).build();
+ final Collection<Connection> connectionsForRelationship = this.context.getConnections(relationship);
+
+ // If the number of connections for this relationship is not 1, then we can't ignore this ROUTE event,
+ // as it may be cloning the FlowFile and adding to multiple connections.
+ if ( connectionsForRelationship.size() == 1 ) {
+ for ( final Map.Entry<FlowFileRecord, StandardRepositoryRecord> entry : records.entrySet() ) {
+ final FlowFileRecord flowFileRecord = entry.getKey();
+ if ( event.getFlowFileUuid().equals(flowFileRecord.getAttribute(CoreAttributes.UUID.key())) ) {
+ final StandardRepositoryRecord repoRecord = entry.getValue();
+ if ( repoRecord.getOriginalQueue() == null ) {
+ return false;
+ }
+
+ final String originalQueueId = repoRecord.getOriginalQueue().getIdentifier();
+ final Connection destinationConnection = connectionsForRelationship.iterator().next();
+ final String destinationQueueId = destinationConnection.getFlowFileQueue().getIdentifier();
+ return originalQueueId.equals(destinationQueueId);
+ }
+ }
+ }
+ }
+
+ return false;
+ }
+
+ @Override
+ public void rollback() {
+ rollback(false);
+ }
+
+ @Override
+ public void rollback(final boolean penalize) {
+ deleteOnCommit.clear();
+ if (records.isEmpty()) {
+ LOG.trace("{} was rolled back, but no events were performed by this ProcessSession", this);
+ acknowledgeRecords();
+ return;
+ }
+
+ resetWriteClaims();
+ resetReadClaim();
+
+ for (final StandardRepositoryRecord record : records.values()) {
+ // remove the working claim if it's different than the original.
+ removeTemporaryClaim(record);
+ }
+
+ final Set<RepositoryRecord> abortedRecords = new HashSet<>();
+ final Set<StandardRepositoryRecord> transferRecords = new HashSet<>();
+ for (final StandardRepositoryRecord record : records.values()) {
+ if (record.isMarkedForAbort()) {
+ removeContent(record.getWorkingClaim());
+ if (record.getCurrentClaim() != null && !record.getCurrentClaim().equals(record.getWorkingClaim())) {
+ // if working & original claim are same, don't remove twice; we only want to remove the original
+ // if it's different from the working. Otherwise, we remove two claimant counts. This causes
+ // an issue if we only updated the flowfile attributes.
+ removeContent(record.getCurrentClaim());
+ }
+ abortedRecords.add(record);
+ } else {
+ transferRecords.add(record);
+ }
+ }
+
+ // Put the FlowFiles that are not marked for abort back to their original queues
+ for (final StandardRepositoryRecord record : transferRecords) {
+ if (record.getOriginal() != null) {
+ final FlowFileQueue originalQueue = record.getOriginalQueue();
+ if (originalQueue != null) {
+ if (penalize) {
+ final long expirationEpochMillis = System.currentTimeMillis() + context.getConnectable().getPenalizationPeriod(TimeUnit.MILLISECONDS);
+ final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getOriginal()).penaltyExpirationTime(expirationEpochMillis).build();
+ originalQueue.put(newFile);
+ } else {
+ originalQueue.put(record.getOriginal());
+ }
+ }
+ }
+ }
+
+ if (!abortedRecords.isEmpty()) {
+ try {
+ context.getFlowFileRepository().updateRepository(abortedRecords);
+ } catch (final IOException ioe) {
+ LOG.error("Unable to update FlowFile repository for aborted records due to {}", ioe.toString());
+ if (LOG.isDebugEnabled()) {
+ LOG.error("", ioe);
+ }
+ }
+ }
+
+ final Connectable connectable = context.getConnectable();
+ final StandardFlowFileEvent flowFileEvent = new StandardFlowFileEvent(connectable.getIdentifier());
+ flowFileEvent.setBytesRead(bytesRead.getValue());
+ flowFileEvent.setBytesWritten(bytesWritten.getValue());
+
+ // update event repository
+ try {
+ context.getFlowFileEventRepository().updateRepository(flowFileEvent);
+ } catch (final Exception e) {
+ LOG.error("Failed to update FlowFileEvent Repository due to " + e);
+ if (LOG.isDebugEnabled()) {
+ LOG.error("", e);
+ }
+ }
+
+ acknowledgeRecords();
+ resetState();
+ }
+
+ private void removeContent(final ContentClaim claim) {
+ if (claim == null) {
+ return;
+ }
+
+ context.getContentRepository().decrementClaimantCount(claim);
+ }
+
+ /**
+ * Destroys a ContentClaim that was being written to but is no longer needed
+ *
+ * @param claim
+ */
+ private void destroyContent(final ContentClaim claim) {
+ if (claim == null) {
+ return;
+ }
+
+ final int decrementedClaimCount = context.getContentRepository().decrementClaimantCount(claim);
+ if (decrementedClaimCount <= 0) {
+ resetWriteClaims(); // Have to ensure that we are not currently writing to the claim before we can destroy it.
+ context.getContentRepository().remove(claim);
+ }
+ }
+
+ private void resetState() {
+ records.clear();
+ recursionSet.clear();
+ contentSizeIn = 0L;
+ contentSizeOut = 0L;
+ flowFilesIn = 0;
+ flowFilesOut = 0;
+ removedCount = 0;
+ removedBytes = 0L;
+ bytesRead.setValue(0L);
+ bytesWritten.setValue(0L);
+ connectionCounts.clear();
+ createdFlowFiles.clear();
+ removedFlowFiles.clear();
+ globalCounters.clear();
+ localCounters.clear();
+
+ generatedProvenanceEvents.clear();
+ forkEventBuilders.clear();
+ provenanceReporter.clear();
+
+ processingStartTime = System.nanoTime();
+ }
+
+ private void acknowledgeRecords() {
+ for (final Map.Entry<Connection, Set<FlowFileRecord>> entry : unacknowledgedFlowFiles.entrySet()) {
+ entry.getKey().getFlowFileQueue().acknowledge(entry.getValue());
+ }
+ unacknowledgedFlowFiles.clear();
+ }
+
+ private String summarizeEvents(final Checkpoint checkpoint) {
+ final Map<Relationship, Set<String>> transferMap = new HashMap<>(); // relationship to flowfile ID's
+ final Set<String> modifiedFlowFileIds = new HashSet<>();
+ int largestTransferSetSize = 0;
+
+ for (final Map.Entry<FlowFileRecord, StandardRepositoryRecord> entry : checkpoint.records.entrySet()) {
+ final FlowFile flowFile = entry.getKey();
+ final StandardRepositoryRecord record = entry.getValue();
+
+ final Relationship relationship = record.getTransferRelationship();
+ if (Relationship.SELF.equals(relationship)) {
+ continue;
+ }
+
+ Set<String> transferIds = transferMap.get(relationship);
+ if (transferIds == null) {
+ transferIds = new HashSet<>();
+ transferMap.put(relationship, transferIds);
+ }
+ transferIds.add(flowFile.getAttribute(CoreAttributes.UUID.key()));
+ largestTransferSetSize = Math.max(largestTransferSetSize, transferIds.size());
+
+ final ContentClaim workingClaim = record.getWorkingClaim();
+ if (workingClaim != null && workingClaim != record.getOriginalClaim() && record.getTransferRelationship() != null) {
+ modifiedFlowFileIds.add(flowFile.getAttribute(CoreAttributes.UUID.key()));
+ }
+ }
+
+ final int numRemoved = checkpoint.removedFlowFiles.size();
+ final int numModified = modifiedFlowFileIds.size();
+ final int numCreated = checkpoint.createdFlowFiles.size();
+
+ final StringBuilder sb = new StringBuilder(512);
+ if (!LOG.isDebugEnabled() && (largestTransferSetSize > VERBOSE_LOG_THRESHOLD || numModified > VERBOSE_LOG_THRESHOLD || numCreated > VERBOSE_LOG_THRESHOLD || numRemoved > VERBOSE_LOG_THRESHOLD)) {
+ if (numCreated > 0) {
+ sb.append("created ").append(numCreated).append(" FlowFiles, ");
+ }
+ if (numModified > 0) {
+ sb.append("modified ").append(modifiedFlowFileIds.size()).append(" FlowFiles, ");
+ }
+ if (numRemoved > 0) {
+ sb.append("removed ").append(numRemoved).append(" FlowFiles, ");
+ }
+ for (final Map.Entry<Relationship, Set<String>> entry : transferMap.entrySet()) {
+ if (entry.getKey() != null) {
+ sb.append("Transferred ").append(entry.getValue().size()).append(" FlowFiles");
+
+ final Relationship relationship = entry.getKey();
+ if (relationship != Relationship.ANONYMOUS) {
+ sb.append(" to '").append(relationship.getName()).append("', ");
+ }
+ }
+ }
+ } else {
+ if (numCreated > 0) {
+ sb.append("created FlowFiles ").append(checkpoint.createdFlowFiles).append(", ");
+ }
+ if (numModified > 0) {
+ sb.append("modified FlowFiles ").append(modifiedFlowFileIds).append(", ");
+ }
+ if (numRemoved > 0) {
+ sb.append("removed FlowFiles ").append(checkpoint.removedFlowFiles).append(", ");
+ }
+ for (final Map.Entry<Relationship, Set<String>> entry : transferMap.entrySet()) {
+ if (entry.getKey() != null) {
+ sb.append("Transferred FlowFiles ").append(entry.getValue());
+
+ final Relationship relationship = entry.getKey();
+ if (relationship != Relationship.ANONYMOUS) {
+ sb.append(" to '").append(relationship.getName()).append("', ");
+ }
+ }
+ }
+ }
+
+ if (sb.length() > 2 && sb.subSequence(sb.length() - 2, sb.length()).equals(", ")) {
+ sb.delete(sb.length() - 2, sb.length());
+ }
+
+ // don't add processing time if we did nothing, because we don't log the summary anyway
+ if (sb.length() > 0) {
+ final long processingNanos = checkpoint.processingTime;
+ sb.append(", Processing Time = ");
+ formatNanos(processingNanos, sb);
+ }
+
+ return sb.toString();
+ }
+
+ private void formatNanos(final long nanos, final StringBuilder sb) {
+ final long seconds = (nanos > 1000000000L) ? (nanos / 1000000000L) : 0L;
+ long millis = (nanos > 1000000L) ? (nanos / 1000000L) : 0L;;
+ final long nanosLeft = nanos % 1000000L;
+
+ if (seconds > 0) {
+ sb.append(seconds).append(" seconds");
+ }
+ if (millis > 0) {
+ if (seconds > 0) {
+ sb.append(", ");
+ millis -= seconds * 1000L;
+ }
+
+ sb.append(millis).append(" millis");
+ }
+ if (seconds == 0 && millis == 0) {
+ sb.append(nanosLeft).append(" nanos");
+ }
+
+ sb.append(" (").append(nanos).append(" nanos)");
+ }
+
+ private void incrementConnectionInputCounts(final Connection connection, final RepositoryRecord record) {
+ StandardFlowFileEvent connectionEvent = connectionCounts.get(connection);
+ if (connectionEvent == null) {
+ connectionEvent = new StandardFlowFileEvent(connection.getIdentifier());
+ connectionCounts.put(connection, connectionEvent);
+ }
+ connectionEvent.setContentSizeIn(connectionEvent.getContentSizeIn() + record.getCurrent().getSize());
+ connectionEvent.setFlowFilesIn(connectionEvent.getFlowFilesIn() + 1);
+ }
+
+ private void incrementConnectionOutputCounts(final Connection connection, final FlowFileRecord record) {
+ StandardFlowFileEvent connectionEvent = connectionCounts.get(connection);
+ if (connectionEvent == null) {
+ connectionEvent = new StandardFlowFileEvent(connection.getIdentifier());
+ connectionCounts.put(connection, connectionEvent);
+ }
+ connectionEvent.setContentSizeOut(connectionEvent.getContentSizeOut() + record.getSize());
+ connectionEvent.setFlowFilesOut(connectionEvent.getFlowFilesOut() + 1);
+ }
+
+ private void registerDequeuedRecord(final FlowFileRecord flowFile, final Connection connection) {
+ final StandardRepositoryRecord record = new StandardRepositoryRecord(connection.getFlowFileQueue(), flowFile);
+ records.put(flowFile, record);
+ flowFilesIn++;
+ contentSizeIn += flowFile.getSize();
+
+ Set<FlowFileRecord> set = unacknowledgedFlowFiles.get(connection);
+ if (set == null) {
+ set = new HashSet<>();
+ unacknowledgedFlowFiles.put(connection, set);
+ }
+ set.add(flowFile);
+
+ incrementConnectionOutputCounts(connection, flowFile);
+ }
+
+ @Override
+ public void adjustCounter(final String name, final long delta, final boolean immediate) {
+ if (immediate) {
+ context.adjustCounter(name, delta);
+ return;
+ }
+
+ adjustCounter(name, delta, localCounters);
+ adjustCounter(name, delta, globalCounters);
+ }
+
+ private void adjustCounter(final String name, final long delta, final Map<String, Long> map) {
+ Long curVal = map.get(name);
+ if (curVal == null) {
+ curVal = Long.valueOf(0L);
+ }
+
+ final long newValue = curVal.longValue() + delta;
+ map.put(name, Long.valueOf(newValue));
+ }
+
+ @Override
+ public FlowFile get() {
+ final List<Connection> connections = context.getPollableConnections();
+ final int numConnections = connections.size();
+ for (int numAttempts = 0; numAttempts < numConnections; numAttempts++) {
+ final Connection conn = connections.get(context.getNextIncomingConnectionIndex() % connections.size());
+ final Set<FlowFileRecord> expired = new HashSet<>();
+ final FlowFileRecord flowFile = conn.getFlowFileQueue().poll(expired);
+ removeExpired(expired, conn);
+
+ if (flowFile != null) {
+ registerDequeuedRecord(flowFile, conn);
+ return flowFile;
+ }
+ }
+
+ return null;
+ }
+
+ @Override
+ public List<FlowFile> get(final int maxResults) {
+ if (maxResults < 0) {
+ throw new IllegalArgumentException();
+ }
+ if (maxResults == 0) {
+ return Collections.emptyList();
+ }
+
+ return get(new QueuePoller() {
+ @Override
+ public List<FlowFileRecord> poll(final FlowFileQueue queue, final Set<FlowFileRecord> expiredRecords) {
+ return queue.poll(new FlowFileFilter() {
+ int polled = 0;
+
+ @Override
+ public FlowFileFilterResult filter(final FlowFile flowFile) {
+ if (++polled <= maxResults) {
+ return FlowFileFilterResult.ACCEPT_AND_CONTINUE;
+ } else {
+ return FlowFileFilterResult.ACCEPT_AND_TERMINATE;
+ }
+ }
+ }, expiredRecords);
+ }
+ }, false);
+ }
+
+ @Override
+ public List<FlowFile> get(final FlowFileFilter filter) {
+ return get(new QueuePoller() {
+ @Override
+ public List<FlowFileRecord> poll(final FlowFileQueue queue, final Set<FlowFileRecord> expiredRecords) {
+ return queue.poll(filter, expiredRecords);
+ }
+ }, true);
+ }
+
+ private List<FlowFile> get(final QueuePoller poller, final boolean lockAllQueues) {
+ final List<Connection> connections = context.getPollableConnections();
+ if (lockAllQueues) {
+ for (final Connection connection : connections) {
+ connection.lock();
+ }
+ }
+
+ try {
+ for (final Connection conn : connections) {
+ final Set<FlowFileRecord> expired = new HashSet<>();
+ final List<FlowFileRecord> newlySelected = poller.poll(conn.getFlowFileQueue(), expired);
+ removeExpired(expired, conn);
+
+ if (newlySelected.isEmpty() && expired.isEmpty()) {
+ continue;
+ }
+
+ for (final FlowFileRecord flowFile : newlySelected) {
+ registerDequeuedRecord(flowFile, conn);
+ }
+
+ return new ArrayList<FlowFile>(newlySelected);
+ }
+
+ return new ArrayList<>();
+ } finally {
+ if (lockAllQueues) {
+ for (final Connection connection : connections) {
+ connection.unlock();
+ }
+ }
+ }
+ }
+
+ @Override
+ public QueueSize getQueueSize() {
+ int flowFileCount = 0;
+ long byteCount = 0L;
+ for (final Connection conn : context.getPollableConnections()) {
+ final QueueSize queueSize = conn.getFlowFileQueue().getActiveQueueSize();
+ flowFileCount += queueSize.getObjectCount();
+ byteCount += queueSize.getByteCount();
+ }
+ return new QueueSize(flowFileCount, byteCount);
+ }
+
+ @Override
- public Set<Relationship> getAvailableRelationships() {
- return context.getAvailableRelationships();
- }
-
- @Override
+ public FlowFile create() {
+ final Map<String, String> attrs = new HashMap<>();
+ attrs.put(CoreAttributes.FILENAME.key(), String.valueOf(System.nanoTime()));
+ attrs.put(CoreAttributes.PATH.key(), DEFAULT_FLOWFILE_PATH);
+ attrs.put(CoreAttributes.UUID.key(), UUID.randomUUID().toString());
+
+ final FlowFileRecord fFile = new StandardFlowFileRecord.Builder().id(context.getNextFlowFileSequence())
+ .addAttributes(attrs)
+ .build();
+ final StandardRepositoryRecord record = new StandardRepositoryRecord(null);
+ record.setWorking(fFile, attrs);
+ records.put(fFile, record);
+ createdFlowFiles.add(fFile.getAttribute(CoreAttributes.UUID.key()));
+ return fFile;
+ }
+
+ @Override
+ public FlowFile clone(final FlowFile example) {
+ return clone(example, 0L, example.getSize());
+ }
+
+ @Override
+ public FlowFile clone(final FlowFile example, final long offset, final long size) {
+ validateRecordState(example);
+ final StandardRepositoryRecord exampleRepoRecord = records.get(example);
+ final FlowFileRecord currRec = exampleRepoRecord.getCurrent();
+ final ContentClaim claim = exampleRepoRecord.getCurrentClaim();
+ if (offset + size > example.getSize()) {
+ throw new FlowFileHandlingException("Specified offset of " + offset + " and size " + size + " exceeds size of " + example.toString());
+ }
+
+ final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder().fromFlowFile(currRec);
+ builder.id(context.getNextFlowFileSequence());
+ builder.contentClaimOffset(currRec.getContentClaimOffset() + offset);
+ builder.size(size);
+
+ final String newUuid = UUID.randomUUID().toString();
+ builder.addAttribute(CoreAttributes.UUID.key(), newUuid);
+
+ final FlowFileRecord clone = builder.build();
+ if (claim != null) {
+ context.getContentRepository().incrementClaimaintCount(claim);
+ }
+ final StandardRepositoryRecord record = new StandardRepositoryRecord(null);
+ record.setWorking(clone, CoreAttributes.UUID.key(), newUuid);
+ records.put(clone, record);
+
+ if (offset == 0L && size == example.getSize()) {
+ provenanceReporter.clone(example, clone);
+ } else {
+ registerForkEvent(example, clone);
+ }
+
+ return clone;
+ }
+
+ private void registerForkEvent(final FlowFile parent, final FlowFile child) {
+ ProvenanceEventBuilder eventBuilder = forkEventBuilders.get(parent);
+ if (eventBuilder == null) {
+ eventBuilder = context.getProvenanceRepository().eventBuilder();
+ eventBuilder.setEventType(ProvenanceEventType.FORK);
+
+ eventBuilder.setFlowFileEntryDate(parent.getEntryDate());
+ eventBuilder.setLineageIdentifiers(parent.getLineageIdentifiers());
+ eventBuilder.setLineageStartDate(parent.getLineageStartDate());
+ eventBuilder.setFlowFileUUID(parent.getAttribute(CoreAttributes.UUID.key()));
+
+ eventBuilder.setComponentId(context.getConnectable().getIdentifier());
+
+ final Connectable connectable = context.getConnectable();
+ final String processorType;
+ if (connectable instanceof ProcessorNode) {
+ processorType = ((ProcessorNode) connectable).getProcessor().getClass().getSimpleName();
+ } else {
+ processorType = connectable.getClass().getSimpleName();
+ }
+ eventBuilder.setComponentType(processorType);
+ eventBuilder.addParentFlowFile(parent);
+
+ updateEventContentClaims(eventBuilder, parent, records.get(parent));
+ forkEventBuilders.put(parent, eventBuilder);
+ }
+
+ eventBuilder.addChildFlowFile(child);
+ }
+
+ private void registerJoinEvent(final FlowFile child, final Collection<FlowFile> parents) {
+ final ProvenanceEventRecord eventRecord = provenanceReporter.generateJoinEvent(parents, child);
+ List<ProvenanceEventRecord> existingRecords = generatedProvenanceEvents.get(child);
+ if (existingRecords == null) {
+ existingRecords = new ArrayList<>();
+ generatedProvenanceEvents.put(child, existingRecords);
+ }
+ existingRecords.add(eventRecord);
+ }
+
+ @Override
+ public FlowFile penalize(final FlowFile flowFile) {
+ validateRecordState(flowFile);
+ final StandardRepositoryRecord record = records.get(flowFile);
+ final long expirationEpochMillis = System.currentTimeMillis() + context.getConnectable().getPenalizationPeriod(TimeUnit.MILLISECONDS);
+ final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).penaltyExpirationTime(expirationEpochMillis).build();
+ record.setWorking(newFile);
+ return newFile;
+ }
+
+ @Override
+ public FlowFile putAttribute(final FlowFile flowFile, final String key, final String value) {
+ validateRecordState(flowFile);
+
+ if (CoreAttributes.UUID.key().equals(key)) {
+ return flowFile;
+ }
+
+ final StandardRepositoryRecord record = records.get(flowFile);
+ final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).addAttribute(key, value).build();
+ record.setWorking(newFile, key, value);
+
+ return newFile;
+ }
+
+ @Override
+ public FlowFile putAllAttributes(final FlowFile flowFile, final Map<String, String> attributes) {
+ validateRecordState(flowFile);
+ final StandardRepositoryRecord record = records.get(flowFile);
+
+ final String originalUuid = flowFile.getAttribute(CoreAttributes.UUID.key());
+
+ final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).addAttributes(attributes);
+ // Ignore the uuid attribute, if passed in
+ ffBuilder.addAttribute(CoreAttributes.UUID.key(), originalUuid);
+ final FlowFileRecord newFile = ffBuilder.build();
+
+ record.setWorking(newFile, attributes);
+ return newFile;
+ }
+
+ @Override
+ public FlowFile removeAttribute(final FlowFile flowFile, final String key) {
+ validateRecordState(flowFile);
+
+ if (CoreAttributes.UUID.key().equals(key)) {
+ return flowFile;
+ }
+
+ final StandardRepositoryRecord record = records.get(flowFile);
+ final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(key).build();
+ record.setWorking(newFile, key, null);
+ return newFile;
+ }
+
+ @Override
+ public FlowFile removeAllAttributes(final FlowFile flowFile, final Set<String> keys) {
+ validateRecordState(flowFile);
+
+ if (keys == null) {
+ return flowFile;
+ }
+
+ final Set<String> keysToRemove;
+ if (keys.contains(CoreAttributes.UUID.key())) {
+ keysToRemove = new HashSet<>(keys);
+ keysToRemove.remove(CoreAttributes.UUID.key());
+ } else {
+ keysToRemove = keys;
+ }
+
+ final StandardRepositoryRecord record = records.get(flowFile);
+ final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(keys).build();
+
+ final Map<String, String> updatedAttrs = new HashMap<>();
+ for (final String key : keys) {
+ updatedAttrs.put(key, null);
+ }
+
+ record.setWorking(newFile, updatedAttrs);
+ return newFile;
+ }
+
+ @Override
+ public FlowFile removeAllAttributes(final FlowFile flowFile, final Pattern keyPattern) {
+ validateRecordState(flowFile);
+ final StandardRepositoryRecord record = records.get(flowFile);
+ final FlowFileRecord newFile = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).removeAttributes(keyPattern).build();
+
+ if (keyPattern == null) {
+ record.setWorking(newFile);
+ } else {
+ final Map<String, String> curAttrs = record.getCurrent().getAttributes();
+
+ final Map<String, String> removed = new HashMap<>();
+ for (final String key : curAttrs.keySet()) {
+ if (CoreAttributes.UUID.key().equals(key)) {
+ continue;
+ }
+
+ if (keyPattern.matcher(key).matches()) {
+ removed.put(key, null);
+ }
+ }
+
+ record.setWorking(newFile, removed);
+ }
+
+ return newFile;
+ }
+
+ @Override
+ public void transfer(final FlowFile flowFile, final Relationship relationship) {
+ validateRecordState(flowFile);
+ final StandardRepositoryRecord record = records.get(flowFile);
+ record.setTransferRelationship(relationship);
+ final int numDestinations = context.getConnections(relationship).size();
+ final int multiplier = Math.max(1, numDestinations);
+
+ boolean autoTerminated = false;
+ boolean selfRelationship = false;
+ if (numDestinations == 0 && context.getConnectable().isAutoTerminated(relationship)) {
+ // auto terminated.
+ autoTerminated = true;
+ } else if (numDestinations == 0 && relationship == Relationship.SELF) {
+ selfRelationship = true;
+ }
+
+ if (autoTerminated) {
+ removedCount += multiplier;
+ removedBytes += flowFile.getSize();
+ } else if (!selfRelationship) {
+ flowFilesOut += multiplier;
+ contentSizeOut += flowFile.getSize() * multiplier;
+ }
+ }
+
+ @Override
+ public void transfer(final FlowFile flowFile) {
+ validateRecordState(flowFile);
+ final StandardRepositoryRecord record = records.get(flowFile);
+ if (record.getOriginalQueue() == null) {
+ throw new IllegalArgumentException("Cannot transfer FlowFiles that are created in this Session back to self");
+ }
+ record.setTransferRelationship(Relationship.SELF);
+ }
+
+ @Override
+ public void transfer(final Collection<FlowFile> flowFiles) {
+ for (final FlowFile flowFile : flowFiles) {
+ transfer(flowFile);
+ }
+ }
+
+ @Override
+ public void transfer(final Collection<FlowFile> flowFiles, final Relationship relationship) {
+ validateRecordState(flowFiles);
+
+ boolean autoTerminated = false;
+ boolean selfRelationship = false;
+ final int numDestinations = context.getConnections(relationship).size();
+ if (numDestinations == 0 && context.getConnectable().isAutoTerminated(relationship)) {
+ // auto terminated.
+ autoTerminated = true;
+ } else if (numDestinations == 0 && relationship == Relationship.SELF) {
+ selfRelationship = true;
+ }
+
+ final int multiplier = Math.max(1, numDestinations);
+
+ long contentSize = 0L;
+ for (final FlowFile flowFile : flowFiles) {
+ final StandardRepositoryRecord record = records.get(flowFile);
+ record.setTransferRelationship(relationship);
+ contentSize += flowFile.getSize() * multiplier;
+ }
+
+ if (autoTerminated) {
+ removedCount += multiplier * flowFiles.size();
+ removedBytes += contentSize;
+ } else if (!selfRelationship) {
+ flowFilesOut += multiplier * flowFiles.size();
+ contentSizeOut += multiplier * contentSize;
+ }
+ }
+
+ @Override
+ public void remove(final FlowFile flowFile) {
+ validateRecordState(flowFile);
+ final StandardRepositoryRecord record = records.get(flowFile);
+ record.markForDelete();
+ removedFlowFiles.add(flowFile.getAttribute(CoreAttributes.UUID.key()));
+
+ // if original connection is null, the FlowFile was created in this session, so we
+ // do not want to count it toward the removed count.
+ if (record.getOriginalQueue() == null) {
+ // if we've generated any Fork events, remove them because the FlowFile was created
+ // and then removed in this session.
+ generatedProvenanceEvents.remove(flowFile);
+ removeForkEvents(flowFile);
+ } else {
+ removedCount++;
+ removedBytes += flowFile.getSize();
+ provenanceReporter.drop(flowFile, flowFile.getAttribute(CoreAttributes.DISCARD_REASON.key()));
+ }
+ }
+
+ @Override
+ public void remove(final Collection<FlowFile> flowFiles) {
+ validateRecordState(flowFiles);
+ for (final FlowFile flowFile : flowFiles) {
+ final StandardRepositoryRecord record = records.get(flowFile);
+ record.markForDelete();
+ removedFlowFiles.add(flowFile.getAttribute(CoreAttributes.UUID.key()));
+
+ // if original connection is null, the FlowFile was created in this session, so we
+ // do not want to count it toward the removed count.
+ if (record.getOriginalQueue() == null) {
+ generatedProvenanceEvents.remove(flowFile);
+ removeForkEvents(flowFile);
+ } else {
+ removedCount++;
+ removedBytes += flowFile.getSize();
+ provenanceReporter.drop(flowFile, flowFile.getAttribute(CoreAttributes.DISCARD_REASON.key()));
+ }
+ }
+ }
+
+ private void removeForkEvents(final FlowFile flowFile) {
+ for (final ProvenanceEventBuilder builder : forkEventBuilders.values()) {
+ final ProvenanceEventRecord event = builder.build();
+
+ if (event.getEventType() == ProvenanceEventType.FORK) {
+ builder.removeChildFlowFile(flowFile);
+ }
+ }
+ }
+
+ public void expireFlowFiles() {
+ final Set<FlowFileRecord> expired = new HashSet<>();
+ final FlowFileFilter filter = new FlowFileFilter() {
+ @Override
+ public FlowFileFilterResult filter(final FlowFile flowFile) {
+ return FlowFileFilterResult.REJECT_AND_CONTINUE;
+ }
+ };
+
+ for (final Connection conn : context.getConnectable().getIncomingConnections()) {
+ do {
+ expired.clear();
+ conn.getFlowFileQueue().poll(filter, expired);
+ removeExpired(expired, conn);
+ } while (!expired.isEmpty());
+ }
+ }
+
+ private void removeExpired(final Set<FlowFileRecord> flowFiles, final Connection connection) {
+ if (flowFiles.isEmpty()) {
+ return;
+ }
+
+ LOG.info("{} {} FlowFiles have expired and will be removed", new Object[]{this, flowFiles.size()});
+ final List<RepositoryRecord> expiredRecords = new ArrayList<>(flowFiles.size());
+
+ final String processorType;
+ final Connectable connectable = context.getConnectable();
+ if (connectable instanceof ProcessorNode) {
+ final ProcessorNode procNode = (ProcessorNode) connectable;
+ processorType = procNode.getProcessor().getClass().getSimpleName();
+ } else {
+ processorType = connectable.getClass().getSimpleName();
+ }
+
+ final StandardProvenanceReporter expiredReporter = new StandardProvenanceReporter(connectable.getIdentifier(),
+ processorType, context.getProvenanceRepository(), this);
+
+ final Map<String, FlowFileRecord> recordIdMap = new HashMap<>();
+ for (final FlowFileRecord flowFile : flowFiles) {
+ recordIdMap.put(flowFile.getAttribute(CoreAttributes.UUID.key()), flowFile);
+
+ final StandardRepositoryRecord record = new StandardRepositoryRecord(connection.getFlowFileQueue(), flowFile);
+ record.markForDelete();
+ expiredRecords.add(record);
+ expiredReporter.expire(flowFile, "Expiration Threshold = " + connection.getFlowFileQueue().getFlowFileExpiration());
+ removeContent(flowFile.getContentClaim());
+
+ final long flowFileLife = System.currentTimeMillis() - flowFile.getEntryDate();
+ final Object terminator = (connectable instanceof ProcessorNode) ? ((ProcessorNode) connectable).getProcessor() : connectable;
+ LOG.info("{} terminated by {} due to FlowFile expiration; life of FlowFile = {} ms", new Object[]{flowFile, terminator, flowFileLife});
+ }
+
+ try {
+ final Iterable<ProvenanceEventRecord> iterable = new Iterable<ProvenanceEventRecord>() {
+ @Override
+ public Iterator<ProvenanceEventRecord> iterator() {
+ final Iterator<ProvenanceEventRecord> expiredEventIterator = expiredReporter.getEvents().iterator();
+ final Iterator<ProvenanceEventRecord> enrichingIterator = new Iterator<ProvenanceEventRecord>() {
+ @Override
+ public boolean hasNext() {
+ return expiredEventIterator.hasNext();
+ }
+
+ @Override
+ public ProvenanceEventRecord next() {
+ final ProvenanceEventRecord event = expiredEventIterator.next();
+ final StandardProvenanceEventRecord.Builder enriched = new StandardProvenanceEventRecord.Builder().fromEvent(event);
+ final FlowFileRecord record = recordIdMap.get(event.getFlowFileUuid());
+ if (record == null) {
+ return null;
+ }
+
+ final ContentClaim claim = record.getContentClaim();
+ if (claim != null) {
+ enriched.setCurrentContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
+ enriched.setPreviousContentClaim(claim.getContainer(), claim.getSection(), claim.getId(), record.getContentClaimOffset(), record.getSize());
+ }
+
+ enriched.setAttributes(record.getAttributes(), Collections.<String, String>emptyMap());
+ return enriched.build();
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+
+ return enrichingIterator;
+ }
+ };
+
+ context.getProvenanceRepository().registerEvents(iterable);
+ context.getFlowFileRepository().updateRepository(expiredRecords);
+ } catch (final IOException e) {
+ LOG.error("Failed to update FlowFile Repository to record expired records due to {}", e);
+ }
+
+ }
+
+ private InputStream getInputStream(final FlowFile flowFile, final ContentClaim claim, final long offset) throws ContentNotFoundException {
+ // If there's no content, don't bother going to the Content Repository because it is generally expensive and we know
+ // that there is no actual content.
+ if (flowFile.getSize() == 0L) {
+ return new ByteArrayInputStream(new byte[0]);
+ }
+
+ try {
+ // If the recursion set is empty, we can use the same input stream that we already have open. However, if
+ // the recursion set is NOT empty, we can't do this because we may be reading the input of FlowFile 1 while in the
+ // callback for reading FlowFile 1 and if we used the same stream we'd be destroying the ability to read from FlowFile 1.
+ if (recursionSet.isEmpty()) {
+ if (currentReadClaim == claim) {
+ if (currentReadClaimStream != null && currentReadClaimStream.getStreamLocation() <= offset) {
+ final long bytesToSkip = offset - currentReadClaimStream.getStreamLocation();
+ if (bytesToSkip > 0) {
+ StreamUtils.skip(currentReadClaimStream, bytesToSkip);
+ }
+
+ return new NonCloseableInputStream(currentReadClaimStream);
+ }
+ }
+
+ final InputStream rawInStream = context.getContentRepository().read(claim);
+
+ if (currentReadClaimStream != null) {
+ currentReadClaimStream.close();
+ }
+
+ currentReadClaim = claim;
+ currentReadClaimStream = new ByteCountingInputStream(rawInStream, new LongHolder(0L));
+ StreamUtils.skip(currentReadClaimStream, offset);
+
+ // Use a non-closeable stream because we want to keep it open after the callback has finished so that we can
+ // reuse the same InputStream for the next FlowFile
+ return new NonCloseableInputStream(currentReadClaimStream);
+ } else {
+ final InputStream rawInStream = context.getContentRepository().read(claim);
+ StreamUtils.skip(rawInStream, offset);
+ return rawInStream;
+ }
+ } catch (final ContentNotFoundException cnfe) {
+ throw cnfe;
+ } catch (final EOFException eof) {
+ throw new ContentNotFoundException(claim, eof);
+ } catch (final IOException ioe) {
+ throw new FlowFileAccessException("Failed to read content of " + flowFile, ioe);
+ }
+ }
+
+ @Override
+ public void read(final FlowFile source, final InputStreamCallback reader) {
+ validateRecordState(source);
+ final StandardRepositoryRecord record = records.get(source);
+
+ try {
+ ensureNotAppending(record.getCurrentClaim());
+ } catch (final IOException e) {
+ throw new FlowFileAccessException("Failed to access ContentClaim for " + source.toString(), e);
+ }
+
+ try (final InputStream rawIn = getInputStream(source, record.getCurrentClaim(), record.getCurrentClaimOffset());
+ final InputStream limitedIn = new LimitedInputStream(rawIn, source.getSize());
+ final ByteCountingInputStream countingStream = new ByteCountingInputStream(limitedIn, this.bytesRead)) {
+
+ // We want to differentiate between IOExceptions thrown by the repository and IOExceptions thrown from
+ // Processor code. As a result, as have the FlowFileAccessInputStream that catches IOException from the repository
+ // and translates into either FlowFileAccessException or ContentNotFoundException. We keep track of any
+ // ContentNotFoundException because if it is thrown, the Processor code may catch it and do something else with it
+ // but in reality, if it is thrown, we want to know about it and handle it, even if the Processor code catches it.
+ final FlowFileAccessInputStream ffais = new FlowFileAccessInputStream(countingStream, source, record.getCurrentClaim());
+ boolean cnfeThrown = false;
+
+ try {
+ recursionSet.add(source);
+ reader.process(ffais);
+ } catch (final ContentNotFoundException cnfe) {
+ cnfeThrown = true;
+ throw cnfe;
+ } finally {
+ recursionSet.remove(source);
+
+ // if cnfeThrown is true, we don't need to re-thrown the Exception; it will propagate.
+ if (!cnfeThrown && ffais.getContentNotFoundException() != null) {
+ throw ffais.getContentNotFoundException();
+ }
+ }
+ } catch (final ContentNotFoundException nfe) {
+ handleContentNotFound(nfe, record);
+ } catch (final IOException ex) {
+ throw new ProcessException("IOException thrown from " + connectableDescription + ": " + ex.toString(), ex);
+ }
+ }
+
+ @Override
+ public FlowFile merge(final Collection<FlowFile> sources, final FlowFile destination) {
+ return merge(sources, destination, null, null, null);
+ }
+
+ @Override
+ public FlowFile merge(final Collection<FlowFile> sources, final FlowFile destination, final byte[] header, final byte[] footer, final byte[] demarcator) {
+ validateRecordState(sources);
+ validateRecordState(destination);
+ if (sources.contains(destination)) {
+ throw new IllegalArgumentException("Destination cannot be within sources");
+ }
+
+ final Collection<StandardRepositoryRecord> sourceRecords = new ArrayList<>();
+ for (final FlowFile source : sources) {
+ final StandardRepositoryRecord record = records.get(source);
+ sourceRecords.add(record);
+
+ try {
+ ensureNotAppending(record.getCurrentClaim());
+ } catch (final IOException e) {
+ throw new FlowFileAccessException("Unable to read from source " + source + " due to " + e.toString(), e);
+ }
+ }
+
+ final StandardRepositoryRecord destinationRecord = records.get(destination);
+ final ContentRepository contentRepo = context.getContentRepository();
+ final ContentClaim newClaim;
+ try {
+ newClaim = contentRepo.create(context.getConnectable().isLossTolerant());
+ claimLog.debug("Creating ContentClaim {} for 'merge' for {}", newClaim, desti
<TRUNCATED>
[2/8] incubator-nifi git commit: Merge branch 'develop' of
https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
----------------------------------------------------------------------
diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
index 0000000,6e5f65d..15591d7
mode 000000,100644..100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessContext.java
@@@ -1,0 -1,261 +1,284 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.util;
+
+ import static java.util.Objects.requireNonNull;
+
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
++import java.util.HashSet;
+ import java.util.LinkedHashMap;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Objects;
++import java.util.Set;
+
+ import org.apache.nifi.components.ConfigurableComponent;
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.components.ValidationResult;
+ import org.apache.nifi.controller.ControllerService;
+ import org.apache.nifi.controller.ControllerServiceLookup;
++import org.apache.nifi.processor.Processor;
++import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.SchedulingContext;
+ import org.junit.Assert;
+
+ public class MockProcessContext extends MockControllerServiceLookup implements SchedulingContext, ControllerServiceLookup {
+
+ private final ConfigurableComponent component;
+ private final Map<PropertyDescriptor, String> properties = new HashMap<>();
+
+ private String annotationData = null;
+ private boolean yieldCalled = false;
+ private boolean enableExpressionValidation = false;
+ private boolean allowExpressionValidation = true;
+
++ private volatile Set<Relationship> unavailableRelationships = new HashSet<>();
++
+ /**
+ * Creates a new MockProcessContext for the given Processor
+ *
+ * @param component
+ */
+ public MockProcessContext(final ConfigurableComponent component) {
+ this.component = Objects.requireNonNull(component);
+ }
+
+ public MockProcessContext(final ControllerService component, final MockProcessContext context) {
+ this(component);
+
+ try {
+ annotationData = context.getControllerServiceAnnotationData(component);
+ final Map<PropertyDescriptor, String> props = context.getControllerServiceProperties(component);
+ properties.putAll(props);
+ } catch (IllegalArgumentException e) {
+ // do nothing...the service is being loaded
+ }
+ }
+
+ @Override
+ public PropertyValue getProperty(final PropertyDescriptor descriptor) {
+ return getProperty(descriptor.getName());
+ }
+
+ @Override
+ public PropertyValue getProperty(final String propertyName) {
+ final PropertyDescriptor descriptor = component.getPropertyDescriptor(propertyName);
+ if (descriptor == null) {
+ return null;
+ }
+
+ final String setPropertyValue = properties.get(descriptor);
+ final String propValue = (setPropertyValue == null) ? descriptor.getDefaultValue() : setPropertyValue;
+ return new MockPropertyValue(propValue, this, (enableExpressionValidation && allowExpressionValidation) ? descriptor : null);
+ }
+
+ @Override
+ public PropertyValue newPropertyValue(final String rawValue) {
+ return new MockPropertyValue(rawValue, this);
+ }
+
+ public ValidationResult setProperty(final String propertyName, final String propertyValue) {
+ return setProperty(new PropertyDescriptor.Builder().name(propertyName).build(), propertyValue);
+ }
+
+ /**
+ * Updates the value of the property with the given PropertyDescriptor to
+ * the specified value IF and ONLY IF the value is valid according to the
+ * descriptor's validator. Otherwise, the property value is not updated. In
+ * either case, the ValidationResult is returned, indicating whether or not
+ * the property is valid
+ *
+ * @param descriptor
+ * @param value
+ * @return
+ */
+ public ValidationResult setProperty(final PropertyDescriptor descriptor, final String value) {
+ requireNonNull(descriptor);
+ requireNonNull(value, "Cannot set property to null value; if the intent is to remove the property, call removeProperty instead");
+ final PropertyDescriptor fullyPopulatedDescriptor = component.getPropertyDescriptor(descriptor.getName());
+
+ final ValidationResult result = fullyPopulatedDescriptor.validate(value, new MockValidationContext(this));
+ String oldValue = properties.put(fullyPopulatedDescriptor, value);
+ if (oldValue == null) {
+ oldValue = fullyPopulatedDescriptor.getDefaultValue();
+ }
+ if ((value == null && oldValue != null) || (value != null && !value.equals(oldValue))) {
+ component.onPropertyModified(fullyPopulatedDescriptor, oldValue, value);
+ }
+
+ return result;
+ }
+
+ public boolean removeProperty(final PropertyDescriptor descriptor) {
+ Objects.requireNonNull(descriptor);
+ final PropertyDescriptor fullyPopulatedDescriptor = component.getPropertyDescriptor(descriptor.getName());
+ String value = null;
+ if (!fullyPopulatedDescriptor.isRequired() && (value = properties.remove(fullyPopulatedDescriptor)) != null) {
+ component.onPropertyModified(fullyPopulatedDescriptor, value, null);
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public void yield() {
+ yieldCalled = true;
+ }
+
+ public boolean isYieldCalled() {
+ return yieldCalled;
+ }
+
+ public void addControllerService(final String serviceIdentifier, final ControllerService controllerService, final Map<PropertyDescriptor, String> properties, final String annotationData) {
+ requireNonNull(controllerService);
+ final ControllerServiceConfiguration config = addControllerService(controllerService);
+ config.setProperties(properties);
+ config.setAnnotationData(annotationData);
+ }
+
+ @Override
+ public int getMaxConcurrentTasks() {
+ return 1;
+ }
+
+ public void setAnnotationData(final String annotationData) {
+ this.annotationData = annotationData;
+ }
+
+ @Override
+ public String getAnnotationData() {
+ return annotationData;
+ }
+
+ @Override
+ public Map<PropertyDescriptor, String> getProperties() {
+ final List<PropertyDescriptor> supported = component.getPropertyDescriptors();
+ if (supported == null || supported.isEmpty()) {
+ return Collections.unmodifiableMap(properties);
+ } else {
+ final Map<PropertyDescriptor, String> props = new LinkedHashMap<>();
+ for (final PropertyDescriptor descriptor : supported) {
+ props.put(descriptor, null);
+ }
+ props.putAll(properties);
+ return props;
+ }
+ }
+
+ /**
+ * Validates the current properties, returning ValidationResults for any
+ * invalid properties. All processor defined properties will be validated.
+ * If they are not included in the in the purposed configuration, the
+ * default value will be used.
+ *
+ * @return Collection of validation result objects for any invalid findings
+ * only. If the collection is empty then the processor is valid. Guaranteed
+ * non-null
+ */
+ public Collection<ValidationResult> validate() {
+ return component.validate(new MockValidationContext(this));
+ }
+
+ public boolean isValid() {
+ for (final ValidationResult result : validate()) {
+ if (!result.isValid()) {
+ return false;
+ }
+ }
+
+ return true;
+ }
+
+ public void assertValid() {
+ final StringBuilder sb = new StringBuilder();
+ int failureCount = 0;
+
+ for (final ValidationResult result : validate()) {
+ if (!result.isValid()) {
+ sb.append(result.toString()).append("\n");
+ failureCount++;
+ }
+ }
+
+ if (failureCount > 0) {
+ Assert.fail("Processor has " + failureCount + " validation failures:\n" + sb.toString());
+ }
+ }
+
+ @Override
+ public String encrypt(final String unencrypted) {
+ return "enc{" + unencrypted + "}";
+ }
+
+ @Override
+ public String decrypt(final String encrypted) {
+ if (encrypted.startsWith("enc{") && encrypted.endsWith("}")) {
+ return encrypted.substring(4, encrypted.length() - 2);
+ }
+ return encrypted;
+ }
+
+ public void setValidateExpressionUsage(final boolean validate) {
+ allowExpressionValidation = validate;
+ }
+
+ public void enableExpressionValidation() {
+ enableExpressionValidation = true;
+ }
+
+ public void disableExpressionValidation() {
+ enableExpressionValidation = false;
+ }
+
+ Map<PropertyDescriptor, String> getControllerServiceProperties(final ControllerService controllerService) {
+ return super.getConfiguration(controllerService.getIdentifier()).getProperties();
+ }
+
+ String getControllerServiceAnnotationData(final ControllerService controllerService) {
+ return super.getConfiguration(controllerService.getIdentifier()).getAnnotationData();
+ }
+
+ @Override
+ public ControllerServiceLookup getControllerServiceLookup() {
+ return this;
+ }
+
+ @Override
+ public void leaseControllerService(final String identifier) {
+ }
+
++ public Set<Relationship> getAvailableRelationships() {
++ if ( !(component instanceof Processor) ) {
++ return Collections.emptySet();
++ }
++
++ final Set<Relationship> relationships = new HashSet<>(((Processor) component).getRelationships());
++ relationships.removeAll(unavailableRelationships);
++ return relationships;
++ }
++
++ public void setUnavailableRelationships(final Set<Relationship> relationships) {
++ this.unavailableRelationships = Collections.unmodifiableSet(new HashSet<>(relationships));
++ }
++
++ public Set<Relationship> getUnavailableRelationships() {
++ return unavailableRelationships;
++ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
----------------------------------------------------------------------
diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
index 0000000,552780c..ea55b34
mode 000000,100644..100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java
@@@ -1,0 -1,1010 +1,1006 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.util;
+
+ import java.io.ByteArrayInputStream;
+ import java.io.ByteArrayOutputStream;
+ import java.io.File;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.nio.file.Files;
+ import java.nio.file.OpenOption;
+ import java.nio.file.Path;
+ import java.nio.file.StandardOpenOption;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.Collection;
+ import java.util.Collections;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Iterator;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Objects;
+ import java.util.Set;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.regex.Pattern;
+
+ import org.junit.Assert;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.processor.FlowFileFilter;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.QueueSize;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.exception.FlowFileAccessException;
+ import org.apache.nifi.processor.exception.FlowFileHandlingException;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.processor.io.InputStreamCallback;
+ import org.apache.nifi.processor.io.OutputStreamCallback;
+ import org.apache.nifi.processor.io.StreamCallback;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+
+ public class MockProcessSession implements ProcessSession {
+
+ private final Map<Relationship, List<MockFlowFile>> transferMap = new ConcurrentHashMap<>();
+ private final MockFlowFileQueue processorQueue;
+ private final Set<Long> beingProcessed = new HashSet<>();
+
+ private final Map<Long, MockFlowFile> currentVersions = new HashMap<>();
+ private final Map<Long, MockFlowFile> originalVersions = new HashMap<>();
+ private final SharedSessionState sharedState;
+ private final Map<String, Long> counterMap = new HashMap<>();
+
+ private boolean committed = false;
+ private boolean rolledback = false;
+ private int removedCount = 0;
+
+ public MockProcessSession(final SharedSessionState sharedState) {
+ this.sharedState = sharedState;
+ this.processorQueue = sharedState.getFlowFileQueue();
+ }
+
+ @Override
+ public void adjustCounter(final String name, final long delta, final boolean immediate) {
+ if (immediate) {
+ sharedState.adjustCounter(name, delta);
+ return;
+ }
+
+ Long counter = counterMap.get(name);
+ if (counter == null) {
+ counter = delta;
+ counterMap.put(name, counter);
+ return;
+ }
+
+ counter = counter + delta;
+ counterMap.put(name, counter);
+ }
+
+ @Override
+ public MockFlowFile clone(final FlowFile flowFile) {
+ validateState(flowFile);
+ final MockFlowFile newFlowFile = new MockFlowFile(sharedState.nextFlowFileId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+ beingProcessed.add(newFlowFile.getId());
+ return newFlowFile;
+ }
+
+ @Override
+ public MockFlowFile clone(final FlowFile flowFile, final long offset, final long size) {
+ validateState(flowFile);
+ if (offset + size > flowFile.getSize()) {
+ throw new FlowFileHandlingException("Specified offset of " + offset + " and size " + size + " exceeds size of " + flowFile.toString());
+ }
+
+ final MockFlowFile newFlowFile = new MockFlowFile(sharedState.nextFlowFileId(), flowFile);
+ final byte[] newContent = Arrays.copyOfRange(((MockFlowFile) flowFile).getData(), (int) offset, (int) (offset + size));
+ newFlowFile.setData(newContent);
+
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+ beingProcessed.add(newFlowFile.getId());
+ return newFlowFile;
+ }
+
+ @Override
+ public void commit() {
+ if (!beingProcessed.isEmpty()) {
+ throw new FlowFileHandlingException("Cannot commit session because the following FlowFiles have not been removed or transferred: " + beingProcessed);
+ }
+ committed = true;
+ beingProcessed.clear();
+ currentVersions.clear();
+ originalVersions.clear();
+
+ for (final Map.Entry<String, Long> entry : counterMap.entrySet()) {
+ sharedState.adjustCounter(entry.getKey(), entry.getValue());
+ }
+
+ counterMap.clear();
+ }
+
+ /**
+ * Clear the 'committed' flag so that we can test that the next iteration of
+ * {@link nifi.processor.Processor#onTrigger} commits or rolls back the
+ * session
+ */
+ public void clearCommited() {
+ committed = false;
+ }
+
+ /**
+ * Clear the 'rolledBack' flag so that we can test that the next iteration
+ * of {@link nifi.processor.Processor#onTrigger} commits or rolls back the
+ * session
+ */
+ public void clearRollback() {
+ rolledback = false;
+ }
+
+ @Override
+ public MockFlowFile create() {
+ final MockFlowFile flowFile = new MockFlowFile(sharedState.nextFlowFileId());
+ currentVersions.put(flowFile.getId(), flowFile);
+ beingProcessed.add(flowFile.getId());
+ return flowFile;
+ }
+
+ @Override
+ public MockFlowFile create(final FlowFile flowFile) {
+ MockFlowFile newFlowFile = create();
+ newFlowFile = (MockFlowFile) inheritAttributes(flowFile, newFlowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+ beingProcessed.add(newFlowFile.getId());
+ return newFlowFile;
+ }
+
+ @Override
+ public MockFlowFile create(final Collection<FlowFile> flowFiles) {
+ MockFlowFile newFlowFile = create();
+ newFlowFile = (MockFlowFile) inheritAttributes(flowFiles, newFlowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+ beingProcessed.add(newFlowFile.getId());
+ return newFlowFile;
+ }
+
+ @Override
+ public void exportTo(final FlowFile flowFile, final OutputStream out) {
+ validateState(flowFile);
+ if (flowFile == null || out == null) {
+ throw new IllegalArgumentException("arguments cannot be null");
+ }
+
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+
+ try {
+ out.write(mock.getData());
+ } catch (IOException e) {
+ throw new FlowFileAccessException(e.toString(), e);
+ }
+ }
+
+ @Override
+ public void exportTo(final FlowFile flowFile, final Path path, final boolean append) {
+ validateState(flowFile);
+ if (flowFile == null || path == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+
+ final OpenOption mode = append ? StandardOpenOption.APPEND : StandardOpenOption.CREATE;
+
+ try (final OutputStream out = Files.newOutputStream(path, mode)) {
+ out.write(mock.getData());
+ } catch (final IOException e) {
+ throw new FlowFileAccessException(e.toString(), e);
+ }
+ }
+
+ @Override
+ public MockFlowFile get() {
+ final MockFlowFile flowFile = processorQueue.poll();
+ if (flowFile != null) {
+ beingProcessed.add(flowFile.getId());
+ currentVersions.put(flowFile.getId(), flowFile);
+ originalVersions.put(flowFile.getId(), flowFile);
+ }
+ return flowFile;
+ }
+
+ @Override
+ public List<FlowFile> get(final int maxResults) {
+ final List<FlowFile> flowFiles = new ArrayList<>(Math.min(500, maxResults));
+ for (int i = 0; i < maxResults; i++) {
+ final MockFlowFile nextFlowFile = get();
+ if (nextFlowFile == null) {
+ return flowFiles;
+ }
+
+ flowFiles.add(nextFlowFile);
+ }
+
+ return flowFiles;
+ }
+
+ @Override
+ public List<FlowFile> get(final FlowFileFilter filter) {
+ final List<FlowFile> flowFiles = new ArrayList<>();
+ final List<MockFlowFile> unselected = new ArrayList<>();
+
+ while (true) {
+ final MockFlowFile flowFile = processorQueue.poll();
+ if (flowFile == null) {
+ break;
+ }
+
+ final FlowFileFilter.FlowFileFilterResult filterResult = filter.filter(flowFile);
+ if (filterResult.isAccept()) {
+ flowFiles.add(flowFile);
+
+ beingProcessed.add(flowFile.getId());
+ currentVersions.put(flowFile.getId(), flowFile);
+ originalVersions.put(flowFile.getId(), flowFile);
+ } else {
+ unselected.add(flowFile);
+ }
+
+ if (!filterResult.isContinue()) {
+ break;
+ }
+ }
+
+ processorQueue.addAll(unselected);
+ return flowFiles;
+ }
+
+ @Override
+ public QueueSize getQueueSize() {
+ return processorQueue.size();
+ }
+
+ @Override
+ public MockFlowFile importFrom(final InputStream in, final FlowFile flowFile) {
+ validateState(flowFile);
+ if (in == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+
+ final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+ try {
+ final byte[] data = readFully(in);
+ newFlowFile.setData(data);
+ return newFlowFile;
+ } catch (final IOException e) {
+ throw new FlowFileAccessException(e.toString(), e);
+ }
+ }
+
+ @Override
+ public MockFlowFile importFrom(final Path path, final boolean keepSourceFile, final FlowFile flowFile) {
+ validateState(flowFile);
+ if (path == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+ MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ try {
+ Files.copy(path, baos);
+ } catch (final IOException e) {
+ throw new FlowFileAccessException(e.toString(), e);
+ }
+
+ newFlowFile.setData(baos.toByteArray());
+ newFlowFile = putAttribute(newFlowFile, CoreAttributes.FILENAME.key(), path.getFileName().toString());
+ return newFlowFile;
+ }
+
- @Override
- public Set<Relationship> getAvailableRelationships() {
- return sharedState.getAvailableRelationships();
- }
+
+ @Override
+ public MockFlowFile merge(final Collection<FlowFile> sources, final FlowFile destination) {
+ for (final FlowFile source : sources) {
+ validateState(source);
+ }
+ validateState(destination);
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ for (final FlowFile flowFile : sources) {
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+ final byte[] data = mock.getData();
+ try {
+ baos.write(data);
+ } catch (final IOException e) {
+ throw new AssertionError("Failed to write to BAOS");
+ }
+ }
+
+ final MockFlowFile newFlowFile = new MockFlowFile(destination.getId(), destination);
+ newFlowFile.setData(baos.toByteArray());
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+
+ return newFlowFile;
+ }
+
+ @Override
+ public MockFlowFile putAllAttributes(final FlowFile flowFile, final Map<String, String> attrs) {
+ validateState(flowFile);
+ if (attrs == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot update attributes of a flow file that I did not create");
+ }
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+ final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+
+ newFlowFile.putAttributes(attrs);
+ return newFlowFile;
+ }
+
+ @Override
+ public MockFlowFile putAttribute(final FlowFile flowFile, final String attrName, final String attrValue) {
+ validateState(flowFile);
+ if (attrName == null || attrValue == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot update attributes of a flow file that I did not create");
+ }
+
+ if ("uuid".equals(attrName)) {
+ Assert.fail("Should not be attempting to set FlowFile UUID via putAttribute. This will be ignored in production");
+ }
+
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+ final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+
+ final Map<String, String> attrs = new HashMap<>();
+ attrs.put(attrName, attrValue);
+ newFlowFile.putAttributes(attrs);
+ return newFlowFile;
+ }
+
+ @Override
+ public void read(final FlowFile flowFile, final InputStreamCallback callback) {
+ if (callback == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+
+ validateState(flowFile);
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+
+ final ByteArrayInputStream bais = new ByteArrayInputStream(mock.getData());
+ try {
+ callback.process(bais);
+ } catch (IOException e) {
+ throw new ProcessException(e.toString(), e);
+ }
+ }
+
+ @Override
+ public void remove(final FlowFile flowFile) {
+ validateState(flowFile);
+ final Iterator<Long> itr = beingProcessed.iterator();
+ while (itr.hasNext()) {
+ final Long ffId = itr.next();
+ if (ffId != null && ffId.equals(flowFile.getId())) {
+ itr.remove();
+ beingProcessed.remove(ffId);
+ removedCount++;
+ currentVersions.remove(ffId);
+ return;
+ }
+ }
+
+ throw new ProcessException(flowFile + " not found in queue");
+ }
+
+ @Override
+ public void remove(final Collection<FlowFile> flowFiles) {
+ for (final FlowFile flowFile : flowFiles) {
+ validateState(flowFile);
+ }
+
+ for (final FlowFile flowFile : flowFiles) {
+ remove(flowFile);
+ }
+ }
+
+ @Override
+ public MockFlowFile removeAllAttributes(final FlowFile flowFile, final Set<String> attrNames) {
+ validateState(flowFile);
+ if (attrNames == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+
+ final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+
+ newFlowFile.removeAttributes(attrNames);
+ return newFlowFile;
+ }
+
+ @Override
+ public MockFlowFile removeAllAttributes(final FlowFile flowFile, final Pattern keyPattern) {
+ validateState(flowFile);
+ if (flowFile == null) {
+ throw new IllegalArgumentException("flowFile cannot be null");
+ }
+ if (keyPattern == null) {
+ return (MockFlowFile) flowFile;
+ }
+
+ final Set<String> attrsToRemove = new HashSet<>();
+ for (final String key : flowFile.getAttributes().keySet()) {
+ if (keyPattern.matcher(key).matches()) {
+ attrsToRemove.add(key);
+ }
+ }
+
+ return removeAllAttributes(flowFile, attrsToRemove);
+ }
+
+ @Override
+ public MockFlowFile removeAttribute(final FlowFile flowFile, final String attrName) {
+ validateState(flowFile);
+ if (attrName == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+ final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+
+ final Set<String> attrNames = new HashSet<>();
+ attrNames.add(attrName);
+ newFlowFile.removeAttributes(attrNames);
+ return newFlowFile;
+ }
+
+ @Override
+ public void rollback() {
+ rollback(false);
+ }
+
+ @Override
+ public void rollback(final boolean penalize) {
+ for (final List<MockFlowFile> list : transferMap.values()) {
+ for (final MockFlowFile flowFile : list) {
+ processorQueue.offer(flowFile);
+ }
+ }
+
+ for (final Long flowFileId : beingProcessed) {
+ final MockFlowFile flowFile = originalVersions.get(flowFileId);
+ if (flowFile != null) {
+ processorQueue.offer(flowFile);
+ }
+ }
+
+ rolledback = true;
+ beingProcessed.clear();
+ currentVersions.clear();
+ originalVersions.clear();
+ transferMap.clear();
+ clearTransferState();
+ }
+
+ @Override
+ public void transfer(final FlowFile flowFile) {
+ validateState(flowFile);
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("I only accept MockFlowFile");
+ }
+
+ beingProcessed.remove(flowFile.getId());
+ processorQueue.offer((MockFlowFile) flowFile);
+ }
+
+ @Override
+ public void transfer(final Collection<FlowFile> flowFiles) {
+ for (final FlowFile flowFile : flowFiles) {
+ transfer(flowFile);
+ }
+ }
+
+ @Override
+ public void transfer(final FlowFile flowFile, final Relationship relationship) {
+ validateState(flowFile);
+ List<MockFlowFile> list = transferMap.get(relationship);
+ if (list == null) {
+ list = new ArrayList<>();
+ transferMap.put(relationship, list);
+ }
+
+ beingProcessed.remove(flowFile.getId());
+ list.add((MockFlowFile) flowFile);
+ }
+
+ @Override
+ public void transfer(final Collection<FlowFile> flowFiles, final Relationship relationship) {
+ for (final FlowFile flowFile : flowFiles) {
+ validateState(flowFile);
+ }
+
+ List<MockFlowFile> list = transferMap.get(relationship);
+ if (list == null) {
+ list = new ArrayList<>();
+ transferMap.put(relationship, list);
+ }
+
+ for (final FlowFile flowFile : flowFiles) {
+ beingProcessed.remove(flowFile.getId());
+ list.add((MockFlowFile) flowFile);
+ }
+ }
+
+ @Override
+ public MockFlowFile write(final FlowFile flowFile, final OutputStreamCallback callback) {
+ validateState(flowFile);
+ if (callback == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ try {
+ callback.process(baos);
+ } catch (final IOException e) {
+ throw new ProcessException(e.toString(), e);
+ }
+
+ final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+
+ newFlowFile.setData(baos.toByteArray());
+ return newFlowFile;
+ }
+
+ @Override
+ public FlowFile append(final FlowFile flowFile, final OutputStreamCallback callback) {
+ validateState(flowFile);
+ if (callback == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ try {
+ baos.write(mock.getData());
+ callback.process(baos);
+ } catch (final IOException e) {
+ throw new ProcessException(e.toString(), e);
+ }
+
+ final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+
+ newFlowFile.setData(baos.toByteArray());
+ return newFlowFile;
+ }
+
+ @Override
+ public MockFlowFile write(final FlowFile flowFile, final StreamCallback callback) {
+ validateState(flowFile);
+ if (callback == null || flowFile == null) {
+ throw new IllegalArgumentException("argument cannot be null");
+ }
+ if (!(flowFile instanceof MockFlowFile)) {
+ throw new IllegalArgumentException("Cannot export a flow file that I did not create");
+ }
+ final MockFlowFile mock = (MockFlowFile) flowFile;
+
+ final ByteArrayInputStream in = new ByteArrayInputStream(mock.getData());
+ final ByteArrayOutputStream out = new ByteArrayOutputStream();
+ try {
+ callback.process(in, out);
+ } catch (final IOException e) {
+ throw new ProcessException(e.toString(), e);
+ }
+
+ final MockFlowFile newFlowFile = new MockFlowFile(mock.getId(), flowFile);
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+ newFlowFile.setData(out.toByteArray());
+
+ return newFlowFile;
+ }
+
+ private byte[] readFully(final InputStream in) throws IOException {
+ final byte[] buffer = new byte[4096];
+ int len;
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ while ((len = in.read(buffer)) >= 0) {
+ baos.write(buffer, 0, len);
+ }
+
+ return baos.toByteArray();
+ }
+
+ public List<MockFlowFile> getFlowFilesForRelationship(final Relationship relationship) {
+ List<MockFlowFile> list = this.transferMap.get(relationship);
+ if (list == null) {
+ list = new ArrayList<>();
+ }
+
+ return list;
+ }
+
+ /**
+ * Returns a List of FlowFiles in the order in which they were transferred
+ * to the given relationship
+ *
+ * @param relationship
+ * @return
+ */
+ public List<MockFlowFile> getFlowFilesForRelationship(final String relationship) {
+ final Relationship procRel = new Relationship.Builder().name(relationship).build();
+ return getFlowFilesForRelationship(procRel);
+ }
+
+ public MockFlowFile createFlowFile(final File file) throws IOException {
+ return createFlowFile(Files.readAllBytes(file.toPath()));
+ }
+
+ public MockFlowFile createFlowFile(final byte[] data) {
+ final MockFlowFile flowFile = create();
+ flowFile.setData(data);
+ return flowFile;
+ }
+
+ public MockFlowFile createFlowFile(final byte[] data, final Map<String, String> attrs) {
+ final MockFlowFile ff = createFlowFile(data);
+ ff.putAttributes(attrs);
+ return ff;
+ }
+
+ @Override
+ public MockFlowFile merge(Collection<FlowFile> sources, FlowFile destination, byte[] header, byte[] footer, byte[] demarcator) {
+ for (final FlowFile flowFile : sources) {
+ validateState(flowFile);
+ }
+ validateState(destination);
+
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ try {
+ if (header != null) {
+ baos.write(header);
+ }
+
+ int count = 0;
+ for (final FlowFile flowFile : sources) {
+ baos.write(((MockFlowFile) flowFile).getData());
+ if (demarcator != null && ++count != sources.size()) {
+ baos.write(demarcator);
+ }
+ }
+
+ if (footer != null) {
+ baos.write(footer);
+ }
+ } catch (final IOException e) {
+ throw new AssertionError("failed to write data to BAOS");
+ }
+
+ final MockFlowFile newFlowFile = new MockFlowFile(destination.getId(), destination);
+ newFlowFile.setData(baos.toByteArray());
+ currentVersions.put(newFlowFile.getId(), newFlowFile);
+
+ return newFlowFile;
+ }
+
+ private void validateState(final FlowFile flowFile) {
+ Objects.requireNonNull(flowFile);
+ final FlowFile currentVersion = currentVersions.get(flowFile.getId());
+ if (currentVersion == null) {
+ throw new FlowFileHandlingException(flowFile + " is not known in this session");
+ }
+
+ if (currentVersion != flowFile) {
+ throw new FlowFileHandlingException(flowFile + " is not the most recent version of this flow file within this session");
+ }
+
+ for (final List<MockFlowFile> flowFiles : transferMap.values()) {
+ if (flowFiles.contains(flowFile)) {
+ throw new IllegalStateException(flowFile + " has already been transferred");
+ }
+ }
+ }
+
+ /**
+ * Inherits the attributes from the given source flow file into another flow
+ * file. The UUID of the source becomes the parent UUID of this flow file.
+ * If a parent uuid had previously been established it will be replaced by
+ * the uuid of the given source
+ *
+ * @param source the FlowFile from which to copy attributes
+ * @param destination the FlowFile to which to copy attributes
+ */
+ private FlowFile inheritAttributes(final FlowFile source, final FlowFile destination) {
+ if (source == null || destination == null || source == destination) {
+ return destination; //don't need to inherit from ourselves
+ }
+ FlowFile updated = putAllAttributes(destination, source.getAttributes());
+ getProvenanceReporter().fork(source, Collections.singletonList(updated));
+ return updated;
+ }
+
+ /**
+ * Inherits the attributes from the given source flow files into the
+ * destination flow file. The UUIDs of the sources becomes the parent UUIDs
+ * of the destination flow file. Only attributes which is common to all
+ * source items is copied into this flow files attributes. Any previously
+ * established parent UUIDs will be replaced by the UUIDs of the sources. It
+ * will capture the uuid of a certain number of source objects and may not
+ * capture all of them. How many it will capture is unspecified.
+ *
+ * @param sources
+ */
+ private FlowFile inheritAttributes(final Collection<FlowFile> sources, final FlowFile destination) {
+ final String uuid = destination.getAttribute(CoreAttributes.UUID.key());
+ final StringBuilder parentUuidBuilder = new StringBuilder();
+ int uuidsCaptured = 0;
+ for (final FlowFile source : sources) {
+ if (source == destination) {
+ continue; //don't want to capture parent uuid of this. Something can't be a child of itself
+ }
+ final String sourceUuid = source.getAttribute(CoreAttributes.UUID.key());
+ if (sourceUuid != null && !sourceUuid.trim().isEmpty()) {
+ uuidsCaptured++;
+ if (parentUuidBuilder.length() > 0) {
+ parentUuidBuilder.append(",");
+ }
+ parentUuidBuilder.append(sourceUuid);
+ }
+
+ if (uuidsCaptured > 100) {
+ break;
+ }
+ }
+
+ FlowFile updated = putAllAttributes(destination, intersectAttributes(sources));
+ getProvenanceReporter().join(sources, updated);
+ return updated;
+ }
+
+ /**
+ * Returns the attributes that are common to every flow file given. The key
+ * and value must match exactly.
+ *
+ * @param flowFileList a list of flow files
+ *
+ * @return the common attributes
+ */
+ private static Map<String, String> intersectAttributes(final Collection<FlowFile> flowFileList) {
+ final Map<String, String> result = new HashMap<>();
+ //trivial cases
+ if (flowFileList == null || flowFileList.isEmpty()) {
+ return result;
+ } else if (flowFileList.size() == 1) {
+ result.putAll(flowFileList.iterator().next().getAttributes());
+ }
+
+ /*
+ * Start with the first attribute map and only put an entry to the
+ * resultant map if it is common to every map.
+ */
+ final Map<String, String> firstMap = flowFileList.iterator().next().getAttributes();
+
+ outer:
+ for (final Map.Entry<String, String> mapEntry : firstMap.entrySet()) {
+ final String key = mapEntry.getKey();
+ final String value = mapEntry.getValue();
+ for (final FlowFile flowFile : flowFileList) {
+ final Map<String, String> currMap = flowFile.getAttributes();
+ final String curVal = currMap.get(key);
+ if (curVal == null || !curVal.equals(value)) {
+ continue outer;
+ }
+ }
+ result.put(key, value);
+ }
+
+ return result;
+ }
+
+ /**
+ * Assert that {@link #commit()} has been called
+ */
+ public void assertCommitted() {
+ Assert.assertTrue("Session was not committed", committed);
+ }
+
+ /**
+ * Assert that {@link #commit()} has not been called
+ */
+ public void assertNotCommitted() {
+ Assert.assertFalse("Session was committed", committed);
+ }
+
+ /**
+ * Assert that {@link #rollback()} has been called
+ */
+ public void assertRolledBack() {
+ Assert.assertTrue("Session was not rolled back", rolledback);
+ }
+
+ /**
+ * Assert that {@link #rollback()} has not been called
+ */
+ public void assertNotRolledBack() {
+ Assert.assertFalse("Session was rolled back", rolledback);
+ }
+
+ /**
+ * Assert that the number of FlowFiles transferred to the given relationship
+ * is equal to the given count
+ *
+ * @param relationship
+ * @param count
+ */
+ public void assertTransferCount(final Relationship relationship, final int count) {
+ final int transferCount = getFlowFilesForRelationship(relationship).size();
+ Assert.assertEquals("Expected " + count + " FlowFiles to be transferred to "
+ + relationship + " but actual transfer count was " + transferCount, count, transferCount);
+ }
+
+ /**
+ * Assert that the number of FlowFiles transferred to the given relationship
+ * is equal to the given count
+ *
+ * @param relationship
+ * @param count
+ */
+ public void assertTransferCount(final String relationship, final int count) {
+ assertTransferCount(new Relationship.Builder().name(relationship).build(), count);
+ }
+
+ /**
+ * Assert that there are no FlowFiles left on the input queue.
+ */
+ public void assertQueueEmpty() {
+ Assert.assertTrue("FlowFile Queue has " + this.processorQueue.size() + " items", this.processorQueue.isEmpty());
+ }
+
+ /**
+ * Assert that at least one FlowFile is on the input queue
+ */
+ public void assertQueueNotEmpty() {
+ Assert.assertFalse("FlowFile Queue is empty", this.processorQueue.isEmpty());
+ }
+
+ /**
+ * Asserts that all FlowFiles that were transferred were transferred to the
+ * given relationship
+ *
+ * @param relationship
+ */
+ public void assertAllFlowFilesTransferred(final String relationship) {
+ assertAllFlowFilesTransferred(new Relationship.Builder().name(relationship).build());
+ }
+
+ /**
+ * Asserts that all FlowFiles that were transferred were transferred to the
+ * given relationship
+ *
+ * @param relationship
+ */
+ public void assertAllFlowFilesTransferred(final Relationship relationship) {
+ for (final Map.Entry<Relationship, List<MockFlowFile>> entry : transferMap.entrySet()) {
+ final Relationship rel = entry.getKey();
+ final List<MockFlowFile> flowFiles = entry.getValue();
+
+ if (!rel.equals(relationship) && flowFiles != null && !flowFiles.isEmpty()) {
+ Assert.fail("Expected all Transferred FlowFiles to go to " + relationship + " but " + flowFiles.size() + " were routed to " + rel);
+ }
+ }
+ }
+
+ /**
+ * Removes all state information about FlowFiles that have been transferred
+ */
+ public void clearTransferState() {
+ this.transferMap.clear();
+ }
+
+ /**
+ * Asserts that all FlowFiles that were transferred were transferred to the
+ * given relationship and that the number of FlowFiles transferred is equal
+ * to <code>count</code>
+ *
+ * @param relationship
+ * @param count
+ */
+ public void assertAllFlowFilesTransferred(final Relationship relationship, final int count) {
+ assertAllFlowFilesTransferred(relationship);
+ assertTransferCount(relationship, count);
+ }
+
+ /**
+ * Asserts that all FlowFiles that were transferred were transferred to the
+ * given relationship and that the number of FlowFiles transferred is equal
+ * to <code>count</code>
+ *
+ * @param relationship
+ * @param count
+ */
+ public void assertAllFlowFilesTransferred(final String relationship, final int count) {
+ assertAllFlowFilesTransferred(new Relationship.Builder().name(relationship).build(), count);
+ }
+
+ /**
+ * Returns the number of FlowFiles that were removed
+ *
+ * @return
+ */
+ public int getRemovedCount() {
+ return removedCount;
+ }
+
+ @Override
+ public ProvenanceReporter getProvenanceReporter() {
+ return sharedState.getProvenanceReporter();
+ }
+
+ @Override
+ public MockFlowFile penalize(final FlowFile flowFile) {
+ validateState(flowFile);
+ final MockFlowFile mockFlowFile = (MockFlowFile) flowFile;
+ mockFlowFile.setPenalized();
+ return mockFlowFile;
+ }
+
+ public byte[] getContentAsByteArray(final MockFlowFile flowFile) {
+ validateState(flowFile);
+ return flowFile.getData();
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
----------------------------------------------------------------------
diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
index 0000000,96bef71..13a87de
mode 000000,100644..100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/SharedSessionState.java
@@@ -1,0 -1,91 +1,72 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.util;
+
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
+ import java.util.concurrent.ConcurrentHashMap;
+ import java.util.concurrent.ConcurrentMap;
+ import java.util.concurrent.atomic.AtomicLong;
+
+ import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+
+ public class SharedSessionState {
+
+ private final MockFlowFileQueue flowFileQueue;
+ private final ProvenanceReporter provenanceReporter;
++ @SuppressWarnings("unused")
+ private final Processor processor;
+ private final AtomicLong flowFileIdGenerator;
+ private final ConcurrentMap<String, AtomicLong> counterMap = new ConcurrentHashMap<>();
+
- private volatile Set<Relationship> unavailableRelationships;
+
+ public SharedSessionState(final Processor processor, final AtomicLong flowFileIdGenerator) {
+ flowFileQueue = new MockFlowFileQueue();
+ provenanceReporter = new MockProvenanceReporter();
- unavailableRelationships = new HashSet<>();
+ this.flowFileIdGenerator = flowFileIdGenerator;
+ this.processor = processor;
+ }
+
- public Set<Relationship> getAvailableRelationships() {
- final Set<Relationship> relationships = new HashSet<>(processor.getRelationships());
- relationships.removeAll(unavailableRelationships);
- return relationships;
- }
-
- public void setUnavailableRelationships(final Set<Relationship> relationships) {
- this.unavailableRelationships = Collections.unmodifiableSet(new HashSet<>(relationships));
- }
-
- public Set<Relationship> getUnavailableRelationships() {
- return unavailableRelationships;
- }
-
+ public MockFlowFileQueue getFlowFileQueue() {
+ return flowFileQueue;
+ }
+
+ public ProvenanceReporter getProvenanceReporter() {
+ return provenanceReporter;
+ }
+
+ public long nextFlowFileId() {
+ return flowFileIdGenerator.getAndIncrement();
+ }
+
+ public void adjustCounter(final String name, final long delta) {
+ AtomicLong counter = counterMap.get(name);
+ if (counter == null) {
+ counter = new AtomicLong(0L);
+ AtomicLong existingCounter = counterMap.putIfAbsent(name, counter);
+ if (existingCounter != null) {
+ counter = existingCounter;
+ }
+ }
+
+ counter.addAndGet(delta);
+ }
+
+ public Long getCounterValue(final String name) {
+ final AtomicLong counterValue = counterMap.get(name);
+ return (counterValue == null) ? null : counterValue.get();
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
----------------------------------------------------------------------
diff --cc nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
index 0000000,54b611d..40d5035
mode 000000,100644..100644
--- a/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
+++ b/nifi/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java
@@@ -1,0 -1,492 +1,492 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.util;
+
+ import static java.util.Objects.requireNonNull;
+
+ import java.io.ByteArrayInputStream;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.lang.reflect.InvocationTargetException;
+ import java.nio.file.Files;
+ import java.nio.file.Path;
+ import java.util.ArrayList;
+ import java.util.Collections;
+ import java.util.Comparator;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.concurrent.Callable;
+ import java.util.concurrent.ExecutorService;
+ import java.util.concurrent.Executors;
+ import java.util.concurrent.Future;
+ import java.util.concurrent.atomic.AtomicInteger;
+ import java.util.concurrent.atomic.AtomicLong;
+
+ import org.apache.nifi.components.AllowableValue;
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.ValidationResult;
+ import org.apache.nifi.controller.ControllerService;
+ import org.apache.nifi.controller.annotation.OnConfigured;
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.Processor;
+ import org.apache.nifi.processor.QueueSize;
+ import org.apache.nifi.processor.Relationship;
+ import org.apache.nifi.processor.annotation.OnAdded;
+ import org.apache.nifi.processor.annotation.OnScheduled;
+ import org.apache.nifi.processor.annotation.OnShutdown;
+ import org.apache.nifi.processor.annotation.OnStopped;
+ import org.apache.nifi.processor.annotation.OnUnscheduled;
+ import org.apache.nifi.processor.annotation.TriggerSerially;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+ import org.apache.nifi.reporting.InitializationException;
+
+ import org.junit.Assert;
+
+ public class StandardProcessorTestRunner implements TestRunner {
+
+ private final Processor processor;
+ private final MockProcessContext context;
+ private final MockFlowFileQueue flowFileQueue;
+ private final MockSessionFactory sessionFactory;
+ private final SharedSessionState sharedState;
+ private final AtomicLong idGenerator;
+ private final boolean triggerSerially;
+
+ private int numThreads = 1;
+ private final AtomicInteger invocations = new AtomicInteger(0);
+
+ StandardProcessorTestRunner(final Processor processor) {
+ this.processor = processor;
+ this.idGenerator = new AtomicLong(0L);
+ this.sharedState = new SharedSessionState(processor, idGenerator);
+ this.flowFileQueue = sharedState.getFlowFileQueue();
+ this.sessionFactory = new MockSessionFactory(sharedState);
+ this.context = new MockProcessContext(processor);
+
+ final MockProcessorInitializationContext mockInitContext = new MockProcessorInitializationContext(processor, context);
+ processor.initialize(mockInitContext);
+
+ try {
+ ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor);
+ } catch (Exception e) {
+ Assert.fail("Could not invoke methods annotated with @OnAdded annotation due to: " + e);
+ }
+
+ triggerSerially = null != processor.getClass().getAnnotation(TriggerSerially.class);
+ }
+
+ @Override
+ public void setValidateExpressionUsage(final boolean validate) {
+ context.setValidateExpressionUsage(validate);
+ }
+
+ @Override
+ public Processor getProcessor() {
+ return processor;
+ }
+
+ @Override
+ public MockProcessContext getProcessContext() {
+ return context;
+ }
+
+ @Override
+ public void run() {
+ run(1);
+ }
+
+ @Override
+ public void run(int iterations) {
+ run(iterations, true);
+ }
+
+ @Override
+ public void run(final int iterations, final boolean stopOnFinish) {
+ run(iterations, stopOnFinish, true);
+ }
+
+ @Override
+ public void run(final int iterations, final boolean stopOnFinish, final boolean initialize) {
+ if (iterations < 1) {
+ throw new IllegalArgumentException();
+ }
+
+ context.assertValid();
+ context.enableExpressionValidation();
+ try {
+ if ( initialize ) {
+ try {
+ ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, context);
+ } catch (Exception e) {
+ e.printStackTrace();
+ Assert.fail("Could not invoke methods annotated with @OnScheduled annotation due to: " + e);
+ }
+ }
+
+ final ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+ @SuppressWarnings("unchecked")
+ final Future<Throwable>[] futures = new Future[iterations];
+ for (int i = 0; i < iterations; i++) {
+ final Future<Throwable> future = executorService.submit(new RunProcessor());
+ futures[i] = future;
+ }
+
+ executorService.shutdown();
+
+ int finishedCount = 0;
+ boolean unscheduledRun = false;
+ for (final Future<Throwable> future : futures) {
+ try {
+ final Throwable thrown = future.get(); // wait for the result
+ if (thrown != null) {
+ throw new AssertionError(thrown);
+ }
+
+ if (++finishedCount == 1) {
+ unscheduledRun = true;
+ try {
+ ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, processor, context);
+ } catch (Exception e) {
+ Assert.fail("Could not invoke methods annotated with @OnUnscheduled annotation due to: " + e);
+ }
+ }
+ } catch (final Exception e) {
+ }
+ }
+
+ if (!unscheduledRun) {
+ try {
+ ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, processor, context);
+ } catch (Exception e) {
+ Assert.fail("Could not invoke methods annotated with @OnUnscheduled annotation due to: " + e);
+ }
+ }
+
+ if (stopOnFinish) {
+ try {
+ ReflectionUtils.invokeMethodsWithAnnotation(OnStopped.class, processor);
+ } catch (Exception e) {
+ Assert.fail("Could not invoke methods annotated with @OnStopped annotation due to: " + e);
+ }
+ }
+ } finally {
+ context.disableExpressionValidation();
+ }
+ }
+
+ @Override
+ public void shutdown() {
+ try {
+ ReflectionUtils.invokeMethodsWithAnnotation(OnShutdown.class, processor);
+ } catch (Exception e) {
+ Assert.fail("Could not invoke methods annotated with @OnShutdown annotation due to: " + e);
+ }
+ }
+
+ private class RunProcessor implements Callable<Throwable> {
+
+ @Override
+ public Throwable call() throws Exception {
+ invocations.incrementAndGet();
+ try {
+ processor.onTrigger(context, sessionFactory);
+ } catch (final Throwable t) {
+ return t;
+ }
+
+ return null;
+ }
+ }
+
+ @Override
+ public ProcessSessionFactory getProcessSessionFactory() {
+ return sessionFactory;
+ }
+
+ @Override
+ public void assertAllFlowFilesTransferred(final String relationship) {
+ for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+ session.assertAllFlowFilesTransferred(relationship);
+ }
+ }
+
+ @Override
+ public void assertAllFlowFilesTransferred(final Relationship relationship) {
+ for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+ session.assertAllFlowFilesTransferred(relationship);
+ }
+ }
+
+ @Override
+ public void assertAllFlowFilesTransferred(final String relationship, final int count) {
+ assertAllFlowFilesTransferred(relationship);
+ assertTransferCount(relationship, count);
+ }
+
+ @Override
+ public void assertAllFlowFilesTransferred(final Relationship relationship, final int count) {
+ assertAllFlowFilesTransferred(relationship);
+ assertTransferCount(relationship, count);
+ }
+
+ @Override
+ public void assertTransferCount(final Relationship relationship, final int count) {
+ Assert.assertEquals(count, getFlowFilesForRelationship(relationship).size());
+ }
+
+ @Override
+ public void assertTransferCount(final String relationship, final int count) {
+ Assert.assertEquals(count, getFlowFilesForRelationship(relationship).size());
+ }
+
+ @Override
+ public void assertValid() {
+ context.assertValid();
+ }
+
+ @Override
+ public void assertNotValid() {
+ Assert.assertFalse("Processor appears to be valid but expected it to be invalid", context.isValid());
+ }
+
+ @Override
+ public boolean isQueueEmpty() {
+ return flowFileQueue.isEmpty();
+ }
+
+ @Override
+ public void assertQueueEmpty() {
+ Assert.assertTrue(flowFileQueue.isEmpty());
+ }
+
+ @Override
+ public void assertQueueNotEmpty() {
+ Assert.assertFalse(flowFileQueue.isEmpty());
+ }
+
+ @Override
+ public void clearTransferState() {
+ for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+ session.clearTransferState();
+ }
+ }
+
+ @Override
+ public void enqueue(final FlowFile... flowFiles) {
+ for (final FlowFile flowFile : flowFiles) {
+ flowFileQueue.offer((MockFlowFile) flowFile);
+ }
+ }
+
+ @Override
+ public void enqueue(final Path path) throws IOException {
+ enqueue(path, new HashMap<String, String>());
+ }
+
+ @Override
+ public void enqueue(final Path path, final Map<String, String> attributes) throws IOException {
+ final Map<String, String> modifiedAttributes = new HashMap<>(attributes);
+ if (!modifiedAttributes.containsKey(CoreAttributes.FILENAME.key())) {
+ modifiedAttributes.put(CoreAttributes.FILENAME.key(), path.toFile().getName());
+ }
+ try (final InputStream in = Files.newInputStream(path)) {
+ enqueue(in, modifiedAttributes);
+ }
+ }
+
+ @Override
+ public void enqueue(final byte[] data) {
+ enqueue(data, new HashMap<String, String>());
+ }
+
+ @Override
+ public void enqueue(final byte[] data, final Map<String, String> attributes) {
+ enqueue(new ByteArrayInputStream(data), attributes);
+ }
+
+ @Override
+ public void enqueue(final InputStream data) {
+ enqueue(data, new HashMap<String, String>());
+ }
+
+ @Override
+ public void enqueue(final InputStream data, final Map<String, String> attributes) {
+ final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator));
+ MockFlowFile flowFile = session.create();
+ flowFile = session.importFrom(data, flowFile);
+ flowFile = session.putAllAttributes(flowFile, attributes);
+ enqueue(flowFile);
+ }
+
+ @Override
+ public byte[] getContentAsByteArray(final MockFlowFile flowFile) {
+ return flowFile.getData();
+ }
+
+ @Override
+ public List<MockFlowFile> getFlowFilesForRelationship(final String relationship) {
+ final Relationship rel = new Relationship.Builder().name(relationship).build();
+ return getFlowFilesForRelationship(rel);
+ }
+
+ @Override
+ public List<MockFlowFile> getFlowFilesForRelationship(final Relationship relationship) {
+ final List<MockFlowFile> flowFiles = new ArrayList<>();
+ for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+ flowFiles.addAll(session.getFlowFilesForRelationship(relationship));
+ }
+
+ Collections.sort(flowFiles, new Comparator<MockFlowFile>() {
+ @Override
+ public int compare(final MockFlowFile o1, final MockFlowFile o2) {
+ return Long.compare(o1.getCreationTime(), o2.getCreationTime());
+ }
+ });
+
+ return flowFiles;
+ }
+
+ @Override
+ public ProvenanceReporter getProvenanceReporter() {
+ return sharedState.getProvenanceReporter();
+ }
+
+ @Override
+ public QueueSize getQueueSize() {
+ return flowFileQueue.size();
+ }
+
+ @Override
+ public Long getCounterValue(final String name) {
+ return sharedState.getCounterValue(name);
+ }
+
+ @Override
+ public int getRemovedCount() {
+ int count = 0;
+ for (final MockProcessSession session : sessionFactory.getCreatedSessions()) {
+ count += session.getRemovedCount();
+ }
+
+ return count;
+ }
+
+ @Override
+ public void setAnnotationData(final String annotationData) {
+ context.setAnnotationData(annotationData);
+ }
+
+ @Override
+ public ValidationResult setProperty(final String propertyName, final String propertyValue) {
+ return context.setProperty(propertyName, propertyValue);
+ }
+
+ @Override
+ public ValidationResult setProperty(final PropertyDescriptor descriptor, final String value) {
+ return context.setProperty(descriptor, value);
+ }
+
+ @Override
+ public ValidationResult setProperty(final PropertyDescriptor descriptor, final AllowableValue value) {
+ return context.setProperty(descriptor, value.getValue());
+ }
+
+ @Override
+ public void setThreadCount(final int threadCount) {
+ if (threadCount > 1 && triggerSerially) {
+ Assert.fail("Cannot set thread-count higher than 1 because the processor is triggered serially");
+ }
+
+ this.numThreads = threadCount;
+ }
+
+ @Override
+ public int getThreadCount() {
+ return numThreads;
+ }
+
+ @Override
+ public void setRelationshipAvailable(final Relationship relationship) {
- final Set<Relationship> unavailable = new HashSet<>(sharedState.getUnavailableRelationships());
++ final Set<Relationship> unavailable = new HashSet<>(context.getUnavailableRelationships());
+ unavailable.remove(relationship);
- sharedState.setUnavailableRelationships(unavailable);
++ context.setUnavailableRelationships(unavailable);
+ }
+
+ @Override
+ public void setRelationshipAvailable(final String relationshipName) {
+ setRelationshipAvailable(new Relationship.Builder().name(relationshipName).build());
+ }
+
+ @Override
+ public void setRelationshipUnavailable(final Relationship relationship) {
- final Set<Relationship> unavailable = new HashSet<>(sharedState.getUnavailableRelationships());
++ final Set<Relationship> unavailable = new HashSet<>(context.getUnavailableRelationships());
+ unavailable.add(relationship);
- sharedState.setUnavailableRelationships(unavailable);
++ context.setUnavailableRelationships(unavailable);
+ }
+
+ @Override
+ public void setRelationshipUnavailable(final String relationshipName) {
+ setRelationshipUnavailable(new Relationship.Builder().name(relationshipName).build());
+ }
+
+ @Override
+ public void addControllerService(final String identifier, final ControllerService service) throws InitializationException {
+ addControllerService(identifier, service, new HashMap<String, String>());
+ }
+
+ @Override
+ public void addControllerService(final String identifier, final ControllerService service, final Map<String, String> properties) throws InitializationException {
+ final MockControllerServiceInitializationContext initContext = new MockControllerServiceInitializationContext(requireNonNull(service), requireNonNull(identifier));
+ service.initialize(initContext);
+
+ final Map<PropertyDescriptor, String> resolvedProps = new HashMap<>();
+ for (final Map.Entry<String, String> entry : properties.entrySet()) {
+ resolvedProps.put(service.getPropertyDescriptor(entry.getKey()), entry.getValue());
+ }
+
+ final MockConfigurationContext configurationContext = new MockConfigurationContext(resolvedProps, context);
+ try {
+ ReflectionUtils.invokeMethodsWithAnnotation(OnConfigured.class, service, configurationContext);
+ } catch (final InvocationTargetException | IllegalAccessException | IllegalArgumentException e) {
+ throw new InitializationException(e);
+ }
+
+ context.addControllerService(identifier, service, resolvedProps, null);
+ }
+
+ @Override
+ public ControllerService getControllerService(final String identifier) {
+ return context.getControllerService(identifier);
+ }
+
+ @Override
+ public <T extends ControllerService> T getControllerService(final String identifier, final Class<T> serviceType) {
+ final ControllerService service = context.getControllerService(identifier);
+ return serviceType.cast(service);
+ }
+
+ @Override
+ public boolean removeProperty(PropertyDescriptor descriptor) {
+ return context.removeProperty(descriptor);
+ }
+
+ }
[3/8] incubator-nifi git commit: Merge branch 'develop' of
https://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
index 0000000,31e5105..da80546
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/PutFileTransfer.java
@@@ -1,0 -1,293 +1,293 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processors.standard;
+
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.stream.io.BufferedInputStream;
+ import org.apache.nifi.logging.ProcessorLog;
+ import org.apache.nifi.processor.AbstractProcessor;
+ import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ 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.processors.standard.util.FileInfo;
+ import org.apache.nifi.processors.standard.util.FileTransfer;
+ import org.apache.nifi.processors.standard.util.SFTPTransfer;
+ import org.apache.nifi.util.ObjectHolder;
+ import org.apache.nifi.util.StopWatch;
+
+ import java.io.File;
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.util.Collections;
+ import java.util.HashSet;
+ import java.util.Set;
+ import java.util.concurrent.TimeUnit;
+
+ /**
+ * Base class for PutFTP & PutSFTP
+ * @param <T>
+ */
+ public abstract class PutFileTransfer<T extends FileTransfer> extends AbstractProcessor {
+
+ public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success").description("FlowFiles that are successfully sent will be routed to success").build();
+ public static final Relationship REL_FAILURE = new Relationship.Builder().name("failure").description("FlowFiles that failed to send to the remote system; failure is usually looped back to this processor").build();
+ public static final Relationship REL_REJECT = new Relationship.Builder().name("reject").description("FlowFiles that were rejected by the destination system").build();
+
+ private final Set<Relationship> relationships;
+
+ public PutFileTransfer() {
+ super();
+ final Set<Relationship> relationships = new HashSet<>();
+ relationships.add(REL_SUCCESS);
+ relationships.add(REL_FAILURE);
+ relationships.add(REL_REJECT);
+ this.relationships = Collections.unmodifiableSet(relationships);
+ }
+
+ @Override
+ public Set<Relationship> getRelationships() {
+ return relationships;
+ }
+
+ protected abstract T getFileTransfer(final ProcessContext context);
+
+ protected void beforePut(final FlowFile flowFile, final ProcessContext context, final T transfer) throws IOException {
+
+ }
+
+ protected void afterPut(final FlowFile flowFile, final ProcessContext context, final T transfer) throws IOException {
+
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context, final ProcessSession session) {
+ FlowFile flowFile = session.get();
+ if (flowFile == null) {
+ return;
+ }
+
+ final ProcessorLog logger = getLogger();
+ final String hostname = context.getProperty(FileTransfer.HOSTNAME).evaluateAttributeExpressions(flowFile).getValue();
+
+ final int maxNumberOfFiles = context.getProperty(FileTransfer.BATCH_SIZE).asInteger();
+ int fileCount = 0;
+ try (final T transfer = getFileTransfer(context)) {
+ do {
+ final String rootPath = context.getProperty(FileTransfer.REMOTE_PATH).evaluateAttributeExpressions(flowFile).getValue();
+ final String workingDirPath;
+ if (rootPath == null) {
+ workingDirPath = null;
+ } else {
+ File workingDirectory = new File(rootPath);
+ if (!workingDirectory.getPath().startsWith("/") && !workingDirectory.getPath().startsWith("\\")) {
+ workingDirectory = new File(transfer.getHomeDirectory(flowFile), workingDirectory.getPath());
+ }
+ workingDirPath = workingDirectory.getPath().replace("\\", "/");
+ }
+
+ final boolean rejectZeroByteFiles = context.getProperty(FileTransfer.REJECT_ZERO_BYTE).asBoolean();
+ final ConflictResult conflictResult = identifyAndResolveConflictFile(context.getProperty(FileTransfer.CONFLICT_RESOLUTION).getValue(),
+ transfer, workingDirPath, flowFile, rejectZeroByteFiles, logger);
+
+ if (conflictResult.isTransfer()) {
+ final StopWatch stopWatch = new StopWatch();
+ stopWatch.start();
+
+ beforePut(flowFile, context, transfer);
+ final FlowFile flowFileToTransfer = flowFile;
+ final ObjectHolder<String> fullPathRef = new ObjectHolder<>(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(SFTPTransfer.CREATE_DIRECTORY).asBoolean()) {
+ transfer.ensureDirectoryExists(flowFileToTransfer, new File(workingDirPath));
+ }
+
+ fullPathRef.set(transfer.put(flowFileToTransfer, workingDirPath, conflictResult.getFileName(), bufferedIn));
+ }
+ }
+ });
+ afterPut(flowFile, context, transfer);
+
+ stopWatch.stop();
+ final String dataRate = stopWatch.calculateDataRate(flowFile.getSize());
+ final long millis = stopWatch.getDuration(TimeUnit.MILLISECONDS);
+ logger.info("Successfully transfered {} to {} on remote host {} in {} milliseconds at a rate of {}",
+ new Object[]{flowFile, fullPathRef.get(), hostname, millis, dataRate});
+
+ String fullPathWithSlash = fullPathRef.get();
+ if (!fullPathWithSlash.startsWith("/")) {
+ fullPathWithSlash = "/" + fullPathWithSlash;
+ }
+ final String destinationUri = transfer.getProtocolName() + "://" + hostname + fullPathWithSlash;
+ session.getProvenanceReporter().send(flowFile, destinationUri, millis);
+ }
+
+ if (conflictResult.isPenalize()) {
+ flowFile = session.penalize(flowFile);
+ }
+
+ session.transfer(flowFile, conflictResult.getRelationship());
+ session.commit();
- } while (isScheduled() && (getRelationships().size() == session.getAvailableRelationships().size()) && (++fileCount < maxNumberOfFiles) && ((flowFile = session.get()) != null));
++ } while (isScheduled() && (getRelationships().size() == context.getAvailableRelationships().size()) && (++fileCount < maxNumberOfFiles) && ((flowFile = session.get()) != null));
+ } catch (final IOException e) {
+ context.yield();
+ logger.error("Unable to transfer {} to remote host {} due to {}", new Object[]{flowFile, hostname, e});
+ flowFile = session.penalize(flowFile);
+ session.transfer(flowFile, REL_FAILURE);
+ } catch (final FlowFileAccessException e) {
+ context.yield();
+ logger.error("Unable to transfer {} to remote host {} due to {}", new Object[]{flowFile, hostname, e.getCause()});
+ flowFile = session.penalize(flowFile);
+ session.transfer(flowFile, REL_FAILURE);
+ } catch (final ProcessException e) {
+ context.yield();
+ logger.error("Unable to transfer {} to remote host {} due to {}: {}; routing to failure", new Object[]{flowFile, hostname, e, e.getCause()});
+ flowFile = session.penalize(flowFile);
+ session.transfer(flowFile, REL_FAILURE);
+ }
+ }
+
+ //Attempts to identify naming or content issues with files before they are transferred.
+ private ConflictResult identifyAndResolveConflictFile(final String conflictResolutionType, final T transfer, final String path, final FlowFile flowFile, final boolean rejectZeroByteFiles, final ProcessorLog logger) throws IOException {
+ Relationship destinationRelationship = REL_SUCCESS;
+ String fileName = flowFile.getAttribute(CoreAttributes.FILENAME.key());
+ boolean transferFile = true;
+ boolean penalizeFile = false;
+
+ //First, check if the file is empty
+ //Reject files that are zero bytes or less
+ if (rejectZeroByteFiles) {
+ final long sizeInBytes = flowFile.getSize();
+ if (sizeInBytes == 0) {
+ logger.warn("Rejecting {} because it is zero bytes", new Object[]{flowFile});
+ return new ConflictResult(REL_REJECT, false, fileName, true);
+ }
+ }
+
+ //Second, check if the user doesn't care about detecting naming conflicts ahead of time
+ if (conflictResolutionType.equalsIgnoreCase(FileTransfer.CONFLICT_RESOLUTION_NONE)) {
+ return new ConflictResult(destinationRelationship, transferFile, fileName, penalizeFile);
+ }
+
+ final FileInfo remoteFileInfo = transfer.getRemoteFileInfo(flowFile, path, fileName);
+ if (remoteFileInfo == null) {
+ return new ConflictResult(destinationRelationship, transferFile, fileName, penalizeFile);
+ }
+
+ if (remoteFileInfo.isDirectory()) {
+ logger.info("Resolving conflict by rejecting {} due to conflicting filename with a directory or file already on remote server", new Object[]{flowFile});
+ return new ConflictResult(REL_REJECT, false, fileName, false);
+ }
+
+ logger.info("Discovered a filename conflict on the remote server for {} so handling using configured Conflict Resolution of {}",
+ new Object[]{flowFile, conflictResolutionType});
+
+ switch (conflictResolutionType.toUpperCase()) {
+ case FileTransfer.CONFLICT_RESOLUTION_REJECT:
+ destinationRelationship = REL_REJECT;
+ transferFile = false;
+ penalizeFile = false;
+ logger.info("Resolving conflict by rejecting {} due to conflicting filename with a directory or file already on remote server", new Object[]{flowFile});
+ break;
+ case FileTransfer.CONFLICT_RESOLUTION_REPLACE:
+ transfer.deleteFile(path, fileName);
+ destinationRelationship = REL_SUCCESS;
+ transferFile = true;
+ penalizeFile = false;
+ logger.info("Resolving filename conflict for {} with remote server by deleting remote file and replacing with flow file", new Object[]{flowFile});
+ break;
+ case FileTransfer.CONFLICT_RESOLUTION_RENAME:
+ boolean uniqueNameGenerated = false;
+ for (int i = 1; i < 100 && !uniqueNameGenerated; i++) {
+ String possibleFileName = i + "." + fileName;
+
+ final FileInfo renamedFileInfo = transfer.getRemoteFileInfo(flowFile, path, possibleFileName);
+ uniqueNameGenerated = (renamedFileInfo == null);
+ if (uniqueNameGenerated) {
+ fileName = possibleFileName;
+ logger.info("Attempting to resolve filename conflict for {} on the remote server by using a newly generated filename of: {}", new Object[]{flowFile, fileName});
+ destinationRelationship = REL_SUCCESS;
+ transferFile = true;
+ penalizeFile = false;
+ break;
+ }
+ }
+ if (!uniqueNameGenerated) {
+ destinationRelationship = REL_REJECT;
+ transferFile = false;
+ penalizeFile = false;
+ logger.info("Could not determine a unique name after 99 attempts for. Switching resolution mode to REJECT for " + flowFile);
+ }
+ break;
+ case FileTransfer.CONFLICT_RESOLUTION_IGNORE:
+ destinationRelationship = REL_SUCCESS;
+ transferFile = false;
+ penalizeFile = false;
+ logger.info("Resolving conflict for {} by not transferring file and and still considering the process a success.", new Object[]{flowFile});
+ break;
+ case FileTransfer.CONFLICT_RESOLUTION_FAIL:
+ destinationRelationship = REL_FAILURE;
+ transferFile = false;
+ penalizeFile = true;
+ logger.info("Resolved filename conflict for {} as configured by routing to FAILURE relationship.", new Object[]{flowFile});
+ default:
+ break;
+ }
+
+ return new ConflictResult(destinationRelationship, transferFile, fileName, penalizeFile);
+ }
+
+ /**
+ * static inner class to hold conflict data
+ */
+ private static class ConflictResult {
+
+ final Relationship relationship;
+ final boolean transferFile;
+ final String newFileName;
+ final boolean penalizeFile;
+
+ public ConflictResult(final Relationship relationship, final boolean transferFileVal, final String newFileNameVal, final boolean penalizeFileVal) {
+ this.relationship = relationship;
+ this.transferFile = transferFileVal;
+ this.newFileName = newFileNameVal;
+ this.penalizeFile = penalizeFileVal;
+ }
+
+ public boolean isTransfer() {
+ return transferFile;
+ }
+
+ public boolean isPenalize() {
+ return penalizeFile;
+ }
+
+ public String getFileName() {
+ return newFileName;
+ }
+
+ public Relationship getRelationship() {
+ return relationship;
+ }
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
index 0000000,cae61f0..1cf5f1f
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/main/java/org/apache/nifi/processors/standard/servlets/ListenHTTPServlet.java
@@@ -1,0 -1,320 +1,325 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processors.standard.servlets;
+
+ import java.io.IOException;
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.security.cert.X509Certificate;
+ import java.util.Enumeration;
+ import java.util.HashMap;
+ import java.util.HashSet;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.ConcurrentMap;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicLong;
+ import java.util.concurrent.atomic.AtomicReference;
+ import java.util.regex.Pattern;
+ import java.util.zip.GZIPInputStream;
+
+ import javax.servlet.ServletConfig;
+ import javax.servlet.ServletContext;
+ import javax.servlet.ServletException;
+ import javax.servlet.http.HttpServlet;
+ import javax.servlet.http.HttpServletRequest;
+ import javax.servlet.http.HttpServletResponse;
+ import javax.ws.rs.Path;
+ import javax.ws.rs.core.MediaType;
+
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.flowfile.attributes.CoreAttributes;
+ import org.apache.nifi.stream.io.BufferedOutputStream;
+ import org.apache.nifi.stream.io.StreamThrottler;
+ import org.apache.nifi.logging.ProcessorLog;
++import org.apache.nifi.processor.ProcessContext;
+ import org.apache.nifi.processor.ProcessSession;
+ import org.apache.nifi.processor.ProcessSessionFactory;
+ import org.apache.nifi.processor.io.OutputStreamCallback;
+ import org.apache.nifi.processors.standard.ListenHTTP;
+ import org.apache.nifi.processors.standard.ListenHTTP.FlowFileEntryTimeWrapper;
+ import org.apache.nifi.util.FlowFileUnpackager;
+ import org.apache.nifi.util.FlowFileUnpackagerV1;
+ import org.apache.nifi.util.FlowFileUnpackagerV2;
+ import org.apache.nifi.util.FlowFileUnpackagerV3;
+ import org.apache.commons.io.IOUtils;
+ import org.apache.commons.lang3.StringUtils;
+
+ @Path(ListenHTTP.URI)
+ public class ListenHTTPServlet extends HttpServlet {
+
+ private static final long serialVersionUID = 5329940480987723163L;
+
+ public static final String FLOWFILE_CONFIRMATION_HEADER = "x-prefer-acknowledge-uri";
+ public static final String LOCATION_HEADER_NAME = "Location";
+ public static final String DEFAULT_FOUND_SUBJECT = "none";
+ public static final String APPLICATION_FLOW_FILE_V1 = "application/flowfile";
+ public static final String APPLICATION_FLOW_FILE_V2 = "application/flowfile-v2";
+ public static final String APPLICATION_FLOW_FILE_V3 = "application/flowfile-v3";
+ public static final String LOCATION_URI_INTENT_NAME = "x-location-uri-intent";
+ public static final String LOCATION_URI_INTENT_VALUE = "flowfile-hold";
+ public static final int FILES_BEFORE_CHECKING_DESTINATION_SPACE = 5;
+ public static final String ACCEPT_HEADER_NAME = "Accept";
+ public static final String ACCEPT_HEADER_VALUE = APPLICATION_FLOW_FILE_V3 + "," + APPLICATION_FLOW_FILE_V2 + "," + APPLICATION_FLOW_FILE_V1 + ",*/*;q=0.8";
+ public static final String ACCEPT_ENCODING_NAME = "Accept-Encoding";
+ public static final String ACCEPT_ENCODING_VALUE = "gzip";
+ public static final String GZIPPED_HEADER = "flowfile-gzipped";
+ public static final String PROTOCOL_VERSION_HEADER = "x-nifi-transfer-protocol-version";
+ public static final String PROTOCOL_VERSION = "3";
+
+ private final AtomicLong filesReceived = new AtomicLong(0L);
+ private final AtomicBoolean spaceAvailable = new AtomicBoolean(true);
+
+ private ProcessorLog logger;
+ private AtomicReference<ProcessSessionFactory> sessionFactoryHolder;
++ private volatile ProcessContext processContext;
+ private Pattern authorizedPattern;
+ private Pattern headerPattern;
+ private ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap;
+ private StreamThrottler streamThrottler;
+
+ /**
+ *
+ * @param config
+ * @throws ServletException
+ */
+ @SuppressWarnings("unchecked")
+ @Override
+ public void init(final ServletConfig config) throws ServletException {
+ final ServletContext context = config.getServletContext();
+ this.logger = (ProcessorLog) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_LOGGER);
+ this.sessionFactoryHolder = (AtomicReference<ProcessSessionFactory>) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_SESSION_FACTORY_HOLDER);
++ this.processContext = (ProcessContext) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_PROCESS_CONTEXT_HOLDER);
+ this.authorizedPattern = (Pattern) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_AUTHORITY_PATTERN);
+ this.headerPattern = (Pattern) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_HEADER_PATTERN);
+ this.flowFileMap = (ConcurrentMap<String, FlowFileEntryTimeWrapper>) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_FLOWFILE_MAP);
+ this.streamThrottler = (StreamThrottler) context.getAttribute(ListenHTTP.CONTEXT_ATTRIBUTE_STREAM_THROTTLER);
+ }
+
+ @Override
+ protected void doHead(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
+ response.addHeader(ACCEPT_ENCODING_NAME, ACCEPT_ENCODING_VALUE);
+ response.addHeader(ACCEPT_HEADER_NAME, ACCEPT_HEADER_VALUE);
+ response.addHeader(PROTOCOL_VERSION_HEADER, PROTOCOL_VERSION);
+ }
+
+ @Override
+ protected void doPost(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
++ final ProcessContext context = processContext;
++
+ ProcessSessionFactory sessionFactory;
+ do {
+ sessionFactory = sessionFactoryHolder.get();
+ if (sessionFactory == null) {
+ try {
+ Thread.sleep(10);
+ } catch (final InterruptedException e) {
+ }
+ }
+ } while (sessionFactory == null);
+
+ final ProcessSession session = sessionFactory.createSession();
+ FlowFile flowFile = null;
+ String holdUuid = null;
+ String foundSubject = null;
+ try {
+ final long n = filesReceived.getAndIncrement() % FILES_BEFORE_CHECKING_DESTINATION_SPACE;
+ if (n == 0 || !spaceAvailable.get()) {
- if (session.getAvailableRelationships().isEmpty()) {
++ if (context.getAvailableRelationships().isEmpty()) {
+ spaceAvailable.set(false);
+ if (logger.isDebugEnabled()) {
+ logger.debug("Received request from " + request.getRemoteHost() + " but no space available; Indicating Service Unavailable");
+ }
+ response.sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE);
+ return;
+ } else {
+ spaceAvailable.set(true);
+ }
+ }
+ response.setHeader("Content-Type", MediaType.TEXT_PLAIN);
+
+ final boolean contentGzipped = Boolean.parseBoolean(request.getHeader(GZIPPED_HEADER));
+
+ final X509Certificate[] certs = (X509Certificate[]) request.getAttribute("javax.servlet.request.X509Certificate");
+ foundSubject = DEFAULT_FOUND_SUBJECT;
+ if (certs != null && certs.length > 0) {
+ for (final X509Certificate cert : certs) {
+ foundSubject = cert.getSubjectDN().getName();
+ if (authorizedPattern.matcher(foundSubject).matches()) {
+ break;
+ } else {
+ logger.warn("Rejecting transfer attempt from " + foundSubject + " because the DN is not authorized, host=" + request.getRemoteHost());
+ response.sendError(HttpServletResponse.SC_FORBIDDEN, "not allowed based on dn");
+ return;
+ }
+ }
+ }
+
+ final String destinationVersion = request.getHeader(PROTOCOL_VERSION_HEADER);
+ Integer protocolVersion = null;
+ if (destinationVersion != null) {
+ try {
+ protocolVersion = Integer.valueOf(destinationVersion);
+ } catch (final NumberFormatException e) {
+ // Value was invalid. Treat as if the header were missing.
+ }
+ }
+
+ final boolean destinationIsLegacyNiFi = (protocolVersion == null);
+ final boolean createHold = Boolean.parseBoolean(request.getHeader(FLOWFILE_CONFIRMATION_HEADER));
+ final String contentType = request.getContentType();
+
+ final InputStream unthrottled = contentGzipped ? new GZIPInputStream(request.getInputStream()) : request.getInputStream();
+
+ final InputStream in = (streamThrottler == null) ? unthrottled : streamThrottler.newThrottledInputStream(unthrottled);
+
+ if (logger.isDebugEnabled()) {
+ logger.debug("Received request from " + request.getRemoteHost() + ", createHold=" + createHold + ", content-type=" + contentType + ", gzip=" + contentGzipped);
+ }
+
+ final AtomicBoolean hasMoreData = new AtomicBoolean(false);
+ final FlowFileUnpackager unpackager;
+ if (APPLICATION_FLOW_FILE_V3.equals(contentType)) {
+ unpackager = new FlowFileUnpackagerV3();
+ } else if (APPLICATION_FLOW_FILE_V2.equals(contentType)) {
+ unpackager = new FlowFileUnpackagerV2();
+ } else if (APPLICATION_FLOW_FILE_V1.equals(contentType)) {
+ unpackager = new FlowFileUnpackagerV1();
+ } else {
+ unpackager = null;
+ }
+
+ final Set<FlowFile> flowFileSet = new HashSet<>();
+
+ do {
+ final long startNanos = System.nanoTime();
+ final Map<String, String> attributes = new HashMap<>();
+ flowFile = session.create();
+ flowFile = session.write(flowFile, new OutputStreamCallback() {
+ @Override
+ public void process(final OutputStream rawOut) throws IOException {
+ try (final BufferedOutputStream bos = new BufferedOutputStream(rawOut, 65536)) {
+ if (unpackager == null) {
+ IOUtils.copy(in, bos);
+ hasMoreData.set(false);
+ } else {
+ attributes.putAll(unpackager.unpackageFlowFile(in, bos));
+
+ if (destinationIsLegacyNiFi) {
+ if (attributes.containsKey("nf.file.name")) {
+ // for backward compatibility with old nifi...
+ attributes.put(CoreAttributes.FILENAME.key(), attributes.remove("nf.file.name"));
+ }
+
+ if (attributes.containsKey("nf.file.path")) {
+ attributes.put(CoreAttributes.PATH.key(), attributes.remove("nf.file.path"));
+ }
+ }
+
+ // remove deprecated FlowFile attribute that was used in older versions of NiFi
+ attributes.remove("parent.uuid");
+
+ hasMoreData.set(unpackager.hasMoreData());
+ }
+ }
+ }
+ });
+
+ final long transferNanos = System.nanoTime() - startNanos;
+ final long transferMillis = TimeUnit.MILLISECONDS.convert(transferNanos, TimeUnit.NANOSECONDS);
+
+ // put metadata on flowfile
+ final String nameVal = request.getHeader(CoreAttributes.FILENAME.key());
+ if (StringUtils.isNotBlank(nameVal)) {
+ attributes.put(CoreAttributes.FILENAME.key(), nameVal);
+ }
+
+ // put arbitrary headers on flow file
+ for(Enumeration<String> headerEnum = request.getHeaderNames();
+ headerEnum.hasMoreElements(); ) {
+ String headerName = headerEnum.nextElement();
+ if (headerPattern != null && headerPattern.matcher(headerName).matches()) {
+ String headerValue = request.getHeader(headerName);
+ attributes.put(headerName, headerValue);
+ }
+ }
+
+ String sourceSystemFlowFileIdentifier = attributes.get(CoreAttributes.UUID.key());
+ if (sourceSystemFlowFileIdentifier != null) {
+ sourceSystemFlowFileIdentifier = "urn:nifi:" + sourceSystemFlowFileIdentifier;
+
+ // If we receveied a UUID, we want to give the FlowFile a new UUID and register the sending system's
+ // identifier as the SourceSystemFlowFileIdentifier field in the Provenance RECEIVE event
+ attributes.put(CoreAttributes.UUID.key(), UUID.randomUUID().toString());
+ }
+
+ flowFile = session.putAllAttributes(flowFile, attributes);
+ session.getProvenanceReporter().receive(flowFile, request.getRequestURL().toString(), sourceSystemFlowFileIdentifier, "Remote DN=" + foundSubject, transferMillis);
+ flowFile = session.putAttribute(flowFile, "restlistener.remote.user.dn", foundSubject);
+ flowFileSet.add(flowFile);
+
+ if (holdUuid == null) {
+ holdUuid = flowFile.getAttribute(CoreAttributes.UUID.key());
+ }
+ } while (hasMoreData.get());
+
+ if (createHold) {
+ String uuid = (holdUuid == null) ? UUID.randomUUID().toString() : holdUuid;
+
+ if (flowFileMap.containsKey(uuid)) {
+ uuid = UUID.randomUUID().toString();
+ }
+
+ final FlowFileEntryTimeWrapper wrapper = new FlowFileEntryTimeWrapper(session, flowFileSet, System.currentTimeMillis());
+ FlowFileEntryTimeWrapper previousWrapper;
+ do {
+ previousWrapper = flowFileMap.putIfAbsent(uuid, wrapper);
+ if (previousWrapper != null) {
+ uuid = UUID.randomUUID().toString();
+ }
+ } while (previousWrapper != null);
+
+ response.setStatus(HttpServletResponse.SC_SEE_OTHER);
+ final String ackUri = ListenHTTP.URI + "/holds/" + uuid;
+ response.addHeader(LOCATION_HEADER_NAME, ackUri);
+ response.addHeader(LOCATION_URI_INTENT_NAME, LOCATION_URI_INTENT_VALUE);
+ response.getOutputStream().write(ackUri.getBytes("UTF-8"));
+ if (logger.isDebugEnabled()) {
+ logger.debug("Ingested {} from Remote Host: [{}] Port [{}] SubjectDN [{}]; placed hold on these {} files with ID {}",
+ new Object[]{flowFileSet, request.getRemoteHost(), request.getRemotePort(), foundSubject, flowFileSet.size(), uuid});
+ }
+ } else {
+ response.setStatus(HttpServletResponse.SC_OK);
+ logger.info("Received from Remote Host: [{}] Port [{}] SubjectDN [{}]; transferring to 'success' {}",
+ new Object[]{request.getRemoteHost(), request.getRemotePort(), foundSubject, flowFile});
+
+ session.transfer(flowFileSet, ListenHTTP.RELATIONSHIP_SUCCESS);
+ session.commit();
+ }
+ } catch (final Throwable t) {
+ session.rollback();
+ if (flowFile == null) {
+ logger.error("Unable to receive file from Remote Host: [{}] SubjectDN [{}] due to {}", new Object[]{request.getRemoteHost(), foundSubject, t});
+ } else {
+ logger.error("Unable to receive file {} from Remote Host: [{}] SubjectDN [{}] due to {}", new Object[]{flowFile, request.getRemoteHost(), foundSubject, t});
+ }
+ response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, t.toString());
+ }
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
----------------------------------------------------------------------
diff --cc nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
index 0000000,a6402e4..ab4c978
mode 000000,100644..100644
--- a/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
+++ b/nifi/nar-bundles/standard-bundle/standard-processors/src/test/java/org/apache/nifi/processors/standard/TestDistributeLoad.java
@@@ -1,0 -1,138 +1,139 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processors.standard;
+
+ import org.apache.nifi.processors.standard.DistributeLoad;
+ import org.apache.nifi.util.TestRunner;
+ import org.apache.nifi.util.TestRunners;
+
+ import org.junit.Assert;
+ import org.junit.BeforeClass;
+ import org.junit.Test;
+
+ public class TestDistributeLoad {
+
+ @BeforeClass
+ public static void before() {
+ System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+ System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+ System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DistributeLoad", "debug");
+ }
+
+ @Test
+ public void testDefaultRoundRobin() {
+ final TestRunner testRunner = TestRunners.newTestRunner(new DistributeLoad());
+ testRunner.setProperty(DistributeLoad.NUM_RELATIONSHIPS, "100");
+
+ for (int i = 0; i < 101; i++) {
+ testRunner.enqueue(new byte[0]);
+ }
+
+ testRunner.run(101);
+ testRunner.assertTransferCount("1", 2);
+ for (int i = 2; i <= 100; i++) {
+ testRunner.assertTransferCount(String.valueOf(i), 1);
+ }
+ }
+
+ @Test
+ public void testWeightedRoundRobin() {
+ final TestRunner testRunner = TestRunners.newTestRunner(new DistributeLoad());
+ testRunner.setProperty(DistributeLoad.NUM_RELATIONSHIPS, "100");
+
+ testRunner.setProperty("1", "5");
+ testRunner.setProperty("2", "3");
+
+ for (int i = 0; i < 106; i++) {
+ testRunner.enqueue(new byte[0]);
+ }
+
+ testRunner.run(108);
+ testRunner.assertTransferCount("1", 5);
+ testRunner.assertTransferCount("2", 3);
+ for (int i = 3; i <= 100; i++) {
+ testRunner.assertTransferCount(String.valueOf(i), 1);
+ }
+ }
+
+ @Test
+ public void testValidationOnAddedProperties() {
+ final TestRunner testRunner = TestRunners.newTestRunner(new DistributeLoad());
+ testRunner.setProperty(DistributeLoad.NUM_RELATIONSHIPS, "100");
+
+ testRunner.setProperty("1", "5");
+
+ try {
+ testRunner.setProperty("1", "0");
+ Assert.fail("Allows property '1' to be set to '0'");
+ } catch (final AssertionError e) {
+ // expected behavior
+ }
+
+ try {
+ testRunner.setProperty("1", "-1");
+ Assert.fail("Allows property '1' to be set to '-1'");
+ } catch (final AssertionError e) {
+ // expected behavior
+ }
+
+ testRunner.setProperty("1", "101");
+ testRunner.setProperty("100", "5");
+
+ try {
+ testRunner.setProperty("101", "5");
+ Assert.fail("Allows property '101' to be set to '5'");
+ } catch (final AssertionError e) {
+ // expected behavior
+ }
+
+ try {
+ testRunner.setProperty("0", "5");
+ Assert.fail("Allows property '0' to be set to '5'");
+ } catch (final AssertionError e) {
+ // expected behavior
+ }
+
+ try {
+ testRunner.setProperty("-1", "5");
+ Assert.fail("Allows property '-1' to be set to '5'");
+ } catch (final AssertionError e) {
+ // expected behavior
+ }
+ }
+
+ @Test
+ public void testNextAvailable() {
+ final TestRunner testRunner = TestRunners.newTestRunner(new DistributeLoad());
+
+ testRunner.setProperty(DistributeLoad.NUM_RELATIONSHIPS.getName(), "100");
+ testRunner.setProperty(DistributeLoad.DISTRIBUTION_STRATEGY.getName(), DistributeLoad.STRATEGY_NEXT_AVAILABLE);
+
+ for (int i = 0; i < 99; i++) {
+ testRunner.enqueue(new byte[0]);
+ }
+
+ testRunner.setRelationshipUnavailable("50");
+
+ testRunner.run(101);
+ testRunner.assertQueueEmpty();
+
+ for (int i = 1; i <= 100; i++) {
++ System.out.println(i);
+ testRunner.assertTransferCount(String.valueOf(i), (i == 50) ? 0 : 1);
+ }
+ }
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
----------------------------------------------------------------------
diff --cc nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
index 0000000,9e04439..7fa183f
mode 000000,100644..100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessContext.java
@@@ -1,0 -1,124 +1,132 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processor;
+
+ import java.util.Map;
++import java.util.Set;
+
+ import org.apache.nifi.components.PropertyDescriptor;
+ import org.apache.nifi.components.PropertyValue;
+ import org.apache.nifi.controller.ControllerServiceLookup;
+
+ /**
+ * <p>
+ * Provides a bridge between a Processor and the NiFi Framework
+ * </p>
+ *
+ * <p>
+ * <b>Note: </b>Implementations of this interface are NOT necessarily
+ * thread-safe.
+ * </p>
+ */
+ public interface ProcessContext {
+
+ /**
+ * Retrieves the current value set for the given descriptor, if a value is
+ * set - else uses the descriptor to determine the appropriate default value
+ *
+ * @param descriptor
+ * @return
+ */
+ PropertyValue getProperty(PropertyDescriptor descriptor);
+
+ /**
+ * Retrieves the current value set for the given descriptor, if a value is
+ * set - else uses the descriptor to determine the appropriate default value
+ *
+ * @param propertyName
+ * @return
+ */
+ PropertyValue getProperty(String propertyName);
+
+ /**
+ * Creates and returns a {@link PropertyValue} object that can be used for
+ * evaluating the value of the given String
+ *
+ * @param rawValue
+ * @return
+ */
+ PropertyValue newPropertyValue(String rawValue);
+
+ /**
+ * <p>
+ * Causes the Processor not to be scheduled for some pre-configured amount
+ * of time. The duration of time for which the processor will not be
+ * scheduled is configured in the same manner as the processor's scheduling
+ * period.
+ * </p>
+ *
+ * <p>
+ * <b>Note: </b>This is NOT a blocking call and does not suspend execution
+ * of the current thread.
+ * </p>
+ */
+ void yield();
+
+ /**
+ * @return the maximum number of threads that may be executing this
+ * processor's code at any given time
+ */
+ int getMaxConcurrentTasks();
+
+ /**
+ * @return the annotation data configured for this processor
+ */
+ String getAnnotationData();
+
+ /**
+ * Returns a Map of all PropertyDescriptors to their configured values. This
+ * Map may or may not be modifiable, but modifying its values will not
+ * change the values of the processor's properties
+ *
+ * @return
+ */
+ Map<PropertyDescriptor, String> getProperties();
+
+ /**
+ * Encrypts the given value using the password provided in the NiFi
+ * Properties
+ *
+ * @param unencrypted
+ * @return
+ */
+ String encrypt(String unencrypted);
+
+ /**
+ * Decrypts the given value using the password provided in the NiFi
+ * Properties
+ *
+ * @param encrypted
+ * @return
+ */
+ String decrypt(String encrypted);
+
+ /**
+ * Provides a {@code ControllerServiceLookup} that can be used to obtain a
+ * Controller Service
+ *
+ * @return
+ */
+ ControllerServiceLookup getControllerServiceLookup();
++
++ /**
++ * @return the set of all relationships for which space is available to
++ * receive new objects
++ */
++ Set<Relationship> getAvailableRelationships();
++
+ }
http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/73384b23/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
----------------------------------------------------------------------
diff --cc nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
index 0000000,09d1bd2..d3de916
mode 000000,100644..100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java
@@@ -1,0 -1,719 +1,713 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.nifi.processor;
+
+ import java.io.InputStream;
+ import java.io.OutputStream;
+ import java.nio.file.Path;
+ import java.util.Collection;
+ import java.util.List;
+ import java.util.Map;
+ import java.util.Set;
+ import java.util.regex.Pattern;
+
+ import org.apache.nifi.flowfile.FlowFile;
+ import org.apache.nifi.processor.exception.FlowFileAccessException;
+ import org.apache.nifi.processor.exception.FlowFileHandlingException;
+ import org.apache.nifi.processor.exception.MissingFlowFileException;
+ import org.apache.nifi.processor.exception.ProcessException;
+ import org.apache.nifi.processor.io.InputStreamCallback;
+ import org.apache.nifi.processor.io.OutputStreamCallback;
+ import org.apache.nifi.processor.io.StreamCallback;
+ import org.apache.nifi.provenance.ProvenanceReporter;
+
+ /**
+ * <p>
+ * A process session encompasses all the behaviors a processor can perform to
+ * obtain, clone, read, modify remove FlowFiles in an atomic unit. A process
+ * session is always tied to a single processor at any one time and ensures no
+ * FlowFile can ever be accessed by any more than one processor at a given time.
+ * The session also ensures that all FlowFiles are always accounted for. The
+ * creator of a ProcessSession is always required to manage the session.</p>
+ *
+ * <p>
+ * A session is not considered thread safe. The session supports a unit of work
+ * that is either committed or rolled back</p>
+ *
+ * <p>
+ * As noted on specific methods and for specific exceptions automated rollback
+ * will occur to ensure consistency of the repository. However, several
+ * situations can result in exceptions yet not cause automated rollback. In
+ * these cases the consistency of the repository will be retained but callers
+ * will be able to indicate whether it should result in rollback or continue on
+ * toward a commit.</p>
+ *
+ * <p>
+ * A process session instance may be used continuously. That is, after each
+ * commit or rollback, the session can be used again.</p>
+ *
+ * @author unattributed
+ */
+ public interface ProcessSession {
+
+ /**
+ * <p>
+ * Commits the current session ensuring all operations against FlowFiles
+ * within this session are atomically persisted. All FlowFiles operated on
+ * within this session must be accounted for by transfer or removal or the
+ * commit will fail.</p>
+ *
+ * <p>
+ * As soon as the commit completes the session is again ready to be used</p>
+ *
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session.
+ * @throws FlowFileHandlingException if not all FlowFiles acted upon within
+ * this session are accounted for by user code such that they have a
+ * transfer identified or where marked for removal. Automated rollback
+ * occurs.
+ * @throws ProcessException if some general fault occurs while persisting
+ * the session. Initiates automatic rollback. The root cause can be obtained
+ * via <code>Exception.getCause()</code>
+ */
+ void commit();
+
+ /**
+ * Reverts any changes made during this session. All FlowFiles are restored
+ * back to their initial session state and back to their original queues. If
+ * this session is already committed or rolled back then no changes will
+ * occur. This method can be called any number of times. Calling this method
+ * is identical to calling {@link #rollback(boolean)} passing
+ * <code>false</code> as the parameter.
+ */
+ void rollback();
+
+ /**
+ * Reverts any changes made during this session. All FlowFiles are restored
+ * back to their initial session state and back to their original queues,
+ * after optionally being penalized. If this session is already committed or
+ * rolled back then no changes will occur. This method can be called any
+ * number of times.
+ *
+ * @param penalize whether or not the FlowFiles that are being restored back
+ * to their queues should be penalized
+ */
+ void rollback(boolean penalize);
+
+ /**
+ * Adjusts counter data for the given counter name and takes care of
+ * registering the counter if not already present. The adjustment occurs
+ * only if and when the ProcessSession is committed.
+ *
+ * @param name the name of the counter
+ * @param delta the delta by which to modify the counter (+ or -)
+ * @param immediate if true, the counter will be updated immediately,
+ * without regard to whether the ProcessSession is commit or rolled back;
+ * otherwise, the counter will be incremented only if and when the
+ * ProcessSession is committed.
+ */
+ void adjustCounter(String name, long delta, boolean immediate);
+
+ /**
+ * @return FlowFile that is next highest priority FlowFile to process.
+ * Otherwise returns null.
+ */
+ FlowFile get();
+
+ /**
+ * Returns up to <code>maxResults</code> FlowFiles from the work queue. If
+ * no FlowFiles are available, returns an empty list. Will not return null.
+ * If multiple incoming queues are present, the behavior is unspecified in
+ * terms of whether all queues or only a single queue will be polled in a
+ * single call.
+ *
+ * @param maxResults the maximum number of FlowFiles to return
+ * @return
+ * @throws IllegalArgumentException if <code>maxResults</code> is less than
+ * 0
+ */
+ List<FlowFile> get(int maxResults);
+
+ /**
+ * <p>
+ * Returns all FlowFiles from all of the incoming queues for which the given
+ * {@link FlowFileFilter} indicates should be accepted. Calls to this method
+ * provide exclusive access to the underlying queues. I.e., no other thread
+ * will be permitted to pull FlowFiles from this Processor's queues or add
+ * FlowFiles to this Processor's incoming queues until this method call has
+ * returned.
+ * </p>
+ *
+ * @param filter
+ * @return
+ */
+ List<FlowFile> get(FlowFileFilter filter);
+
+ /**
+ * @return the QueueSize that represents the number of FlowFiles and their
+ * combined data size for all FlowFiles waiting to be processed by the
+ * Processor that owns this ProcessSession, regardless of which Connection
+ * the FlowFiles live on
+ */
+ QueueSize getQueueSize();
+
+ /**
- * @return the set of all relationships for which space is available to
- * receive new objects
- */
- Set<Relationship> getAvailableRelationships();
-
- /**
+ * Creates a new FlowFile in the repository with no content and without any
+ * linkage to a parent FlowFile. This method is appropriate only when data
+ * is received or created from an external system. Otherwise, this method
+ * should be avoided and should instead use {@link #create(FlowFile)} or
+ * {@link #create(Collection<FlowFile>)}.
+ *
+ * When this method is used, a Provenance CREATE or RECEIVE Event should be
+ * generated. See the {@link #getProvenanceReporter()} method and
+ * {@link ProvenanceReporter} class for more information
+ *
+ * @return newly created FlowFile
+ */
+ FlowFile create();
+
+ /**
+ * Creates a new FlowFile in the repository with no content but with a
+ * parent linkage to <code>parent</code>. The newly created FlowFile will
+ * inherit all of the parent's attributes except for the UUID. This method
+ * will automatically generate a Provenance FORK event or a Provenance JOIN
+ * event, depending on whether or not other FlowFiles are generated from the
+ * same parent before the ProcessSession is committed.
+ *
+ * @param parent
+ * @return
+ */
+ FlowFile create(FlowFile parent);
+
+ /**
+ * Creates a new FlowFile in the repository with no content but with a
+ * parent linkage to the FlowFiles specified by the parents Collection. The
+ * newly created FlowFile will inherit all of the attributes that are in
+ * common to all parents (except for the UUID, which will be in common if
+ * only a single parent exists). This method will automatically generate a
+ * Provenance JOIN event.
+ *
+ * @param parents
+ * @return
+ */
+ FlowFile create(Collection<FlowFile> parents);
+
+ /**
+ * Creates a new FlowFile that is a clone of the given FlowFile as of the
+ * time this is called, both in content and attributes. This method
+ * automatically emits a Provenance CLONE Event.
+ *
+ * @param example FlowFile to be the source of cloning - given FlowFile must
+ * be a part of the given session
+ * @return FlowFile that is a clone of the given example
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content
+ * @throws NullPointerException if the argument null
+ */
+ FlowFile clone(FlowFile example);
+
+ /**
+ * Creates a new FlowFile whose parent is the given FlowFile. The content of
+ * the new FlowFile will be a subset of the byte sequence of the given
+ * FlowFile starting at the specified offset and with the length specified.
+ * The new FlowFile will contain all of the attributes of the original. This
+ * method automatically emits a Provenance FORK Event (or a Provenance CLONE
+ * Event, if the offset is 0 and the size is exactly equal to the size of
+ * the example FlowFile).
+ *
+ * @param example
+ * @param offset
+ * @param size
+ * @return a FlowFile with the specified size whose parent is first argument
+ * to this function
+ *
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session, or if the
+ * specified offset + size exceeds that of the size of the example FlowFile.
+ * Automatic rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ */
+ FlowFile clone(FlowFile example, long offset, long size);
+
+ /**
+ * Sets a penalty for the given FlowFile which will make it unavailable to
+ * be operated on any further during the penalty period.
+ *
+ * @param flowFile to penalize
+ * @return FlowFile the new FlowFile reference to use
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws NullPointerException if the argument null
+ */
+ FlowFile penalize(FlowFile flowFile);
+
+ /**
+ * Updates the given FlowFiles attributes with the given key/value pair. If
+ * the key is named {@code uuid}, this attribute will be ignored.
+ *
+ * @param flowFile to update
+ * @param key of attribute
+ * @param value of attribute
+ * @return FlowFile the updated FlowFile
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws NullPointerException if an argument is null
+ */
+ FlowFile putAttribute(FlowFile flowFile, String key, String value);
+
+ /**
+ * Updates the given FlowFiles attributes with the given key/value pairs. If
+ * the map contains a key named {@code uuid}, this attribute will be
+ * ignored.
+ *
+ * @param flowFile to update
+ * @param attributes the attributes to add to the given FlowFile
+ * @return FlowFile the updated FlowFile
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws NullPointerException if an argument is null
+ */
+ FlowFile putAllAttributes(FlowFile flowFile, Map<String, String> attributes);
+
+ /**
+ * Removes the given FlowFile attribute with the given key. If the key is
+ * named {@code uuid}, this method will return the same FlowFile without
+ * removing any attribute.
+ *
+ * @param flowFile to update
+ * @param key of attribute
+ * @return FlowFile the updated FlowFile
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws NullPointerException if the argument null
+ */
+ FlowFile removeAttribute(FlowFile flowFile, String key);
+
+ /**
+ * Removes the attributes with the given keys from the given FlowFile. If
+ * the set of keys contains the value {@code uuid}, this key will be ignored
+ *
+ * @param flowFile to update
+ * @param keys of attribute
+ * @return FlowFile the updated FlowFile
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws NullPointerException if the argument null
+ */
+ FlowFile removeAllAttributes(FlowFile flowFile, Set<String> keys);
+
+ /**
+ * Remove all attributes from the given FlowFile that have keys which match
+ * the given pattern. If the pattern matches the key {@code uuid}, this key
+ * will not be removed.
+ *
+ * @param flowFile to update
+ * @param keyPattern may be null; if supplied is matched against each of the
+ * FlowFile attribute keys
+ * @return FlowFile containing only attributes which did not meet the key
+ * pattern
+ */
+ FlowFile removeAllAttributes(FlowFile flowFile, Pattern keyPattern);
+
+ /**
+ * Transfers the given FlowFile to the appropriate destination processor
+ * work queue(s) based on the given relationship. If the relationship leads
+ * to more than one destination the state of the FlowFile is replicated such
+ * that each destination receives an exact copy of the FlowFile though each
+ * will have its own unique identity. The destination processors will not be
+ * able to operate on the given FlowFile until this session is committed or
+ * until the ownership of the session is migrated to another processor. If
+ * ownership of the session is passed to a destination processor then that
+ * destination processor will have immediate visibility of the transferred
+ * FlowFiles within the session.
+ *
+ * @param flowFile
+ * @param relationship
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws NullPointerException if the argument null
+ * @throws IllegalArgumentException if given relationship is not a known or
+ * registered relationship
+ */
+ void transfer(FlowFile flowFile, Relationship relationship);
+
+ /**
+ * Transfers the given FlowFile back to the work queue from which it was
+ * pulled. The processor will not be able to operate on the given FlowFile
+ * until this session is committed. Any modifications that have been made to
+ * the FlowFile will be maintained. FlowFiles that are created by the
+ * processor cannot be transferred back to themselves via this method.
+ *
+ * @param flowFile
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws IllegalArgumentException if the FlowFile was created by this
+ * processor
+ * @throws NullPointerException if the argument null
+ */
+ void transfer(FlowFile flowFile);
+
+ /**
+ * Transfers the given FlowFiles back to the work queues from which the
+ * FlowFiles were pulled. The processor will not be able to operate on the
+ * given FlowFile until this session is committed. Any modifications that
+ * have been made to the FlowFile will be maintained. FlowFiles that are
+ * created by the processor cannot be transferred back to themselves via
+ * this method.
+ *
+ * @param flowFiles
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFiles are already
+ * transferred or removed or don't belong to this session. Automatic
+ * rollback will occur.
+ * @throws IllegalArgumentException if the FlowFile was created by this
+ * processor
+ * @throws NullPointerException if the argument null
+ */
+ void transfer(Collection<FlowFile> flowFiles);
+
+ /**
+ * Transfers the given FlowFile to the appropriate destination processor
+ * work queue(s) based on the given relationship. If the relationship leads
+ * to more than one destination the state of the FlowFile is replicated such
+ * that each destination receives an exact copy of the FlowFile though each
+ * will have its own unique identity. The destination processors will not be
+ * able to operate on the given FlowFile until this session is committed or
+ * until the ownership of the session is migrated to another processor. If
+ * ownership of the session is passed to a destination processor then that
+ * destination processor will have immediate visibility of the transferred
+ * FlowFiles within the session.
+ *
+ * @param flowFiles
+ * @param relationship
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws NullPointerException if the argument null
+ * @throws IllegalArgumentException if given relationship is not a known or
+ * registered relationship
+ */
+ void transfer(Collection<FlowFile> flowFiles, Relationship relationship);
+
+ /**
+ * Ends the managed persistence for the given FlowFile. The persistent
+ * attributes for the FlowFile are deleted and so is the content assuming
+ * nothing else references it and this FlowFile will no longer be available
+ * for further operation.
+ *
+ * @param flowFile
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ */
+ void remove(FlowFile flowFile);
+
+ /**
+ * Ends the managed persistence for the given FlowFiles. The persistent
+ * attributes for the FlowFile are deleted and so is the content assuming
+ * nothing else references it and this FlowFile will no longer be available
+ * for further operation.
+ *
+ * @param flowFiles
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if any of the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ */
+ void remove(Collection<FlowFile> flowFiles);
+
+ /**
+ * Executes the given callback against the contents corresponding to the
+ * given FlowFile.
+ *
+ * @param source
+ * @param reader
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content
+ */
+ void read(FlowFile source, InputStreamCallback reader);
+
+ /**
+ * Combines the content of all given source FlowFiles into a single given
+ * destination FlowFile.
+ *
+ * @param sources
+ * @param destination
+ * @return updated destination FlowFile (new size, etc...)
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws IllegalArgumentException if the given destination is contained
+ * within the sources
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content. The state of the destination will be as it was prior to
+ * this call.
+ */
+ FlowFile merge(Collection<FlowFile> sources, FlowFile destination);
+
+ /**
+ * Combines the content of all given source FlowFiles into a single given
+ * destination FlowFile.
+ *
+ * @param sources
+ * @param destination
+ * @param header bytes that will be added to the beginning of the merged
+ * output. May be null or empty.
+ * @param footer bytes that will be added to the end of the merged output.
+ * May be null or empty.
+ * @param demarcator bytes that will be placed in between each object merged
+ * together. May be null or empty.
+ * @return updated destination FlowFile (new size, etc...)
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws IllegalArgumentException if the given destination is contained
+ * within the sources
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content. The state of the destination will be as it was prior to
+ * this call.
+ */
+ FlowFile merge(Collection<FlowFile> sources, FlowFile destination, byte[] header, byte[] footer, byte[] demarcator);
+
+ /**
+ * Executes the given callback against the content corresponding to the
+ * given FlowFile
+ *
+ * @param source
+ * @param writer
+ * @return updated FlowFile
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be referenced, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content
+ */
+ FlowFile write(FlowFile source, OutputStreamCallback writer);
+
+ /**
+ * Executes the given callback against the content corresponding to the
+ * given flow file
+ *
+ * @param source
+ * @param writer
+ * @return updated FlowFile
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content
+ */
+ FlowFile write(FlowFile source, StreamCallback writer);
+
+ /**
+ * Executes the given callback against the content corresponding to the
+ * given FlowFile, such that any data written to the OutputStream of the
+ * content will be appended to the end of FlowFile.
+ *
+ * @param source
+ * @param writer
+ * @return
+ */
+ FlowFile append(FlowFile source, OutputStreamCallback writer);
+
+ /**
+ * Writes to the given FlowFile all content from the given content path.
+ *
+ * @param source the file from which content will be obtained
+ * @param keepSourceFile if true the content is simply copied; if false the
+ * original content might be used in a destructive way for efficiency such
+ * that the repository will have the data but the original data will be
+ * gone. If false the source object will be removed or gone once imported.
+ * It will not be restored if the session is rolled back so this must be
+ * used with caution. In some cases it can result in tremendous efficiency
+ * gains but is also dangerous.
+ * @param destination the FlowFile whose content will be updated
+ * @return the updated destination FlowFile (new size)
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content
+ */
+ FlowFile importFrom(Path source, boolean keepSourceFile, FlowFile destination);
+
+ /**
+ * Writes to the given FlowFile all content from the given content path.
+ *
+ * @param source the file from which content will be obtained
+ * @param destination the FlowFile whose content will be updated
+ * @return the updated destination FlowFile (new size)
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content
+ */
+ FlowFile importFrom(InputStream source, FlowFile destination);
+
+ /**
+ * Writes the content of the given FlowFile to the given destination path.
+ *
+ * @param flowFile
+ * @param destination
+ * @param append if true will append to the current content at the given
+ * path; if false will replace any current content
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content
+ */
+ void exportTo(FlowFile flowFile, Path destination, boolean append);
+
+ /**
+ * Writes the content of the given FlowFile to the given destination stream
+ *
+ * @param flowFile
+ * @param destination
+ * @throws IllegalStateException if detected that this method is being
+ * called from within a callback of another method in this session and for
+ * the given FlowFile(s)
+ * @throws FlowFileHandlingException if the given FlowFile is already
+ * transferred or removed or doesn't belong to this session. Automatic
+ * rollback will occur.
+ * @throws MissingFlowFileException if the given FlowFile content cannot be
+ * found. The FlowFile should no longer be reference, will be internally
+ * destroyed, and the session is automatically rolled back and what is left
+ * of the FlowFile is destroyed.
+ * @throws FlowFileAccessException if some IO problem occurs accessing
+ * FlowFile content
+ */
+ void exportTo(FlowFile flowFile, OutputStream destination);
+
+ /**
+ * Returns a ProvenanceReporter that is tied to this ProcessSession.
+ *
+ * @return
+ */
+ ProvenanceReporter getProvenanceReporter();
+ }