Merge remote-tracking branch 'apache/master' into NIFI-1156

This commit is contained in:
Jeremy Dyer 2015-11-30 10:12:39 -05:00
commit 3e26ed8105
134 changed files with 7230 additions and 2145 deletions

3
NOTICE
View File

@ -4,3 +4,6 @@ Copyright 2014-2015 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
This product includes the following work from the Apache Hadoop project:
BoundedByteArrayOutputStream.java adapted to SoftLimitBoundedByteArrayOutputStream.java

View File

@ -24,14 +24,30 @@ import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;
/**
* <p>
* Marker annotation a {@link org.apache.nifi.processor.Processor Processor}
* implementation can use to indicate that the Processor should still be
* triggered even when it has no data in its work queue. By default, Processors
* which have no non-self incoming edges will be triggered even if there is no
* work in its queue. However, Processors that have non-self incoming edges will
* only be triggered if they have work in their queue or they present this
* annotation.
* triggered even when it has no data in its work queue.
* </p>
*
* <p>
* A Processor is scheduled to be triggered based on its configured Scheduling Period
* and Scheduling Strategy. However, when the scheduling period elapses, the Processor
* will not be scheduled if it has no work to do. Normally, a Processor is said to have
* work to do if one of the following circumstances is true:
* </p>
*
* <ul>
* <li>An incoming Connection has data in its queue</li>
* <li>The Processor has no incoming Connections.</li>
* <li>All incoming Connections are self-loops (both the source and destination of the Connection are the same Processor).
* </ul>
*
* <p>
* If the Processor needs to be triggered to run even when the above conditions are all
* <code>false</code>, the Processor's class can be annotated with this annotation, which
* will cause the Processor to be triggered, even if its incoming queues are empty.
* </p>
*/
@Documented
@Target({ElementType.TYPE})

View File

@ -132,10 +132,18 @@ public interface ProcessContext {
*/
boolean hasIncomingConnection();
/**
* @return <code>true</code> if the processor has one or more incoming connections for
* which the source of the connection is NOT the processor; returns <code>false</code> if
* the processor has no incoming connections or if all incoming connections are self-loops
* (i.e., the processor is also the source of all incoming connections).
*/
boolean hasNonLoopConnection();
/**
* @param relationship a relationship to check for connections
* @return true if the relationship has one or more outbound connections,
* false otherwise
* false otherwise
*/
boolean hasConnection(Relationship relationship);

View File

@ -4,6 +4,10 @@ Copyright 2014-2015 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
This product includes the following work from the Apache Hadoop project:
BoundedByteArrayOutputStream.java which was adapted to SoftLimitBoundedByteArrayOutputStream.java
===========================================
Apache Software License v2
===========================================

View File

@ -627,13 +627,14 @@ public class RunNiFi {
final Properties nifiProps = loadProperties(logger);
final String secretKey = nifiProps.getProperty("secret.key");
final String pid = nifiProps.getProperty("pid");
try (final Socket socket = new Socket()) {
logger.debug("Connecting to NiFi instance");
socket.setSoTimeout(60000);
socket.setSoTimeout(10000);
socket.connect(new InetSocketAddress("localhost", port));
logger.debug("Established connection to NiFi instance.");
socket.setSoTimeout(60000);
socket.setSoTimeout(10000);
logger.debug("Sending SHUTDOWN Command to port {}", port);
final OutputStream out = socket.getOutputStream();
@ -654,7 +655,6 @@ public class RunNiFi {
if (SHUTDOWN_CMD.equals(response)) {
logger.info("Apache NiFi has accepted the Shutdown Command and is shutting down now");
final String pid = nifiProps.getProperty("pid");
if (pid != null) {
final Properties bootstrapProperties = new Properties();
try (final FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
@ -703,7 +703,13 @@ public class RunNiFi {
logger.error("When sending SHUTDOWN command to NiFi, got unexpected response {}", response);
}
} catch (final IOException ioe) {
logger.error("Failed to send shutdown command to port {} due to {}", new Object[]{port, ioe.toString(), ioe});
if (pid == null) {
logger.error("Failed to send shutdown command to port {} due to {}. No PID found for the NiFi process, so unable to kill process; "
+ "the process should be killed manually.", new Object[] {port, ioe.toString()});
} else {
logger.error("Failed to send shutdown command to port {} due to {}. Will kill the NiFi Process with PID {}.", new Object[] {port, ioe.toString(), pid});
killProcessTree(pid, logger);
}
} finally {
if (lockFile.exists() && !lockFile.delete()) {
logger.error("Failed to delete lock file {}; this file should be cleaned up manually", lockFile);

View File

@ -259,6 +259,7 @@ public class StandardLineageResult implements ComputeLineageResult {
case FORK:
case JOIN:
case REPLAY:
case FETCH:
case CLONE: {
// For events that create FlowFile nodes, we need to create the FlowFile Nodes and associated Edges, as appropriate
for (final String childUuid : record.getChildUuids()) {

View File

@ -728,6 +728,7 @@ public final class StandardProvenanceEventRecord implements ProvenanceEventRecor
}
break;
case RECEIVE:
case FETCH:
case SEND:
assertSet(transitUri, "Transit URI");
break;

View File

@ -34,6 +34,7 @@ import javax.net.ssl.TrustManager;
import javax.net.ssl.TrustManagerFactory;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.util.file.FileUtils;
public class SSLContextFactory {
@ -58,13 +59,23 @@ public class SSLContextFactory {
// prepare the keystore
final KeyStore keyStore = KeyStore.getInstance(keystoreType);
keyStore.load(new FileInputStream(keystore), keystorePass);
final FileInputStream keyStoreStream = new FileInputStream(keystore);
try {
keyStore.load(keyStoreStream, keystorePass);
} finally {
FileUtils.closeQuietly(keyStoreStream);
}
final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
keyManagerFactory.init(keyStore, keystorePass);
// prepare the truststore
final KeyStore trustStore = KeyStore.getInstance(truststoreType);
trustStore.load(new FileInputStream(truststore), truststorePass);
final FileInputStream trustStoreStream = new FileInputStream(truststore);
try {
trustStore.load(trustStoreStream, truststorePass);
} finally {
FileUtils.closeQuietly(trustStoreStream);
}
final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
trustManagerFactory.init(trustStore);

View File

@ -41,75 +41,75 @@ public class TestLeakyBucketThrottler {
final byte[] data = new byte[1024 * 1024 * 4];
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final OutputStream throttledOut = throttler.newThrottledOutputStream(baos);
try (final OutputStream throttledOut = throttler.newThrottledOutputStream(baos)) {
final long start = System.currentTimeMillis();
throttledOut.write(data);
throttler.close();
final long millis = System.currentTimeMillis() - start;
// should take 4 sec give or take
assertTrue(millis > 3000);
assertTrue(millis < 6000);
final long start = System.currentTimeMillis();
throttledOut.write(data);
throttler.close();
final long millis = System.currentTimeMillis() - start;
// should take 4 sec give or take
assertTrue(millis > 3000);
assertTrue(millis < 6000);
}
}
@Test(timeout = 10000)
public void testInputStreamInterface() throws IOException {
// throttle rate at 1 MB/sec
final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024);
final byte[] data = new byte[1024 * 1024 * 4];
final ByteArrayInputStream bais = new ByteArrayInputStream(data);
final InputStream throttledIn = throttler.newThrottledInputStream(bais);
// throttle rate at 1 MB/sec
try ( final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024);
final ByteArrayInputStream bais = new ByteArrayInputStream(data);
final InputStream throttledIn = throttler.newThrottledInputStream(bais);
final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final byte[] buffer = new byte[4096];
final long start = System.currentTimeMillis();
int len;
while ((len = throttledIn.read(buffer)) > 0) {
baos.write(buffer, 0, len);
}
final byte[] buffer = new byte[4096];
final long start = System.currentTimeMillis();
int len;
while ((len = throttledIn.read(buffer)) > 0) {
baos.write(buffer, 0, len);
final long millis = System.currentTimeMillis() - start;
// should take 4 sec give or take
assertTrue(millis > 3000);
assertTrue(millis < 6000);
}
throttler.close();
final long millis = System.currentTimeMillis() - start;
// should take 4 sec give or take
assertTrue(millis > 3000);
assertTrue(millis < 6000);
baos.close();
}
@Test(timeout = 10000)
public void testDirectInterface() throws IOException, InterruptedException {
// throttle rate at 1 MB/sec
final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024);
try (final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024);
final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
// create 3 threads, each sending ~2 MB
final List<Thread> threads = new ArrayList<Thread>();
for (int i = 0; i < 3; i++) {
final Thread t = new WriterThread(i, throttler, baos);
threads.add(t);
}
// create 3 threads, each sending ~2 MB
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
final List<Thread> threads = new ArrayList<Thread>();
for (int i = 0; i < 3; i++) {
final Thread t = new WriterThread(i, throttler, baos);
threads.add(t);
final long start = System.currentTimeMillis();
for (final Thread t : threads) {
t.start();
}
for (final Thread t : threads) {
t.join();
}
final long elapsed = System.currentTimeMillis() - start;
throttler.close();
// To send 15 MB, it should have taken at least 5 seconds and no more than 7 seconds, to
// allow for busy-ness and the fact that we could write a tiny bit more than the limit.
assertTrue(elapsed > 5000);
assertTrue(elapsed < 7000);
// ensure bytes were copied out appropriately
assertEquals(3 * (2 * 1024 * 1024 + 1), baos.getBufferLength());
assertEquals((byte) 'A', baos.getUnderlyingBuffer()[baos.getBufferLength() - 1]);
}
final long start = System.currentTimeMillis();
for (final Thread t : threads) {
t.start();
}
for (final Thread t : threads) {
t.join();
}
final long elapsed = System.currentTimeMillis() - start;
throttler.close();
// To send 15 MB, it should have taken at least 5 seconds and no more than 7 seconds, to
// allow for busy-ness and the fact that we could write a tiny bit more than the limit.
assertTrue(elapsed > 5000);
assertTrue(elapsed < 7000);
// ensure bytes were copied out appropriately
assertEquals(3 * (2 * 1024 * 1024 + 1), baos.getBufferLength());
assertEquals((byte) 'A', baos.getUnderlyingBuffer()[baos.getBufferLength() - 1]);
}
private static class WriterThread extends Thread {

View File

@ -381,8 +381,8 @@ Bootstrap Properties
The _bootstrap.conf_ file in the _conf_ directory allows users to configure settings for how NiFi should be started.
This includes parameters, such as the size of the Java Heap, what Java command to run, and Java System Properties.
Here, we will address the different properties that are made available in the file. Any chances to this file will
take affect only after NiFi has been stopped and restarted.
Here, we will address the different properties that are made available in the file. Any changes to this file will
take effect only after NiFi has been stopped and restarted.
|====
|*Property*|*Description*
@ -509,16 +509,14 @@ The first section of the _nifi.properties_ file is for the Core Properties. Thes
|nifi.flowcontroller.graceful.shutdown.period|Indicates the shutdown period. The default value is 10 sec.
|nifi.flowservice.writedelay.interval|When many changes are made to the flow.xml, this property specifies how long to wait before writing out the changes, so as to batch the changes into a single write. The default value is 500 ms.
|nifi.administrative.yield.duration|If a component allows an unexpected exception to escape, it is considered a bug. As a result, the framework will pause (or administratively yield) the component for this amount of time. This is done so that the component does not use up massive amounts of system resources, since it is known to have problems in the existing state. The default value is 30 sec.
|nifi.bored.yield.duration|When a component has no work to do (i.e., is "bored"), this is the amount of time it will wait before checking to see if it has new data to work on. This way, it does not use up CPU resources by checking for new work too often. When setting this property, be aware that it could add extra latency for components that do not constantly have work to do, as once they go into this "bored" state, they will wait this amount of time before checking for more work. The default value is 10 millis.
|nifi.authority.provider.configuration.file*|This is the location of the file that specifies how user access is authenticated. The default value is ./conf/authority-providers.xml.
|nifi.reporting.task.configuration.file*|This is the location of the Reporting Tasks file. The default value is ./conf/reporting-tasks.xml.
|nifi.controller.service.configuration.file*|This is the location of the Controller Services file. The default value is ./conf/controller-services.xml.
|nifi.templates.directory*|This is the location of the directory where flow templates are saved. The default value is ./conf/templates.l
|nifi.ui.banner.text|This is banner text that may be configured to display at the top of the User Interface. It is blank by default.
|nifi.ui.autorefresh.interval|The interval at which the User Interface auto-refreshes. The default value is 30 sec.
|nifi.nar.library.directory|The location of the nar library. The default value is ./lib and probably should be left as is.
|nifi.nar.working.directory|The location of the nar working directory. The default value is ./work/nar and probably should be left as is.
|nifi.documentation.working.directory|The documentation working directory. The default value is ./work/docs/components and probably should be left as is.
|nifi.bored.yield.duration|When a component has no work to do (i.e., is "bored"), this is the amount of time it will wait before checking to see if it has new data to work on. This way, it does not use up CPU resources by checking for new work too often. When setting this property, be aware that it could add extra latency for components that do not constantly have work to do, as once they go into this "bored" state, they will wait this amount of time before checking for more work. The default value is 10 millis.
|====
*H2 Settings* +
@ -633,6 +631,7 @@ Providing three total locations, including _nifi.provenance.repository.director
|nifi.provenance.repository.indexed.fields|This is a comma-separated list of the fields that should be indexed and made searchable. Fields that are not indexed will not be searchable. Valid fields are: EventType, FlowFileUUID, Filename, TransitURI, ProcessorID, AlternateIdentifierURI, ContentType, Relationship, Details. The default value is: EventType, FlowFileUUID, Filename, ProcessorID.
|nifi.provenance.repository.indexed.attributes|This is a comma-separated list of FlowFile Attributes that should be indexed and made searchable. It is blank by default.
|nifi.provenance.repository.index.shard.size|Large values for the shard size will result in more Java heap usage when searching the Provenance Repository but should provide better performance. The default value is 500 MB.
|nifi.provenance.repository.max.attribute.length|Indicates the maximum length that a FlowFile attribute can be when retrieving a Provenance Event from the repository. If the length of any attribute exceeds this value, it will be truncated when the event is retrieved. The default is 65536.
|====
*Volatile Provenance Repository Properties* +
@ -766,6 +765,15 @@ Only configure these properties for the cluster manager.
|nifi.cluster.manager.safemode.duration|Upon restart of an already existing cluster, this is the amount of time that the cluster manager waits for the primary node to connect before giving up and selecting another node to be the primary node. The default value is 0 sec, which means to wait forever. If the administrator does not care which node is the primary node, this value can be changed to some amount of time other than 0 sec.
|====
*Kerberos* +
|====
|*Property*|*Description*
|nifi.kerberos.krb5.file*|The location of the krb5 file, if used. It is blank by default. Note that this property is not used to authenticate NiFi users.
Rather, it is made available for extension points, such as Hadoop-based Processors, to use. At this time, only a single krb5 file is allowed to
be specified per NiFi instance, so this property is configured here rather than in individual Processors.
|====
NOTE: *For Upgrading* - Take care when configuring the properties above that are marked with an asterisk (*). To make the upgrade process easier, it is advisable to change the default configurations to locations outside the main root installation directory. In this way, these items can remain in their configured location through an upgrade, and NiFi can find all the repositories and configuration files and pick up where it left off as soon as the old version is stopped and the new version is started. Furthermore, the administrator may reuse this _nifi.properties_ file and any other configuration files without having to re-configure them each time an upgrade takes place. As previously noted, it is important to check for any changes in the _nifi.properties_ file of the new version when upgrading and make sure they are reflected in the _nifi.properties_ file you use.

Binary file not shown.

After

Width:  |  Height:  |  Size: 30 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 163 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 97 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 98 KiB

Binary file not shown.

After

Width:  |  Height:  |  Size: 106 KiB

View File

@ -37,8 +37,7 @@ use a supported web browser to view the User Interface. Supported web browsers i
* Google Chrome 36+
* Safari 8
Note that there is a known issue in Internet Explorer (IE) 10 and 11 that can cause problems when moving items on the NiFi graph. If you encounter this problem, we suggest using a browser other than IE. This known issue is described here: https://connect.microsoft.com/IE/Feedback/Details/1050422.
Note that there is a known issue in Internet Explorer (IE) 10 and 11 that can cause problems when moving items on the NiFi canvas. If you encounter this problem, we suggest using a browser other than IE. This known issue is described here: https://connect.microsoft.com/IE/Feedback/Details/1050422.
[template="glossary", id="terminology"]
@ -100,6 +99,11 @@ Terminology
As a result, several components may be combined together to make a larger building block from which to create a dataflow.
These templates can also be exported as XML and imported into another NiFi instance, allowing these building blocks to be shared.
*flow.xml.gz*: Everything the DFM puts onto the NiFi User Interface canvas is written, in real time, to one file called the flow.xml.gz. This file is located in the nifi/conf directory.
Any change made on the canvas is automatically saved to this file, without the user needing to click a "save" button. In addition, the user may create a back-up copy of this file at any time
by selecting the Controller Settings button in the far-right section of the tool bar and clicking "Back-up flow" on the General tab. By default, this action saves a copy of the current flow in the nifi/conf/archive directory.
See <<Controller_Settings>> for a description of where the "Back-up flow" button may be found. (Note that in a NiFi Cluster, the NiFi Cluster Manager's copy of this file is named flow.tar, whereas this file is still named flow.xml.gz on the nodes.)
[[User_Interface]]
NiFi User Interface
@ -123,8 +127,8 @@ Along the top of the of the screen is a toolbar that contains several of these s
To the left is the Components Toolbar. This toolbar consists of the different components that can be dragged onto the canvas.
Next to the Components Toolbar is the Actions Toolbar. This toolbar consists of buttons to manipulate the existing
components on the graph. To the right of the Actions Toolbar is the Search Toolbar. This toolbar consists of a single
Search field that allows users to easily find components on the graph. Users are able to search by component name,
components on the canvas. To the right of the Actions Toolbar is the Search Toolbar. This toolbar consists of a single
Search field that allows users to easily find components on the canvas. Users are able to search by component name,
type, identifier, configuration properties, and their values.
The Management Toolbar sits to the right-hand side of the screen. This toolbar consists of buttons that are
@ -133,8 +137,8 @@ and configure system properties, such as how many system resources should be pro
image::nifi-toolbar-components.png["NiFi Components Toolbar"]
Next, we have segments that provide capabilities to easily navigate around the graph. On the left-hand side is a toolbar that
provides the ability to pan around the graph and zoom in and out. On the right-hand side is a “Birds-Eye View” of the dataflow.
Next, we have segments that provide capabilities to easily navigate around the canvas. On the left-hand side is a toolbar that
provides the ability to pan around the canvas and zoom in and out. On the right-hand side is a “Birds-Eye View” of the dataflow.
This provides a high-level view of the dataflow and allows the user to quickly and easily pan across large portions of the dataflow.
Along the top of the screen is a trail of breadcrumbs. As users navigate into and out of Process Groups, the breadcrumbs show
the depth in the flow and each Process Group that was entered to reach this depth. Each of the Process Groups listed in the breadcrumbs
@ -143,8 +147,8 @@ is a link that will take you back up to that level in the flow.
image::nifi-navigation.png["NiFi Navigation"]
[[status_bar]]
Below the breadcrumbs lives the Status bar. The Status bar provides information about how many Processors exist in the graph in
each state (Stopped, Running, Invalid, Disabled), how many Remote Process Groups exist on the graph in each state
Below the breadcrumbs lives the Status bar. The Status bar provides information about how many Processors exist on the canvas in
each state (Stopped, Running, Invalid, Disabled), how many Remote Process Groups exist on the canvas in each state
(Transmitting, Not Transmitting), the number of threads that are currently active in the flow, the amount of data that currently
exists in the flow, and the timestamp at which all of this information was last refreshed. If there are any System-Level bulletins,
these are shown in the Status bar as well. Additionally, if the instance of NiFi is clustered, the Status bar shows how many nodes
@ -175,7 +179,7 @@ image:iconProcessor.png["Processor", width=32]
*Processor*: The Processor is the most commonly used component, as it is responsible for data ingress, egress, routing, and
manipulating. There are many different types of Processors. In fact, this is a very common Extension Point in NiFi,
meaning that many vendors may implement their own Processors to perform whatever functions are necessary for their use case.
When a Processor is dragged onto the graph, the user is presented with a dialog to choose which type of Processor to use:
When a Processor is dragged onto the canvas, the user is presented with a dialog to choose which type of Processor to use:
image::add-processor.png["Add Processor Dialog"]
@ -191,7 +195,26 @@ image::add-processor-with-tag-cloud.png["Add Processor with Tag Cloud"]
Clicking the `Add` button or double-clicking on a Processor Type will add the selected Processor to the canvas at the
location that it was dropped.
*Note*: For any component added to the graph, it is possible to select it with the mouse and move it anywhere on the graph. Also, it is possible to select multiple items at once by either holding down the Shift key and selecting each item or by holding down the Shift key and dragging a selection box around the desired components.
*Note*: For any component added to the canvas, it is possible to select it with the mouse and move it anywhere on the canvas. Also, it is possible to select multiple items at once by either holding down the Shift key and selecting each item or by holding down the Shift key and dragging a selection box around the desired components.
Once a Processor has been dragged onto the canvas, the user may interact with it by right-clicking on the Processor and selecting an option from
context menu.
image::nifi-processor-menu.png["Processor Menu", width=300]
The following options are available:
- *Configure*: This option allows the user to establish or change the configuration of the Processor. (See <<Configuring_a_Processor>>.)
- *Start* or *Stop*: This option allows the user to start or stop a Processor; the option will be either Start or Stop, depending on the current state of the Processor.
- *Stats*: This option opens a graphical representation of the Processor's statistical information over time.
- *Upstream connections*: This option allows the user to see and "jump to" upstream connections that are coming into the Processor. This is particularly useful when processors connect into and out of other Process Groups.
- *Downstream connections*: This option allows the user to see and "jump to" downstream connections that are going out of the Processor. This is particularly useful when processors connect into and out of other Process Groups.
- *Usage*: This option takes the user to the Processor's usage documentation.
- *Change color*: This option allows the user to change the color of the Processor, which can make the visual management of large flows easier.
- *Center in view*: This option centers the view of the canvas on the given Processor.
- *Copy*: This option places a copy of the selected Processor on the clipboard, so that it may be pasted elsewhere on the canvas by right-clicking on the canvas and selecting Paste. The Copy/Paste actions also may be done using the keystrokes Ctrl-C (Command-C) and Ctrl-V (Command-V).
- *Delete*: This option allows the DFM to delete a Processor from the canvas.
[[input_port]]
@ -228,6 +251,24 @@ image:iconProcessGroup.png["Process Group", width=32]
and maintain. When a Process Group is dragged onto the canvas, the DFM is prompted to name the Process Group. All Process
Groups within the same parent group must have unique names. The Process Group will then be nested within that parent group.
Once a Process Group has been dragged onto the canvas, the user may interact with it by right-clicking on the Process Group and selecting an option from
context menu.
image::nifi-process-group-menu.png["Process Group Menu", width=300]
The following options are available:
- *Configure*: This option allows the user to establish or change the configuration of the Process Group.
- *Enter group*: This option allows the user to enter the Process Group. It is also possible to double-click on the Process Group to enter it.
- *Start*: This option allows the user to start a Process Group.
- *Stop*: This option allows the user to stop a Process Group.
- *Stats*: This option opens a graphical representation of the Process Group's statistical information over time.
- *Upstream connections*: This option allows the user to see and "jump to" upstream connections that are coming into the Process Group.
- *Downstream connections*: This option allows the user to see and "jump to" downstream connections that are going out of the Process Group.
- *Center in view*: This option centers the view of the canvas on the given Process Group.
- *Copy*: This option places a copy of the selected Process Group on the clipboard, so that it may be pasted elsewhere on the canvas by right-clicking on the canvas and selecting Paste. The Copy/Paste actions also may be done using the keystrokes Ctrl-C (Command-C) and Ctrl-V (Command-V).
- *Delete*: This option allows the DFM to delete a Process Group.
[[remote_process_group]]
@ -241,6 +282,27 @@ how busy each node is. This information is then used to load balance the data th
then interrogated periodically to determine information about any nodes that are dropped from or added to the cluster and to
recalculate the load balancing based on each node's load. For more information, see the section on <<site-to-site,Site-to-Site>>.
Once a Remote Process Group has been dragged onto the canvas, the user may interact with it by right-clicking on the Remote Process Group and selecting an option from
context menu.
image::nifi-rpg-menu.png["Remote Process Group Menu", width=300]
The following options are available:
- *Configure*: This option allows the user to establish or change the configuration of the Remote Process Group.
- *Remote Ports*: This option allows the user to see input ports and/or output ports that exist on the remote instance of NiFi that the Remote Process Group is connected to. Note that if the Site-to-Site configuration is secure, only the ports that the connecting NiFi has been given accessed to will be visible.
- *Enable transmission*: Makes the transmission of data between NiFi instances active. (See <<Remote_Group_Transmission>> )
- *Disable transmission*: Disables the transmission of data between NiFi instances.
- *Stats*: This option opens a graphical representation of the Remote Process Group's statistical information over time.
- *Upstream connections*: This option allows the user to see and "jump to" upstream connections that are coming into the Remote Process Group.
- *Downstream connections*: This option allows the user to see and "jump to" downstream connections that are going out of the Remote Process Group.
- *Refresh*: This option refreshes the view of the status of the remote NiFi instance.
- *Go to*: This option opens a view of the remote NiFi instance in a new tab of the browser. Note that if the Site-to-Site configuration is secure, the user must have access to the remote NiFi instance in order to view it.
- *Center in view*: This option centers the view of the canvas on the given Remote Process Group.
- *Copy*: This option places a copy of the selected Process Group on the clipboard, so that it may be pasted elsewhere on the canvas by right-clicking on the canvas and selecting Paste. The Copy/Paste actions also may be done using the keystrokes Ctrl-C (Command-C) and Ctrl-V (Command-V).
- *Delete*: This option allows the DFM to delete a Remote Process Group from the canvas.
[[funnel]]
image:iconFunnel.png["Funnel", width=32]
@ -276,11 +338,10 @@ choosing `Configure...`
[[Configuring_a_Processor]]
=== Configuring a Processor
Once a Processor has been dragged onto the Canvas, it is ready to configure. This is done by right-clicking on the
Processor and clicking the `Configure...` option from the context menu. The configuration dialog is opened with four
To configure a processor, right-click on the Processor and select the `Configure...` option from the context menu. The configuration dialog is opened with four
different tabs, each of which is discussed below. Once you have finished configuring the Processor, you can apply
the changes by clicking the `Apply` button or cancel all changes by clicking the `Cancel` button.
@ -298,7 +359,7 @@ image::settings-tab.png["Settings Tab"]
This tab contains several different configuration items. First, it allows the DFM to change the name of the Processor.
The name of a Processor by default is the same as the Processor type. Next to the Processor Name is a checkbox, indicating
whether the Processor is Enabled. When a Processor is added to the graph, it is enabled. If the
whether the Processor is Enabled. When a Processor is added to the canvas, it is enabled. If the
Processor is disabled, it cannot be started. The disabled state is used to indicate that when a group of Processors is started,
such as when a DFM starts an entire Process Group, this (disabled) Processor should be excluded.
@ -460,8 +521,12 @@ for all the Processors that are available. Clicking on the desired Processor in
While DFMs have the ability to create Controller Services from the Configure Processor window, there is also a central place within the User Interface for adding and configuring both Controller Services and Reporting Tasks. To get there, click on the Controller Settings button in the Management section of the toolbar.
[[Controller_Settings]]
==== Controller Settings
image:controller-settings-button.png["Controller Settings Button", width=200]
The Controller Settings window has three tabs across the top: General, Controller Services, and Reporting Tasks. The General tab is for settings that pertain to general information about the NiFi instance. For example, here, the DFM can provide a unique name for the overall dataflow, as well as comments that describe the flow. Be aware that this information is visible to any other NiFi instance that connects remotely to this instance (using Remote Process Groups, a.k.a., Site-to-Site).
The General tab also provides settings for the overall maximum thread counts of the instance, as well as the ability to click "Back-up flow" to create a backup copy of the current flow, which is saved by default in the /conf/archive directory.
@ -472,7 +537,7 @@ To the right of the General tab is the Controller Services tab. From this tab, t
image:controller-services-tab.png["Controller Services Tab", width=900]
The Add Controller Service window opens. This window is similar to the Add Processor window. It provides a list of the available Controller Services on the right and a tag cloud, showing the most common catagory tags used for Controller Services, on the left. The DFM may click any tag in the tag cloud in order to narrow down the list of Controller Services to those that fit the categories desired. The DFM may also use the Filter field at the top of the window to search for the desired Contoller Service. Upon selecting a Controller Service from the list, the DFM can see a description of the the service below. Select the desired controller service and click Add, or simply double-click the name of the service to add it.
The Add Controller Service window opens. This window is similar to the Add Processor window. It provides a list of the available Controller Services on the right and a tag cloud, showing the most common category tags used for Controller Services, on the left. The DFM may click any tag in the tag cloud in order to narrow down the list of Controller Services to those that fit the categories desired. The DFM may also use the Filter field at the top of the window to search for the desired Controller Service. Upon selecting a Controller Service from the list, the DFM can see a description of the the service below. Select the desired controller service and click Add, or simply double-click the name of the service to add it.
image:add-controller-service-window.png["Add Controller Service Window", width=700]
@ -513,10 +578,10 @@ The Comments tab is just an open-text field, where the DFM may include comments
When you want to run the Reporting Task, click the Start button in the far-right column of the Reporting Tasks tab.
[[Connecting_Components]]
=== Connecting Components
Once processors and other components have been added to the graph and configured, the next step is to connect them
Once processors and other components have been added to the canvas and configured, the next step is to connect them
to one another so that NiFi knows what to do with each FlowFile after it has been processed. This is accomplished by creating a
Connection between each component. When the user hovers the mouse over the center of a component, a new Connection icon (
image:addConnect.png["Connection Bubble"]
@ -557,7 +622,7 @@ File expiration is a concept by which data that cannot be processed in a timely
This is useful, for example, when the volume of data is expected to exceed the volume that can be sent to a remote site.
In this case, the expiration can be used in conjunction with Prioritizers to ensure that the highest priority data is
processed first and then anything that cannot be processed within a certain time period (one hour, for example) can be dropped. The expiration period is based on the time that the data entered the NiFi instance. In other words, if the file expiration on a given connection is set to '1 hour', and a file that has been in the NiFi instance for one hour reaches that connection, it will expire. The default
value of `0 sec` indicates that the data will never expire. When a file expiration other than '0 sec' is set, a small clock icon appears on the connection label, so the DFM can see it at-a-glance when looking at a flow on the graph.
value of `0 sec` indicates that the data will never expire. When a file expiration other than '0 sec' is set, a small clock icon appears on the connection label, so the DFM can see it at-a-glance when looking at a flow on the canvas.
NiFi provides two configuration elements for Back Pressure. These thresholds indicate how much data should be
@ -581,10 +646,28 @@ The following prioritizers are available:
- *FirstInFirstOutPrioritizer*: Given two FlowFiles, the on that reached the connection first will be processed first.
- *NewestFlowFileFirstPrioritizer*: Given two FlowFiles, the one that is newest in the dataflow will be processed first.
- *OldestFlowFileFirstPrioritizer*: Given two FlowFiles, the on that is oldest in the dataflow will be processed first. This is the default scheme that is used if no prioritizers are selected.
- *PriorityAttributePrioritizer*: Given two FlowFiles that both have a "priority" attribute, the one that has the highest priority value will be prprocessed first. Note that an UpdateAttribute processor should be used to add the "priority" attribute to the FlowFiles before they reach a connection that has this prioritizer set. Values for the "priority" attribute may be alphanumeric, where "a" is a higher priority than "z", and "1" is a higher priority than "9", for example.
- *PriorityAttributePrioritizer*: Given two FlowFiles that both have a "priority" attribute, the one that has the highest priority value will be processed first. Note that an UpdateAttribute processor should be used to add the "priority" attribute to the FlowFiles before they reach a connection that has this prioritizer set. Values for the "priority" attribute may be alphanumeric, where "a" is a higher priority than "z", and "1" is a higher priority than "9", for example.
*Note*: After a connection has been drawn between two components, the connection's configuration may be changed, and the connection may be moved to a new destination; however, the processors on either side of the connection must be stopped before a configuration or destination change may be made.
image:nifi-connection.png["Connection", width=300]
To change a connection's configuration or interact with the connection in other ways, right-click on the connection to open the connection context menu.
image:nifi-connection-menu.png["Connection Menu", width=200]
The following options are available:
- *Configure*: This option allows the user to change the configuration of the connection.
- *Stats*: This option opens a graphical representation of the connection's statistical information over time.
- *Bring to front*: This option brings the connection to the front of the canvas if something else (such as another connection) is overlapping it.
- *Go to source*: This option can be useful if there is a long distance between the connection's source and destination components on the canvas. By clicking this option, the view of the canvas will jump to the source of the connection.
- *Go to destination*: Similar to the "Go to source" option, this option changes the view to the destination component on the canvas and can be useful if there is a long distance between two connected components.
- *Empty queue*: This option allows the DFM to clear the queue of FlowFiles that may be waiting to be processed. This option can be especially useful during testing, when the DFM is not concerned about deleting data from the queue. When this option is selected, users must confirm that they want to delete the data in the queue.
- *Delete*: This option allows the DFM to delete a connection between two components. Note that the components on both sides of the connection must be stopped and the connection must be empty before it can be deleted.
=== Processor Validation
Before trying to start a Processor, it's important to make sure that the Processor's configuration is valid.
@ -647,7 +730,7 @@ link:administration-guide.html[Admin Guide].
This allows new capabilities to be added while still maintaining backward compatibility with all older instances. Additionally, if a vulnerability
or deficiency is ever discovered in a protocol, it allows a newer version of NiFi to forbid communication over the compromised versions of the protocol.
In order to communicate with a remote NiFi instance via Site-to-Site, simply drag a <<remote_process_group,Remote Process Group>> onto the graph
In order to communicate with a remote NiFi instance via Site-to-Site, simply drag a <<remote_process_group,Remote Process Group>> onto the canvas
and enter the URL of the remote NiFi instance (for more information on the components of a Remote Process Group, see
<<Remote_Group_Transmission,Remote Process Group Transmission>> section of this guide.) The URL is the same
URL you would use to go to that instance's User Interface. At that point, you can drag a connection to or from the Remote Process Group
@ -664,7 +747,7 @@ communicate with. For information on configuring NiFi to run securely, see the
link:administration-guide.html[Admin Guide].
In order to allow another NiFi instance to push data to your local instance, you can simply drag an <<input_port,Input Port>> onto the Root Process Group
of your graph. After entering a name for the port, it will be added to your flow. You can now right-click on the Input Port and choose Configure in order
of your canvas. After entering a name for the port, it will be added to your flow. You can now right-click on the Input Port and choose Configure in order
to adjust the name and the number of concurrent tasks that are used for the port. If Site-to-Site is configured to run securely, you will also be given
the ability to adjust who has access to the port. If secure, only those who have been granted access to communicate with the port will be able to see
that the port exists.
@ -698,7 +781,7 @@ This section has described the steps required to build a dataflow. Now, to put i
consists of just two processors: GenerateFlowFile and LogAttribute. These processors are normally used for testing, but they can also be used
to build a quick flow for demonstration purposes and see NiFi in action.
After you drag the GenerateFlowFile and LogAttribute processors to the graph and connect them (using the guidelines provided above), configure them as follows:
After you drag the GenerateFlowFile and LogAttribute processors to the canvas and connect them (using the guidelines provided above), configure them as follows:
* Generate FlowFile
** On the Scheduling tab, set Run schedule to: 5 sec. Note that the GenerateFlowFile processor can create many FlowFiles very quickly; that's why setting the Run schedule is important so that this flow does not overwhelm the system NiFi is running on.
@ -717,7 +800,7 @@ Now see the following section on how to start and stop the dataflow. When the da
== Command and Control of DataFlow
== Command and Control of the DataFlow
When a component is added to the NiFi canvas, it is in the Stopped state. In order to cause the component to
be triggered, the component must be started. Once started, the component can be stopped at any time. From a
@ -802,7 +885,7 @@ Only Ports and Processors can be enabled and disabled.
=== Remote Process Group Transmission
Remote Process Groups provide a mechanism for sending data to or retrieving data from a remote instance
of NiFi. When a Remote Process Group (RPG) is added to the canvas, it is added with the Transmision Disabled,
of NiFi. When a Remote Process Group (RPG) is added to the canvas, it is added with the Transmission Disabled,
as indicated by the icon (
image:iconTransmissionInactive.png["Transmission Disabled"]
) in the top-left corner. When Transmission is Disabled, it can be enabled by right-clicking on the
@ -857,7 +940,7 @@ or not compression should be used when transmitting data to or from this Port.
== Navigating within a DataFlow
NiFi provides various mechanisms for getting around a dataflow. The <<User_Interface>> section discussed various ways to navigate around
the NiFi graph; however, once a flow exists on the graph, there are additional ways to get from one component to another. The <<User Interface>> section showed that when multiple Process Groups exist in a flow, breadcrumbs appear under the toolbar, providing a way to navigate between them. In addition, to enter a Process Group that is currently visible on the graph, simply double-click it, thereby "drilling down" into it. Connections also provide a way to jump from one location to another within the flow. Right-click on a connection and select "Go to source" or "Go to destination" in order to jump to one end of the connection or another. This can be very useful in large, complex dataflows, where the connection lines may be long and span large areas of the graph. Finally, all components provide the ability to jump forward or backward within the flow. Right-click any component (e.g., a processor, process group, port, etc.) and select either "Upstream connections" or "Downstream connections". A dialog window will open, showing the available upstream or downstream connections that the user may jump to. This can be especially useful when trying to follow a dataflow in a backward direction. It is typically easy to follow the path of a dataflow from start to finish, drilling down into nested process groups; however, it can be more difficult to follow the dataflow in the other direction.
the NiFi canvas; however, once a flow exists on the canvas, there are additional ways to get from one component to another. The <<User Interface>> section showed that when multiple Process Groups exist in a flow, breadcrumbs appear under the toolbar, providing a way to navigate between them. In addition, to enter a Process Group that is currently visible on the canvas, simply double-click it, thereby "drilling down" into it. Connections also provide a way to jump from one location to another within the flow. Right-click on a connection and select "Go to source" or "Go to destination" in order to jump to one end of the connection or another. This can be very useful in large, complex dataflows, where the connection lines may be long and span large areas of the canvas. Finally, all components provide the ability to jump forward or backward within the flow. Right-click any component (e.g., a processor, process group, port, etc.) and select either "Upstream connections" or "Downstream connections". A dialog window will open, showing the available upstream or downstream connections that the user may jump to. This can be especially useful when trying to follow a dataflow in a backward direction. It is typically easy to follow the path of a dataflow from start to finish, drilling down into nested process groups; however, it can be more difficult to follow the dataflow in the other direction.
@ -869,7 +952,7 @@ health and status. The Status bar provides information about the overall system
(See <<status_bar>> above for more information). Processors, Process Groups, and Remote Process Groups
provide fine-grained details about their operations. Connections and Process Groups provide information
about the amount of data in their queues. The Summary Page provides information about all of the components
on the graph in a tabular format and also provides System Diagnostics information that includes disk usage,
on the canvas in a tabular format and also provides System Diagnostics information that includes disk usage,
CPU utilization, and Java Heap and Garbage Collection information. In a clustered environment, this
information is available per-node or as aggregates across the entire cluster. We will explore each of these
monitoring artifacts below.
@ -1118,7 +1201,7 @@ image:iconNotSecure.png["Not Secure"]
will be counted.
** image:iconTransmissionActive.png["Transmitting"]
*Transmitting Ports*: The number of Output Ports from whcih this NiFi is connected and currently configured
*Transmitting Ports*: The number of Output Ports from which this NiFi is connected and currently configured
to pull data from. Ports can be turned on and off by enabling and disabling transmission on the Remote Process
Group (see <<Remote_Group_Transmission>>) or via the <<Remote_Port_Configuration>> dialog.
@ -1158,7 +1241,7 @@ image:iconSummary.png["Summary"]
image::summary-table.png["Summary Table"]
This dialog provides a great deal of information about each of the components on the graph. Below, we have annotated
This dialog provides a great deal of information about each of the components on the canvas. Below, we have annotated
the different elements within the dialog in order to make the discussion of the dialog easier.
image::summary-annotated.png["Summary Table Annotated"]
@ -1294,7 +1377,7 @@ error message if unable to create the template for some reason.
.Note
********************************************************************************************************************
It is important to note that if any Processor that is Templated has a sensitive property (such as a password), the value of that
sensitive property is not included in the Template. As a result, when dragging the Template onto the graph, newly
sensitive property is not included in the Template. As a result, when dragging the Template onto the canvas, newly
created Processors may not be valid if they are missing values for their sensitive properties. Additionally, any
Connection that was selected when making the Template is not included in the Template if either the source or the
destination of the Connection is not also included in the Template.
@ -1458,9 +1541,9 @@ image:expanded-events.png["Expanded Events", width=300]
Other Management Features
-------------------------
In addition to the Summary Page, Data Provenance Page, Template Management Page, and Bulletin Board Page, there are other tools in the Management Toolbar (See <<User_Interface>>) that are useful to the DFM. The Flow Configuration History, which is available by clicking on the clock icon ( image:iconFlowHistory.png["Flow History", width=28] ) in the Management Toolbar, shows all the changes that have been made to the dataflow graph. The history can aid in troubleshooting, such as if a recent change to the dataflow has caused a problem and needs to be fixed. The DFM can see what changes have been made and adjust the flow as needed to fix the problem. While NiFi does not have an "undo" feature, the DFM can make new changes to the dataflow that will fix the problem.
In addition to the Summary Page, Data Provenance Page, Template Management Page, and Bulletin Board Page, there are other tools in the Management Toolbar (See <<User_Interface>>) that are useful to the DFM. The Flow Configuration History, which is available by clicking on the clock icon ( image:iconFlowHistory.png["Flow History", width=28] ) in the Management Toolbar, shows all the changes that have been made to the dataflow. The history can aid in troubleshooting, such as if a recent change to the dataflow has caused a problem and needs to be fixed. The DFM can see what changes have been made and adjust the flow as needed to fix the problem. While NiFi does not have an "undo" feature, the DFM can make new changes to the dataflow that will fix the problem.
Two other tools in the Management Toolbar are the Controller Settings page ( image:iconSettings.png["Settings", width=28] ) and the Users page ( image:iconUsers.png["Users", width=28] ). The Controller Settings page provides the ability to change the name of the NiFi instance, add comments describing the NiFi instance, set the maximum number of threads that are available to the application, and create a back-up copy of the dataflow(s) currently on the graph. It also provides tabs where DFMs may add and configure Controller Services and Reporting Tasks (see <<Controller_Services_and_Reporting_Tasks>>). The Users page is used to manage user access, which is described in the Admin Guide.
Two other tools in the Management Toolbar are the Controller Settings page ( image:iconSettings.png["Settings", width=28] ) and the Users page ( image:iconUsers.png["Users", width=28] ). The Controller Settings page provides the ability to change the name of the NiFi instance, add comments describing the NiFi instance, set the maximum number of threads that are available to the application, and create a back-up copy of the dataflow(s) currently on the canvas. It also provides tabs where DFMs may add and configure Controller Services and Reporting Tasks (see <<Controller_Services_and_Reporting_Tasks>>). The Users page is used to manage user access, which is described in the Admin Guide.

View File

@ -17,6 +17,7 @@
package org.apache.nifi.processors;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.Collections;
import java.util.HashSet;
@ -57,13 +58,16 @@ public class WriteResourceToStream extends AbstractProcessor {
relationships.add(REL_SUCCESS);
relationships.add(REL_FAILURE);
this.relationships = Collections.unmodifiableSet(relationships);
final InputStream resourceStream = Thread.currentThread()
.getContextClassLoader().getResourceAsStream("file.txt");
try {
this.resourceData = IOUtils.toString(Thread.currentThread()
.getContextClassLoader().getResourceAsStream("file.txt"));
this.resourceData = IOUtils.toString(resourceStream);
} catch (IOException e) {
throw new RuntimeException("Unable to load resources", e);
} finally {
IOUtils.closeQuietly(resourceStream);
}
}
@Override

View File

@ -48,7 +48,8 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
private boolean yieldCalled = false;
private boolean enableExpressionValidation = false;
private boolean allowExpressionValidation = true;
private boolean incomingConnection = true;
private volatile boolean incomingConnection = true;
private volatile boolean nonLoopConnection = true;
private volatile Set<Relationship> connections = new HashSet<>();
private volatile Set<Relationship> unavailableRelationships = new HashSet<>();
@ -305,6 +306,15 @@ public class MockProcessContext extends MockControllerServiceLookup implements S
return this.connections.contains(relationship);
}
public void setNonLoopConnection(final boolean hasNonLoopConnection) {
this.nonLoopConnection = hasNonLoopConnection;
}
@Override
public boolean hasNonLoopConnection() {
return nonLoopConnection;
}
public void addConnection(final Relationship relationship) {
this.connections.add(relationship);
}

View File

@ -1010,8 +1010,10 @@ public class MockProcessSession implements ProcessSession {
public MockFlowFile penalize(final FlowFile flowFile) {
validateState(flowFile);
final MockFlowFile mockFlowFile = (MockFlowFile) flowFile;
mockFlowFile.setPenalized();
return mockFlowFile;
final MockFlowFile newFlowFile = new MockFlowFile(mockFlowFile.getId(), flowFile);
currentVersions.put(newFlowFile.getId(), newFlowFile);
newFlowFile.setPenalized();
return newFlowFile;
}
public byte[] getContentAsByteArray(final MockFlowFile flowFile) {

View File

@ -523,6 +523,11 @@ public class StandardProcessorTestRunner implements TestRunner {
context.setIncomingConnection(hasIncomingConnection);
}
@Override
public void setNonLoopConnection(final boolean hasNonLoopConnection) {
context.setNonLoopConnection(hasNonLoopConnection);
}
@Override
public void addConnection(Relationship relationship) {
context.addConnection(relationship);

View File

@ -492,10 +492,18 @@ public interface TestRunner {
* Indicates to the framework that the configured processor has one or more
* incoming connections.
*
* @param hasIncomingConnection whether or not the configured processor has an incoming connection
* @param hasIncomingConnection whether or not the configured processor should behave as though it has an incoming connection
*/
void setIncomingConnection(boolean hasIncomingConnection);
/**
* Indicates to the framework that the configured processor has one or more incoming
* connections for which the processor is not also the source.
*
* @param hasNonLoopConnection whether or not the configured processor should behave as though it has a non-looping incoming connection
*/
void setNonLoopConnection(boolean hasNonLoopConnection);
/**
* Indicates to the Framework that the configured processor has a connection for the given Relationship.
*

View File

@ -0,0 +1,58 @@
/*
* 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 org.apache.nifi.flowfile.FlowFile;
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.ProcessException;
import org.junit.Test;
import java.util.Collections;
import java.util.Set;
public class TestMockProcessSession {
@Test(expected = AssertionError.class)
public void testPenalizeFlowFileFromProcessor() {
TestRunners.newTestRunner(PoorlyBehavedProcessor.class).run();
}
protected static class PoorlyBehavedProcessor extends AbstractProcessor {
private static final Relationship REL_FAILURE = new Relationship.Builder()
.name("failure")
.build();
private final Set<Relationship> relationships = Collections.singleton(REL_FAILURE);
@Override
public Set<Relationship> getRelationships() {
return relationships;
}
@Override
public void onTrigger(final ProcessContext ctx, final ProcessSession session) throws ProcessException {
final FlowFile file = session.create();
session.penalize(file);
session.transfer(file, REL_FAILURE);
}
}
}

View File

@ -38,6 +38,7 @@ import org.apache.nifi.annotation.behavior.SideEffectFree;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
@ -51,6 +52,7 @@ import org.apache.nifi.processor.io.StreamCallback;
@SideEffectFree
@SupportsBatching
@Tags({"avro", "convert", "json"})
@InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("Converts a Binary Avro record into a JSON object. This processor provides a direct mapping of an Avro field to a JSON field, such "
+ "that the resulting JSON will have the same hierarchical structure as the Avro document. Note that the Avro schema information will be lost, as this "

View File

@ -26,6 +26,8 @@ import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.model.DeleteObjectRequest;
import com.amazonaws.services.s3.model.DeleteVersionRequest;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
@ -40,6 +42,7 @@ import org.apache.nifi.processor.util.StandardValidators;
@SupportsBatching
@SeeAlso({PutS3Object.class})
@Tags({"Amazon", "S3", "AWS", "Archive", "Delete"})
@InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("Deletes FlowFiles on an Amazon S3 Bucket. " +
"If attempting to delete a file that does not exist, FlowFile is routed to success.")
public class DeleteS3Object extends AbstractS3Processor {

View File

@ -30,6 +30,8 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
@ -56,6 +58,7 @@ import com.microsoft.eventhubs.client.ResilientEventHubReceiver;
@Tags({ "azure", "microsoft", "cloud", "eventhub", "events", "streaming", "streams" })
@CapabilityDescription("Receives messages from a Microsoft Azure Event Hub, writing the contents of the Azure message to the content of the FlowFile")
@InputRequirement(Requirement.INPUT_FORBIDDEN)
@WritesAttributes({
@WritesAttribute(attribute = "eventhub.enqueued.timestamp", description = "The time (in milliseconds since epoch, UTC) at which the message was enqueued in the Azure Event Hub"),
@WritesAttribute(attribute = "eventhub.offset", description = "The offset into the partition at which the message was stored"),

View File

@ -28,6 +28,8 @@ import java.util.concurrent.TimeUnit;
import javax.jms.DeliveryMode;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
@ -53,6 +55,7 @@ import com.microsoft.eventhubs.client.EventHubSender;
@SupportsBatching
@Tags({ "microsoft", "azure", "cloud", "eventhub", "events", "streams", "streaming" })
@InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("Sends the contents of a FlowFile to a Windows Azure Event Hub. Note: the content of the FlowFile will be buffered into memory before being sent, "
+ "so care should be taken to avoid sending FlowFiles to this Processor that exceed the amount of Java Heap Space available.")
public class PutAzureEventHub extends AbstractProcessor {

View File

@ -137,12 +137,10 @@ public abstract class AbstractCouchbaseProcessor extends AbstractProcessor {
}
private CouchbaseClusterControllerService getClusterService(final ProcessContext context) {
if (clusterService == null) {
synchronized (AbstractCouchbaseProcessor.class) {
if (clusterService == null) {
clusterService = context.getProperty(COUCHBASE_CLUSTER_SERVICE)
synchronized (AbstractCouchbaseProcessor.class) {
if (clusterService == null) {
clusterService = context.getProperty(COUCHBASE_CLUSTER_SERVICE)
.asControllerService(CouchbaseClusterControllerService.class);
}
}
}

View File

@ -25,6 +25,8 @@ import java.util.Map;
import java.util.Set;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.ReadsAttribute;
import org.apache.nifi.annotation.behavior.ReadsAttributes;
import org.apache.nifi.annotation.behavior.WritesAttribute;
@ -57,6 +59,7 @@ import com.couchbase.client.java.document.RawJsonDocument;
@Tags({"nosql", "couchbase", "database", "put"})
@CapabilityDescription("Put a document to Couchbase Server via Key/Value access.")
@SeeAlso({CouchbaseClusterControllerService.class})
@InputRequirement(Requirement.INPUT_REQUIRED)
@ReadsAttributes({
@ReadsAttribute(attribute = "uuid", description = "Used as a document id if 'Document Id' is not specified"),
})

View File

@ -59,12 +59,8 @@ public class FlowFileEvent implements Event {
@Override
public Map<String, String> getHeaders() {
if (!headersLoaded) {
synchronized (headers) {
if (headersLoaded) {
return headers;
}
synchronized (headers) {
if (!headersLoaded) {
headers.putAll(flowFile.getAttributes());
headers.put(ENTRY_DATE_HEADER, Long.toString(flowFile.getEntryDate()));
headers.put(ID_HEADER, Long.toString(flowFile.getId()));
@ -76,7 +72,6 @@ public class FlowFileEvent implements Event {
}
headers.put(LINEAGE_START_DATE_HEADER, Long.toString(flowFile.getLineageStartDate()));
headers.put(SIZE_HEADER, Long.toString(flowFile.getSize()));
headersLoaded = true;
}
}
@ -94,11 +89,7 @@ public class FlowFileEvent implements Event {
@Override
public byte[] getBody() {
if (bodyLoaded) {
return body;
}
synchronized (bodyLock ) {
synchronized (bodyLock) {
if (!bodyLoaded) {
if (flowFile.getSize() > Integer.MAX_VALUE) {
throw new RuntimeException("Can't get body of Event because the backing FlowFile is too large (" + flowFile.getSize() + " bytes)");

View File

@ -88,6 +88,11 @@ public class MockProcessContext implements ProcessContext {
return true;
}
@Override
public boolean hasNonLoopConnection() {
return true;
}
@Override
public boolean hasConnection(Relationship relationship) {
return false;

View File

@ -3402,12 +3402,12 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
@Override
public void run() {
try {
((StreamingOutput) nodeResponse.getResponse().getEntity()).write(
new OutputStream() {
@Override
public void write(final int b) { /* drain response */ }
}
);
try (final OutputStream drain = new OutputStream() {
@Override
public void write(final int b) { /* drain response */ }
}) {
((StreamingOutput) nodeResponse.getResponse().getEntity()).write(drain);
}
} catch (final IOException | WebApplicationException ex) {
logger.info("Failed clearing out non-client response buffer due to: " + ex, ex);
}

View File

@ -862,6 +862,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
if (originalSize.getObjectCount() == 0) {
dropRequest.setDroppedSize(originalSize);
dropRequest.setState(DropFlowFileState.COMPLETE);
dropRequestMap.put(requestIdentifier, dropRequest);
return dropRequest;
}

View File

@ -964,6 +964,18 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
}
}
List<Connection> getIncomingNonLoopConnections() {
final List<Connection> connections = getIncomingConnections();
final List<Connection> nonLoopConnections = new ArrayList<>(connections.size());
for (final Connection connection : connections) {
if (!connection.getSource().equals(this)) {
nonLoopConnections.add(connection);
}
}
return nonLoopConnections;
}
@Override
public boolean isValid() {
readLock.lock();
@ -991,13 +1003,13 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
case INPUT_ALLOWED:
break;
case INPUT_FORBIDDEN: {
if (!getIncomingConnections().isEmpty()) {
if (!getIncomingNonLoopConnections().isEmpty()) {
return false;
}
break;
}
case INPUT_REQUIRED: {
if (getIncomingConnections().isEmpty()) {
if (getIncomingNonLoopConnections().isEmpty()) {
return false;
}
break;
@ -1045,7 +1057,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
case INPUT_ALLOWED:
break;
case INPUT_FORBIDDEN: {
final int incomingConnCount = getIncomingConnections().size();
final int incomingConnCount = getIncomingNonLoopConnections().size();
if (incomingConnCount != 0) {
results.add(new ValidationResult.Builder()
.explanation("Processor does not allow upstream connections but currently has " + incomingConnCount)
@ -1056,7 +1068,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
break;
}
case INPUT_REQUIRED: {
if (getIncomingConnections().isEmpty()) {
if (getIncomingNonLoopConnections().isEmpty()) {
results.add(new ValidationResult.Builder()
.explanation("Processor requires an upstream connection but currently has none")
.subject("Upstream Connections")

View File

@ -58,6 +58,7 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.regex.Pattern;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
@ -808,7 +809,13 @@ public class FileSystemRepository implements ContentRepository {
final Path path = getPath(claim, true);
final FileInputStream fis = new FileInputStream(path.toFile());
if (claim.getOffset() > 0L) {
StreamUtils.skip(fis, claim.getOffset());
try {
StreamUtils.skip(fis, claim.getOffset());
} catch(IOException ioe) {
IOUtils.closeQuietly(fis);
throw ioe;
}
}
// see javadocs for claim.getLength() as to why we do this.

View File

@ -247,6 +247,10 @@ public final class StandardFlowFileRecord implements FlowFile, FlowFileRecord {
public Builder removeAttributes(final String... keys) {
if (keys != null) {
for (final String key : keys) {
if (CoreAttributes.UUID.key().equals(key)) {
continue;
}
bAttributes.remove(key);
}
}
@ -256,6 +260,10 @@ public final class StandardFlowFileRecord implements FlowFile, FlowFileRecord {
public Builder removeAttributes(final Set<String> keys) {
if (keys != null) {
for (final String key : keys) {
if (CoreAttributes.UUID.key().equals(key)) {
continue;
}
bAttributes.remove(key);
}
}
@ -267,6 +275,11 @@ public final class StandardFlowFileRecord implements FlowFile, FlowFileRecord {
final Iterator<String> iterator = bAttributes.keySet().iterator();
while (iterator.hasNext()) {
final String key = iterator.next();
if (CoreAttributes.UUID.key().equals(key)) {
continue;
}
if (keyPattern.matcher(key).matches()) {
iterator.remove();
}

View File

@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.regex.Pattern;
import org.apache.commons.io.IOUtils;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.ProcessorNode;
@ -102,8 +103,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
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<String, Long> counters = 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
@ -396,11 +396,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
}
}
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()) {
for (final Map.Entry<String, Long> entry : checkpoint.counters.entrySet()) {
adjustCounter(entry.getKey(), entry.getValue(), true);
}
@ -456,6 +452,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
bytesSent += event.getFileSize();
break;
case RECEIVE:
case FETCH:
flowFilesReceived++;
bytesReceived += event.getFileSize();
break;
@ -615,7 +612,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
if (registeredTypes.contains(ProvenanceEventType.CREATE)
|| registeredTypes.contains(ProvenanceEventType.FORK)
|| registeredTypes.contains(ProvenanceEventType.JOIN)
|| registeredTypes.contains(ProvenanceEventType.RECEIVE)) {
|| registeredTypes.contains(ProvenanceEventType.RECEIVE)
|| registeredTypes.contains(ProvenanceEventType.FETCH)) {
creationEventRegistered = true;
}
}
@ -990,8 +988,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
connectionCounts.clear();
createdFlowFiles.clear();
removedFlowFiles.clear();
globalCounters.clear();
localCounters.clear();
counters.clear();
generatedProvenanceEvents.clear();
forkEventBuilders.clear();
@ -1164,8 +1161,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
return;
}
adjustCounter(name, delta, localCounters);
adjustCounter(name, delta, globalCounters);
adjustCounter(name, delta, counters);
}
private void adjustCounter(final String name, final long delta, final Map<String, Long> map) {
@ -1409,14 +1405,18 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
validateRecordState(flowFile);
final StandardRepositoryRecord record = records.get(flowFile);
final String originalUuid = flowFile.getAttribute(CoreAttributes.UUID.key());
final Map<String, String> updatedAttributes;
if (attributes.containsKey(CoreAttributes.UUID.key())) {
updatedAttributes = new HashMap<>(attributes);
updatedAttributes.remove(CoreAttributes.UUID.key());
} else {
updatedAttributes = attributes;
}
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 StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().fromFlowFile(record.getCurrent()).addAttributes(updatedAttributes);
final FlowFileRecord newFile = ffBuilder.build();
record.setWorking(newFile, attributes);
record.setWorking(newFile, updatedAttributes);
return newFile;
}
@ -1442,19 +1442,15 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
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) {
if (CoreAttributes.UUID.key().equals(key)) {
continue;
}
updatedAttrs.put(key, null);
}
@ -1759,7 +1755,12 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
return new DisableOnCloseInputStream(currentReadClaimStream);
} else {
final InputStream rawInStream = context.getContentRepository().read(claim);
StreamUtils.skip(rawInStream, offset);
try {
StreamUtils.skip(rawInStream, offset);
} catch(IOException ioe) {
IOUtils.closeQuietly(rawInStream);
throw ioe;
}
return rawInStream;
}
} catch (final ContentNotFoundException cnfe) {
@ -2546,8 +2547,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
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<String, Long> counters = new HashMap<>();
private final Set<Path> deleteOnCommit = new HashSet<>();
private final Set<String> removedFlowFiles = new HashSet<>();
@ -2573,8 +2573,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
this.records.putAll(session.records);
this.connectionCounts.putAll(session.connectionCounts);
this.unacknowledgedFlowFiles.putAll(session.unacknowledgedFlowFiles);
this.localCounters.putAll(session.localCounters);
this.globalCounters.putAll(session.globalCounters);
this.counters.putAll(session.counters);
this.deleteOnCommit.addAll(session.deleteOnCommit);
this.removedFlowFiles.addAll(session.removedFlowFiles);

View File

@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.io.IOUtils;
import org.apache.nifi.controller.repository.claim.ContentClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaim;
import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
@ -401,14 +402,22 @@ public class VolatileContentRepository implements ContentRepository {
@Override
public long exportTo(ContentClaim claim, OutputStream destination) throws IOException {
final InputStream in = read(claim);
return StreamUtils.copy(in, destination);
try {
return StreamUtils.copy(in, destination);
} finally {
IOUtils.closeQuietly(in);
}
}
@Override
public long exportTo(ContentClaim claim, OutputStream destination, long offset, long length) throws IOException {
final InputStream in = read(claim);
StreamUtils.skip(in, offset);
StreamUtils.copy(in, destination, length);
try {
StreamUtils.skip(in, offset);
StreamUtils.copy(in, destination, length);
} finally {
IOUtils.closeQuietly(in);
}
return length;
}

View File

@ -37,6 +37,7 @@ 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;
import org.apache.nifi.util.Connectables;
/**
* This class is essentially an empty shell for {@link Connectable}s that are not Processors
@ -196,6 +197,11 @@ public class ConnectableProcessContext implements ProcessContext {
return connectable.hasIncomingConnection();
}
@Override
public boolean hasNonLoopConnection() {
return Connectables.hasNonLoopConnection(connectable);
}
@Override
public boolean hasConnection(Relationship relationship) {
Set<Connection> connections = connectable.getConnections(relationship);

View File

@ -72,30 +72,45 @@ public class ContinuallyRunProcessorTask implements Callable<Boolean> {
this.processContext = processContext;
}
static boolean isRunOnCluster(final ProcessorNode procNode, final boolean isClustered, final boolean isPrimary) {
return !procNode.isIsolated() || !isClustered || isPrimary;
}
static boolean isYielded(final ProcessorNode procNode) {
return procNode.getYieldExpiration() >= System.currentTimeMillis();
}
static boolean isWorkToDo(final ProcessorNode procNode) {
return procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || !Connectables.hasNonLoopConnection(procNode) || Connectables.flowFilesQueued(procNode);
}
@Override
@SuppressWarnings("deprecation")
public Boolean call() {
// make sure processor is not yielded
boolean shouldRun = (procNode.getYieldExpiration() < System.currentTimeMillis());
if (!shouldRun) {
if (isYielded(procNode)) {
return false;
}
// make sure that either we're not clustered or this processor runs on all nodes or that this is the primary node
shouldRun = !procNode.isIsolated() || !flowController.isClustered() || flowController.isPrimary();
if (!shouldRun) {
if (!isRunOnCluster(procNode, flowController.isClustered(), flowController.isPrimary())) {
return false;
}
// make sure that either proc has incoming FlowFiles or has no incoming connections or is annotated with @TriggerWhenEmpty
shouldRun = procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || Connectables.flowFilesQueued(procNode);
if (!shouldRun) {
// Make sure processor has work to do. This means that it meets one of these criteria:
// * It is annotated with @TriggerWhenEmpty
// * It has data in an incoming Connection
// * It has no incoming connections
// * All incoming connections are self-loops
if (!isWorkToDo(procNode)) {
return true;
}
if (numRelationships > 0) {
final int requiredNumberOfAvailableRelationships = procNode.isTriggerWhenAnyDestinationAvailable() ? 1 : numRelationships;
shouldRun = context.isRelationshipAvailabilitySatisfied(requiredNumberOfAvailableRelationships);
if (!context.isRelationshipAvailabilitySatisfied(requiredNumberOfAvailableRelationships)) {
return false;
}
}
final long batchNanos = procNode.getRunDuration(TimeUnit.NANOSECONDS);
@ -112,10 +127,6 @@ public class ContinuallyRunProcessorTask implements Callable<Boolean> {
batch = false;
}
if (!shouldRun) {
return false;
}
scheduleState.incrementActiveThreadCount();
final long startNanos = System.nanoTime();
@ -123,6 +134,7 @@ public class ContinuallyRunProcessorTask implements Callable<Boolean> {
int invocationCount = 0;
try {
try (final AutoCloseable ncl = NarCloseable.withNarLoader()) {
boolean shouldRun = true;
while (shouldRun) {
procNode.onTrigger(processContext, sessionFactory);
invocationCount++;
@ -135,10 +147,14 @@ public class ContinuallyRunProcessorTask implements Callable<Boolean> {
return false;
}
shouldRun = procNode.isTriggerWhenEmpty() || !procNode.hasIncomingConnection() || Connectables.flowFilesQueued(procNode);
shouldRun = shouldRun && (procNode.getYieldExpiration() < System.currentTimeMillis());
if (!isWorkToDo(procNode)) {
break;
}
if (isYielded(procNode)) {
break;
}
if (shouldRun && numRelationships > 0) {
if (numRelationships > 0) {
final int requiredNumberOfAvailableRelationships = procNode.isTriggerWhenAnyDestinationAvailable() ? 1 : numRelationships;
shouldRun = context.isRelationshipAvailabilitySatisfied(requiredNumberOfAvailableRelationships);
}

View File

@ -63,7 +63,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void warn(String msg, final Throwable t) {
//warn("{} " + msg, new Object[]{component}, t);
if (!isWarnEnabled()) {
return;
}
msg = "{} " + msg;
final Object[] os = {component, t.toString(), t};
logger.warn(msg, os);
@ -72,6 +75,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void warn(String msg, Object[] os) {
if (!isWarnEnabled()) {
return;
}
if (lastArgIsException(os)) {
warn(msg, translateException(os), (Throwable) os[os.length - 1]);
} else {
@ -84,6 +91,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void warn(String msg, Object[] os, final Throwable t) {
if (!isWarnEnabled()) {
return;
}
os = addProcessorAndThrowable(os, t);
msg = "{} " + msg + ": {}";
@ -96,6 +107,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void warn(String msg) {
if (!isWarnEnabled()) {
return;
}
msg = "{} " + msg;
final Object[] os = {component};
logger.warn(msg, component);
@ -104,6 +119,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void trace(String msg, Throwable t) {
if (!isTraceEnabled()) {
return;
}
msg = "{} " + msg;
final Object[] os = {component, t.toString(), t};
logger.trace(msg, os);
@ -112,6 +131,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void trace(String msg, Object[] os) {
if (!isTraceEnabled()) {
return;
}
msg = "{} " + msg;
os = addProcessor(os);
logger.trace(msg, os);
@ -120,6 +143,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void trace(String msg) {
if (!isTraceEnabled()) {
return;
}
msg = "{} " + msg;
final Object[] os = {component};
logger.trace(msg, os);
@ -128,6 +155,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void trace(String msg, Object[] os, Throwable t) {
if (!isTraceEnabled()) {
return;
}
os = addProcessorAndThrowable(os, t);
msg = "{} " + msg + ": {}";
@ -163,6 +194,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void info(String msg, Throwable t) {
if (!isInfoEnabled()) {
return;
}
msg = "{} " + msg;
final Object[] os = {component, t.toString()};
@ -175,6 +210,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void info(String msg, Object[] os) {
if (!isInfoEnabled()) {
return;
}
msg = "{} " + msg;
os = addProcessor(os);
@ -184,6 +223,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void info(String msg) {
if (!isInfoEnabled()) {
return;
}
msg = "{} " + msg;
final Object[] os = {component};
@ -193,6 +236,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void info(String msg, Object[] os, Throwable t) {
if (!isInfoEnabled()) {
return;
}
os = addProcessorAndThrowable(os, t);
msg = "{} " + msg + ": {}";
@ -210,6 +257,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void error(String msg, Throwable t) {
if (!isErrorEnabled()) {
return;
}
msg = "{} " + msg;
Object[] os = {component, t.toString()};
logger.error(msg, os);
@ -222,6 +273,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void error(String msg, Object[] os) {
if (!isErrorEnabled()) {
return;
}
if (lastArgIsException(os)) {
error(msg, translateException(os), (Throwable) os[os.length - 1]);
} else {
@ -234,6 +289,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void error(String msg) {
if (!isErrorEnabled()) {
return;
}
msg = "{} " + msg;
final Object[] os = {component};
@ -254,6 +313,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void error(String msg, Object[] os, Throwable t) {
if (!isErrorEnabled()) {
return;
}
os = addProcessorAndThrowable(os, t);
msg = "{} " + msg + ": {}";
@ -266,6 +329,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void debug(String msg, Throwable t) {
if (!isDebugEnabled()) {
return;
}
msg = "{} " + msg;
final Object[] os = {component};
@ -275,6 +342,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void debug(String msg, Object[] os) {
if (!isDebugEnabled()) {
return;
}
os = addProcessor(os);
msg = "{} " + msg;
@ -284,6 +355,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void debug(String msg, Object[] os, Throwable t) {
if (!isDebugEnabled()) {
return;
}
os = addProcessorAndThrowable(os, t);
msg = "{} " + msg + ": {}";
@ -296,6 +371,10 @@ public class SimpleProcessLogger implements ProcessorLog {
@Override
public void debug(String msg) {
if (!isDebugEnabled()) {
return;
}
msg = "{} " + msg;
final Object[] os = {component};

View File

@ -32,6 +32,7 @@ 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;
import org.apache.nifi.util.Connectables;
public class StandardProcessContext implements ProcessContext, ControllerServiceLookup {
@ -184,6 +185,11 @@ public class StandardProcessContext implements ProcessContext, ControllerService
return procNode.hasIncomingConnection();
}
@Override
public boolean hasNonLoopConnection() {
return Connectables.hasNonLoopConnection(procNode);
}
@Override
public boolean hasConnection(Relationship relationship) {
Set<Connection> connections = procNode.getConnections(relationship);

View File

@ -117,6 +117,11 @@ public class StandardSchedulingContext implements SchedulingContext {
return processContext.hasIncomingConnection();
}
@Override
public boolean hasNonLoopConnection() {
return processContext.hasNonLoopConnection();
}
@Override
public boolean hasConnection(Relationship relationship) {
return processContext.hasConnection(relationship);

View File

@ -17,6 +17,7 @@
package org.apache.nifi.util;
import java.util.Collection;
import java.util.List;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.Connection;
@ -53,4 +54,15 @@ public class Connectables {
return false;
}
public static boolean hasNonLoopConnection(final Connectable connectable) {
final List<Connection> connections = connectable.getIncomingConnections();
for (final Connection connection : connections) {
if (!connection.getSource().equals(connectable)) {
return true;
}
}
return false;
}
}

View File

@ -37,13 +37,16 @@ import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
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.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.regex.Pattern;
import org.apache.nifi.connectable.Connectable;
import org.apache.nifi.connectable.ConnectableType;
@ -58,6 +61,7 @@ import org.apache.nifi.controller.repository.claim.StandardContentClaim;
import org.apache.nifi.controller.repository.claim.StandardResourceClaim;
import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.groups.ProcessGroup;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.FlowFileAccessException;
@ -469,6 +473,42 @@ public class TestStandardProcessSession {
assertEquals(1, provenanceRepo.getEvents(0L, 100000).size());
}
@Test
public void testUuidAttributeCannotBeUpdated() {
String originalUuid = "11111111-1111-1111-1111-111111111111";
final FlowFileRecord flowFileRecord1 = new StandardFlowFileRecord.Builder()
.id(1L)
.addAttribute("uuid", originalUuid)
.entryDate(System.currentTimeMillis())
.build();
flowFileQueue.put(flowFileRecord1);
FlowFile flowFile = session.get();
assertNotNull(flowFile);
final String uuid = CoreAttributes.UUID.key();
final String newUuid = "22222222-2222-2222-2222-222222222222";
flowFile = session.putAttribute(flowFile, uuid, newUuid);
assertEquals(originalUuid, flowFile.getAttribute(uuid));
final Map<String, String> uuidMap = new HashMap<>(1);
uuidMap.put(uuid, newUuid);
flowFile = session.putAllAttributes(flowFile, uuidMap);
assertEquals(originalUuid, flowFile.getAttribute(uuid));
flowFile = session.removeAllAttributes(flowFile, Pattern.compile("uuid"));
assertEquals(originalUuid, flowFile.getAttribute(uuid));
flowFile = session.removeAllAttributes(flowFile, Collections.singleton(uuid));
assertEquals(originalUuid, flowFile.getAttribute(uuid));
flowFile = session.removeAttribute(flowFile, uuid);
assertEquals(originalUuid, flowFile.getAttribute(uuid));
}
@Test
public void testUpdateAttributesThenJoin() throws IOException {
final FlowFileRecord flowFileRecord1 = new StandardFlowFileRecord.Builder()

View File

@ -0,0 +1,82 @@
/*
* 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.tasks;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.when;
import java.util.Collections;
import org.apache.nifi.connectable.Connection;
import org.apache.nifi.controller.ProcessorNode;
import org.apache.nifi.controller.queue.FlowFileQueue;
import org.junit.Test;
import org.mockito.Mockito;
public class TestContinuallyRunProcessorTask {
@Test
public void testIsWorkToDo() {
System.setProperty("nifi.properties.file.path", "src/test/resources/nifi.properties");
final ProcessorNode procNode = Mockito.mock(ProcessorNode.class);
Mockito.when(procNode.hasIncomingConnection()).thenReturn(false);
// There is work to do because there are no incoming connections.
assertTrue(ContinuallyRunProcessorTask.isWorkToDo(procNode));
// Test with only a single connection that is self-looping and empty
final Connection selfLoopingConnection = Mockito.mock(Connection.class);
when(selfLoopingConnection.getSource()).thenReturn(procNode);
when(selfLoopingConnection.getDestination()).thenReturn(procNode);
when(procNode.hasIncomingConnection()).thenReturn(true);
when(procNode.getIncomingConnections()).thenReturn(Collections.singletonList(selfLoopingConnection));
assertTrue(ContinuallyRunProcessorTask.isWorkToDo(procNode));
// Test with only a single connection that is self-looping and empty
final FlowFileQueue flowFileQueue = Mockito.mock(FlowFileQueue.class);
when(flowFileQueue.isActiveQueueEmpty()).thenReturn(true);
final FlowFileQueue nonEmptyQueue = Mockito.mock(FlowFileQueue.class);
when(nonEmptyQueue.isActiveQueueEmpty()).thenReturn(false);
when(selfLoopingConnection.getFlowFileQueue()).thenReturn(nonEmptyQueue);
assertTrue(ContinuallyRunProcessorTask.isWorkToDo(procNode));
// Test with only a non-looping Connection that has no FlowFiles
final Connection emptyConnection = Mockito.mock(Connection.class);
when(emptyConnection.getSource()).thenReturn(Mockito.mock(ProcessorNode.class));
when(emptyConnection.getDestination()).thenReturn(procNode);
when(emptyConnection.getFlowFileQueue()).thenReturn(flowFileQueue);
when(procNode.getIncomingConnections()).thenReturn(Collections.singletonList(emptyConnection));
assertFalse(ContinuallyRunProcessorTask.isWorkToDo(procNode));
// test when the queue has data
final Connection nonEmptyConnection = Mockito.mock(Connection.class);
when(nonEmptyConnection.getSource()).thenReturn(Mockito.mock(ProcessorNode.class));
when(nonEmptyConnection.getDestination()).thenReturn(procNode);
when(nonEmptyConnection.getFlowFileQueue()).thenReturn(nonEmptyQueue);
when(procNode.getIncomingConnections()).thenReturn(Collections.singletonList(nonEmptyConnection));
assertTrue(ContinuallyRunProcessorTask.isWorkToDo(procNode));
}
}

View File

@ -53,6 +53,13 @@ public class TestSimpleProcessLogger {
Field loggerField = componentLog.getClass().getDeclaredField("logger");
loggerField.setAccessible(true);
logger = mock(Logger.class);
when(logger.isDebugEnabled()).thenReturn(true);
when(logger.isInfoEnabled()).thenReturn(true);
when(logger.isWarnEnabled()).thenReturn(true);
when(logger.isErrorEnabled()).thenReturn(true);
when(logger.isTraceEnabled()).thenReturn(true);
loggerField.set(componentLog, logger);
} catch (Exception e) {
e.printStackTrace();

View File

@ -38,8 +38,10 @@ import java.util.Map;
import java.util.Set;
import java.util.jar.JarEntry;
import java.util.jar.JarFile;
import javax.servlet.DispatcherType;
import javax.servlet.ServletContext;
import org.apache.nifi.NiFiServer;
import org.apache.nifi.controller.FlowSerializationException;
import org.apache.nifi.controller.FlowSynchronizationException;
@ -51,6 +53,7 @@ import org.apache.nifi.services.FlowService;
import org.apache.nifi.util.NiFiProperties;
import org.apache.nifi.web.NiFiWebContext;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.web.ContentAccess;
import org.apache.nifi.ui.extension.UiExtension;
@ -372,24 +375,25 @@ public class JettyServer implements NiFiServer {
}
// get an input stream for the nifi-processor configuration file
BufferedReader in = new BufferedReader(new InputStreamReader(jarFile.getInputStream(jarEntry)));
try (BufferedReader in = new BufferedReader(new InputStreamReader(jarFile.getInputStream(jarEntry)))) {
// read in each configured type
String rawComponentType;
while ((rawComponentType = in.readLine()) != null) {
// extract the component type
final String componentType = extractComponentType(rawComponentType);
if (componentType != null) {
List<String> extensions = uiExtensions.get(uiExtensionType);
// read in each configured type
String rawComponentType;
while ((rawComponentType = in.readLine()) != null) {
// extract the component type
final String componentType = extractComponentType(rawComponentType);
if (componentType != null) {
List<String> extensions = uiExtensions.get(uiExtensionType);
// if there are currently no extensions for this type create it
if (extensions == null) {
extensions = new ArrayList<>();
uiExtensions.put(uiExtensionType, extensions);
// if there are currently no extensions for this type create it
if (extensions == null) {
extensions = new ArrayList<>();
uiExtensions.put(uiExtensionType, extensions);
}
// add the specified type
extensions.add(componentType);
}
// add the specified type
extensions.add(componentType);
}
}
}
@ -437,37 +441,34 @@ public class JettyServer implements NiFiServer {
*/
private List<String> getWarExtensions(final File war, final String path) {
List<String> processorTypes = new ArrayList<>();
// load the jar file and attempt to find the nifi-processor entry
JarFile jarFile = null;
try {
// load the jar file and attempt to find the nifi-processor entry
jarFile = new JarFile(war);
JarEntry jarEntry = jarFile.getJarEntry(path);
// ensure the nifi-processor entry was found
if (jarEntry != null) {
// get an input stream for the nifi-processor configuration file
BufferedReader in = new BufferedReader(new InputStreamReader(jarFile.getInputStream(jarEntry)));
try (final BufferedReader in = new BufferedReader(
new InputStreamReader(jarFile.getInputStream(jarEntry)))) {
// read in each configured type
String rawProcessorType;
while ((rawProcessorType = in.readLine()) != null) {
// extract the processor type
final String processorType = extractComponentType(rawProcessorType);
if (processorType != null) {
processorTypes.add(processorType);
// read in each configured type
String rawProcessorType;
while ((rawProcessorType = in.readLine()) != null) {
// extract the processor type
final String processorType = extractComponentType(rawProcessorType);
if (processorType != null) {
processorTypes.add(processorType);
}
}
}
}
} catch (IOException ioe) {
logger.warn(String.format("Unable to inspect %s for a custom processor UI.", war));
logger.warn("Unable to inspect {} for a custom processor UI.", new Object[]{war, ioe});
} finally {
try {
// close the jar file - which closes all input streams obtained via getInputStream above
if (jarFile != null) {
jarFile.close();
}
} catch (IOException ioe) {
}
IOUtils.closeQuietly(jarFile);
}
return processorTypes;

View File

@ -148,7 +148,7 @@ public final class DtoFactory {
}
};
final int MAX_BULLETINS_PER_COMPONENT = 5;
final static int MAX_BULLETINS_PER_COMPONENT = 5;
private ControllerServiceLookup controllerServiceLookup;

View File

@ -499,7 +499,13 @@ public class StandardConnectionDAO extends ComponentDAO implements ConnectionDAO
public DropFlowFileStatus deleteFlowFileDropRequest(String groupId, String connectionId, String dropRequestId) {
final Connection connection = locateConnection(groupId, connectionId);
final FlowFileQueue queue = connection.getFlowFileQueue();
return queue.cancelDropFlowFileRequest(dropRequestId);
final DropFlowFileStatus dropFlowFileStatus = queue.cancelDropFlowFileRequest(dropRequestId);
if (dropFlowFileStatus == null) {
throw new ResourceNotFoundException(String.format("Unable to find drop request with id '%s'.", dropRequestId));
}
return dropFlowFileStatus;
}
/* setters */

View File

@ -43,6 +43,7 @@
<div id="general-links" class="component-links">
<ul>
<li class="component-item"><a class="component-link overview" href="html/overview.html" target="component-usage">Overview</a></li>
<li class="component-item"><a class="component-link getting-started" href="html/getting-started.html" target="component-usage">Getting Started</a></li>
<li class="component-item"><a class="component-link user-guide" href="html/user-guide.html" target="component-usage">User Guide</a></li>
<li class="component-item"><a class="component-link expression-language-guide" href="html/expression-language-guide.html" target="component-usage">Expression Language Guide</a></li>
<li class="component-item"><a class="component-link admin-guide" href="html/administration-guide.html" target="component-usage">Admin Guide</a></li>

View File

@ -262,7 +262,7 @@ $(document).ready(function () {
});
// listen for on the rest api and user guide and developer guide and admin guide and overview
$('a.rest-api, a.user-guide, a.developer-guide, a.admin-guide, a.overview, a.expression-language-guide').on('click', function() {
$('a.rest-api, a.user-guide, a.developer-guide, a.admin-guide, a.overview, a.expression-language-guide, a.getting-started').on('click', function() {
selectComponent($(this).text());
});

View File

@ -768,34 +768,8 @@ nf.Actions = (function () {
// remove the component/connection in question
nf[selectionData.type].remove(selectionData.component.id);
// if the source processor is part of the response, we
// have just removed a relationship. must update the status
// of the source processor in case its validity has changed
if (nf.CanvasUtils.isConnection(selection)) {
var sourceComponentId = nf.CanvasUtils.getConnectionSourceComponentId(selectionData.component);
var source = d3.select('#id-' + sourceComponentId);
var sourceData = source.datum();
// update the source status if necessary
if (nf.CanvasUtils.isProcessor(source)) {
nf.Processor.reload(sourceData.component);
} else if (nf.CanvasUtils.isInputPort(source)) {
nf.Port.reload(sourceData.component);
} else if (nf.CanvasUtils.isRemoteProcessGroup(source)) {
nf.RemoteProcessGroup.reload(sourceData.component);
}
var destinationComponentId = nf.CanvasUtils.getConnectionDestinationComponentId(selectionData.component);
var destination = d3.select('#id-' + destinationComponentId);
var destinationData = destination.datum();
// update the destination component accordingly
if (nf.CanvasUtils.isProcessor(destination)) {
nf.Processor.reload(destinationData.component);
} else if (nf.CanvasUtils.isRemoteProcessGroup(destination)) {
nf.RemoteProcessGroup.reload(destinationData.component);
}
} else {
// if the selection is a connection, reload the source and destination accordingly
if (nf.CanvasUtils.isConnection(selection) === false) {
var connections = nf.Connection.getComponentConnections(selectionData.component.id);
if (connections.length > 0) {
var ids = [];
@ -846,40 +820,16 @@ nf.Actions = (function () {
}
});
// refresh all component types as necessary (handle components that have been removed)
// remove all the non connections in the snippet first
components.forEach(function (type, ids) {
nf[type].remove(ids);
if (type !== 'Connection') {
nf[type].remove(ids);
}
});
// if some connections were removed
if (snippet.connections > 0) {
selection.filter(function (d) {
return d.type === 'Connection';
}).each(function (d) {
// add the source to refresh if its not already going to be refreshed
var sourceComponentId = nf.CanvasUtils.getConnectionSourceComponentId(d.component);
var source = d3.select('#id-' + sourceComponentId);
var sourceData = source.datum();
// update the source status if necessary - if the source was already removed
// as part of this operation the reloading has no affect
if (nf.CanvasUtils.isProcessor(source)) {
nf.Processor.reload(sourceData.component);
} else if (nf.CanvasUtils.isInputPort(source)) {
nf.Port.reload(sourceData.component);
} else if (nf.CanvasUtils.isRemoteProcessGroup(source)) {
nf.RemoteProcessGroup.reload(sourceData.component);
}
// add the destination to refresh if its not already going to be refreshed
var destinationComponentId = nf.CanvasUtils.getConnectionDestinationComponentId(d.component);
var destination = d3.select('#id-' + destinationComponentId);
var destinationData = destination.datum();
if (nf.CanvasUtils.isRemoteProcessGroup(destination)) {
nf.RemoteProcessGroup.reload(destinationData.component);
}
});
// then remove all the connections
if (components.has('Connection')) {
nf.Connection.remove(components.get('Connection'));
}
// refresh the birdseye/toolbar
@ -1018,7 +968,7 @@ nf.Actions = (function () {
$('#drop-request-status-message').text(dropRequest.state);
// update the current number of enqueued flowfiles
if (nf.Common.isDefinedAndNotNull(dropRequest.currentCount)) {
if (nf.Common.isDefinedAndNotNull(connection.status) && nf.Common.isDefinedAndNotNull(dropRequest.currentCount)) {
connection.status.queued = dropRequest.current;
nf.Connection.refresh(connection.id);
}

View File

@ -955,6 +955,44 @@ nf.CanvasUtils = (function () {
return '';
},
/**
* Reloads a connection's source and destination.
*
* @param {string} sourceComponentId The connection source id
* @param {string} destinationComponentId The connection destination id
*/
reloadConnectionSourceAndDestination: function (sourceComponentId, destinationComponentId) {
if (nf.Common.isBlank(sourceComponentId) === false) {
var source = d3.select('#id-' + sourceComponentId);
if (source.empty() === false) {
var sourceData = source.datum();
// update the source status if necessary
if (nf.CanvasUtils.isProcessor(source)) {
nf.Processor.reload(sourceData.component);
} else if (nf.CanvasUtils.isInputPort(source)) {
nf.Port.reload(sourceData.component);
} else if (nf.CanvasUtils.isRemoteProcessGroup(source)) {
nf.RemoteProcessGroup.reload(sourceData.component);
}
}
}
if (nf.Common.isBlank(destinationComponentId) === false) {
var destination = d3.select('#id-' + destinationComponentId);
if (destination.empty() === false) {
var destinationData = destination.datum();
// update the destination component accordingly
if (nf.CanvasUtils.isProcessor(destination)) {
nf.Processor.reload(destinationData.component);
} else if (nf.CanvasUtils.isRemoteProcessGroup(destination)) {
nf.RemoteProcessGroup.reload(destinationData.component);
}
}
}
},
/**
* Returns the component id of the source of this processor. If the connection is attached
* to a port in a [sub|remote] group, the component id will be that of the group. Otherwise

View File

@ -860,21 +860,8 @@ nf.ConnectionConfiguration = (function () {
'connections': [response.connection]
}, true);
// update the source component accordingly
if (nf.CanvasUtils.isProcessor(source)) {
nf.Processor.reload(sourceData.component);
} else if (nf.CanvasUtils.isInputPort(source)) {
nf.Port.reload(sourceData.component);
} else if (nf.CanvasUtils.isRemoteProcessGroup(source)) {
nf.RemoteProcessGroup.reload(sourceData.component);
}
// update the destination component accordingly
if (nf.CanvasUtils.isProcessor(destination)) {
nf.Processor.reload(destinationData.component);
} else if (nf.CanvasUtils.isRemoteProcessGroup(destination)) {
nf.RemoteProcessGroup.reload(destinationData.component);
}
// reload the connections source/destination components
nf.CanvasUtils.reloadConnectionSourceAndDestination(sourceComponentId, destinationComponentId);
// update component visibility
nf.Canvas.View.updateVisibility();
@ -899,13 +886,10 @@ nf.ConnectionConfiguration = (function () {
// get the source details
var sourceComponentId = $('#connection-source-component-id').val();
var source = d3.select('#id-' + sourceComponentId);
var sourceData = source.datum();
// get the destination details
var destinationComponentId = $('#connection-destination-component-id').val();
var destination = d3.select('#id-' + destinationComponentId);
var destinationData = destination.datum();
var destinationType = nf.CanvasUtils.getConnectableTypeForDestination(destination);
// get the destination details
@ -950,21 +934,8 @@ nf.ConnectionConfiguration = (function () {
// update this connection
nf.Connection.set(connection);
// update the source component accordingly
if (nf.CanvasUtils.isProcessor(source)) {
nf.Processor.reload(sourceData.component);
} else if (nf.CanvasUtils.isInputPort(source)) {
nf.Port.reload(sourceData.component);
} else if (nf.CanvasUtils.isRemoteProcessGroup(source)) {
nf.RemoteProcessGroup.reload(sourceData.component);
}
// update the destination component accordingly
if (nf.CanvasUtils.isProcessor(destination)) {
nf.Processor.reload(destinationData.component);
} else if (nf.CanvasUtils.isRemoteProcessGroup(destination)) {
nf.RemoteProcessGroup.reload(destinationData.component);
}
// reload the connections source/destination components
nf.CanvasUtils.reloadConnectionSourceAndDestination(sourceComponentId, destinationComponentId);
}
}).fail(function (xhr, status, error) {
if (xhr.status === 400 || xhr.status === 404 || xhr.status === 409) {

View File

@ -1033,6 +1033,12 @@ nf.Connection = (function () {
// removes the specified connections
var removeConnections = function (removed) {
// consider reloading source/destination of connection being removed
removed.each(function (d) {
nf.CanvasUtils.reloadConnectionSourceAndDestination(d.component.source.id, d.component.destination.id);
});
// remove the connection
removed.remove();
};
@ -1142,6 +1148,7 @@ nf.Connection = (function () {
// get the corresponding connection
var connection = d3.select(this.parentNode);
var connectionData = connection.datum();
var previousDestinationId = connectionData.component.destination.id;
// attempt to select a new destination
var destination = d3.select('g.connectable-destination');
@ -1153,7 +1160,10 @@ nf.Connection = (function () {
// prompt for the new port if appropriate
if (nf.CanvasUtils.isProcessGroup(destination) || nf.CanvasUtils.isRemoteProcessGroup(destination)) {
// user will select new port and updated connect details will be set accordingly
nf.ConnectionConfiguration.showConfiguration(connection, destination).fail(function () {
nf.ConnectionConfiguration.showConfiguration(connection, destination).done(function () {
// reload the previous destination
nf.CanvasUtils.reloadConnectionSourceAndDestination(null, previousDestinationId);
}).fail(function () {
// reset the connection
connection.call(updateConnections, true, false);
});
@ -1192,13 +1202,17 @@ nf.Connection = (function () {
data: updatedConnectionData,
dataType: 'json'
}).done(function (response) {
var connectionData = response.connection;
var updatedConnectionData = response.connection;
// update the revision
nf.Client.setRevision(response.revision);
// refresh to update the label
nf.Connection.set(connectionData);
nf.Connection.set(updatedConnectionData);
// reload the previous destination and the new source/destination
nf.CanvasUtils.reloadConnectionSourceAndDestination(null, previousDestinationId);
nf.CanvasUtils.reloadConnectionSourceAndDestination(updatedConnectionData.source.id, updatedConnectionData.destination.id);
}).fail(function (xhr, status, error) {
if (xhr.status === 400 || xhr.status === 404 || xhr.status === 409) {
nf.Dialog.showOkDialog({

View File

@ -1203,6 +1203,11 @@ nf.ProvenanceTable = (function () {
formatEventDetail('Relationship', event.relationship);
}
// conditionally show FETCH details
if (event.eventType === 'FETCH') {
formatEventDetail('Transit Uri', event.transitUri);
}
// conditionally show the cluster node identifier
if (nf.Common.isDefinedAndNotNull(event.clusterNodeId)) {
// save the cluster node id

View File

@ -19,6 +19,7 @@ package org.apache.nifi.processors.hadoop;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assume.assumeTrue;
import java.io.File;
import java.io.FileInputStream;
@ -44,10 +45,23 @@ import org.apache.nifi.util.MockProcessContext;
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 PutHDFSTest {
@BeforeClass
public static void setUp() throws Exception{
/*
* Running Hadoop on Windows requires a special build which will produce required binaries and native modules [1]. Since functionality
* provided by this module and validated by these test does not have any native implication we do not distribute required binaries and native modules
* to support running these tests in Windows environment, therefore they are ignored. You can also get more info from this StackOverflow thread [2]
*
* [1] https://wiki.apache.org/hadoop/Hadoop2OnWindows
* [2] http://stackoverflow.com/questions/19620642/failed-to-locate-the-winutils-binary-in-the-hadoop-binary-path
*/
}
@Test
public void testValidators() {
TestRunner runner = TestRunners.newTestRunner(PutHDFS.class);
@ -159,6 +173,9 @@ public class PutHDFSTest {
@Test
public void testPutFile() throws IOException {
// Refer to comment in the BeforeClass method for an explanation
assumeTrue(isNotWindows());
TestRunner runner = TestRunners.newTestRunner(PutHDFS.class);
runner.setProperty(PutHDFS.DIRECTORY, "target/test-classes");
runner.setProperty(PutHDFS.CONFLICT_RESOLUTION, "replace");
@ -182,6 +199,9 @@ public class PutHDFSTest {
@Test
public void testPutFileWithException() throws IOException {
// Refer to comment in the BeforeClass method for an explanation
assumeTrue(isNotWindows());
String dirName = "target/testPutFileWrongPermissions";
File file = new File(dirName);
file.mkdirs();
@ -213,4 +233,8 @@ public class PutHDFSTest {
fs.setPermission(p, new FsPermission(FsAction.EXECUTE, FsAction.EXECUTE, FsAction.EXECUTE));
fs.delete(p, true);
}
private boolean isNotWindows() {
return !System.getProperty("os.name").startsWith("Windows");
}
}

View File

@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.HashMap;
@ -71,9 +70,11 @@ public class TestCreateHadoopSequenceFile {
}
@Test
public void testSimpleCase() throws FileNotFoundException {
public void testSimpleCase() throws IOException {
for (File inFile : inFiles) {
controller.enqueue(new FileInputStream(inFile));
try (FileInputStream fin = new FileInputStream(inFile) ) {
controller.enqueue(fin);
}
}
controller.run(3);
@ -88,7 +89,9 @@ public class TestCreateHadoopSequenceFile {
@Test
public void testSequenceFileSaysValueIsBytesWritable() throws UnsupportedEncodingException, IOException {
for (File inFile : inFiles) {
controller.enqueue(new FileInputStream(inFile));
try (FileInputStream fin = new FileInputStream(inFile) ){
controller.enqueue(fin);
}
}
controller.run(3);
@ -118,35 +121,39 @@ public class TestCreateHadoopSequenceFile {
}
@Test
public void testMergedTarData() throws FileNotFoundException {
public void testMergedTarData() throws IOException {
Map<String, String> attributes = new HashMap<>();
attributes.put(CoreAttributes.MIME_TYPE.key(), "application/tar");
controller.enqueue(new FileInputStream("src/test/resources/testdata/13545312236534130.tar"), attributes);
controller.run();
List<MockFlowFile> successSeqFiles = controller.getFlowFilesForRelationship(CreateHadoopSequenceFile.RELATIONSHIP_SUCCESS);
assertEquals(1, successSeqFiles.size());
final byte[] data = successSeqFiles.iterator().next().toByteArray();
// Data should be greater than 1000000 because that's the size of 2 of our input files,
// and the file size should contain all of that plus headers, but the headers should only
// be a couple hundred bytes.
assertTrue(data.length > 1000000);
assertTrue(data.length < 1501000);
try (final FileInputStream fin = new FileInputStream("src/test/resources/testdata/13545312236534130.tar")) {
controller.enqueue(fin, attributes);
controller.run();
List<MockFlowFile> successSeqFiles = controller.getFlowFilesForRelationship(CreateHadoopSequenceFile.RELATIONSHIP_SUCCESS);
assertEquals(1, successSeqFiles.size());
final byte[] data = successSeqFiles.iterator().next().toByteArray();
// Data should be greater than 1000000 because that's the size of 2 of our input files,
// and the file size should contain all of that plus headers, but the headers should only
// be a couple hundred bytes.
assertTrue(data.length > 1000000);
assertTrue(data.length < 1501000);
}
}
@Test
public void testMergedZipData() throws IOException {
Map<String, String> attributes = new HashMap<>();
attributes.put(CoreAttributes.MIME_TYPE.key(), "application/zip");
controller.enqueue(new FileInputStream("src/test/resources/testdata/13545423550275052.zip"), attributes);
controller.run();
List<MockFlowFile> successSeqFiles = controller.getFlowFilesForRelationship(CreateHadoopSequenceFile.RELATIONSHIP_SUCCESS);
assertEquals(1, successSeqFiles.size());
final byte[] data = successSeqFiles.iterator().next().toByteArray();
// Data should be greater than 1000000 because that's the size of 2 of our input files,
// and the file size should contain all of that plus headers, but the headers should only
// be a couple hundred bytes.
assertTrue(data.length > 1000000);
assertTrue(data.length < 1501000);
try (FileInputStream fin = new FileInputStream("src/test/resources/testdata/13545423550275052.zip")){
controller.enqueue(fin, attributes);
controller.run();
List<MockFlowFile> successSeqFiles = controller.getFlowFilesForRelationship(CreateHadoopSequenceFile.RELATIONSHIP_SUCCESS);
assertEquals(1, successSeqFiles.size());
final byte[] data = successSeqFiles.iterator().next().toByteArray();
// Data should be greater than 1000000 because that's the size of 2 of our input files,
// and the file size should contain all of that plus headers, but the headers should only
// be a couple hundred bytes.
assertTrue(data.length > 1000000);
assertTrue(data.length < 1501000);
}
// FileOutputStream fos = new FileOutputStream("zip-3-randoms.sf");
// fos.write(data);
// fos.flush();
@ -157,16 +164,19 @@ public class TestCreateHadoopSequenceFile {
public void testMergedFlowfilePackagedData() throws IOException {
Map<String, String> attributes = new HashMap<>();
attributes.put(CoreAttributes.MIME_TYPE.key(), "application/flowfile-v3");
controller.enqueue(new FileInputStream("src/test/resources/testdata/13545479542069498.pkg"), attributes);
controller.run();
List<MockFlowFile> successSeqFiles = controller.getFlowFilesForRelationship(CreateHadoopSequenceFile.RELATIONSHIP_SUCCESS);
assertEquals(1, successSeqFiles.size());
final byte[] data = successSeqFiles.iterator().next().toByteArray();
// Data should be greater than 1000000 because that's the size of 2 of our input files,
// and the file size should contain all of that plus headers, but the headers should only
// be a couple hundred bytes.
assertTrue(data.length > 1000000);
assertTrue(data.length < 1501000);
try ( final FileInputStream fin = new FileInputStream("src/test/resources/testdata/13545479542069498.pkg")) {
controller.enqueue(fin, attributes);
controller.run();
List<MockFlowFile> successSeqFiles = controller.getFlowFilesForRelationship(CreateHadoopSequenceFile.RELATIONSHIP_SUCCESS);
assertEquals(1, successSeqFiles.size());
final byte[] data = successSeqFiles.iterator().next().toByteArray();
// Data should be greater than 1000000 because that's the size of 2 of our input files,
// and the file size should contain all of that plus headers, but the headers should only
// be a couple hundred bytes.
assertTrue(data.length > 1000000);
assertTrue(data.length < 1501000);
}
// FileOutputStream fos = new FileOutputStream("flowfilePkg-3-randoms.sf");
// fos.write(data);
// fos.flush();

View File

@ -50,6 +50,10 @@
<artifactId>commons-lang3</artifactId>
<version>3.4</version>
</dependency>
<dependency>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-mapper-asl</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>

View File

@ -0,0 +1,183 @@
/*
* 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.hbase;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.hbase.put.PutFlowFile;
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.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
/**
* Base class for processors that put data to HBase.
*/
public abstract class AbstractPutHBase extends AbstractProcessor {
protected static final PropertyDescriptor HBASE_CLIENT_SERVICE = new PropertyDescriptor.Builder()
.name("HBase Client Service")
.description("Specifies the Controller Service to use for accessing HBase.")
.required(true)
.identifiesControllerService(HBaseClientService.class)
.build();
protected static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
.name("Table Name")
.description("The name of the HBase Table to put data into")
.required(true)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
protected static final PropertyDescriptor ROW_ID = new PropertyDescriptor.Builder()
.name("Row Identifier")
.description("Specifies the Row ID to use when inserting data into HBase")
.required(false) // not all sub-classes will require this
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
protected static final PropertyDescriptor COLUMN_FAMILY = new PropertyDescriptor.Builder()
.name("Column Family")
.description("The Column Family to use when inserting data into HBase")
.required(true)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
protected static final PropertyDescriptor COLUMN_QUALIFIER = new PropertyDescriptor.Builder()
.name("Column Qualifier")
.description("The Column Qualifier to use when inserting data into HBase")
.required(true)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
protected static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
.name("Batch Size")
.description("The maximum number of FlowFiles to process in a single execution. The FlowFiles will be " +
"grouped by table, and a single Put per table will be performed.")
.required(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.defaultValue("25")
.build();
protected static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("A FlowFile is routed to this relationship after it has been successfully stored in HBase")
.build();
protected static final Relationship REL_FAILURE = new Relationship.Builder()
.name("failure")
.description("A FlowFile is routed to this relationship if it cannot be sent to HBase")
.build();
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
List<FlowFile> flowFiles = session.get(batchSize);
if (flowFiles == null || flowFiles.size() == 0) {
return;
}
final Map<String,List<PutFlowFile>> tablePuts = new HashMap<>();
// Group FlowFiles by HBase Table
for (final FlowFile flowFile : flowFiles) {
final PutFlowFile putFlowFile = createPut(session, context, flowFile);
if (putFlowFile == null) {
// sub-classes should log appropriate error messages before returning null
session.transfer(flowFile, REL_FAILURE);
} else if (!putFlowFile.isValid()) {
if (StringUtils.isBlank(putFlowFile.getTableName())) {
getLogger().error("Missing table name for FlowFile {}; routing to failure", new Object[]{flowFile});
} else if (StringUtils.isBlank(putFlowFile.getRow())) {
getLogger().error("Missing row id for FlowFile {}; routing to failure", new Object[]{flowFile});
} else if (putFlowFile.getColumns() == null || putFlowFile.getColumns().isEmpty()) {
getLogger().error("No columns provided for FlowFile {}; routing to failure", new Object[]{flowFile});
} else {
// really shouldn't get here, but just in case
getLogger().error("Failed to produce a put for FlowFile {}; routing to failure", new Object[]{flowFile});
}
session.transfer(flowFile, REL_FAILURE);
} else {
List<PutFlowFile> putFlowFiles = tablePuts.get(putFlowFile.getTableName());
if (putFlowFiles == null) {
putFlowFiles = new ArrayList<>();
tablePuts.put(putFlowFile.getTableName(), putFlowFiles);
}
putFlowFiles.add(putFlowFile);
}
}
getLogger().debug("Sending {} FlowFiles to HBase in {} put operations", new Object[]{flowFiles.size(), tablePuts.size()});
final long start = System.nanoTime();
final List<PutFlowFile> successes = new ArrayList<>();
final HBaseClientService hBaseClientService = context.getProperty(HBASE_CLIENT_SERVICE).asControllerService(HBaseClientService.class);
for (Map.Entry<String, List<PutFlowFile>> entry : tablePuts.entrySet()) {
try {
hBaseClientService.put(entry.getKey(), entry.getValue());
successes.addAll(entry.getValue());
} catch (Exception e) {
getLogger().error(e.getMessage(), e);
for (PutFlowFile putFlowFile : entry.getValue()) {
getLogger().error("Failed to send {} to HBase due to {}; routing to failure", new Object[]{putFlowFile.getFlowFile(), e});
final FlowFile failure = session.penalize(putFlowFile.getFlowFile());
session.transfer(failure, REL_FAILURE);
}
}
}
final long sendMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
getLogger().debug("Sent {} FlowFiles to HBase successfully in {} milliseconds", new Object[]{successes.size(), sendMillis});
for (PutFlowFile putFlowFile : successes) {
session.transfer(putFlowFile.getFlowFile(), REL_SUCCESS);
final String details = "Put " + putFlowFile.getColumns().size() + " cells to HBase";
session.getProvenanceReporter().send(putFlowFile.getFlowFile(), getTransitUri(putFlowFile), details, sendMillis);
}
}
protected String getTransitUri(PutFlowFile putFlowFile) {
return "hbase://" + putFlowFile.getTableName() + "/" + putFlowFile.getRow();
}
/**
* Sub-classes provide the implementation to create a put from a FlowFile.
*
* @param session
* the current session
* @param context
* the current context
* @param flowFile
* the FlowFile to create a Put from
*
* @return a PutFlowFile instance for the given FlowFile
*/
protected abstract PutFlowFile createPut(final ProcessSession session, final ProcessContext context, final FlowFile flowFile);
}

View File

@ -41,6 +41,7 @@ import org.apache.nifi.hbase.scan.ResultCell;
import org.apache.nifi.hbase.scan.ResultHandler;
import org.apache.nifi.hbase.util.ObjectSerDe;
import org.apache.nifi.hbase.util.StringSerDe;
import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
@ -83,7 +84,7 @@ import java.util.regex.Pattern;
@WritesAttribute(attribute = "hbase.table", description = "The name of the HBase table that the data was pulled from"),
@WritesAttribute(attribute = "mime.type", description = "Set to application/json to indicate that output is JSON")
})
public class GetHBase extends AbstractHBaseProcessor {
public class GetHBase extends AbstractProcessor {
static final Pattern COLUMNS_PATTERN = Pattern.compile("\\w+(:\\w+)?(?:,\\w+(:\\w+)?)*");

View File

@ -16,7 +16,6 @@
*/
package org.apache.nifi.hbase;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.EventDriven;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
@ -24,91 +23,36 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.hbase.put.PutColumn;
import org.apache.nifi.hbase.put.PutFlowFile;
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.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.stream.io.StreamUtils;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
@EventDriven
@SupportsBatching
@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
@Tags({"hadoop", "hbase"})
@CapabilityDescription("Adds the Contents of a FlowFile to HBase as the value of a single cell")
public class PutHBaseCell extends AbstractProcessor {
protected static final PropertyDescriptor HBASE_CLIENT_SERVICE = new PropertyDescriptor.Builder()
.name("HBase Client Service")
.description("Specifies the Controller Service to use for accessing HBase.")
.required(true)
.identifiesControllerService(HBaseClientService.class)
.build();
protected static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
.name("Table Name")
.description("The name of the HBase Table to put data into")
.required(true)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
static final PropertyDescriptor ROW = new PropertyDescriptor.Builder()
.name("Row Identifier")
.description("Specifies the Row ID to use when inserting data into HBase")
.required(true)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
static final PropertyDescriptor COLUMN_FAMILY = new PropertyDescriptor.Builder()
.name("Column Family")
.description("The Column Family to use when inserting data into HBase")
.required(true)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
static final PropertyDescriptor COLUMN_QUALIFIER = new PropertyDescriptor.Builder()
.name("Column Qualifier")
.description("The Column Qualifier to use when inserting data into HBase")
.required(true)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
.name("Batch Size")
.description("The maximum number of FlowFiles to process in a single execution. The FlowFiles will be " +
"grouped by table, and a single Put per table will be performed.")
.required(true)
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.defaultValue("25")
.build();
static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("A FlowFile is routed to this relationship after it has been successfully stored in HBase")
.build();
static final Relationship FAILURE = new Relationship.Builder()
.name("failure")
.description("A FlowFile is routed to this relationship if it cannot be sent to HBase")
.build();
public class PutHBaseCell extends AbstractPutHBase {
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(HBASE_CLIENT_SERVICE);
properties.add(TABLE_NAME);
properties.add(ROW);
properties.add(ROW_ID);
properties.add(COLUMN_FAMILY);
properties.add(COLUMN_QUALIFIER);
properties.add(BATCH_SIZE);
@ -119,84 +63,27 @@ public class PutHBaseCell extends AbstractProcessor {
public Set<Relationship> getRelationships() {
final Set<Relationship> rels = new HashSet<>();
rels.add(REL_SUCCESS);
rels.add(FAILURE);
rels.add(REL_FAILURE);
return rels;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
List<FlowFile> flowFiles = session.get(batchSize);
if (flowFiles == null || flowFiles.size() == 0) {
return;
}
protected PutFlowFile createPut(final ProcessSession session, final ProcessContext context, final FlowFile flowFile) {
final String tableName = context.getProperty(TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue();
final String row = context.getProperty(ROW_ID).evaluateAttributeExpressions(flowFile).getValue();
final String columnFamily = context.getProperty(COLUMN_FAMILY).evaluateAttributeExpressions(flowFile).getValue();
final String columnQualifier = context.getProperty(COLUMN_QUALIFIER).evaluateAttributeExpressions(flowFile).getValue();
final Map<String,List<PutFlowFile>> tablePuts = new HashMap<>();
// Group FlowFiles by HBase Table
for (final FlowFile flowFile : flowFiles) {
final String tableName = context.getProperty(TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue();
final String row = context.getProperty(ROW).evaluateAttributeExpressions(flowFile).getValue();
final String columnFamily = context.getProperty(COLUMN_FAMILY).evaluateAttributeExpressions(flowFile).getValue();
final String columnQualifier = context.getProperty(COLUMN_QUALIFIER).evaluateAttributeExpressions(flowFile).getValue();
if (StringUtils.isBlank(tableName) || StringUtils.isBlank(row) || StringUtils.isBlank(columnFamily) || StringUtils.isBlank(columnQualifier)) {
getLogger().error("Invalid FlowFile {} missing table, row, column familiy, or column qualifier; routing to failure", new Object[]{flowFile});
session.transfer(flowFile, FAILURE);
} else {
final byte[] buffer = new byte[(int) flowFile.getSize()];
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream in) throws IOException {
StreamUtils.fillBuffer(in, buffer);
}
});
final PutFlowFile putFlowFile = new PutFlowFile(tableName, row, columnFamily, columnQualifier, buffer, flowFile);
List<PutFlowFile> putFlowFiles = tablePuts.get(tableName);
if (putFlowFiles == null) {
putFlowFiles = new ArrayList<>();
tablePuts.put(tableName, putFlowFiles);
}
putFlowFiles.add(putFlowFile);
final byte[] buffer = new byte[(int) flowFile.getSize()];
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream in) throws IOException {
StreamUtils.fillBuffer(in, buffer);
}
}
});
getLogger().debug("Sending {} FlowFiles to HBase in {} put operations", new Object[] {flowFiles.size(), tablePuts.size()});
final long start = System.nanoTime();
final List<PutFlowFile> successes = new ArrayList<>();
final HBaseClientService hBaseClientService = context.getProperty(HBASE_CLIENT_SERVICE).asControllerService(HBaseClientService.class);
for (Map.Entry<String, List<PutFlowFile>> entry : tablePuts.entrySet()) {
try {
hBaseClientService.put(entry.getKey(), entry.getValue());
successes.addAll(entry.getValue());
} catch (Exception e) {
getLogger().error(e.getMessage(), e);
for (PutFlowFile putFlowFile : entry.getValue()) {
getLogger().error("Failed to send {} to HBase due to {}; routing to failure", new Object[]{putFlowFile.getFlowFile(), e});
final FlowFile failure = session.penalize(putFlowFile.getFlowFile());
session.transfer(failure, FAILURE);
}
}
}
final long sendMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
getLogger().debug("Sent {} FlowFiles to HBase successfully in {} milliseconds", new Object[] {successes.size(), sendMillis});
for (PutFlowFile putFlowFile : successes) {
session.transfer(putFlowFile.getFlowFile(), REL_SUCCESS);
session.getProvenanceReporter().send(putFlowFile.getFlowFile(), getTransitUri(putFlowFile), sendMillis);
}
}
protected String getTransitUri(PutFlowFile putFlowFile) {
return "hbase://" + putFlowFile.getTableName() + "/" + putFlowFile.getRow() + "/" + putFlowFile.getColumnFamily()
+ ":" + putFlowFile.getColumnQualifier();
final Collection<PutColumn> columns = Collections.singletonList(new PutColumn(columnFamily, columnQualifier, buffer));
return new PutFlowFile(tableName, row, columns, flowFile);
}
}

View File

@ -0,0 +1,230 @@
/*
* 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.hbase;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.EventDriven;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.hbase.put.PutColumn;
import org.apache.nifi.hbase.put.PutFlowFile;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.util.ObjectHolder;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.map.ObjectMapper;
import java.io.BufferedInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
@EventDriven
@SupportsBatching
@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
@Tags({"hadoop", "hbase", "put", "json"})
@CapabilityDescription("Adds rows to HBase based on the contents of incoming JSON documents. Each FlowFile must contain a single " +
"UTF-8 encoded JSON document, and any FlowFiles where the root element is not a single document will be routed to failure. " +
"Each JSON field name and value will become a column qualifier and value of the HBase row. Any fields with a null value " +
"will be skipped, and fields with a complex value will be handled according to the Complex Field Strategy. " +
"The row id can be specified either directly on the processor through the Row Identifier property, or can be extracted from the JSON " +
"document by specifying the Row Identifier Field Name property. This processor will hold the contents of all FlowFiles for the given batch " +
"in memory at one time.")
public class PutHBaseJSON extends AbstractPutHBase {
protected static final PropertyDescriptor ROW_FIELD_NAME = new PropertyDescriptor.Builder()
.name("Row Identifier Field Name")
.description("Specifies the name of a JSON element whose value should be used as the row id for the given JSON document.")
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
protected static final String FAIL_VALUE = "Fail";
protected static final String WARN_VALUE = "Warn";
protected static final String IGNORE_VALUE = "Ignore";
protected static final String TEXT_VALUE = "Text";
protected static final AllowableValue COMPLEX_FIELD_FAIL = new AllowableValue(FAIL_VALUE, FAIL_VALUE, "Route entire FlowFile to failure if any elements contain complex values.");
protected static final AllowableValue COMPLEX_FIELD_WARN = new AllowableValue(WARN_VALUE, WARN_VALUE, "Provide a warning and do not include field in row sent to HBase.");
protected static final AllowableValue COMPLEX_FIELD_IGNORE = new AllowableValue(IGNORE_VALUE, IGNORE_VALUE, "Silently ignore and do not include in row sent to HBase.");
protected static final AllowableValue COMPLEX_FIELD_TEXT = new AllowableValue(TEXT_VALUE, TEXT_VALUE, "Use the string representation of the complex field as the value of the given column.");
protected static final PropertyDescriptor COMPLEX_FIELD_STRATEGY = new PropertyDescriptor.Builder()
.name("Complex Field Strategy")
.description("Indicates how to handle complex fields, i.e. fields that do not have a single text value.")
.expressionLanguageSupported(false)
.required(true)
.allowableValues(COMPLEX_FIELD_FAIL, COMPLEX_FIELD_WARN, COMPLEX_FIELD_IGNORE, COMPLEX_FIELD_TEXT)
.defaultValue(COMPLEX_FIELD_TEXT.getValue())
.build();
@Override
public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(HBASE_CLIENT_SERVICE);
properties.add(TABLE_NAME);
properties.add(ROW_ID);
properties.add(ROW_FIELD_NAME);
properties.add(COLUMN_FAMILY);
properties.add(BATCH_SIZE);
properties.add(COMPLEX_FIELD_STRATEGY);
return properties;
}
@Override
public Set<Relationship> getRelationships() {
final Set<Relationship> rels = new HashSet<>();
rels.add(REL_SUCCESS);
rels.add(REL_FAILURE);
return rels;
}
@Override
protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
final Collection<ValidationResult> results = new ArrayList<>();
final String rowId = validationContext.getProperty(ROW_ID).getValue();
final String rowFieldName = validationContext.getProperty(ROW_FIELD_NAME).getValue();
if (StringUtils.isBlank(rowId) && StringUtils.isBlank(rowFieldName)) {
results.add(new ValidationResult.Builder()
.subject(this.getClass().getSimpleName())
.explanation("Row Identifier or Row Identifier Field Name is required")
.valid(false)
.build());
} else if (!StringUtils.isBlank(rowId) && !StringUtils.isBlank(rowFieldName)) {
results.add(new ValidationResult.Builder()
.subject(this.getClass().getSimpleName())
.explanation("Row Identifier and Row Identifier Field Name can not be used together")
.valid(false)
.build());
}
return results;
}
@Override
protected PutFlowFile createPut(final ProcessSession session, final ProcessContext context, final FlowFile flowFile) {
final String tableName = context.getProperty(TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue();
final String rowId = context.getProperty(ROW_ID).evaluateAttributeExpressions(flowFile).getValue();
final String rowFieldName = context.getProperty(ROW_FIELD_NAME).evaluateAttributeExpressions(flowFile).getValue();
final String columnFamily = context.getProperty(COLUMN_FAMILY).evaluateAttributeExpressions(flowFile).getValue();
final boolean extractRowId = !StringUtils.isBlank(rowFieldName);
final String complexFieldStrategy = context.getProperty(COMPLEX_FIELD_STRATEGY).getValue();
// Parse the JSON document
final ObjectMapper mapper = new ObjectMapper();
final ObjectHolder<JsonNode> rootNodeRef = new ObjectHolder<>(null);
try {
session.read(flowFile, new InputStreamCallback() {
@Override
public void process(final InputStream in) throws IOException {
try (final InputStream bufferedIn = new BufferedInputStream(in)) {
rootNodeRef.set(mapper.readTree(bufferedIn));
}
}
});
} catch (final ProcessException pe) {
getLogger().error("Failed to parse {} as JSON due to {}; routing to failure", new Object[]{flowFile, pe.toString()}, pe);
return null;
}
final JsonNode rootNode = rootNodeRef.get();
if (rootNode.isArray()) {
getLogger().error("Root node of JSON must be a single document, found array for {}; routing to failure", new Object[]{flowFile});
return null;
}
final Collection<PutColumn> columns = new ArrayList<>();
final ObjectHolder<String> rowIdHolder = new ObjectHolder<>(null);
// convert each field/value to a column for the put, skip over nulls and arrays
final Iterator<String> fieldNames = rootNode.getFieldNames();
while (fieldNames.hasNext()) {
final String fieldName = fieldNames.next();
final ObjectHolder<String> fieldValueHolder = new ObjectHolder<>(null);
final JsonNode fieldNode = rootNode.get(fieldName);
if (fieldNode.isNull()) {
getLogger().debug("Skipping {} because value was null", new Object[]{fieldName});
} else if (fieldNode.isValueNode()) {
fieldValueHolder.set(fieldNode.asText());
} else {
// for non-null, non-value nodes, determine what to do based on the handling strategy
switch (complexFieldStrategy) {
case FAIL_VALUE:
getLogger().error("Complex value found for {}; routing to failure", new Object[]{fieldName});
return null;
case WARN_VALUE:
getLogger().warn("Complex value found for {}; skipping", new Object[]{fieldName});
break;
case TEXT_VALUE:
// use toString() here because asText() is only guaranteed to be supported on value nodes
// some other types of nodes, like ArrayNode, provide toString implementations
fieldValueHolder.set(fieldNode.toString());
break;
case IGNORE_VALUE:
// silently skip
break;
default:
break;
}
}
// if we have a field value, then see if this is the row id field, if so store the value for later
// otherwise add a new column where the fieldName and fieldValue are the column qualifier and value
if (fieldValueHolder.get() != null) {
if (extractRowId && fieldName.equals(rowFieldName)) {
rowIdHolder.set(fieldValueHolder.get());
} else {
columns.add(new PutColumn(columnFamily, fieldName, fieldValueHolder.get().getBytes(StandardCharsets.UTF_8)));
}
}
}
// if we are expecting a field name to use for the row id and the incoming document doesn't have it
// log an error message so the user can see what the field names were and return null so it gets routed to failure
if (extractRowId && rowIdHolder.get() == null) {
final String fieldNameStr = StringUtils.join(rootNode.getFieldNames(), ",");
getLogger().error("Row ID field named '{}' not found in field names '{}'; routing to failure", new Object[] {rowFieldName, fieldNameStr});
return null;
}
final String putRowId = (extractRowId ? rowIdHolder.get() : rowId);
return new PutFlowFile(tableName, putRowId, columns, flowFile);
}
}

View File

@ -14,4 +14,5 @@
# limitations under the License.
org.apache.nifi.hbase.GetHBase
org.apache.nifi.hbase.PutHBaseCell
org.apache.nifi.hbase.PutHBaseCell
org.apache.nifi.hbase.PutHBaseJSON

View File

@ -0,0 +1,87 @@
/*
* 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.hbase;
import org.apache.nifi.hbase.put.PutColumn;
import org.apache.nifi.hbase.put.PutFlowFile;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertTrue;
public class HBaseTestUtil {
public static void verifyPut(final String row, final String columnFamily, final Map<String,String> columns, final List<PutFlowFile> puts) {
boolean foundPut = false;
for (final PutFlowFile put : puts) {
if (!row.equals(put.getRow())) {
continue;
}
if (put.getColumns() == null || put.getColumns().size() != columns.size()) {
continue;
}
// start off assuming we have all the columns
boolean foundAllColumns = true;
for (Map.Entry<String, String> entry : columns.entrySet()) {
// determine if we have the current expected column
boolean foundColumn = false;
for (PutColumn putColumn : put.getColumns()) {
final String colVal = new String(putColumn.getBuffer(), StandardCharsets.UTF_8);
if (columnFamily.equals(putColumn.getColumnFamily()) && entry.getKey().equals(putColumn.getColumnQualifier())
&& entry.getValue().equals(colVal)) {
foundColumn = true;
break;
}
}
// if we didn't have the current expected column we know we don't have all expected columns
if (!foundColumn) {
foundAllColumns = false;
break;
}
}
// if we found all the expected columns this was a match so we can break
if (foundAllColumns) {
foundPut = true;
break;
}
}
assertTrue(foundPut);
}
public static void verifyEvent(final List<ProvenanceEventRecord> events, final String uri, final ProvenanceEventType eventType) {
boolean foundEvent = false;
for (final ProvenanceEventRecord event : events) {
if (event.getTransitUri().equals(uri) && event.getEventType().equals(eventType)) {
foundEvent = true;
break;
}
}
assertTrue(foundEvent);
}
}

View File

@ -17,6 +17,7 @@
package org.apache.nifi.hbase;
import org.apache.nifi.controller.AbstractControllerService;
import org.apache.nifi.hbase.put.PutColumn;
import org.apache.nifi.hbase.put.PutFlowFile;
import org.apache.nifi.hbase.scan.Column;
import org.apache.nifi.hbase.scan.ResultCell;
@ -33,7 +34,7 @@ import java.util.Map;
public class MockHBaseClientService extends AbstractControllerService implements HBaseClientService {
private Map<String,ResultCell[]> results = new HashMap<>();
private Map<String, List<PutFlowFile>> puts = new HashMap<>();
private Map<String, List<PutFlowFile>> flowFilePuts = new HashMap<>();
private boolean throwException = false;
@Override
@ -42,7 +43,12 @@ public class MockHBaseClientService extends AbstractControllerService implements
throw new IOException("exception");
}
this.puts.put(tableName, new ArrayList<>(puts));
this.flowFilePuts.put(tableName, new ArrayList<>(puts));
}
@Override
public void put(String tableName, String rowId, Collection<PutColumn> columns) throws IOException {
throw new UnsupportedOperationException();
}
@Override
@ -92,8 +98,8 @@ public class MockHBaseClientService extends AbstractControllerService implements
results.put(rowKey, cellArray);
}
public Map<String, List<PutFlowFile>> getPuts() {
return puts;
public Map<String, List<PutFlowFile>> getFlowFilePuts() {
return flowFilePuts;
}
public void setThrowException(boolean throwException) {

View File

@ -16,6 +16,7 @@
*/
package org.apache.nifi.hbase;
import org.apache.nifi.hbase.put.PutColumn;
import org.apache.nifi.hbase.put.PutFlowFile;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.MockFlowFile;
@ -43,7 +44,7 @@ public class TestPutHBaseCell {
final TestRunner runner = TestRunners.newTestRunner(PutHBaseCell.class);
runner.setProperty(PutHBaseCell.TABLE_NAME, tableName);
runner.setProperty(PutHBaseCell.ROW, row);
runner.setProperty(PutHBaseCell.ROW_ID, row);
runner.setProperty(PutHBaseCell.COLUMN_FAMILY, columnFamily);
runner.setProperty(PutHBaseCell.COLUMN_QUALIFIER, columnQualifier);
runner.setProperty(PutHBaseCell.BATCH_SIZE, "1");
@ -58,12 +59,14 @@ public class TestPutHBaseCell {
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
outFile.assertContentEquals(content);
assertNotNull(hBaseClient.getPuts());
assertEquals(1, hBaseClient.getPuts().size());
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
List<PutFlowFile> puts = hBaseClient.getPuts().get(tableName);
List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(tableName);
assertEquals(1, puts.size());
verifyPut(row, columnFamily, columnQualifier, content, puts.get(0));
assertEquals(1, runner.getProvenanceEvents().size());
}
@Test
@ -89,12 +92,14 @@ public class TestPutHBaseCell {
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
outFile.assertContentEquals(content);
assertNotNull(hBaseClient.getPuts());
assertEquals(1, hBaseClient.getPuts().size());
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
List<PutFlowFile> puts = hBaseClient.getPuts().get(tableName);
List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(tableName);
assertEquals(1, puts.size());
verifyPut(row, columnFamily, columnQualifier, content, puts.get(0));
assertEquals(1, runner.getProvenanceEvents().size());
}
@Test
@ -115,7 +120,9 @@ public class TestPutHBaseCell {
runner.run();
runner.assertTransferCount(PutHBaseCell.REL_SUCCESS, 0);
runner.assertTransferCount(PutHBaseCell.FAILURE, 1);
runner.assertTransferCount(PutHBaseCell.REL_FAILURE, 1);
assertEquals(0, runner.getProvenanceEvents().size());
}
@Test
@ -142,7 +149,9 @@ public class TestPutHBaseCell {
runner.run();
runner.assertTransferCount(PutHBaseCell.REL_SUCCESS, 1);
runner.assertTransferCount(PutHBaseCell.FAILURE, 1);
runner.assertTransferCount(PutHBaseCell.REL_FAILURE, 1);
assertEquals(1, runner.getProvenanceEvents().size());
}
@Test
@ -171,13 +180,15 @@ public class TestPutHBaseCell {
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
outFile.assertContentEquals(content1);
assertNotNull(hBaseClient.getPuts());
assertEquals(1, hBaseClient.getPuts().size());
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
List<PutFlowFile> puts = hBaseClient.getPuts().get(tableName);
List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(tableName);
assertEquals(2, puts.size());
verifyPut(row1, columnFamily, columnQualifier, content1, puts.get(0));
verifyPut(row2, columnFamily, columnQualifier, content2, puts.get(1));
assertEquals(2, runner.getProvenanceEvents().size());
}
@Test
@ -202,7 +213,9 @@ public class TestPutHBaseCell {
runner.enqueue(content2.getBytes("UTF-8"), attributes2);
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.FAILURE, 2);
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_FAILURE, 2);
assertEquals(0, runner.getProvenanceEvents().size());
}
@Test
@ -229,13 +242,15 @@ public class TestPutHBaseCell {
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
outFile.assertContentEquals(content1);
assertNotNull(hBaseClient.getPuts());
assertEquals(1, hBaseClient.getPuts().size());
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
List<PutFlowFile> puts = hBaseClient.getPuts().get(tableName);
List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(tableName);
assertEquals(2, puts.size());
verifyPut(row, columnFamily, columnQualifier, content1, puts.get(0));
verifyPut(row, columnFamily, columnQualifier, content2, puts.get(1));
assertEquals(2, runner.getProvenanceEvents().size());
}
private Map<String, String> getAtrributeMapWithEL(String tableName, String row, String columnFamily, String columnQualifier) {
@ -250,7 +265,7 @@ public class TestPutHBaseCell {
private TestRunner getTestRunnerWithEL(PutHBaseCell proc) {
final TestRunner runner = TestRunners.newTestRunner(proc);
runner.setProperty(PutHBaseCell.TABLE_NAME, "${hbase.tableName}");
runner.setProperty(PutHBaseCell.ROW, "${hbase.row}");
runner.setProperty(PutHBaseCell.ROW_ID, "${hbase.row}");
runner.setProperty(PutHBaseCell.COLUMN_FAMILY, "${hbase.columnFamily}");
runner.setProperty(PutHBaseCell.COLUMN_QUALIFIER, "${hbase.columnQualifier}");
return runner;
@ -266,9 +281,14 @@ public class TestPutHBaseCell {
private void verifyPut(String row, String columnFamily, String columnQualifier, String content, PutFlowFile put) {
assertEquals(row, put.getRow());
assertEquals(columnFamily, put.getColumnFamily());
assertEquals(columnQualifier, put.getColumnQualifier());
assertEquals(content, new String(put.getBuffer(), StandardCharsets.UTF_8));
assertNotNull(put.getColumns());
assertEquals(1, put.getColumns().size());
final PutColumn column = put.getColumns().iterator().next();
assertEquals(columnFamily, column.getColumnFamily());
assertEquals(columnQualifier, column.getColumnQualifier());
assertEquals(content, new String(column.getBuffer(), StandardCharsets.UTF_8));
}
}

View File

@ -0,0 +1,423 @@
/*
* 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.hbase;
import org.apache.nifi.hbase.put.PutFlowFile;
import org.apache.nifi.provenance.ProvenanceEventRecord;
import org.apache.nifi.provenance.ProvenanceEventType;
import org.apache.nifi.reporting.InitializationException;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Test;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
public class TestPutHBaseJSON {
public static final String DEFAULT_TABLE_NAME = "nifi";
public static final String DEFAULT_ROW = "row1";
public static final String DEFAULT_COLUMN_FAMILY = "family1";
@Test
public void testCustomValidate() throws InitializationException {
// missing row id and row id field name should be invalid
TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
getHBaseClientService(runner);
runner.assertNotValid();
// setting both properties should still be invalid
runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, "rowId");
runner.setProperty(PutHBaseJSON.ROW_FIELD_NAME, "rowFieldName");
runner.assertNotValid();
// only a row id field name should make it valid
runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_FIELD_NAME, "rowFieldName");
runner.assertValid();
// only a row id should make it valid
runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, "rowId");
runner.assertValid();
}
@Test
public void testSingleJsonDocAndProvidedRowId() throws IOException, InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, DEFAULT_ROW);
final String content = "{ \"field1\" : \"value1\", \"field2\" : \"value2\" }";
runner.enqueue(content.getBytes("UTF-8"));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
outFile.assertContentEquals(content);
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
final List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(DEFAULT_TABLE_NAME);
assertEquals(1, puts.size());
final Map<String,String> expectedColumns = new HashMap<>();
expectedColumns.put("field1", "value1");
expectedColumns.put("field2", "value2");
HBaseTestUtil.verifyPut(DEFAULT_ROW, DEFAULT_COLUMN_FAMILY, expectedColumns, puts);
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
assertEquals(1, events.size());
final ProvenanceEventRecord event = events.get(0);
assertEquals("hbase://" + DEFAULT_TABLE_NAME + "/" + DEFAULT_ROW, event.getTransitUri());
}
@Test
public void testSingJsonDocAndExtractedRowId() throws IOException, InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_FIELD_NAME, "rowField");
final String content = "{ \"rowField\" : \"myRowId\", \"field1\" : \"value1\", \"field2\" : \"value2\" }";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
outFile.assertContentEquals(content);
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
final List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(DEFAULT_TABLE_NAME);
assertEquals(1, puts.size());
// should be a put with row id of myRowId, and rowField shouldn't end up in the columns
final Map<String,String> expectedColumns1 = new HashMap<>();
expectedColumns1.put("field1", "value1");
expectedColumns1.put("field2", "value2");
HBaseTestUtil.verifyPut("myRowId", DEFAULT_COLUMN_FAMILY, expectedColumns1, puts);
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
assertEquals(1, events.size());
HBaseTestUtil.verifyEvent(runner.getProvenanceEvents(), "hbase://" + DEFAULT_TABLE_NAME + "/myRowId", ProvenanceEventType.SEND);
}
@Test
public void testSingJsonDocAndExtractedRowIdMissingField() throws IOException, InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_FIELD_NAME, "rowField");
final String content = "{ \"field1\" : \"value1\", \"field2\" : \"value2\" }";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_FAILURE, 1);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_FAILURE).get(0);
outFile.assertContentEquals(content);
// should be no provenance events
assertEquals(0, runner.getProvenanceEvents().size());
// no puts should have made it to the client
assertEquals(0, hBaseClient.getFlowFilePuts().size());
}
@Test
public void testMultipleJsonDocsRouteToFailure() throws IOException, InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, DEFAULT_ROW);
final String content1 = "{ \"field1\" : \"value1\", \"field2\" : \"value2\" }";
final String content2 = "{ \"field3\" : \"value3\", \"field4\" : \"value4\" }";
final String content = "[ " + content1 + " , " + content2 + " ]";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_FAILURE, 1);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_FAILURE).get(0);
outFile.assertContentEquals(content);
// should be no provenance events
assertEquals(0, runner.getProvenanceEvents().size());
// no puts should have made it to the client
assertEquals(0, hBaseClient.getFlowFilePuts().size());
}
@Test
public void testELWithProvidedRowId() throws IOException, InitializationException {
final TestRunner runner = getTestRunner("${hbase.table}", "${hbase.colFamily}", "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, "${hbase.rowId}");
final Map<String,String> attributes = new HashMap<>();
attributes.put("hbase.table", "myTable");
attributes.put("hbase.colFamily", "myColFamily");
attributes.put("hbase.rowId", "myRowId");
final String content = "{ \"field1\" : \"value1\", \"field2\" : \"value2\" }";
runner.enqueue(content.getBytes("UTF-8"), attributes);
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
outFile.assertContentEquals(content);
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
final List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get("myTable");
assertEquals(1, puts.size());
final Map<String,String> expectedColumns = new HashMap<>();
expectedColumns.put("field1", "value1");
expectedColumns.put("field2", "value2");
HBaseTestUtil.verifyPut("myRowId", "myColFamily", expectedColumns, puts);
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
assertEquals(1, events.size());
HBaseTestUtil.verifyEvent(runner.getProvenanceEvents(), "hbase://myTable/myRowId", ProvenanceEventType.SEND);
}
@Test
public void testELWithExtractedRowId() throws IOException, InitializationException {
final TestRunner runner = getTestRunner("${hbase.table}", "${hbase.colFamily}", "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_FIELD_NAME, "${hbase.rowField}");
final Map<String,String> attributes = new HashMap<>();
attributes.put("hbase.table", "myTable");
attributes.put("hbase.colFamily", "myColFamily");
attributes.put("hbase.rowField", "field1");
final String content = "{ \"field1\" : \"value1\", \"field2\" : \"value2\" }";
runner.enqueue(content.getBytes("UTF-8"), attributes);
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_SUCCESS).get(0);
outFile.assertContentEquals(content);
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
final List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get("myTable");
assertEquals(1, puts.size());
final Map<String,String> expectedColumns = new HashMap<>();
expectedColumns.put("field2", "value2");
HBaseTestUtil.verifyPut("value1", "myColFamily", expectedColumns, puts);
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
assertEquals(1, events.size());
HBaseTestUtil.verifyEvent(runner.getProvenanceEvents(), "hbase://myTable/value1", ProvenanceEventType.SEND);
}
@Test
public void testNullAndArrayElementsWithWarnStrategy() throws InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, DEFAULT_ROW);
runner.setProperty(PutHBaseJSON.COMPLEX_FIELD_STRATEGY, PutHBaseJSON.COMPLEX_FIELD_WARN.getValue());
// should route to success because there is at least one valid field
final String content = "{ \"field1\" : [{ \"child_field1\" : \"child_value1\" }], \"field2\" : \"value2\", \"field3\" : null }";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
final List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(DEFAULT_TABLE_NAME);
assertEquals(1, puts.size());
// should have skipped field1 and field3
final Map<String,String> expectedColumns = new HashMap<>();
expectedColumns.put("field2", "value2");
HBaseTestUtil.verifyPut(DEFAULT_ROW, DEFAULT_COLUMN_FAMILY, expectedColumns, puts);
}
@Test
public void testNullAndArrayElementsWithIgnoreStrategy() throws InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, DEFAULT_ROW);
runner.setProperty(PutHBaseJSON.COMPLEX_FIELD_STRATEGY, PutHBaseJSON.COMPLEX_FIELD_IGNORE.getValue());
// should route to success because there is at least one valid field
final String content = "{ \"field1\" : [{ \"child_field1\" : \"child_value1\" }], \"field2\" : \"value2\", \"field3\" : null }";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
final List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(DEFAULT_TABLE_NAME);
assertEquals(1, puts.size());
// should have skipped field1 and field3
final Map<String,String> expectedColumns = new HashMap<>();
expectedColumns.put("field2", "value2");
HBaseTestUtil.verifyPut(DEFAULT_ROW, DEFAULT_COLUMN_FAMILY, expectedColumns, puts);
}
@Test
public void testNullAndArrayElementsWithFailureStrategy() throws InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, DEFAULT_ROW);
runner.setProperty(PutHBaseJSON.COMPLEX_FIELD_STRATEGY, PutHBaseJSON.COMPLEX_FIELD_FAIL.getValue());
// should route to success because there is at least one valid field
final String content = "{ \"field1\" : [{ \"child_field1\" : \"child_value1\" }], \"field2\" : \"value2\", \"field3\" : null }";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_FAILURE, 1);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_FAILURE).get(0);
outFile.assertContentEquals(content);
// should be no provenance events
assertEquals(0, runner.getProvenanceEvents().size());
// no puts should have made it to the client
assertEquals(0, hBaseClient.getFlowFilePuts().size());
}
@Test
public void testNullAndArrayElementsWithTextStrategy() throws InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, DEFAULT_ROW);
runner.setProperty(PutHBaseJSON.COMPLEX_FIELD_STRATEGY, PutHBaseJSON.COMPLEX_FIELD_TEXT.getValue());
// should route to success because there is at least one valid field
final String content = "{ \"field1\" : [{ \"child_field1\" : \"child_value1\" }], \"field2\" : \"value2\", \"field3\" : null }";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
final List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(DEFAULT_TABLE_NAME);
assertEquals(1, puts.size());
// should have skipped field1 and field3
final Map<String,String> expectedColumns = new HashMap<>();
expectedColumns.put("field1", "[{\"child_field1\":\"child_value1\"}]");
expectedColumns.put("field2", "value2");
HBaseTestUtil.verifyPut(DEFAULT_ROW, DEFAULT_COLUMN_FAMILY, expectedColumns, puts);
}
@Test
public void testNestedDocWithTextStrategy() throws InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, DEFAULT_ROW);
runner.setProperty(PutHBaseJSON.COMPLEX_FIELD_STRATEGY, PutHBaseJSON.COMPLEX_FIELD_TEXT.getValue());
// should route to success because there is at least one valid field
final String content = "{ \"field1\" : { \"child_field1\" : \"child_value1\" }, \"field2\" : \"value2\", \"field3\" : null }";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_SUCCESS);
assertNotNull(hBaseClient.getFlowFilePuts());
assertEquals(1, hBaseClient.getFlowFilePuts().size());
final List<PutFlowFile> puts = hBaseClient.getFlowFilePuts().get(DEFAULT_TABLE_NAME);
assertEquals(1, puts.size());
// should have skipped field1 and field3
final Map<String,String> expectedColumns = new HashMap<>();
expectedColumns.put("field1", "{\"child_field1\":\"child_value1\"}");
expectedColumns.put("field2", "value2");
HBaseTestUtil.verifyPut(DEFAULT_ROW, DEFAULT_COLUMN_FAMILY, expectedColumns, puts);
}
@Test
public void testAllElementsAreNullOrArrays() throws InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
final MockHBaseClientService hBaseClient = getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, DEFAULT_ROW);
runner.setProperty(PutHBaseJSON.COMPLEX_FIELD_STRATEGY, PutHBaseJSON.COMPLEX_FIELD_WARN.getValue());
// should route to failure since it would produce a put with no columns
final String content = "{ \"field1\" : [{ \"child_field1\" : \"child_value1\" }], \"field2\" : null }";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_FAILURE, 1);
final MockFlowFile outFile = runner.getFlowFilesForRelationship(PutHBaseCell.REL_FAILURE).get(0);
outFile.assertContentEquals(content);
// should be no provenance events
assertEquals(0, runner.getProvenanceEvents().size());
// no puts should have made it to the client
assertEquals(0, hBaseClient.getFlowFilePuts().size());
}
@Test
public void testInvalidJson() throws InitializationException {
final TestRunner runner = getTestRunner(DEFAULT_TABLE_NAME, DEFAULT_COLUMN_FAMILY, "1");
getHBaseClientService(runner);
runner.setProperty(PutHBaseJSON.ROW_ID, DEFAULT_ROW);
final String content = "NOT JSON";
runner.enqueue(content.getBytes(StandardCharsets.UTF_8));
runner.run();
runner.assertAllFlowFilesTransferred(PutHBaseCell.REL_FAILURE, 1);
}
private TestRunner getTestRunner(String table, String columnFamily, String batchSize) {
final TestRunner runner = TestRunners.newTestRunner(PutHBaseJSON.class);
runner.setProperty(PutHBaseJSON.TABLE_NAME, table);
runner.setProperty(PutHBaseJSON.COLUMN_FAMILY, columnFamily);
runner.setProperty(PutHBaseJSON.BATCH_SIZE, batchSize);
return runner;
}
private MockHBaseClientService getHBaseClientService(final TestRunner runner) throws InitializationException {
final MockHBaseClientService hBaseClient = new MockHBaseClientService();
runner.addControllerService("hbaseClient", hBaseClient);
runner.enableControllerService(hBaseClient);
runner.setProperty(PutHBaseCell.HBASE_CLIENT_SERVICE, "hbaseClient");
return hBaseClient;
}
}

View File

@ -65,6 +65,11 @@
<artifactId>nifi-mock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-simple</artifactId>

View File

@ -24,14 +24,15 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
@ -39,9 +40,7 @@ import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.Validator;
import org.apache.nifi.flowfile.FlowFile;
@ -56,6 +55,7 @@ import org.apache.nifi.processor.util.StandardValidators;
import kafka.consumer.Consumer;
import kafka.consumer.ConsumerConfig;
import kafka.consumer.ConsumerIterator;
import kafka.consumer.ConsumerTimeoutException;
import kafka.consumer.KafkaStream;
import kafka.javaapi.consumer.ConsumerConnector;
import kafka.message.MessageAndMetadata;
@ -65,13 +65,21 @@ import kafka.message.MessageAndMetadata;
@CapabilityDescription("Fetches messages from Apache Kafka")
@Tags({"Kafka", "Apache", "Get", "Ingest", "Ingress", "Topic", "PubSub"})
@WritesAttributes({
@WritesAttribute(attribute = "kafka.topic", description = "The name of the Kafka Topic from which the message was received"),
@WritesAttribute(attribute = "kafka.key", description = "The key of the Kafka message, if it exists and batch size is 1. If"
+ " the message does not have a key, or if the batch size is greater than 1, this attribute will not be added"),
@WritesAttribute(attribute = "kafka.partition", description = "The partition of the Kafka Topic from which the message was received. This attribute is added only if the batch size is 1"),
@WritesAttribute(attribute = "kafka.offset", description = "The offset of the message within the Kafka partition. This attribute is added only if the batch size is 1")})
@WritesAttribute(attribute = "kafka.topic", description = "The name of the Kafka Topic from which the message was received"),
@WritesAttribute(attribute = "kafka.key", description = "The key of the Kafka message, if it exists and batch size is 1. If"
+ " the message does not have a key, or if the batch size is greater than 1, this attribute will not be added"),
@WritesAttribute(attribute = "kafka.partition", description = "The partition of the Kafka Topic from which the message was received. This attribute is added only if the batch size is 1"),
@WritesAttribute(attribute = "kafka.offset", description = "The offset of the message within the Kafka partition. This attribute is added only if the batch size is 1")})
@DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
description = "These properties will be added on the Kafka configuration after loading any provided configuration properties."
+ " In the event a dynamic property represents a property that was already set as part of the static properties, its value wil be"
+ " overriden with warning message describing the override."
+ " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration.")
public class GetKafka extends AbstractProcessor {
public static final String SMALLEST = "smallest";
public static final String LARGEST = "largest";
public static final PropertyDescriptor ZOOKEEPER_CONNECTION_STRING = new PropertyDescriptor.Builder()
.name("ZooKeeper Connection String")
.description("The Connection String to use in order to connect to ZooKeeper. This is often a comma-separated list of <host>:<port>"
@ -141,12 +149,20 @@ public class GetKafka extends AbstractProcessor {
.expressionLanguageSupported(false)
.build();
public static final PropertyDescriptor GROUP_ID = new PropertyDescriptor.Builder()
.name("Group ID")
.description("A Group ID is used to identify consumers that are within the same consumer group")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.build();
.name("Group ID")
.description("A Group ID is used to identify consumers that are within the same consumer group")
.required(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(false)
.build();
public static final PropertyDescriptor AUTO_OFFSET_RESET = new PropertyDescriptor.Builder()
.name("Auto Offset Reset")
.description("Automatically reset the offset to the smallest or largest offset available on the broker")
.required(true)
.allowableValues(SMALLEST, LARGEST)
.defaultValue(LARGEST)
.build();
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
@ -156,20 +172,18 @@ public class GetKafka extends AbstractProcessor {
private final BlockingQueue<ConsumerIterator<byte[], byte[]>> streamIterators = new LinkedBlockingQueue<>();
private volatile ConsumerConnector consumer;
final Lock interruptionLock = new ReentrantLock();
// guarded by interruptionLock
private final Set<Thread> interruptableThreads = new HashSet<>();
private final AtomicBoolean consumerStreamsReady = new AtomicBoolean();
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final PropertyDescriptor clientNameWithDefault = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(CLIENT_NAME)
.defaultValue("NiFi-" + getIdentifier())
.build();
.fromPropertyDescriptor(CLIENT_NAME)
.defaultValue("NiFi-" + getIdentifier())
.build();
final PropertyDescriptor groupIdWithDefault = new PropertyDescriptor.Builder()
.fromPropertyDescriptor(GROUP_ID)
.defaultValue(getIdentifier())
.build();
.fromPropertyDescriptor(GROUP_ID)
.defaultValue(getIdentifier())
.build();
final List<PropertyDescriptor> props = new ArrayList<>();
props.add(ZOOKEEPER_CONNECTION_STRING);
@ -181,6 +195,7 @@ public class GetKafka extends AbstractProcessor {
props.add(groupIdWithDefault);
props.add(KAFKA_TIMEOUT);
props.add(ZOOKEEPER_TIMEOUT);
props.add(AUTO_OFFSET_RESET);
return props;
}
@ -191,26 +206,66 @@ public class GetKafka extends AbstractProcessor {
return relationships;
}
@OnScheduled
public void createConsumers(final ProcessContext context) {
final String topic = context.getProperty(TOPIC).getValue();
final Map<String, Integer> topicCountMap = new HashMap<>(1);
topicCountMap.put(topic, context.getMaxConcurrentTasks());
final Properties props = new Properties();
props.setProperty("zookeeper.connect", context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue());
props.setProperty("group.id", context.getProperty(GROUP_ID).getValue());
props.setProperty("client.id", context.getProperty(CLIENT_NAME).getValue());
props.setProperty("auto.commit.interval.ms", String.valueOf(context.getProperty(ZOOKEEPER_COMMIT_DELAY).asTimePeriod(TimeUnit.MILLISECONDS)));
props.setProperty("auto.commit.enable", "true"); // just be explicit
props.setProperty("auto.offset.reset", "smallest");
props.setProperty("zk.connectiontimeout.ms", context.getProperty(ZOOKEEPER_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).toString());
props.setProperty("auto.offset.reset", context.getProperty(AUTO_OFFSET_RESET).getValue());
props.setProperty("zookeeper.connection.timeout.ms", context.getProperty(ZOOKEEPER_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).toString());
props.setProperty("socket.timeout.ms", context.getProperty(KAFKA_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).toString());
for (final Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
PropertyDescriptor descriptor = entry.getKey();
if (descriptor.isDynamic()) {
if (props.containsKey(descriptor.getName())) {
this.getLogger().warn("Overriding existing property '" + descriptor.getName() + "' which had value of '"
+ props.getProperty(descriptor.getName()) + "' with dynamically set value '" + entry.getValue() + "'.");
}
props.setProperty(descriptor.getName(), entry.getValue());
}
}
/*
* Unless user sets it to some explicit value we are setting it to the
* lowest possible value of 1 millisecond to ensure the
* consumerStream.hasNext() doesn't block. See
* http://kafka.apache.org/documentation.html#configuration) as well as
* comment in 'catch ConsumerTimeoutException' in onTrigger() for more
* explanation as to the reasoning behind it.
*/
if (!props.containsKey("consumer.timeout.ms")) {
this.getLogger().info("Setting 'consumer.timeout.ms' to 1 milliseconds to avoid consumer"
+ " block in the event when no events are present in Kafka topic. If you wish to change this value "
+ " set it as dynamic property. If you wish to explicitly enable consumer block (at your own risk)"
+ " set its value to -1.");
props.setProperty("consumer.timeout.ms", "1");
}
final ConsumerConfig consumerConfig = new ConsumerConfig(props);
consumer = Consumer.createJavaConsumerConnector(consumerConfig);
final Map<String, Integer> topicCountMap = new HashMap<>(1);
int partitionCount = KafkaUtils.retrievePartitionCountForTopic(context.getProperty(ZOOKEEPER_CONNECTION_STRING).getValue(), context.getProperty(TOPIC).getValue());
int concurrentTaskToUse = context.getMaxConcurrentTasks();
if (context.getMaxConcurrentTasks() < partitionCount){
this.getLogger().warn("The amount of concurrent tasks '" + context.getMaxConcurrentTasks() + "' configured for "
+ "this processor is less than the amount of partitions '" + partitionCount + "' for topic '" + context.getProperty(TOPIC).getValue() + "'. "
+ "Consider making it equal to the amount of partition count for most efficient event consumption.");
} else if (context.getMaxConcurrentTasks() > partitionCount){
concurrentTaskToUse = partitionCount;
this.getLogger().warn("The amount of concurrent tasks '" + context.getMaxConcurrentTasks() + "' configured for "
+ "this processor is greater than the amount of partitions '" + partitionCount + "' for topic '" + context.getProperty(TOPIC).getValue() + "'. "
+ "Therefore those tasks would never see a message. To avoid that the '" + partitionCount + "'(partition count) will be used to consume events");
}
topicCountMap.put(topic, concurrentTaskToUse);
final Map<String, List<KafkaStream<byte[], byte[]>>> consumerMap = consumer.createMessageStreams(topicCountMap);
final List<KafkaStream<byte[], byte[]>> streams = consumerMap.get(topic);
@ -219,10 +274,12 @@ public class GetKafka extends AbstractProcessor {
for (final KafkaStream<byte[], byte[]> stream : streams) {
streamIterators.add(stream.iterator());
}
this.consumerStreamsReady.set(true);
}
@OnStopped
public void shutdownConsumer() {
this.consumerStreamsReady.set(false);
if (consumer != null) {
try {
consumer.commitOffsets();
@ -232,75 +289,57 @@ public class GetKafka extends AbstractProcessor {
}
}
@OnUnscheduled
public void interruptIterators() {
// Kafka doesn't provide a non-blocking API for pulling messages. We can, however,
// interrupt the Threads. We do this when the Processor is stopped so that we have the
// ability to shutdown the Processor.
interruptionLock.lock();
try {
for (final Thread t : interruptableThreads) {
t.interrupt();
}
interruptableThreads.clear();
} finally {
interruptionLock.unlock();
}
}
protected ConsumerIterator<byte[], byte[]> getStreamIterator() {
return streamIterators.poll();
@Override
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
return new PropertyDescriptor.Builder()
.description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
.name(propertyDescriptorName).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).dynamic(true)
.build();
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
final ConsumerIterator<byte[], byte[]> iterator = getStreamIterator();
if (iterator == null) {
return;
/*
* Will ensure that consumer streams are ready upon the first invocation
* of onTrigger. Will be reset to 'false' in the event of exception
*/
synchronized (this.consumerStreamsReady) {
if (!this.consumerStreamsReady.get()) {
this.createConsumers(context);
}
}
ConsumerIterator<byte[], byte[]> iterator = this.getStreamIterator();
if (iterator != null) {
this.consumeFromKafka(context, session, iterator);
}
}
protected ConsumerIterator<byte[], byte[]> getStreamIterator() {
return this.streamIterators.poll();
}
private void consumeFromKafka(final ProcessContext context, final ProcessSession session,
ConsumerIterator<byte[], byte[]> iterator) throws ProcessException {
final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
final String demarcator = context.getProperty(MESSAGE_DEMARCATOR).getValue().replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
final byte[] demarcatorBytes = demarcator.getBytes(StandardCharsets.UTF_8);
final String topic = context.getProperty(TOPIC).getValue();
FlowFile flowFile = null;
FlowFile flowFile = session.create();
final Map<String, String> attributes = new HashMap<>();
attributes.put("kafka.topic", topic);
final long start = System.nanoTime();
int msgCount = 0;
try {
// add the current thread to the Set of those to be interrupted if processor stopped.
interruptionLock.lock();
try {
interruptableThreads.add(Thread.currentThread());
} finally {
interruptionLock.unlock();
}
final long start = System.nanoTime();
flowFile = session.create();
final Map<String, String> attributes = new HashMap<>();
attributes.put("kafka.topic", topic);
int numMessages = 0;
for (int msgCount = 0; msgCount < batchSize; msgCount++) {
// if the processor is stopped, iterator.hasNext() will throw an Exception.
// In this case, we just break out of the loop.
try {
if (!iterator.hasNext()) {
break;
}
} catch (final Exception e) {
break;
}
for (; msgCount < batchSize && iterator.hasNext(); msgCount++) {
final MessageAndMetadata<byte[], byte[]> mam = iterator.next();
if (mam == null) {
return;
}
final byte[] key = mam.key();
if (batchSize == 1) {
final byte[] key = mam.key();
// the kafka.key, kafka.offset, and kafka.partition attributes are added only
// for a batch size of 1.
if (key != null) {
@ -322,33 +361,26 @@ public class GetKafka extends AbstractProcessor {
out.write(mam.message());
}
});
numMessages++;
}
// If we received no messages, remove the FlowFile. Otherwise, send to success.
if (flowFile.getSize() == 0L) {
session.remove(flowFile);
} else {
flowFile = session.putAllAttributes(flowFile, attributes);
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
session.getProvenanceReporter().receive(flowFile, "kafka://" + topic, "Received " + numMessages + " Kafka messages", millis);
getLogger().info("Successfully received {} from Kafka with {} messages in {} millis", new Object[]{flowFile, numMessages, millis});
session.transfer(flowFile, REL_SUCCESS);
}
this.releaseFlowFile(flowFile, session, attributes, start, topic, msgCount);
} catch (ConsumerTimeoutException e) {
/*
* By default Kafka blocks indefinitely if topic is empty via
* stream.hasNext(). If 'consumer.timeout.ms' property is set (see
* http://kafka.apache.org/documentation.html#configuration) the
* hasNext() will fail with this exception. To this processor it
* simply means there are no messages and current task should exit
* in non-failure releasing the flow file if it was able to
* accumulate any events.
*/
this.releaseFlowFile(flowFile, session, attributes, start, topic, msgCount);
} catch (final Exception e) {
this.shutdownConsumer();
getLogger().error("Failed to receive FlowFile from Kafka due to {}", new Object[]{e});
if (flowFile != null) {
session.remove(flowFile);
}
} finally {
// Remove the current thread from the Set of Threads to interrupt.
interruptionLock.lock();
try {
interruptableThreads.remove(Thread.currentThread());
} finally {
interruptionLock.unlock();
}
// Add the iterator back to the queue
if (iterator != null) {
streamIterators.offer(iterator);
@ -356,4 +388,22 @@ public class GetKafka extends AbstractProcessor {
}
}
}
/**
* Will release flow file. Releasing of the flow file in the context of this
* operation implies the following:
*
* If Empty then remove from session and return
* If has something then transfer to REL_SUCCESS
*/
private void releaseFlowFile(FlowFile flowFile, ProcessSession session, Map<String, String> attributes, long start, String topic, int msgCount){
if (flowFile.getSize() == 0L) {
session.remove(flowFile);
} else {
flowFile = session.putAllAttributes(flowFile, attributes);
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
session.getProvenanceReporter().receive(flowFile, "kafka://" + topic, "Received " + msgCount + " Kafka messages", millis);
getLogger().info("Successfully received {} from Kafka with {} messages in {} millis", new Object[]{flowFile, msgCount, millis});
session.transfer(flowFile, REL_SUCCESS);
}
}
}

View File

@ -0,0 +1,56 @@
package org.apache.nifi.processors.kafka;
import java.util.Collections;
import org.I0Itec.zkclient.ZkClient;
import org.I0Itec.zkclient.exception.ZkMarshallingError;
/*
* 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.
*/
import org.I0Itec.zkclient.serialize.ZkSerializer;
import kafka.admin.AdminUtils;
import kafka.api.TopicMetadata;
import kafka.utils.ZKStringSerializer;
import scala.collection.JavaConversions;
/**
* Utility class to support interruction with Kafka internals.
*
*/
class KafkaUtils {
/**
* Will retrieve the amount of partitions for a given Kafka topic.
*/
static int retrievePartitionCountForTopic(String zookeeperConnectionString, String topicName) {
ZkClient zkClient = new ZkClient(zookeeperConnectionString);
zkClient.setZkSerializer(new ZkSerializer() {
@Override
public byte[] serialize(Object o) throws ZkMarshallingError {
return ZKStringSerializer.serialize(o);
}
@Override
public Object deserialize(byte[] bytes) throws ZkMarshallingError {
return ZKStringSerializer.deserialize(bytes);
}
});
scala.collection.Set<TopicMetadata> topicMetadatas = AdminUtils
.fetchTopicMetadataFromZk(JavaConversions.asScalaSet(Collections.singleton(topicName)), zkClient);
return topicMetadatas.size();
}
}

View File

@ -27,6 +27,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Properties;
import java.util.Set;
import java.util.Map.Entry;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@ -44,6 +45,7 @@ import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SupportsBatching;
@ -81,6 +83,11 @@ import scala.actors.threadpool.Arrays;
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka. The messages to send may be individual FlowFiles or may be delimited, using a "
+ "user-specified delimiter, such as a new-line.")
@TriggerWhenEmpty // because we have a queue of sessions that are ready to be committed
@DynamicProperty(name = "The name of a Kafka configuration property.", value = "The value of a given Kafka configuration property.",
description = "These properties will be added on the Kafka configuration after loading any provided configuration properties."
+ " In the event a dynamic property represents a property that was already set as part of the static properties, its value wil be"
+ " overriden with warning message describing the override."
+ " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration.")
public class PutKafka extends AbstractSessionFactoryProcessor {
private static final String SINGLE_BROKER_REGEX = ".*?\\:\\d{3,5}";
@ -356,6 +363,18 @@ public class PutKafka extends AbstractSessionFactoryProcessor {
properties.setProperty("retries", "0");
properties.setProperty("block.on.buffer.full", "false");
for (final Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
PropertyDescriptor descriptor = entry.getKey();
if (descriptor.isDynamic()) {
if (properties.containsKey(descriptor.getName())) {
this.getLogger().warn("Overriding existing property '" + descriptor.getName() + "' which had value of '"
+ properties.getProperty(descriptor.getName()) + "' with dynamically set value '"
+ entry.getValue() + "'.");
}
properties.setProperty(descriptor.getName(), entry.getValue());
}
}
return properties;
}
@ -397,6 +416,14 @@ public class PutKafka extends AbstractSessionFactoryProcessor {
return partitionInfos.get(partitionIdx).partition();
}
@Override
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
return new PropertyDescriptor.Builder()
.description("Specifies the value for '" + propertyDescriptorName + "' Kafka Configuration.")
.name(propertyDescriptorName).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).dynamic(true)
.build();
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
FlowFileMessageBatch batch;

View File

@ -36,6 +36,8 @@ import org.apache.avro.generic.GenericData.Record;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.hadoop.conf.Configuration;
import org.apache.nifi.annotation.behavior.DynamicProperty;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
@ -62,6 +64,7 @@ import com.google.common.collect.Lists;
@Tags({ "avro", "convert", "kite" })
@CapabilityDescription("Convert records from one Avro schema to another, including support for flattening and simple type conversions")
@InputRequirement(Requirement.INPUT_REQUIRED)
@DynamicProperty(name = "Field name from input schema",
value = "Field name for output schema",
description = "Explicit mappings from input schema to output schema, which supports renaming fields and stepping into nested records on the input schema using notation like parent.id")

View File

@ -27,6 +27,8 @@ import java.util.List;
import java.util.Set;
import org.apache.commons.io.IOUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
@ -48,6 +50,7 @@ import com.mongodb.client.MongoCollection;
import com.mongodb.client.MongoCursor;
@Tags({ "mongodb", "read", "get" })
@InputRequirement(Requirement.INPUT_FORBIDDEN)
@CapabilityDescription("Creates FlowFiles from documents in MongoDB")
public class GetMongo extends AbstractMongoProcessor {
public static final Validator DOCUMENT_VALIDATOR = new Validator() {

View File

@ -26,6 +26,8 @@ import java.util.List;
import java.util.Set;
import org.apache.nifi.annotation.behavior.EventDriven;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.PropertyDescriptor;
@ -46,6 +48,7 @@ import com.mongodb.client.model.UpdateOptions;
@EventDriven
@Tags({ "mongodb", "insert", "update", "write", "put" })
@InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("Writes the contents of a FlowFile to MongoDB")
public class PutMongo extends AbstractMongoProcessor {
static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")

View File

@ -1784,6 +1784,7 @@ public class PersistentProvenanceRepository implements ProvenanceEventRepository
try {
Thread.sleep(100L);
} catch (final InterruptedException ie) {
Thread.currentThread().interrupt();
}
}

View File

@ -347,6 +347,8 @@ public class StandardRecordReader implements RecordReader {
} else if (eventType == ProvenanceEventType.RECEIVE) {
builder.setTransitUri(readNullableString(dis));
builder.setSourceSystemFlowFileIdentifier(readNullableString(dis));
} else if (eventType == ProvenanceEventType.FETCH) {
builder.setTransitUri(readNullableString(dis));
} else if (eventType == ProvenanceEventType.SEND) {
builder.setTransitUri(readNullableString(dis));
} else if (eventType == ProvenanceEventType.ADDINFO) {

View File

@ -235,6 +235,8 @@ public class StandardRecordWriter implements RecordWriter {
} else if (recordType == ProvenanceEventType.RECEIVE) {
writeNullableString(out, record.getTransitUri());
writeNullableString(out, record.getSourceSystemFlowFileIdentifier());
} else if (recordType == ProvenanceEventType.FETCH) {
writeNullableString(out, record.getTransitUri());
} else if (recordType == ProvenanceEventType.SEND) {
writeNullableString(out, record.getTransitUri());
} else if (recordType == ProvenanceEventType.ADDINFO) {

View File

@ -17,15 +17,15 @@
package org.apache.nifi.provenance.lucene;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
@ -44,12 +44,9 @@ import org.apache.lucene.search.TopDocs;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DocsReader {
class DocsReader {
private final Logger logger = LoggerFactory.getLogger(DocsReader.class);
public DocsReader(final List<File> storageDirectories) {
}
public Set<ProvenanceEventRecord> read(final TopDocs topDocs, final IndexReader indexReader, final Collection<Path> allProvenanceLogFiles,
final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
if (retrievalCount.get() >= maxResults) {
@ -100,101 +97,61 @@ public class DocsReader {
}
}
if ( record == null ) {
throw new IOException("Failed to find Provenance Event " + d);
} else {
return record;
if (record == null) {
logger.warn("Failed to read Provenance Event for '" + d + "'. The event file may be missing or corrupted");
}
return record;
}
public Set<ProvenanceEventRecord> read(final List<Document> docs, final Collection<Path> allProvenanceLogFiles,
final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
final AtomicInteger retrievalCount, final int maxResults, final int maxAttributeChars) throws IOException {
if (retrievalCount.get() >= maxResults) {
return Collections.emptySet();
}
LuceneUtil.sortDocsForRetrieval(docs);
RecordReader reader = null;
String lastStorageFilename = null;
final Set<ProvenanceEventRecord> matchingRecords = new LinkedHashSet<>();
final long start = System.nanoTime();
Set<ProvenanceEventRecord> matchingRecords = new LinkedHashSet<>();
Map<String, List<Document>> byStorageNameDocGroups = LuceneUtil.groupDocsByStorageFileName(docs);
int eventsReadThisFile = 0;
int logFileCount = 0;
final Set<String> storageFilesToSkip = new HashSet<>();
int eventsReadThisFile = 0;
for (String storageFileName : byStorageNameDocGroups.keySet()) {
File provenanceEventFile = LuceneUtil.getProvenanceLogFile(storageFileName, allProvenanceLogFiles);
if (provenanceEventFile != null) {
try (RecordReader reader = RecordReaders.newRecordReader(provenanceEventFile, allProvenanceLogFiles,
maxAttributeChars)) {
try {
for (final Document d : docs) {
final String storageFilename = d.getField(FieldNames.STORAGE_FILENAME).stringValue();
if ( storageFilesToSkip.contains(storageFilename) ) {
continue;
}
try {
if (reader != null && storageFilename.equals(lastStorageFilename)) {
matchingRecords.add(getRecord(d, reader));
eventsReadThisFile++;
if ( retrievalCount.incrementAndGet() >= maxResults ) {
break;
}
} else {
logger.debug("Opening log file {}", storageFilename);
logFileCount++;
if (reader != null) {
reader.close();
}
final List<File> potentialFiles = LuceneUtil.getProvenanceLogFiles(storageFilename, allProvenanceLogFiles);
if (potentialFiles.isEmpty()) {
logger.warn("Could not find Provenance Log File with basename {} in the "
+ "Provenance Repository; assuming file has expired and continuing without it", storageFilename);
storageFilesToSkip.add(storageFilename);
continue;
}
if (potentialFiles.size() > 1) {
throw new FileNotFoundException("Found multiple Provenance Log Files with basename " +
storageFilename + " in the Provenance Repository");
}
for (final File file : potentialFiles) {
try {
if (reader != null) {
logger.debug("Read {} records from previous file", eventsReadThisFile);
}
reader = RecordReaders.newRecordReader(file, allProvenanceLogFiles, maxAttributeChars);
matchingRecords.add(getRecord(d, reader));
eventsReadThisFile = 1;
if ( retrievalCount.incrementAndGet() >= maxResults ) {
break;
}
} catch (final IOException e) {
throw new IOException("Failed to retrieve record " + d + " from Provenance File " + file + " due to " + e, e);
}
Iterator<Document> docIter = byStorageNameDocGroups.get(storageFileName).iterator();
while (docIter.hasNext() && retrievalCount.incrementAndGet() < maxResults){
ProvenanceEventRecord eRec = this.getRecord(docIter.next(), reader);
if (eRec != null) {
matchingRecords.add(eRec);
eventsReadThisFile++;
}
}
} finally {
lastStorageFilename = storageFilename;
} catch (Exception e) {
logger.warn("Failed while trying to read Provenance Events. The event file '"
+ provenanceEventFile.getAbsolutePath() +
"' may be missing or corrupted.", e);
}
}
} finally {
if (reader != null) {
reader.close();
} else {
logger.warn("Could not find Provenance Log File with "
+ "basename {} in the Provenance Repository; assuming "
+ "file has expired and continuing without it", storageFileName);
}
}
logger.debug("Read {} records from previous file", eventsReadThisFile);
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
logger.debug("Took {} ms to read {} events from {} prov log files", millis, matchingRecords.size(), logFileCount);
logger.debug("Took {} ms to read {} events from {} prov log files", millis, matchingRecords.size(),
logFileCount);
return matchingRecords;
}
}

View File

@ -89,7 +89,7 @@ public class IndexSearch {
return sqr;
}
final DocsReader docsReader = new DocsReader(repository.getConfiguration().getStorageDirectories());
final DocsReader docsReader = new DocsReader();
matchingRecords = docsReader.read(topDocs, searcher.getIndexReader(), repository.getAllLogFiles(), retrievedCount,
provenanceQuery.getMaxResults(), maxAttributeChars);

View File

@ -93,7 +93,7 @@ public class LineageQuery {
final TopDocs uuidQueryTopDocs = searcher.search(query, MAX_QUERY_RESULTS);
final long searchEnd = System.nanoTime();
final DocsReader docsReader = new DocsReader(repo.getConfiguration().getStorageDirectories());
final DocsReader docsReader = new DocsReader();
final Set<ProvenanceEventRecord> recs = docsReader.read(uuidQueryTopDocs, searcher.getIndexReader(), repo.getAllLogFiles(),
new AtomicInteger(0), Integer.MAX_VALUE, maxAttributeChars);

View File

@ -22,7 +22,9 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.provenance.SearchableFields;
@ -128,8 +130,14 @@ public class LuceneUtil {
return luceneQuery;
}
/**
* Will sort documents by filename and then file offset so that we can
* retrieve the records efficiently
*
* @param documents
* list of {@link Document}s
*/
public static void sortDocsForRetrieval(final List<Document> documents) {
// sort by filename and then file offset so that we can retrieve the records efficiently
Collections.sort(documents, new Comparator<Document>() {
@Override
public int compare(final Document o1, final Document o2) {
@ -160,4 +168,30 @@ public class LuceneUtil {
}
});
}
/**
* Will group documents based on the {@link FieldNames#STORAGE_FILENAME}.
*
* @param documents
* list of {@link Document}s which will be sorted via
* {@link #sortDocsForRetrieval(List)} for more efficient record
* retrieval.
* @return a {@link Map} of document groups with
* {@link FieldNames#STORAGE_FILENAME} as key and {@link List} of
* {@link Document}s as value.
*/
public static Map<String, List<Document>> groupDocsByStorageFileName(final List<Document> documents) {
Map<String, List<Document>> documentGroups = new HashMap<>();
for (Document document : documents) {
String fileName = document.get(FieldNames.STORAGE_FILENAME);
if (!documentGroups.containsKey(fileName)) {
documentGroups.put(fileName, new ArrayList<Document>());
}
documentGroups.get(fileName).add(document);
}
for (List<Document> groupedDocuments : documentGroups.values()) {
sortDocsForRetrieval(groupedDocuments);
}
return documentGroups;
}
}

View File

@ -20,9 +20,11 @@ import static org.apache.nifi.provenance.TestUtil.createFlowFile;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import java.io.File;
import java.io.FileFilter;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
@ -36,6 +38,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.zip.GZIPOutputStream;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.core.SimpleAnalyzer;
@ -48,6 +51,7 @@ import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.FSDirectory;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.provenance.lineage.EventNode;
import org.apache.nifi.provenance.lineage.Lineage;
import org.apache.nifi.provenance.lineage.LineageEdge;
@ -869,6 +873,72 @@ public class TestPersistentProvenanceRepository {
}
}
/**
* Here the event file is simply corrupted by virtue of not having any event
* records while having correct headers
*/
@Test
public void testWithWithEventFileMissingRecord() throws Exception {
File eventFile = this.prepCorruptedEventFileTests();
final Query query = new Query(UUID.randomUUID().toString());
query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.ComponentID, "foo-*"));
query.setMaxResults(100);
DataOutputStream in = new DataOutputStream(new GZIPOutputStream(new FileOutputStream(eventFile)));
in.writeUTF("BlahBlah");
in.writeInt(4);
in.close();
assertTrue(eventFile.exists());
final QueryResult result = repo.queryEvents(query);
assertEquals(10, result.getMatchingEvents().size());
}
/**
* Here the event file is simply corrupted by virtue of being empty (0
* bytes)
*/
@Test
public void testWithWithEventFileCorrupted() throws Exception {
File eventFile = this.prepCorruptedEventFileTests();
final Query query = new Query(UUID.randomUUID().toString());
query.addSearchTerm(SearchTerms.newSearchTerm(SearchableFields.ComponentID, "foo-*"));
query.setMaxResults(100);
DataOutputStream in = new DataOutputStream(new GZIPOutputStream(new FileOutputStream(eventFile)));
in.close();
final QueryResult result = repo.queryEvents(query);
assertEquals(10, result.getMatchingEvents().size());
}
private File prepCorruptedEventFileTests() throws Exception {
RepositoryConfiguration config = createConfiguration();
config.setMaxStorageCapacity(1024L * 1024L);
config.setMaxEventFileLife(500, TimeUnit.MILLISECONDS);
config.setMaxEventFileCapacity(1024L * 1024L);
config.setSearchableFields(new ArrayList<>(SearchableFields.getStandardFields()));
config.setDesiredIndexSize(10);
repo = new PersistentProvenanceRepository(config, DEFAULT_ROLLOVER_MILLIS);
repo.initialize(getEventReporter());
String uuid = UUID.randomUUID().toString();
for (int i = 0; i < 20; i++) {
ProvenanceEventRecord record = repo.eventBuilder().fromFlowFile(mock(FlowFile.class))
.setEventType(ProvenanceEventType.CREATE).setComponentId("foo-" + i).setComponentType("myComponent")
.setFlowFileUUID(uuid).build();
repo.registerEvent(record);
if (i == 9) {
repo.waitForRollover();
Thread.sleep(2000L);
}
}
repo.waitForRollover();
File eventFile = new File(config.getStorageDirectories().get(0), "10.prov.gz");
assertTrue(eventFile.delete());
return eventFile;
}
@Test
public void testIndexDirectoryRemoved() throws InterruptedException, IOException, ParseException {
final RepositoryConfiguration config = createConfiguration();

View File

@ -156,7 +156,7 @@ public class GetTwitter extends AbstractProcessor {
private final BlockingQueue<Event> eventQueue = new LinkedBlockingQueue<>(1000);
private volatile Client client;
private volatile BlockingQueue<String> messageQueue = new LinkedBlockingQueue<>(10000);
private volatile BlockingQueue<String> messageQueue = new LinkedBlockingQueue<>(5000);
@Override
protected void init(final ProcessorInitializationContext context) {
@ -222,8 +222,6 @@ public class GetTwitter extends AbstractProcessor {
@OnScheduled
public void onScheduled(final ProcessContext context) throws MalformedURLException {
messageQueue = new LinkedBlockingQueue<>(100000);
final String endpointName = context.getProperty(ENDPOINT).getValue();
final Authentication oauth = new OAuth1(context.getProperty(CONSUMER_KEY).getValue(),
context.getProperty(CONSUMER_SECRET).getValue(),

View File

@ -124,11 +124,6 @@ language governing permissions and limitations under the License. -->
<groupId>net.sf.saxon</groupId>
<artifactId>Saxon-HE</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-mock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-socket-utils</artifactId>
@ -137,11 +132,6 @@ language governing permissions and limitations under the License. -->
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-load-distribution-service-api</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-distributed-cache-client-service</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>joda-time</groupId>
<artifactId>joda-time</artifactId>
@ -154,20 +144,10 @@ language governing permissions and limitations under the License. -->
<groupId>org.apache.activemq</groupId>
<artifactId>activemq-client</artifactId>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>activemq-broker</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.jayway.jsonpath</groupId>
<artifactId>json-path</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-ssl-context-service</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.tika</groupId>
<artifactId>tika-core</artifactId>
@ -189,7 +169,37 @@ language governing permissions and limitations under the License. -->
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-mapper-asl</artifactId>
</dependency>
<dependency>
<groupId>com.squareup.okhttp</groupId>
<artifactId>okhttp</artifactId>
<version>2.5.0</version>
</dependency>
<dependency>
<groupId>com.burgstaller</groupId>
<artifactId>okhttp-digest</artifactId>
<version>0.4</version>
<type>jar</type>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-mock</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-distributed-cache-client-service</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.activemq</groupId>
<artifactId>activemq-broker</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-ssl-context-service</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derby</artifactId>

View File

@ -66,7 +66,7 @@ public abstract class AbstractSyslogProcessor extends AbstractProcessor {
BODY("syslog.body"),
VALID("syslog.valid"),
PROTOCOL("syslog.protocol"),
PORT("syslog.pprt");
PORT("syslog.port");
private String key;

View File

@ -66,7 +66,7 @@ public class AttributesToJSON extends AbstractProcessor {
public static final String DESTINATION_ATTRIBUTE = "flowfile-attribute";
public static final String DESTINATION_CONTENT = "flowfile-content";
private final String APPLICATION_JSON = "application/json";
private static final String APPLICATION_JSON = "application/json";
public static final PropertyDescriptor ATTRIBUTES_LIST = new PropertyDescriptor.Builder()

View File

@ -561,25 +561,27 @@ public class ConvertJSONToSQL extends AbstractProcessor {
public static TableSchema from(final Connection conn, final String catalog, final String tableName,
final boolean translateColumnNames, final boolean includePrimaryKeys) throws SQLException {
final ResultSet colrs = conn.getMetaData().getColumns(catalog, null, tableName, "%");
try (final ResultSet colrs = conn.getMetaData().getColumns(catalog, null, tableName, "%")) {
final List<ColumnDescription> cols = new ArrayList<>();
while (colrs.next()) {
final ColumnDescription col = ColumnDescription.from(colrs);
cols.add(col);
}
final Set<String> primaryKeyColumns = new HashSet<>();
if (includePrimaryKeys) {
final ResultSet pkrs = conn.getMetaData().getPrimaryKeys(catalog, null, tableName);
while (pkrs.next()) {
final String colName = pkrs.getString("COLUMN_NAME");
primaryKeyColumns.add(normalizeColumnName(colName, translateColumnNames));
final List<ColumnDescription> cols = new ArrayList<>();
while (colrs.next()) {
final ColumnDescription col = ColumnDescription.from(colrs);
cols.add(col);
}
}
return new TableSchema(cols, translateColumnNames, primaryKeyColumns);
final Set<String> primaryKeyColumns = new HashSet<>();
if (includePrimaryKeys) {
try (final ResultSet pkrs = conn.getMetaData().getPrimaryKeys(catalog, null, tableName)) {
while (pkrs.next()) {
final String colName = pkrs.getString("COLUMN_NAME");
primaryKeyColumns.add(normalizeColumnName(colName, translateColumnNames));
}
}
}
return new TableSchema(cols, translateColumnNames, primaryKeyColumns);
}
}
}

View File

@ -171,7 +171,7 @@ public class EncodeContent extends AbstractProcessor {
}
}
private class EncodeBase64 implements StreamCallback {
private static class EncodeBase64 implements StreamCallback {
@Override
public void process(InputStream in, OutputStream out) throws IOException {
@ -181,7 +181,7 @@ public class EncodeContent extends AbstractProcessor {
}
}
private class DecodeBase64 implements StreamCallback {
private static class DecodeBase64 implements StreamCallback {
@Override
public void process(InputStream in, OutputStream out) throws IOException {
@ -191,7 +191,7 @@ public class EncodeContent extends AbstractProcessor {
}
}
private class EncodeBase32 implements StreamCallback {
private static class EncodeBase32 implements StreamCallback {
@Override
public void process(InputStream in, OutputStream out) throws IOException {
@ -201,7 +201,7 @@ public class EncodeContent extends AbstractProcessor {
}
}
private class DecodeBase32 implements StreamCallback {
private static class DecodeBase32 implements StreamCallback {
@Override
public void process(InputStream in, OutputStream out) throws IOException {
@ -213,7 +213,7 @@ public class EncodeContent extends AbstractProcessor {
private static final byte[] HEX_CHARS = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'};
private class EncodeHex implements StreamCallback {
private static class EncodeHex implements StreamCallback {
@Override
public void process(InputStream in, OutputStream out) throws IOException {
@ -231,7 +231,7 @@ public class EncodeContent extends AbstractProcessor {
}
}
private class DecodeHex implements StreamCallback {
private static class DecodeHex implements StreamCallback {
@Override
public void process(InputStream in, OutputStream out) throws IOException {

View File

@ -29,6 +29,8 @@ import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.stream.io.StreamUtils;
@ -54,6 +56,7 @@ import org.apache.nifi.annotation.documentation.SeeAlso;
@SideEffectFree
@SupportsBatching
@Tags({"deprecated"})
@InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("WARNING: This has been deprecated and will be removed in 0.2.0. \n\n Use ExtractText instead.")
@SeeAlso(ExtractText.class)
@Deprecated

View File

@ -129,7 +129,11 @@ public class ExecuteSQL extends AbstractProcessor {
FlowFile incoming = null;
if (context.hasIncomingConnection()) {
incoming = session.get();
if (incoming == null) {
// If we have no FlowFile, and all incoming connections are self-loops then we can continue on.
// However, if we have no FlowFile and we have connections coming from other Processors, then
// we know that we should run only if we have a FlowFile.
if (incoming == null && context.hasNonLoopConnection()) {
return;
}
}

View File

@ -30,6 +30,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
@ -59,6 +60,7 @@ import org.apache.nifi.processor.io.InputStreamCallback;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.standard.util.ArgumentUtils;
import org.apache.nifi.processors.standard.util.SoftLimitBoundedByteArrayOutputStream;
import org.apache.nifi.stream.io.BufferedInputStream;
import org.apache.nifi.stream.io.BufferedOutputStream;
import org.apache.nifi.stream.io.StreamUtils;
@ -126,7 +128,7 @@ import org.apache.nifi.stream.io.StreamUtils;
@CapabilityDescription("Executes an external command on the contents of a flow file, and creates a new flow file with the results of the command.")
@DynamicProperty(name = "An environment variable name", value = "An environment variable value", description = "These environment variables are passed to the process spawned by this Processor")
@WritesAttributes({
@WritesAttribute(attribute = "execution.command", description = "The name of the command executed to create the new FlowFile"),
@WritesAttribute(attribute = "execution.command", description = "The name of the command executed"),
@WritesAttribute(attribute = "execution.command.args", description = "The semi-colon delimited list of arguments"),
@WritesAttribute(attribute = "execution.status", description = "The exit status code returned from executing the command"),
@WritesAttribute(attribute = "execution.error", description = "Any error messages returned from executing the command")})
@ -140,14 +142,10 @@ public class ExecuteStreamCommand extends AbstractProcessor {
.name("output stream")
.description("The destination path for the flow file created from the command's output")
.build();
private static final Set<Relationship> RELATIONSHIPS;
private AtomicReference<Set<Relationship>> relationships = new AtomicReference<>();
static {
Set<Relationship> rels = new HashSet<>();
rels.add(OUTPUT_STREAM_RELATIONSHIP);
rels.add(ORIGINAL_RELATIONSHIP);
RELATIONSHIPS = Collections.unmodifiableSet(rels);
}
private final static Set<Relationship> OUTPUT_STREAM_RELATIONSHIP_SET;
private final static Set<Relationship> ATTRIBUTE_RELATIONSHIP_SET;
private static final Validator ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR = StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true);
static final PropertyDescriptor EXECUTION_COMMAND = new PropertyDescriptor.Builder()
@ -195,6 +193,22 @@ public class ExecuteStreamCommand extends AbstractProcessor {
.defaultValue("false")
.build();
static final PropertyDescriptor PUT_OUTPUT_IN_ATTRIBUTE = new PropertyDescriptor.Builder()
.name("Output Destination Attribute")
.description("If set, the output of the stream command will be put into an attribute of the original FlowFile instead of a separate "
+ "FlowFile. There will no longer be a relationship for 'output stream'. The value of this property will be the key for the output attribute.")
.addValidator(StandardValidators.ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR)
.build();
static final PropertyDescriptor PUT_ATTRIBUTE_MAX_LENGTH = new PropertyDescriptor.Builder()
.name("Max Attribute Length")
.description("If routing the output of the stream command to an attribute, the number of characters put to the attribute value "
+ "will be at most this amount. This is important because attributes are held in memory and large attributes will quickly "
+ "cause out of memory issues. If the output goes longer than this value, it will truncated to fit. Consider making this smaller if able.")
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
.defaultValue("256")
.build();
private static final Validator characterValidator = new StandardValidators.StringLengthValidator(1, 1);
static final PropertyDescriptor ARG_DELIMITER = new PropertyDescriptor.Builder()
@ -216,19 +230,44 @@ public class ExecuteStreamCommand extends AbstractProcessor {
props.add(IGNORE_STDIN);
props.add(WORKING_DIR);
props.add(ARG_DELIMITER);
props.add(PUT_OUTPUT_IN_ATTRIBUTE);
props.add(PUT_ATTRIBUTE_MAX_LENGTH);
PROPERTIES = Collections.unmodifiableList(props);
Set<Relationship> outputStreamRelationships = new HashSet<>();
outputStreamRelationships.add(OUTPUT_STREAM_RELATIONSHIP);
outputStreamRelationships.add(ORIGINAL_RELATIONSHIP);
OUTPUT_STREAM_RELATIONSHIP_SET = Collections.unmodifiableSet(outputStreamRelationships);
Set<Relationship> attributeRelationships = new HashSet<>();
attributeRelationships.add(ORIGINAL_RELATIONSHIP);
ATTRIBUTE_RELATIONSHIP_SET = Collections.unmodifiableSet(attributeRelationships);
}
private ProcessorLog logger;
@Override
public Set<Relationship> getRelationships() {
return RELATIONSHIPS;
return relationships.get();
}
@Override
protected void init(ProcessorInitializationContext context) {
logger = getLogger();
relationships.set(OUTPUT_STREAM_RELATIONSHIP_SET);
}
@Override
public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) {
if (descriptor.equals(PUT_OUTPUT_IN_ATTRIBUTE)) {
if (newValue != null) {
relationships.set(ATTRIBUTE_RELATIONSHIP_SET);
} else {
relationships.set(OUTPUT_STREAM_RELATIONSHIP_SET);
}
}
}
@Override
@ -254,6 +293,10 @@ public class ExecuteStreamCommand extends AbstractProcessor {
}
final ArrayList<String> args = new ArrayList<>();
final boolean putToAttribute = context.getProperty(PUT_OUTPUT_IN_ATTRIBUTE).isSet();
final Integer attributeSize = context.getProperty(PUT_ATTRIBUTE_MAX_LENGTH).asInteger();
final String attributeName = context.getProperty(PUT_OUTPUT_IN_ATTRIBUTE).getValue();
final String executeCommand = context.getProperty(EXECUTION_COMMAND).evaluateAttributeExpressions(inputFlowFile).getValue();
args.add(executeCommand);
final String commandArguments = context.getProperty(EXECUTION_ARGUMENTS).getValue();
@ -300,10 +343,17 @@ public class ExecuteStreamCommand extends AbstractProcessor {
final BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(pes))) {
int exitCode = -1;
final BufferedOutputStream bos = new BufferedOutputStream(pos);
FlowFile outputStreamFlowFile = session.create(inputFlowFile);
ProcessStreamWriterCallback callback = new ProcessStreamWriterCallback(ignoreStdin, bos, bis, logger, session, outputStreamFlowFile, process);
FlowFile outputFlowFile = putToAttribute ? inputFlowFile : session.create(inputFlowFile);
ProcessStreamWriterCallback callback = new ProcessStreamWriterCallback(ignoreStdin, bos, bis, logger,
attributeName, session, outputFlowFile, process,putToAttribute,attributeSize);
session.read(inputFlowFile, callback);
outputStreamFlowFile = callback.outputStreamFlowFile;
outputFlowFile = callback.outputFlowFile;
if (putToAttribute) {
outputFlowFile = session.putAttribute(outputFlowFile, attributeName, new String(callback.outputBuffer, 0, callback.size));
}
exitCode = callback.exitCode;
logger.debug("Execution complete for command: {}. Exited with code: {}", new Object[]{executeCommand, exitCode});
@ -321,21 +371,29 @@ public class ExecuteStreamCommand extends AbstractProcessor {
int length = strBldr.length() > 4000 ? 4000 : strBldr.length();
attributes.put("execution.error", strBldr.substring(0, length));
final Relationship outputFlowFileRelationship = putToAttribute ? ORIGINAL_RELATIONSHIP : OUTPUT_STREAM_RELATIONSHIP;
if (exitCode == 0) {
logger.info("Transferring flow file {} to 'output stream'", new Object[]{outputStreamFlowFile});
logger.info("Transferring flow file {} to {}",
new Object[]{outputFlowFile,outputFlowFileRelationship.getName()});
} else {
logger.error("Transferring flow file {} to 'output stream'. Executable command {} ended in an error: {}",
new Object[]{outputStreamFlowFile, executeCommand, strBldr.toString()});
logger.error("Transferring flow file {} to {}. Executable command {} ended in an error: {}",
new Object[]{outputFlowFile,outputFlowFileRelationship.getName(), executeCommand, strBldr.toString()});
}
attributes.put("execution.status", Integer.toString(exitCode));
attributes.put("execution.command", executeCommand);
attributes.put("execution.command.args", commandArguments);
outputStreamFlowFile = session.putAllAttributes(outputStreamFlowFile, attributes);
session.transfer(outputStreamFlowFile, OUTPUT_STREAM_RELATIONSHIP);
logger.info("Transferring flow file {} to original", new Object[]{inputFlowFile});
inputFlowFile = session.putAllAttributes(inputFlowFile, attributes);
session.transfer(inputFlowFile, ORIGINAL_RELATIONSHIP);
outputFlowFile = session.putAllAttributes(outputFlowFile, attributes);
// This transfer will transfer the FlowFile that received the stream out put to it's destined relationship.
// In the event the stream is put to the an attribute of the original, it will be transferred here.
session.transfer(outputFlowFile, outputFlowFileRelationship);
if (!putToAttribute) {
logger.info("Transferring flow file {} to original", new Object[]{inputFlowFile});
inputFlowFile = session.putAllAttributes(inputFlowFile, attributes);
session.transfer(inputFlowFile, ORIGINAL_RELATIONSHIP);
}
} catch (final IOException ex) {
// could not close Process related streams
@ -348,59 +406,97 @@ public class ExecuteStreamCommand extends AbstractProcessor {
static class ProcessStreamWriterCallback implements InputStreamCallback {
final boolean ignoreStdin;
final OutputStream stdInWritable;
final InputStream stdOutReadable;
final OutputStream stdinWritable;
final InputStream stdoutReadable;
final ProcessorLog logger;
final ProcessSession session;
final Process process;
FlowFile outputStreamFlowFile;
FlowFile outputFlowFile;
int exitCode;
final boolean putToAttribute;
final int attributeSize;
final String attributeName;
public ProcessStreamWriterCallback(boolean ignoreStdin, OutputStream stdInWritable, InputStream stdOutReadable,
ProcessorLog logger, ProcessSession session, FlowFile outputStreamFlowFile, Process process) {
byte[] outputBuffer;
int size;
public ProcessStreamWriterCallback(boolean ignoreStdin, OutputStream stdinWritable, InputStream stdoutReadable,ProcessorLog logger, String attributeName,
ProcessSession session, FlowFile outputFlowFile, Process process, boolean putToAttribute, int attributeSize) {
this.ignoreStdin = ignoreStdin;
this.stdInWritable = stdInWritable;
this.stdOutReadable = stdOutReadable;
this.stdinWritable = stdinWritable;
this.stdoutReadable = stdoutReadable;
this.logger = logger;
this.session = session;
this.outputStreamFlowFile = outputStreamFlowFile;
this.outputFlowFile = outputFlowFile;
this.process = process;
this.putToAttribute = putToAttribute;
this.attributeSize = attributeSize;
this.attributeName = attributeName;
}
@Override
public void process(final InputStream incomingFlowFileIS) throws IOException {
outputStreamFlowFile = session.write(outputStreamFlowFile, new OutputStreamCallback() {
if (putToAttribute) {
try (SoftLimitBoundedByteArrayOutputStream softLimitBoundedBAOS = new SoftLimitBoundedByteArrayOutputStream(attributeSize)) {
readStdoutReadable(ignoreStdin, stdinWritable, logger, incomingFlowFileIS);
final long longSize = StreamUtils.copy(stdoutReadable, softLimitBoundedBAOS);
@Override
public void process(OutputStream out) throws IOException {
// Because the outputstream has a cap that the copy doesn't know about, adjust
// the actual size
if (longSize > (long) attributeSize) { // Explicit cast for readability
size = attributeSize;
} else{
size = (int) longSize; // Note: safe cast, longSize is limited by attributeSize
}
Thread writerThread = new Thread(new Runnable() {
outputBuffer = softLimitBoundedBAOS.getBuffer();
stdoutReadable.close();
@Override
public void run() {
if (!ignoreStdin) {
try {
StreamUtils.copy(incomingFlowFileIS, stdInWritable);
} catch (IOException e) {
logger.error("Failed to write flow file to stdIn due to {}", new Object[]{e}, e);
}
}
// MUST close the output stream to the stdIn so that whatever is reading knows
// there is no more data
IOUtils.closeQuietly(stdInWritable);
}
});
writerThread.setDaemon(true);
writerThread.start();
StreamUtils.copy(stdOutReadable, out);
try {
exitCode = process.waitFor();
} catch (InterruptedException e) {
logger.warn("Command Execution Process was interrupted", e);
}
}
});
} else {
outputFlowFile = session.write(outputFlowFile, new OutputStreamCallback() {
@Override
public void process(OutputStream out) throws IOException {
readStdoutReadable(ignoreStdin, stdinWritable, logger, incomingFlowFileIS);
StreamUtils.copy(stdoutReadable, out);
try {
exitCode = process.waitFor();
} catch (InterruptedException e) {
logger.warn("Command Execution Process was interrupted", e);
}
}
});
}
}
}
private static void readStdoutReadable(final boolean ignoreStdin, final OutputStream stdinWritable,
final ProcessorLog logger, final InputStream incomingFlowFileIS) throws IOException {
Thread writerThread = new Thread(new Runnable() {
@Override
public void run() {
if (!ignoreStdin) {
try {
StreamUtils.copy(incomingFlowFileIS, stdinWritable);
} catch (IOException e) {
// This is unlikely to occur, and isn't handled at the moment
// Bug captured in NIFI-1194
logger.error("Failed to write flow file to stdin due to {}", new Object[]{e}, e);
}
}
// MUST close the output stream to the stdin so that whatever is reading knows
// there is no more data.
IOUtils.closeQuietly(stdinWritable);
}
});
writerThread.setDaemon(true);
writerThread.start();
}
}

View File

@ -0,0 +1,311 @@
/*
* 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.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.nio.file.CopyOption;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardCopyOption;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.StringUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.components.AllowableValue;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.flowfile.FlowFile;
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.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.util.StopWatch;
@InputRequirement(Requirement.INPUT_REQUIRED)
@Tags({"local", "files", "filesystem", "ingest", "ingress", "get", "source", "input"})
@CapabilityDescription("Reads the contents of a file from disk and streams it into the contents of an incoming FlowFile. Once this is done, the file is optionally moved elsewhere or deleted "
+ "to help keep the file system organized.")
@SeeAlso({GetFile.class, PutFile.class, ListFile.class})
public class FetchFile extends AbstractProcessor {
static final AllowableValue COMPLETION_NONE = new AllowableValue("None", "None", "Leave the file as-is");
static final AllowableValue COMPLETION_MOVE = new AllowableValue("Move File", "Move File", "Moves the file to the directory specified by the <Move Destination Directory> property");
static final AllowableValue COMPLETION_DELETE = new AllowableValue("Delete File", "Delete File", "Deletes the original file from the file system");
static final AllowableValue CONFLICT_REPLACE = new AllowableValue("Replace File", "Replace File", "The newly ingested file should replace the existing file in the Destination Directory");
static final AllowableValue CONFLICT_KEEP_INTACT = new AllowableValue("Keep Existing", "Keep Existing", "The existing file should in the Destination Directory should stay intact and the newly "
+ "ingested file should be deleted");
static final AllowableValue CONFLICT_FAIL = new AllowableValue("Fail", "Fail", "The existing destination file should remain intact and the incoming FlowFile should be routed to failure");
static final AllowableValue CONFLICT_RENAME = new AllowableValue("Rename", "Rename", "The existing destination file should remain intact. The newly ingested file should be moved to the "
+ "destination directory but be renamed to a random filename");
static final PropertyDescriptor FILENAME = new PropertyDescriptor.Builder()
.name("File to Fetch")
.description("The fully-qualified filename of the file to fetch from the file system")
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.expressionLanguageSupported(true)
.defaultValue("${absolute.path}/${filename}")
.required(true)
.build();
static final PropertyDescriptor COMPLETION_STRATEGY = new PropertyDescriptor.Builder()
.name("Completion Strategy")
.description("Specifies what to do with the original file on the file system once it has been pulled into NiFi")
.expressionLanguageSupported(false)
.allowableValues(COMPLETION_NONE, COMPLETION_MOVE, COMPLETION_DELETE)
.defaultValue(COMPLETION_NONE.getValue())
.required(true)
.build();
static final PropertyDescriptor MOVE_DESTINATION_DIR = new PropertyDescriptor.Builder()
.name("Move Destination Directory")
.description("The directory to the move the original file to once it has been fetched from the file system. This property is ignored unless the Completion Strategy is set to \"Move File\". "
+ "If the directory does not exist, it will be created.")
.expressionLanguageSupported(true)
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.required(false)
.build();
static final PropertyDescriptor CONFLICT_STRATEGY = new PropertyDescriptor.Builder()
.name("Move Conflict Strategy")
.description("If Completion Strategy is set to Move File and a file already exists in the destination directory with the same name, this property specifies "
+ "how that naming conflict should be resolved")
.allowableValues(CONFLICT_RENAME, CONFLICT_REPLACE, CONFLICT_KEEP_INTACT, CONFLICT_FAIL)
.defaultValue(CONFLICT_RENAME.getValue())
.required(true)
.build();
static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("Any FlowFile that is successfully fetched from the file system will be transferred to this Relationship.")
.build();
static final Relationship REL_NOT_FOUND = new Relationship.Builder()
.name("not.found")
.description("Any FlowFile that could not be fetched from the file system because the file could not be found will be transferred to this Relationship.")
.build();
static final Relationship REL_PERMISSION_DENIED = new Relationship.Builder()
.name("permission.denied")
.description("Any FlowFile that could not be fetched from the file system due to the user running NiFi not having sufficient permissions will be transferred to this Relationship.")
.build();
static final Relationship REL_FAILURE = new Relationship.Builder()
.name("failure")
.description(
"Any FlowFile that could not be fetched from the file system for any reason other than insufficient permissions or the file not existing will be transferred to this Relationship.")
.build();
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(FILENAME);
properties.add(COMPLETION_STRATEGY);
properties.add(MOVE_DESTINATION_DIR);
properties.add(CONFLICT_STRATEGY);
return properties;
}
@Override
public Set<Relationship> getRelationships() {
final Set<Relationship> relationships = new HashSet<>();
relationships.add(REL_SUCCESS);
relationships.add(REL_NOT_FOUND);
relationships.add(REL_PERMISSION_DENIED);
relationships.add(REL_FAILURE);
return relationships;
}
@Override
protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
final List<ValidationResult> results = new ArrayList<>();
if (COMPLETION_MOVE.getValue().equalsIgnoreCase(validationContext.getProperty(COMPLETION_STRATEGY).getValue())) {
if (!validationContext.getProperty(MOVE_DESTINATION_DIR).isSet()) {
results.add(new ValidationResult.Builder().subject(MOVE_DESTINATION_DIR.getName()).input(null).valid(false).explanation(
MOVE_DESTINATION_DIR.getName() + " must be specified if " + COMPLETION_STRATEGY.getName() + " is set to " + COMPLETION_MOVE.getDisplayName()).build());
}
}
return results;
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
FlowFile flowFile = session.get();
if (flowFile == null) {
return;
}
final StopWatch stopWatch = new StopWatch(true);
final String filename = context.getProperty(FILENAME).evaluateAttributeExpressions(flowFile).getValue();
final File file = new File(filename);
// Verify that file exists
if (!file.exists()) {
getLogger().error("Could not fetch file {} from file system for {} because the file does not exist; routing to not.found", new Object[] {file, flowFile});
session.getProvenanceReporter().route(flowFile, REL_NOT_FOUND);
session.transfer(session.penalize(flowFile), REL_NOT_FOUND);
return;
}
// Verify read permission on file
final String user = System.getProperty("user.name");
if (!isReadable(file)) {
getLogger().error("Could not fetch file {} from file system for {} due to user {} not having sufficient permissions to read the file; routing to permission.denied",
new Object[] {file, flowFile, user});
session.getProvenanceReporter().route(flowFile, REL_PERMISSION_DENIED);
session.transfer(session.penalize(flowFile), REL_PERMISSION_DENIED);
return;
}
// If configured to move the file and fail if unable to do so, check that the existing file does not exist and that we have write permissions
// for the parent file.
final String completionStrategy = context.getProperty(COMPLETION_STRATEGY).getValue();
final String targetDirectoryName = context.getProperty(MOVE_DESTINATION_DIR).evaluateAttributeExpressions(flowFile).getValue();
if (targetDirectoryName != null) {
final File targetDir = new File(targetDirectoryName);
if (COMPLETION_MOVE.getValue().equalsIgnoreCase(completionStrategy)) {
if (targetDir.exists() && (!isWritable(targetDir) || !isDirectory(targetDir))) {
getLogger().error("Could not fetch file {} from file system for {} because Completion Strategy is configured to move the original file to {}, "
+ "but that is not a directory or user {} does not have permissions to write to that directory",
new Object[] {file, flowFile, targetDir, user});
session.transfer(flowFile, REL_FAILURE);
return;
}
final String conflictStrategy = context.getProperty(CONFLICT_STRATEGY).getValue();
if (CONFLICT_FAIL.getValue().equalsIgnoreCase(conflictStrategy)) {
final File targetFile = new File(targetDir, file.getName());
if (targetFile.exists()) {
getLogger().error("Could not fetch file {} from file system for {} because Completion Strategy is configured to move the original file to {}, "
+ "but a file with name {} already exists in that directory and the Move Conflict Strategy is configured for failure",
new Object[] {file, flowFile, targetDir, file.getName()});
session.transfer(flowFile, REL_FAILURE);
return;
}
}
}
}
// import content from file system
try (final FileInputStream fis = new FileInputStream(file)) {
flowFile = session.importFrom(fis, flowFile);
} catch (final IOException ioe) {
getLogger().error("Could not fetch file {} from file system for {} due to {}; routing to failure", new Object[] {file, flowFile, ioe.toString()}, ioe);
session.transfer(session.penalize(flowFile), REL_FAILURE);
return;
}
session.getProvenanceReporter().modifyContent(flowFile, "Replaced content of FlowFile with contents of " + file.toURI(), stopWatch.getElapsed(TimeUnit.MILLISECONDS));
session.transfer(flowFile, REL_SUCCESS);
// It is critical that we commit the session before we perform the Completion Strategy. Otherwise, we could have a case where we
// ingest the file, delete/move the file, and then NiFi is restarted before the session is committed. That would result in data loss.
// As long as we commit the session right here, before we perform the Completion Strategy, we are safe.
session.commit();
// Attempt to perform the Completion Strategy action
Exception completionFailureException = null;
if (COMPLETION_DELETE.getValue().equalsIgnoreCase(completionStrategy)) {
// convert to path and use Files.delete instead of file.delete so that if we fail, we know why
try {
delete(file);
} catch (final IOException ioe) {
completionFailureException = ioe;
}
} else if (COMPLETION_MOVE.getValue().equalsIgnoreCase(completionStrategy)) {
final File targetDirectory = new File(targetDirectoryName);
final File targetFile = new File(targetDirectory, file.getName());
try {
if (targetFile.exists()) {
final String conflictStrategy = context.getProperty(CONFLICT_STRATEGY).getValue();
if (CONFLICT_KEEP_INTACT.getValue().equalsIgnoreCase(conflictStrategy)) {
// don't move, just delete the original
Files.delete(file.toPath());
} else if (CONFLICT_RENAME.getValue().equalsIgnoreCase(conflictStrategy)) {
// rename to add a random UUID but keep the file extension if it has one.
final String simpleFilename = targetFile.getName();
final String newName;
if (simpleFilename.contains(".")) {
newName = StringUtils.substringBeforeLast(simpleFilename, ".") + "-" + UUID.randomUUID().toString() + "." + StringUtils.substringAfterLast(simpleFilename, ".");
} else {
newName = simpleFilename + "-" + UUID.randomUUID().toString();
}
move(file, new File(targetDirectory, newName), false);
} else if (CONFLICT_REPLACE.getValue().equalsIgnoreCase(conflictStrategy)) {
move(file, targetFile, true);
}
} else {
move(file, targetFile, false);
}
} catch (final IOException ioe) {
completionFailureException = ioe;
}
}
// Handle completion failures
if (completionFailureException != null) {
getLogger().warn("Successfully fetched the content from {} for {} but failed to perform Completion Action due to {}; routing to success",
new Object[] {file, flowFile, completionFailureException}, completionFailureException);
}
}
//
// The following set of methods exist purely for testing purposes
//
protected void move(final File source, final File target, final boolean overwrite) throws IOException {
final File targetDirectory = target.getParentFile();
// convert to path and use Files.move instead of file.renameTo so that if we fail, we know why
final Path targetPath = target.toPath();
if (!targetDirectory.exists()) {
Files.createDirectories(targetDirectory.toPath());
}
final CopyOption[] copyOptions = overwrite ? new CopyOption[] {StandardCopyOption.REPLACE_EXISTING} : new CopyOption[] {};
Files.move(source.toPath(), targetPath, copyOptions);
}
protected void delete(final File file) throws IOException {
Files.delete(file.toPath());
}
protected boolean isReadable(final File file) {
return file.canRead();
}
protected boolean isWritable(final File file) {
return file.canWrite();
}
protected boolean isDirectory(final File file) {
return file.isDirectory();
}
}

View File

@ -91,7 +91,7 @@ import org.apache.nifi.processor.util.StandardValidators;
@WritesAttribute(attribute = "file.permissions", description = "The read/write/execute permissions of the file. May not work on all file systems"),
@WritesAttribute(attribute = "absolute.path", description = "The full/absolute path from where a file was picked up. The current 'path' "
+ "attribute is still populated, but may be a relative path")})
@SeeAlso(PutFile.class)
@SeeAlso({PutFile.class, FetchFile.class})
public class GetFile extends AbstractProcessor {
public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()

View File

@ -49,13 +49,13 @@ import java.util.regex.Pattern;
import javax.net.ssl.SSLContext;
import org.apache.commons.lang3.StringUtils;
import org.apache.http.Header;
import org.apache.http.HttpHost;
import org.apache.http.HttpResponse;
import org.apache.http.auth.AuthScope;
import org.apache.http.auth.UsernamePasswordCredentials;
import org.apache.http.client.CredentialsProvider;
import org.apache.http.client.HttpClient;
import org.apache.http.client.config.RequestConfig;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.config.Registry;
@ -64,11 +64,12 @@ import org.apache.http.conn.HttpClientConnectionManager;
import org.apache.http.conn.socket.ConnectionSocketFactory;
import org.apache.http.conn.socket.PlainConnectionSocketFactory;
import org.apache.http.conn.ssl.SSLConnectionSocketFactory;
import org.apache.http.conn.ssl.SSLContexts;
import org.apache.http.conn.ssl.TrustSelfSignedStrategy;
import org.apache.http.impl.client.BasicCredentialsProvider;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClientBuilder;
import org.apache.http.impl.conn.BasicHttpClientConnectionManager;
import org.apache.http.ssl.SSLContextBuilder;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.WritesAttribute;
@ -320,19 +321,26 @@ public class GetHTTP extends AbstractSessionFactoryProcessor {
private SSLContext createSSLContext(final SSLContextService service)
throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException, KeyManagementException, UnrecoverableKeyException {
final KeyStore truststore = KeyStore.getInstance(service.getTrustStoreType());
try (final InputStream in = new FileInputStream(new File(service.getTrustStoreFile()))) {
truststore.load(in, service.getTrustStorePassword().toCharArray());
final SSLContextBuilder sslContextBuilder = new SSLContextBuilder();
if (StringUtils.isNotBlank(service.getTrustStoreFile())) {
final KeyStore truststore = KeyStore.getInstance(service.getTrustStoreType());
try (final InputStream in = new FileInputStream(new File(service.getTrustStoreFile()))) {
truststore.load(in, service.getTrustStorePassword().toCharArray());
}
sslContextBuilder.loadTrustMaterial(truststore, new TrustSelfSignedStrategy());
}
final KeyStore keystore = KeyStore.getInstance(service.getKeyStoreType());
try (final InputStream in = new FileInputStream(new File(service.getKeyStoreFile()))) {
keystore.load(in, service.getKeyStorePassword().toCharArray());
if (StringUtils.isNotBlank(service.getKeyStoreFile())){
final KeyStore keystore = KeyStore.getInstance(service.getKeyStoreType());
try (final InputStream in = new FileInputStream(new File(service.getKeyStoreFile()))) {
keystore.load(in, service.getKeyStorePassword().toCharArray());
}
sslContextBuilder.loadKeyMaterial(keystore, service.getKeyStorePassword().toCharArray());
}
final SSLContext sslContext = SSLContexts.custom().loadTrustMaterial(truststore, new TrustSelfSignedStrategy()).loadKeyMaterial(keystore, service.getKeyStorePassword().toCharArray()).build();
return sslContext;
return sslContextBuilder.build();
}
@Override
@ -430,7 +438,7 @@ public class GetHTTP extends AbstractSessionFactoryProcessor {
}
// create the http client
final HttpClient client = clientBuilder.build();
final CloseableHttpClient client = clientBuilder.build();
// create request
final HttpGet get = new HttpGet(url);
@ -531,7 +539,6 @@ public class GetHTTP extends AbstractSessionFactoryProcessor {
logger.error("Failed to process due to {}; rolling back session", new Object[]{t.getMessage()}, t);
throw t;
}
} finally {
conMan.shutdown();
}

View File

@ -0,0 +1,370 @@
/*
* 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.File;
import java.io.FileFilter;
import java.io.IOException;
import java.nio.file.FileStore;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.nio.file.attribute.BasicFileAttributeView;
import java.nio.file.attribute.BasicFileAttributes;
import java.nio.file.attribute.FileOwnerAttributeView;
import java.nio.file.attribute.PosixFileAttributeView;
import java.nio.file.attribute.PosixFilePermissions;
import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.regex.Pattern;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.SeeAlso;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.DataUnit;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.standard.util.FileInfo;
@TriggerSerially
@InputRequirement(Requirement.INPUT_FORBIDDEN)
@Tags({"file", "get", "list", "ingest", "source", "filesystem"})
@CapabilityDescription("Retrieves a listing of files from the local filesystem. For each file that is listed, " +
"creates a FlowFile that represents the file so that it can be fetched in conjunction with ListFile. This " +
"Processor is designed to run on Primary Node only in a cluster. If the primary node changes, the new " +
"Primary Node will pick up where the previous node left off without duplicating all of the data. Unlike " +
"GetFile, this Processor does not delete any data from the local filesystem.")
@WritesAttributes({
@WritesAttribute(attribute="filename", description="The name of the file that was read from filesystem."),
@WritesAttribute(attribute="path", description="The path is set to the relative path of the file's directory " +
"on filesystem compared to the Input Directory property. For example, if Input Directory is set to " +
"/tmp, then files picked up from /tmp will have the path attribute set to \"/\". If the Recurse " +
"Subdirectories property is set to true and a file is picked up from /tmp/abc/1/2/3, then the path " +
"attribute will be set to \"abc/1/2/3/\"."),
@WritesAttribute(attribute="absolute.path", description="The absolute.path is set to the absolute path of " +
"the file's directory on filesystem. For example, if the Input Directory property is set to /tmp, " +
"then files picked up from /tmp will have the path attribute set to \"/tmp/\". If the Recurse " +
"Subdirectories property is set to true and a file is picked up from /tmp/abc/1/2/3, then the path " +
"attribute will be set to \"/tmp/abc/1/2/3/\"."),
@WritesAttribute(attribute="fs.owner", description="The user that owns the file in filesystem"),
@WritesAttribute(attribute="fs.group", description="The group that owns the file in filesystem"),
@WritesAttribute(attribute="fs.lastModified", description="The timestamp of when the file in filesystem was " +
"last modified, as milliseconds since midnight Jan 1, 1970 UTC"),
@WritesAttribute(attribute="fs.length", description="The number of bytes in the file in filesystem"),
@WritesAttribute(attribute="fs.permissions", description="The permissions for the file in filesystem. This " +
"is formatted as 3 characters for the owner, 3 for the group, and 3 for other users. For example " +
"rw-rw-r--")
})
@SeeAlso({GetFile.class, PutFile.class, FetchFile.class})
public class ListFile extends AbstractListProcessor<FileInfo> {
public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
.name("Input Directory")
.description("The input directory from which files to pull files")
.required(true)
.addValidator(StandardValidators.createDirectoryExistsValidator(true, false))
.expressionLanguageSupported(true)
.build();
public static final PropertyDescriptor RECURSE = new PropertyDescriptor.Builder()
.name("Recurse Subdirectories")
.description("Indicates whether to list files from subdirectories of the directory")
.required(true)
.allowableValues("true", "false")
.defaultValue("true")
.build();
public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
.name("File Filter")
.description("Only files whose names match the given regular expression will be picked up")
.required(true)
.defaultValue("[^\\.].*")
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
.build();
public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
.name("Path Filter")
.description("When " + RECURSE.getName() + " is true, then only subdirectories whose path matches the given regular expression will be scanned")
.required(false)
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
.build();
public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
.name("Minimum File Age")
.description("The minimum age that a file must be in order to be pulled; any file younger than this amount of time (according to last modification date) will be ignored")
.required(true)
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.defaultValue("0 sec")
.build();
public static final PropertyDescriptor MAX_AGE = new PropertyDescriptor.Builder()
.name("Maximum File Age")
.description("The maximum age that a file must be in order to be pulled; any file older than this amount of time (according to last modification date) will be ignored")
.required(false)
.addValidator(StandardValidators.createTimePeriodValidator(100, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
.build();
public static final PropertyDescriptor MIN_SIZE = new PropertyDescriptor.Builder()
.name("Minimum File Size")
.description("The minimum size that a file must be in order to be pulled")
.required(true)
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
.defaultValue("0 B")
.build();
public static final PropertyDescriptor MAX_SIZE = new PropertyDescriptor.Builder()
.name("Maximum File Size")
.description("The maximum size that a file can be in order to be pulled")
.required(false)
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
.build();
public static final PropertyDescriptor IGNORE_HIDDEN_FILES = new PropertyDescriptor.Builder()
.name("Ignore Hidden Files")
.description("Indicates whether or not hidden files should be ignored")
.allowableValues("true", "false")
.defaultValue("true")
.required(true)
.build();
private List<PropertyDescriptor> properties;
private Set<Relationship> relationships;
private final AtomicReference<FileFilter> fileFilterRef = new AtomicReference<>();
public static final String FILE_CREATION_TIME_ATTRIBUTE = "file.creationTime";
public static final String FILE_LAST_MODIFY_TIME_ATTRIBUTE = "file.lastModifiedTime";
public static final String FILE_LAST_ACCESS_TIME_ATTRIBUTE = "file.lastAccessTime";
public static final String FILE_SIZE_ATTRIBUTE = "file.size";
public static final String FILE_OWNER_ATTRIBUTE = "file.owner";
public static final String FILE_GROUP_ATTRIBUTE = "file.group";
public static final String FILE_PERMISSIONS_ATTRIBUTE = "file.permissions";
public static final String FILE_MODIFY_DATE_ATTR_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
@Override
protected void init(final ProcessorInitializationContext context) {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(DIRECTORY);
properties.add(RECURSE);
properties.add(FILE_FILTER);
properties.add(PATH_FILTER);
properties.add(MIN_AGE);
properties.add(MAX_AGE);
properties.add(MIN_SIZE);
properties.add(MAX_SIZE);
properties.add(IGNORE_HIDDEN_FILES);
this.properties = Collections.unmodifiableList(properties);
final Set<Relationship> relationships = new HashSet<>();
relationships.add(REL_SUCCESS);
this.relationships = Collections.unmodifiableSet(relationships);
}
@Override
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return properties;
}
@Override
public Set<Relationship> getRelationships() {
return relationships;
}
@OnScheduled
public void onScheduled(final ProcessContext context) {
fileFilterRef.set(createFileFilter(context));
}
@Override
protected Map<String, String> createAttributes(final FileInfo fileInfo, final ProcessContext context) {
final Map<String, String> attributes = new HashMap<>();
final String fullPath = fileInfo.getFullPathFileName();
final File file = new File(fullPath);
final Path filePath = file.toPath();
final Path directoryPath = new File(getPath(context)).toPath();
final Path relativePath = directoryPath.relativize(filePath.getParent());
String relativePathString = relativePath.toString() + "/";
final Path absPath = filePath.toAbsolutePath();
final String absPathString = absPath.getParent().toString() + "/";
attributes.put(CoreAttributes.PATH.key(), relativePathString);
attributes.put(CoreAttributes.FILENAME.key(), fileInfo.getFileName());
attributes.put(CoreAttributes.ABSOLUTE_PATH.key(), absPathString);
try {
FileStore store = Files.getFileStore(filePath);
if (store.supportsFileAttributeView("basic")) {
try {
final DateFormat formatter = new SimpleDateFormat(FILE_MODIFY_DATE_ATTR_FORMAT, Locale.US);
BasicFileAttributeView view = Files.getFileAttributeView(filePath, BasicFileAttributeView.class);
BasicFileAttributes attrs = view.readAttributes();
attributes.put(FILE_SIZE_ATTRIBUTE, Long.toString(attrs.size()));
attributes.put(FILE_LAST_MODIFY_TIME_ATTRIBUTE, formatter.format(new Date(attrs.lastModifiedTime().toMillis())));
attributes.put(FILE_CREATION_TIME_ATTRIBUTE, formatter.format(new Date(attrs.creationTime().toMillis())));
attributes.put(FILE_LAST_ACCESS_TIME_ATTRIBUTE, formatter.format(new Date(attrs.lastAccessTime().toMillis())));
} catch (Exception ignore) {
} // allow other attributes if these fail
}
if (store.supportsFileAttributeView("owner")) {
try {
FileOwnerAttributeView view = Files.getFileAttributeView(filePath, FileOwnerAttributeView.class);
attributes.put(FILE_OWNER_ATTRIBUTE, view.getOwner().getName());
} catch (Exception ignore) {
} // allow other attributes if these fail
}
if (store.supportsFileAttributeView("posix")) {
try {
PosixFileAttributeView view = Files.getFileAttributeView(filePath, PosixFileAttributeView.class);
attributes.put(FILE_PERMISSIONS_ATTRIBUTE, PosixFilePermissions.toString(view.readAttributes().permissions()));
attributes.put(FILE_GROUP_ATTRIBUTE, view.readAttributes().group().getName());
} catch (Exception ignore) {
} // allow other attributes if these fail
}
} catch (IOException ioe) {
// well then this FlowFile gets none of these attributes
getLogger().warn("Error collecting attributes for file {}, message is {}",
new Object[]{absPathString, ioe.getMessage()});
}
return attributes;
}
@Override
protected String getPath(final ProcessContext context) {
return context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue();
}
@Override
protected List<FileInfo> performListing(final ProcessContext context, final Long minTimestamp) throws IOException {
final File path = new File(getPath(context));
final Boolean recurse = context.getProperty(RECURSE).asBoolean();
return scanDirectory(path, fileFilterRef.get(), recurse, minTimestamp);
}
@Override
protected boolean isListingResetNecessary(final PropertyDescriptor property) {
return DIRECTORY.equals(property)
|| RECURSE.equals(property)
|| FILE_FILTER.equals(property)
|| PATH_FILTER.equals(property)
|| MIN_AGE.equals(property)
|| MAX_AGE.equals(property)
|| MIN_SIZE.equals(property)
|| MAX_SIZE.equals(property)
|| IGNORE_HIDDEN_FILES.equals(property);
}
private List<FileInfo> scanDirectory(final File path, final FileFilter filter, final Boolean recurse,
final Long minTimestamp) throws IOException {
final List<FileInfo> listing = new ArrayList<>();
File[] files = path.listFiles();
if (files != null) {
for (File file : files) {
if (file.isDirectory()) {
if (recurse) {
listing.addAll(scanDirectory(file, filter, true, minTimestamp));
}
} else {
if ((minTimestamp == null || file.lastModified() >= minTimestamp) && filter.accept(file)) {
listing.add(new FileInfo.Builder()
.directory(file.isDirectory())
.filename(file.getName())
.fullPathFileName(file.getAbsolutePath())
.lastModifiedTime(file.lastModified())
.build());
}
}
}
}
return listing;
}
private FileFilter createFileFilter(final ProcessContext context) {
final long minSize = context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue();
final Double maxSize = context.getProperty(MAX_SIZE).asDataSize(DataUnit.B);
final long minAge = context.getProperty(MIN_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
final Long maxAge = context.getProperty(MAX_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
final boolean ignoreHidden = context.getProperty(IGNORE_HIDDEN_FILES).asBoolean();
final Pattern filePattern = Pattern.compile(context.getProperty(FILE_FILTER).getValue());
final String indir = context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue();
final boolean recurseDirs = context.getProperty(RECURSE).asBoolean();
final String pathPatternStr = context.getProperty(PATH_FILTER).getValue();
final Pattern pathPattern = (!recurseDirs || pathPatternStr == null) ? null : Pattern.compile(pathPatternStr);
return new FileFilter() {
@Override
public boolean accept(final File file) {
if (minSize > file.length()) {
return false;
}
if (maxSize != null && maxSize < file.length()) {
return false;
}
final long fileAge = System.currentTimeMillis() - file.lastModified();
if (minAge > fileAge) {
return false;
}
if (maxAge != null && maxAge < fileAge) {
return false;
}
if (ignoreHidden && file.isHidden()) {
return false;
}
if (pathPattern != null) {
Path reldir = Paths.get(indir).relativize(file.toPath()).getParent();
if (reldir != null && !reldir.toString().isEmpty()) {
if (!pathPattern.matcher(reldir.toString()).matches()) {
return false;
}
}
}
//Verify that we have at least read permissions on the file we're considering grabbing
if (!Files.isReadable(file.toPath())) {
return false;
}
return filePattern.matcher(file.getName()).matches();
}
};
}
}

View File

@ -23,6 +23,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.commons.io.IOUtils;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.ProcessContext;
@ -92,7 +93,13 @@ public abstract class ListFileTransfer extends AbstractListProcessor<FileInfo> {
@Override
protected List<FileInfo> performListing(final ProcessContext context, final Long minTimestamp) throws IOException {
final FileTransfer transfer = getFileTransfer(context);
final List<FileInfo> listing = transfer.getListing();
final List<FileInfo> listing;
try {
listing = transfer.getListing();
} finally {
IOUtils.closeQuietly(transfer);
}
if (minTimestamp == null) {
return listing;
}

View File

@ -238,8 +238,9 @@ public class ListenSyslog extends AbstractSyslogProcessor {
initialEvent = syslogEvents.poll();
}
// if nothing in either queue then just return
// if nothing in either queue then yield and return
if (initialEvent == null) {
context.yield();
return;
}

View File

@ -49,6 +49,7 @@ import javax.net.ssl.SSLSession;
import javax.security.cert.X509Certificate;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.io.IOUtils;
import org.apache.http.Header;
import org.apache.http.HttpException;
import org.apache.http.HttpHost;
@ -637,6 +638,7 @@ public class PostHTTP extends AbstractProcessor {
+ "configured to deliver FlowFiles; rolling back session", new Object[]{url});
session.rollback();
context.yield();
IOUtils.closeQuietly(client);
return;
}
} else {

Some files were not shown because too many files have changed in this diff Show More