From 48a4c2a32061ada03b7243318c2bb255feb9317b Mon Sep 17 00:00:00 2001
From: Aldrin Piri
Date: Wed, 8 Jul 2015 14:22:17 -0400
Subject: [PATCH 01/77] Adding Linux configuration information as well as
linking to the administration guide within the development quickstart.
---
nifi-site/src/pages/markdown/quickstart.md | 53 +++++++++++++++++++++-
1 file changed, 52 insertions(+), 1 deletion(-)
diff --git a/nifi-site/src/pages/markdown/quickstart.md b/nifi-site/src/pages/markdown/quickstart.md
index 379cda99c3..0851a4dac5 100644
--- a/nifi-site/src/pages/markdown/quickstart.md
+++ b/nifi-site/src/pages/markdown/quickstart.md
@@ -29,6 +29,57 @@ Then checkout the 'develop' branch
git checkout develop
```
+
+### Linux Operating System Configuration
+
+_NOTE_: If you are building on Linux, consider these best practices. Typical Linux defaults are not necessarily well tuned for the needs of an IO intensive application like NiFi.
+For all of these areas, your distribution's requirements may vary. Use these sections as advice, but consult your distribution-specific documentation for how best to achieve these recommendations.
+
+
+#### Maximum File Handles
+
+NiFi will at any one time potentially have a very large number of file handles open. Increase the limits by
+editing '/etc/security/limits.conf' to add something like
+
+ * hard nofile 50000
+ * soft nofile 50000
+
+#### Maximum Forked Processes
+
+NiFi may be configured to generate a significant number of threads. To increase the allowable number edit '/etc/security/limits.conf'
+ * hard nproc 10000
+ * soft nproc 10000
+
+And your distribution may require an edit to /etc/security/limits.d/90-nproc.conf by adding
+ * soft nproc 10000
+
+#### Increase the number of TCP socket ports available
+This is particularly important if your flow will be setting up and tearing down a large number of sockets in small period of time.
+
+ sudo sysctl -w net.ipv4.ip_local_port_range="10000 65000"
+
+#### Set how long sockets stay in a TIMED_WAIT state when closed
+You don't want your sockets to sit and linger too long given that you want to be able to quickly setup and teardown new sockets. It is a good idea to read more about
+it but to adjust do something like
+
+ sudo sysctl -w net.ipv4.netfilter.ip_conntrack_tcp_timeout_time_wait="1"
+
+
+#### Tell Linux you never want NiFi to swap
+Swapping is fantastic for some applications. It isn't good for something like
+NiFi that always wants to be running. To tell Linux you'd like swapping off you
+can edit '/etc/sysctl.conf' to add the following line
+
+ vm.swappiness = 0
+
+#### Disable partition atime
+For the partitions handling the various NiFi repos turn off things like 'atime'.
+Doing so can cause a surprising bump in throughput. Edit the '/etc/fstab' file
+and for the partition(s) of interest add the 'noatime' option.
+
+#### Additional guidance
+Additional information on system administration and settings can be located in our [Administrator's Guide][adminguide].
+
### Build steps
1. You need a recent Java 7 (or newer) JDK.
@@ -86,7 +137,7 @@ is ready for use:
2014-12-09 00:42:03,540 INFO [main] org.apache.nifi.web.server.JettyServer NiFi has started. The UI is available at the following URLs:
-
+[adminguide]: https://nifi.incubator.apache.org/docs/nifi-docs/html/administration-guide.html
[maven]: http://maven.apache.org/
[jira]: https://issues.apache.org/jira/browse/NIFI
[git]: http://git-scm.com/
From 7f6f404baa1826462384de6c35be823189a36821 Mon Sep 17 00:00:00 2001
From: joewitt
Date: Sat, 11 Jul 2015 09:44:56 -0400
Subject: [PATCH 02/77] NIFI-685 This closes #67. Was merged/closed in JIRA
but not github
From 8bd20510ee348e497b6039e083b32e37a4d3a20e Mon Sep 17 00:00:00 2001
From: Mark Payne
Date: Mon, 13 Jul 2015 14:17:42 -0400
Subject: [PATCH 03/77] NIFI-762: Allow user to set keystore and truststore
properties instead of setting sslcontext
---
.../org/apache/nifi/events/EventReporter.java | 4 +-
.../nifi/remote/client/KeystoreType.java | 24 ++
.../nifi/remote/client/SiteToSiteClient.java | 257 +++++++++++++++++-
.../remote/client/SiteToSiteClientConfig.java | 32 ++-
.../nifi/controller/FlowController.java | 2 +
.../remote/StandardRemoteProcessGroup.java | 2 +
.../nifi/remote/StandardRootGroupPort.java | 2 +
.../TestPersistentProvenanceRepository.java | 2 +
8 files changed, 311 insertions(+), 14 deletions(-)
create mode 100644 nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/KeystoreType.java
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java b/nifi/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java
index 76702f174b..d645d60d6e 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/events/EventReporter.java
@@ -16,12 +16,14 @@
*/
package org.apache.nifi.events;
+import java.io.Serializable;
+
import org.apache.nifi.reporting.Severity;
/**
* Implementations MUST be thread-safe
*/
-public interface EventReporter {
+public interface EventReporter extends Serializable {
void reportEvent(Severity severity, String category, String message);
}
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/KeystoreType.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/KeystoreType.java
new file mode 100644
index 0000000000..63c3d63c20
--- /dev/null
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/KeystoreType.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.remote.client;
+
+import java.io.Serializable;
+
+public enum KeystoreType implements Serializable {
+ PKCS12,
+ JKS;
+}
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
index 5c4ce55d20..78237b997c 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClient.java
@@ -18,11 +18,17 @@ package org.apache.nifi.remote.client;
import java.io.Closeable;
import java.io.File;
+import java.io.FileInputStream;
import java.io.IOException;
+import java.io.InputStream;
import java.io.Serializable;
+import java.security.KeyStore;
+import java.security.SecureRandom;
import java.util.concurrent.TimeUnit;
+import javax.net.ssl.KeyManagerFactory;
import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManagerFactory;
import org.apache.nifi.events.EventReporter;
import org.apache.nifi.remote.Transaction;
@@ -143,6 +149,12 @@ public interface SiteToSiteClient extends Closeable {
private long penalizationNanos = TimeUnit.SECONDS.toNanos(3);
private long idleExpirationNanos = TimeUnit.SECONDS.toNanos(30L);
private SSLContext sslContext;
+ private String keystoreFilename;
+ private String keystorePass;
+ private KeystoreType keystoreType;
+ private String truststoreFilename;
+ private String truststorePass;
+ private KeystoreType truststoreType;
private EventReporter eventReporter;
private File peerPersistenceFile;
private boolean useCompression;
@@ -164,6 +176,12 @@ public interface SiteToSiteClient extends Closeable {
this.penalizationNanos = config.getPenalizationPeriod(TimeUnit.NANOSECONDS);
this.idleExpirationNanos = config.getIdleConnectionExpiration(TimeUnit.NANOSECONDS);
this.sslContext = config.getSslContext();
+ this.keystoreFilename = config.getKeystoreFilename();
+ this.keystorePass = config.getKeystorePassword();
+ this.keystoreType = config.getKeystoreType();
+ this.truststoreFilename = config.getTruststoreFilename();
+ this.truststorePass = config.getTruststorePassword();
+ this.truststoreType = config.getTruststoreType();
this.eventReporter = config.getEventReporter();
this.peerPersistenceFile = config.getPeerPersistenceFile();
this.useCompression = config.isUseCompression();
@@ -240,7 +258,12 @@ public interface SiteToSiteClient extends Closeable {
* secure. The remote instance of NiFi always determines whether or not
* Site-to-Site communications are secure (i.e., the client will always
* use secure or non-secure communications, depending on what the server
- * dictates).
+ * dictates). Note: The SSLContext provided by this method will be
+ * ignored if using a Serializable Configuration (see {@link #buildSerializableConfig()}).
+ * If a Serializable Configuration is required and communications are to be
+ * secure, the {@link #keystoreFilename(String)}, {@link #keystorePass(String)},
+ * {@link #keystoreType}, {@link #truststoreFilename}, {@link #truststorePass(String)},
+ * and {@link #truststoreType(KeystoreType)} methods must be used instead.
*
* @param sslContext the context
* @return the builder
@@ -250,6 +273,131 @@ public interface SiteToSiteClient extends Closeable {
return this;
}
+ /**
+ * @return the filename to use for the Keystore in order to communicate securely
+ * with the remote instance of NiFi
+ */
+ public String getKeystoreFilename() {
+ return keystoreFilename;
+ }
+
+ /**
+ * Sets the filename to use for the Keystore in order to communicate securely
+ * with the remote instance of NiFi
+ *
+ * @param keystoreFilename the filename to use for the Keystore in order to communicate securely
+ * with the remote instance of NiFi
+ * @return the builder
+ */
+ public Builder keystoreFilename(final String keystoreFilename) {
+ this.keystoreFilename = keystoreFilename;
+ return this;
+ }
+
+ /**
+ * @return the password to use for the Keystore in order to communicate securely
+ * with the remote instance of NiFi
+ */
+ public String getKeystorePass() {
+ return keystorePass;
+ }
+
+ /**
+ * Sets the password to use for the Keystore in order to communicate securely
+ * with the remote instance of NiFi
+ *
+ * @param keystorePass the password to use for the Keystore in order to communicate securely
+ * with the remote instance of NiFi
+ * @return the builder
+ */
+ public Builder keystorePass(final String keystorePass) {
+ this.keystorePass = keystorePass;
+ return this;
+ }
+
+ /**
+ * @return the type of Keystore to use in order to communicate securely
+ * with the remote instance of NiFi
+ */
+ public KeystoreType getKeystoreType() {
+ return keystoreType;
+ }
+
+ /**
+ * Sets the type of the Keystore to use in order to communicate securely
+ * with the remote instance of NiFi
+ *
+ * @param keystoreType the type of the Keystore to use in order to communicate securely
+ * with the remote instance of NiFi
+ * @return the builder
+ */
+ public Builder keystoreType(final KeystoreType keystoreType) {
+ this.keystoreType = keystoreType;
+ return this;
+ }
+
+ /**
+ * @return the filename to use for the Truststore in order to communicate securely
+ * with the remote instance of NiFi
+ */
+ public String getTruststoreFilename() {
+ return truststoreFilename;
+ }
+
+ /**
+ * Sets the filename to use for the Truststore in order to communicate securely
+ * with the remote instance of NiFi
+ *
+ * @param truststoreFilename the filename to use for the Truststore in order to communicate securely
+ * with the remote instance of NiFi
+ * @return the builder
+ */
+ public Builder truststoreFilename(final String truststoreFilename) {
+ this.truststoreFilename = truststoreFilename;
+ return this;
+ }
+
+ /**
+ * @return the password to use for the Truststore in order to communicate securely
+ * with the remote instance of NiFi
+ */
+ public String getTruststorePass() {
+ return truststorePass;
+ }
+
+ /**
+ * Sets the password to use for the Truststore in order to communicate securely
+ * with the remote instance of NiFi
+ *
+ * @param truststorePass the filename to use for the Truststore in order to communicate securely
+ * with the remote instance of NiFi
+ */
+ public Builder truststorePass(final String truststorePass) {
+ this.truststorePass = truststorePass;
+ return this;
+ }
+
+ /**
+ * @return the type of the Truststore to use in order to communicate securely
+ * with the remote instance of NiFi
+ */
+ public KeystoreType getTruststoreType() {
+ return truststoreType;
+ }
+
+ /**
+ * Sets the password type of the Truststore to use in order to communicate securely
+ * with the remote instance of NiFi
+ *
+ * @param truststoreType the type of the Truststore to use in order to communicate securely
+ * with the remote instance of NiFi
+ * @return the builder
+ */
+ public Builder truststoreType(final KeystoreType truststoreType) {
+ this.truststoreType = truststoreType;
+ return this;
+ }
+
/**
* Provides an EventReporter that can be used by the client in order to
* report any events that could be of interest when communicating with
@@ -365,8 +513,8 @@ public interface SiteToSiteClient extends Closeable {
* but does not create a SiteToSiteClient
*/
public SiteToSiteClientConfig buildConfig() {
- final SiteToSiteClientConfig config = new SiteToSiteClientConfig() {
- private static final long serialVersionUID = 1323119754841633818L;
+ return new SiteToSiteClientConfig() {
+ private static final long serialVersionUID = 1L;
@Override
public boolean isUseCompression() {
@@ -420,29 +568,57 @@ public interface SiteToSiteClient extends Closeable {
@Override
public long getPreferredBatchDuration(final TimeUnit timeUnit) {
- return timeUnit.convert(Builder.this.batchNanos, TimeUnit.NANOSECONDS);
+ return timeUnit.convert(batchNanos, TimeUnit.NANOSECONDS);
}
@Override
public long getPreferredBatchSize() {
- return Builder.this.batchSize;
+ return batchSize;
}
@Override
public int getPreferredBatchCount() {
- return Builder.this.batchCount;
+ return batchCount;
+ }
+
+ @Override
+ public String getKeystoreFilename() {
+ return keystoreFilename;
+ }
+
+ @Override
+ public String getKeystorePassword() {
+ return keystorePass;
+ }
+
+ @Override
+ public KeystoreType getKeystoreType() {
+ return keystoreType;
+ }
+
+ @Override
+ public String getTruststoreFilename() {
+ return truststoreFilename;
+ }
+
+ @Override
+ public String getTruststorePassword() {
+ return truststorePass;
+ }
+
+ @Override
+ public KeystoreType getTruststoreType() {
+ return truststoreType;
}
};
-
- return config;
}
/**
* @return a new SiteToSiteClient that can be used to send and receive
- * data with remote instances of NiFi
+ * data with remote instances of NiFi
*
* @throws IllegalStateException if either the url is not set or neither
- * the port name nor port identifier is set.
+ * the port name nor port identifier is set.
*/
public SiteToSiteClient build() {
if (url == null) {
@@ -450,7 +626,7 @@ public interface SiteToSiteClient extends Closeable {
}
if (portName == null && portIdentifier == null) {
- throw new IllegalStateException("Must specify either Port Name or Port Identifier to builder Site-to-Site client");
+ throw new IllegalStateException("Must specify either Port Name or Port Identifier to build Site-to-Site client");
}
return new SocketClient(buildConfig());
@@ -493,7 +669,58 @@ public interface SiteToSiteClient extends Closeable {
* @return the SSL Context that is configured for this builder
*/
public SSLContext getSslContext() {
- return sslContext;
+ if (sslContext != null) {
+ return sslContext;
+ }
+
+ final KeyManagerFactory keyManagerFactory;
+ if (keystoreFilename != null && keystorePass != null && keystoreType != null) {
+ try {
+ // prepare the keystore
+ final KeyStore keyStore = KeyStore.getInstance(getKeystoreType().name());
+ try (final InputStream keyStoreStream = new FileInputStream(new File(getKeystoreFilename()))) {
+ keyStore.load(keyStoreStream, getKeystorePass().toCharArray());
+ }
+ keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+ keyManagerFactory.init(keyStore, getKeystorePass().toCharArray());
+ } catch (final Exception e) {
+ throw new RuntimeException("Failed to load Keystore", e);
+ }
+ } else {
+ keyManagerFactory = null;
+ }
+
+ final TrustManagerFactory trustManagerFactory;
+ if (truststoreFilename != null && truststorePass != null && truststoreType != null) {
+ try {
+ // prepare the truststore
+ final KeyStore trustStore = KeyStore.getInstance(getTruststoreType().name());
+ try (final InputStream trustStoreStream = new FileInputStream(new File(getTruststoreFilename()))) {
+ trustStore.load(trustStoreStream, getTruststorePass().toCharArray());
+ }
+ trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+ trustManagerFactory.init(trustStore);
+ } catch (final Exception e) {
+ throw new RuntimeException("Failed to load Truststore", e);
+ }
+ } else {
+ trustManagerFactory = null;
+ }
+
+ if (keyManagerFactory != null || trustManagerFactory != null) {
+ try {
+ // initialize the ssl context
+ final SSLContext sslContext = SSLContext.getInstance("TLS");
+ sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
+ sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
+
+ return sslContext;
+ } catch (final Exception e) {
+ throw new RuntimeException("Created keystore and truststore but failed to initialize SSLContext");
+ }
+ } else {
+ return null;
+ }
}
/**
@@ -535,4 +762,10 @@ public interface SiteToSiteClient extends Closeable {
return portIdentifier;
}
}
+
+
+ public abstract class SerializableSiteToSiteClientConfig implements SiteToSiteClientConfig, Serializable {
+ private static final long serialVersionUID = 1L;
+
+ }
}
diff --git a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
index c4b0d220e6..50a0d3c31e 100644
--- a/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
+++ b/nifi/nifi-commons/nifi-site-to-site-client/src/main/java/org/apache/nifi/remote/client/SiteToSiteClientConfig.java
@@ -57,9 +57,39 @@ public interface SiteToSiteClientConfig extends Serializable {
*/
SSLContext getSslContext();
+ /**
+ * @return the filename to use for the keystore, or null if none is configured
+ */
+ String getKeystoreFilename();
+
+ /**
+ * @return the password to use for the keystore, or null if none is configured
+ */
+ String getKeystorePassword();
+
+ /**
+ * @return the Type of the keystore, or null if none is configured
+ */
+ KeystoreType getKeystoreType();
+
+ /**
+ * @return the filename to use for the truststore, or null if none is configured
+ */
+ String getTruststoreFilename();
+
+ /**
+ * @return the password to use for the truststore, or null if none is configured
+ */
+ String getTruststorePassword();
+
+ /**
+ * @return the type of the truststore, or null if none is configured
+ */
+ KeystoreType getTruststoreType();
+
/**
* @return the file that is to be used for persisting the nodes of a remote
- * cluster, if any
+ * cluster, if any
*/
File getPeerPersistenceFile();
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 255a35a988..3d78b3a935 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -528,6 +528,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
private static EventReporter createEventReporter(final BulletinRepository bulletinRepository) {
return new EventReporter() {
+ private static final long serialVersionUID = 1L;
+
@Override
public void reportEvent(final Severity severity, final String category, final String message) {
final Bulletin bulletin = BulletinFactory.createBulletin(category, severity.name(), message);
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index 61516d0d3a..bd934461e9 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -160,6 +160,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
final BulletinRepository bulletinRepository = flowController.getBulletinRepository();
eventReporter = new EventReporter() {
+ private static final long serialVersionUID = 1L;
+
@Override
public void reportEvent(final Severity severity, final String category, final String message) {
final String groupId = StandardRemoteProcessGroup.this.getProcessGroup().getIdentifier();
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java
index 9eadec02d1..66fd303141 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-site-to-site/src/main/java/org/apache/nifi/remote/StandardRootGroupPort.java
@@ -104,6 +104,8 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
this.scheduler = scheduler;
setYieldPeriod("100 millis");
eventReporter = new EventReporter() {
+ private static final long serialVersionUID = 1L;
+
@Override
public void reportEvent(final Severity severity, final String category, final String message) {
final String groupId = StandardRootGroupPort.this.getProcessGroup().getIdentifier();
diff --git a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
index 16f0312839..3737588d38 100644
--- a/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-provenance-repository-bundle/nifi-persistent-provenance-repository/src/test/java/org/apache/nifi/provenance/TestPersistentProvenanceRepository.java
@@ -135,6 +135,8 @@ public class TestPersistentProvenanceRepository {
private EventReporter getEventReporter() {
return new EventReporter() {
+ private static final long serialVersionUID = 1L;
+
@Override
public void reportEvent(Severity severity, String category, String message) {
System.out.println(severity + " : " + category + " : " + message);
From bb64e70e6fbda12a9e0388b5c2240d96d34ac6bf Mon Sep 17 00:00:00 2001
From: Alan Jackoway
Date: Tue, 7 Jul 2015 17:28:26 -0400
Subject: [PATCH 04/77] NIFI-751 Add Processor To Convert Avro Formats
Implemented a new NiFi processor that allows avro records to be converted from one Avro schema
to another. This supports..
* Flattening records using . notation like "parent.id"
* Simple type conversions to String or base primitive types.
* Specifying field renames using dynamic properties.
Signed-off-by: joewitt
---
.../processors/kite/AvroRecordConverter.java | 320 +++++++++++++++++
.../processors/kite/ConvertAvroSchema.java | 339 ++++++++++++++++++
.../org.apache.nifi.processor.Processor | 1 +
.../additionalDetails.html | 142 ++++++++
.../kite/TestAvroRecordConverter.java | 201 +++++++++++
.../kite/TestConvertAvroSchema.java | 216 +++++++++++
6 files changed, 1219 insertions(+)
create mode 100644 nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AvroRecordConverter.java
create mode 100644 nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
create mode 100644 nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/resources/docs/org.apache.nifi.processors.kite.ConvertAvroSchema/additionalDetails.html
create mode 100644 nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestAvroRecordConverter.java
create mode 100644 nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestConvertAvroSchema.java
diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AvroRecordConverter.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AvroRecordConverter.java
new file mode 100644
index 0000000000..68e6c98342
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/AvroRecordConverter.java
@@ -0,0 +1,320 @@
+/*
+ * 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.kite;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Scanner;
+
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.avro.generic.IndexedRecord;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * Responsible for converting records of one Avro type to another. Supports
+ * syntax like "record.field" to unpack fields and will try to do simple type
+ * conversion.
+ */
+public class AvroRecordConverter {
+ private final Schema inputSchema;
+ private final Schema outputSchema;
+ // Store this from output field to input field so we can look up by output.
+ private final Map fieldMapping;
+
+ /**
+ * @param inputSchema
+ * Schema of input record objects
+ * @param outputSchema
+ * Schema of output record objects
+ * @param fieldMapping
+ * Map from field name in input record to field name in output
+ * record.
+ */
+ public AvroRecordConverter(Schema inputSchema, Schema outputSchema,
+ Map fieldMapping) {
+ this.inputSchema = inputSchema;
+ this.outputSchema = outputSchema;
+ // Need to reverse this map.
+ this.fieldMapping = Maps
+ .newHashMapWithExpectedSize(fieldMapping.size());
+ for (Map.Entry entry : fieldMapping.entrySet()) {
+ this.fieldMapping.put(entry.getValue(), entry.getKey());
+ }
+ }
+
+ /**
+ * @return Any fields in the output schema that are not mapped or are mapped
+ * by a non-existent input field.
+ */
+ public Collection getUnmappedFields() {
+ List result = Lists.newArrayList();
+ for (Field f : outputSchema.getFields()) {
+ String fieldName = f.name();
+ if (fieldMapping.containsKey(fieldName)) {
+ fieldName = fieldMapping.get(fieldName);
+ }
+
+ Schema currentSchema = inputSchema;
+ while (fieldName.contains(".")) {
+ // Recurse down the schema to find the right field.
+ int dotIndex = fieldName.indexOf('.');
+ String entityName = fieldName.substring(0, dotIndex);
+ // Get the schema. In case we had an optional record, choose
+ // just the record.
+ currentSchema = getNonNullSchema(currentSchema);
+ if (currentSchema.getField(entityName) == null) {
+ // Tried to step into a schema that doesn't exist. Break out
+ // of the loop
+ break;
+ }
+ currentSchema = currentSchema.getField(entityName).schema();
+ fieldName = fieldName.substring(dotIndex + 1);
+ }
+ if (currentSchema == null
+ || getNonNullSchema(currentSchema).getField(fieldName) == null) {
+ result.add(f.name());
+ }
+ }
+ return result;
+ }
+
+ /**
+ * Converts one record to another given a input and output schema plus
+ * explicit mappings for certain target fields.
+ *
+ * @param input
+ * Input record to convert conforming to the inputSchema this
+ * converter was created with.
+ * @return Record converted to the outputSchema this converter was created
+ * with.
+ * @throws AvroConversionException
+ * When schemas do not match or illegal conversions are
+ * attempted, such as when numeric data fails to parse.
+ */
+ public Record convert(Record input) throws AvroConversionException {
+ Record result = new Record(outputSchema);
+ for (Field outputField : outputSchema.getFields()) {
+ // Default to matching by name
+ String inputFieldName = outputField.name();
+ if (fieldMapping.containsKey(outputField.name())) {
+ inputFieldName = fieldMapping.get(outputField.name());
+ }
+
+ IndexedRecord currentRecord = input;
+ Schema currentSchema = getNonNullSchema(inputSchema);
+ while (inputFieldName.contains(".")) {
+ // Recurse down the schema to find the right field.
+ int dotIndex = inputFieldName.indexOf('.');
+ String entityName = inputFieldName.substring(0, dotIndex);
+ // Get the record object
+ Object innerRecord = currentRecord.get(currentSchema.getField(
+ entityName).pos());
+ if (innerRecord == null) {
+ // Probably hit a null record here. Just break out of the
+ // loop so that null object will be passed to convertData
+ // below.
+ currentRecord = null;
+ break;
+ }
+ if (innerRecord != null
+ && !(innerRecord instanceof IndexedRecord)) {
+ throw new AvroConversionException(inputFieldName
+ + " stepped through a non-record");
+ }
+ currentRecord = (IndexedRecord) innerRecord;
+
+ // Get the schema. In case we had an optional record, choose
+ // just the record.
+ currentSchema = currentSchema.getField(entityName).schema();
+ currentSchema = getNonNullSchema(currentSchema);
+ inputFieldName = inputFieldName.substring(dotIndex + 1);
+ }
+
+ // Current should now be in the right place to read the record.
+ Field f = currentSchema.getField(inputFieldName);
+ if (currentRecord == null) {
+ // We may have stepped into a null union type and gotten a null
+ // result.
+ Schema s = null;
+ if (f != null) {
+ s = f.schema();
+ }
+ result.put(outputField.name(),
+ convertData(null, s, outputField.schema()));
+ } else {
+ result.put(
+ outputField.name(),
+ convertData(currentRecord.get(f.pos()), f.schema(),
+ outputField.schema()));
+ }
+ }
+ return result;
+ }
+
+ public Schema getInputSchema() {
+ return inputSchema;
+ }
+
+ public Schema getOutputSchema() {
+ return outputSchema;
+ }
+
+ /**
+ * Converts the data from one schema to another. If the types are the same,
+ * no change will be made, but simple conversions will be attempted for
+ * other types.
+ *
+ * @param content
+ * The data to convert, generally taken from a field in an input
+ * Record.
+ * @param inputSchema
+ * The schema of the content object
+ * @param outputSchema
+ * The schema to convert to.
+ * @return The content object, converted to the output schema.
+ * @throws AvroConversionException
+ * When conversion is impossible, either because the output type
+ * is not supported or because numeric data failed to parse.
+ */
+ private Object convertData(Object content, Schema inputSchema,
+ Schema outputSchema) throws AvroConversionException {
+ if (content == null) {
+ // No conversion can happen here.
+ if (supportsNull(outputSchema)) {
+ return null;
+ }
+ throw new AvroConversionException("Output schema " + outputSchema
+ + " does not support null");
+ }
+
+ Schema nonNillInput = getNonNullSchema(inputSchema);
+ Schema nonNillOutput = getNonNullSchema(outputSchema);
+ if (nonNillInput.getType().equals(nonNillOutput.getType())) {
+ return content;
+ } else {
+ if (nonNillOutput.getType() == Schema.Type.STRING) {
+ return content.toString();
+ }
+
+ // For the non-string cases of these, we will try to convert through
+ // string using Scanner to validate types. This means we could
+ // return questionable results when a String starts with a number
+ // but then contains other content
+ Scanner scanner = new Scanner(content.toString());
+ switch (nonNillOutput.getType()) {
+ case LONG:
+ if (scanner.hasNextLong()) {
+ return scanner.nextLong();
+ } else {
+ throw new AvroConversionException("Cannot convert "
+ + content + " to long");
+ }
+ case INT:
+ if (scanner.hasNextInt()) {
+ return scanner.nextInt();
+ } else {
+ throw new AvroConversionException("Cannot convert "
+ + content + " to int");
+ }
+ case DOUBLE:
+ if (scanner.hasNextDouble()) {
+ return scanner.nextDouble();
+ } else {
+ throw new AvroConversionException("Cannot convert "
+ + content + " to double");
+ }
+ case FLOAT:
+ if (scanner.hasNextFloat()) {
+ return scanner.nextFloat();
+ } else {
+ throw new AvroConversionException("Cannot convert "
+ + content + " to float");
+ }
+ default:
+ throw new AvroConversionException("Cannot convert to type "
+ + nonNillOutput.getType());
+ }
+ }
+ }
+
+ /**
+ * If s is a union schema of some type with null, returns that type.
+ * Otherwise just return schema itself.
+ *
+ * Does not handle unions of schemas with anything except null and one type.
+ *
+ * @param s
+ * Schema to remove nillable from.
+ * @return The Schema of the non-null part of a the union, if the input was
+ * a union type. Otherwise returns the input schema.
+ */
+ protected static Schema getNonNullSchema(Schema s) {
+ // Handle the case where s is a union type. Assert that this must be a
+ // union that only includes one non-null type.
+ if (s.getType() == Schema.Type.UNION) {
+ List types = s.getTypes();
+ boolean foundOne = false;
+ Schema result = s;
+ for (Schema type : types) {
+ if (!type.getType().equals(Schema.Type.NULL)) {
+ Preconditions.checkArgument(foundOne == false,
+ "Cannot handle union of two non-null types");
+ foundOne = true;
+ result = type;
+ }
+ }
+ return result;
+ } else {
+ return s;
+ }
+ }
+
+ protected static boolean supportsNull(Schema s) {
+ if (s.getType() == Schema.Type.NULL) {
+ return true;
+ } else if (s.getType() == Schema.Type.UNION) {
+ for (Schema type : s.getTypes()) {
+ if (type.getType() == Schema.Type.NULL) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Exception thrown when Avro conversion fails.
+ */
+ public class AvroConversionException extends Exception {
+ public AvroConversionException(String string, IOException e) {
+ super(string, e);
+ }
+
+ public AvroConversionException(String string) {
+ super(string);
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
new file mode 100644
index 0000000000..0d9f6586b5
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
@@ -0,0 +1,339 @@
+/*
+ * 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.kite;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.CodecFactory;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.file.DataFileWriter;
+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.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.flowfile.FlowFile;
+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.StreamCallback;
+import org.apache.nifi.processors.kite.AvroRecordConverter.AvroConversionException;
+import org.apache.nifi.util.LongHolder;
+import org.kitesdk.data.DatasetException;
+import org.kitesdk.data.DatasetIOException;
+import org.kitesdk.data.SchemaNotFoundException;
+import org.kitesdk.data.spi.DefaultConfiguration;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+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")
+@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")
+public class ConvertAvroSchema extends AbstractKiteProcessor {
+
+ private static final Relationship SUCCESS = new Relationship.Builder()
+ .name("success")
+ .description("Avro content that converted successfully").build();
+
+ private static final Relationship FAILURE = new Relationship.Builder()
+ .name("failure").description("Avro content that failed to convert")
+ .build();
+
+ /**
+ * Makes sure the output schema is a valid output schema and that all its
+ * fields can be mapped either automatically or are explicitly mapped.
+ */
+ protected static final Validator MAPPED_SCHEMA_VALIDATOR = new Validator() {
+ @Override
+ public ValidationResult validate(String subject, String uri,
+ ValidationContext context) {
+ Configuration conf = getConfiguration(context.getProperty(
+ CONF_XML_FILES).getValue());
+ String inputUri = context.getProperty(INPUT_SCHEMA).getValue();
+ String error = null;
+
+ final boolean elPresent = context
+ .isExpressionLanguageSupported(subject)
+ && context.isExpressionLanguagePresent(uri);
+ if (!elPresent) {
+ try {
+ Schema outputSchema = getSchema(uri, conf);
+ Schema inputSchema = getSchema(inputUri, conf);
+ // Get the explicitly mapped fields. This is identical to
+ // logic in onTrigger, but ValidationContext and
+ // ProcessContext share no ancestor, so we cannot generalize
+ // the code.
+ Map fieldMapping = new HashMap<>();
+ for (final Map.Entry entry : context
+ .getProperties().entrySet()) {
+ if (entry.getKey().isDynamic()) {
+ fieldMapping.put(entry.getKey().getName(),
+ entry.getValue());
+ }
+ }
+ AvroRecordConverter converter = new AvroRecordConverter(
+ inputSchema, outputSchema, fieldMapping);
+ Collection unmappedFields = converter
+ .getUnmappedFields();
+ if (unmappedFields.size() > 0) {
+ error = "The following fields are unmapped: "
+ + unmappedFields;
+ }
+
+ } catch (SchemaNotFoundException e) {
+ error = e.getMessage();
+ }
+ }
+ return new ValidationResult.Builder().subject(subject).input(uri)
+ .explanation(error).valid(error == null).build();
+ }
+ };
+
+ @VisibleForTesting
+ static final PropertyDescriptor INPUT_SCHEMA = new PropertyDescriptor.Builder()
+ .name("Input Schema").description("Avro Schema of Input Flowfiles")
+ .addValidator(SCHEMA_VALIDATOR).expressionLanguageSupported(true)
+ .required(true).build();
+
+ @VisibleForTesting
+ static final PropertyDescriptor OUTPUT_SCHEMA = new PropertyDescriptor.Builder()
+ .name("Output Schema")
+ .description("Avro Schema of Output Flowfiles")
+ .addValidator(MAPPED_SCHEMA_VALIDATOR).expressionLanguageSupported(true)
+ .required(true).build();
+
+ private static final List PROPERTIES = ImmutableList
+ . builder()
+ .addAll(AbstractKiteProcessor.getProperties()).add(INPUT_SCHEMA)
+ .add(OUTPUT_SCHEMA).build();
+
+ private static final Set RELATIONSHIPS = ImmutableSet
+ . builder().add(SUCCESS).add(FAILURE).build();
+
+ private static final Pattern AVRO_FIELDNAME_PATTERN = Pattern
+ .compile("[A-Za-z_][A-Za-z0-9_\\.]*");
+
+ /**
+ * Validates that the input and output fields (from dynamic properties) are
+ * all valid avro field names including "." to step into records.
+ */
+ protected static final Validator AVRO_FIELDNAME_VALIDATOR = new Validator() {
+ @Override
+ public ValidationResult validate(final String subject,
+ final String value, final ValidationContext context) {
+ if (context.isExpressionLanguageSupported(subject)
+ && context.isExpressionLanguagePresent(value)) {
+ return new ValidationResult.Builder().subject(subject)
+ .input(value)
+ .explanation("Expression Language Present").valid(true)
+ .build();
+ }
+
+ String reason = "";
+ if (!AVRO_FIELDNAME_PATTERN.matcher(subject).matches()) {
+ reason = subject + " is not a valid Avro fieldname";
+ }
+ if (!AVRO_FIELDNAME_PATTERN.matcher(value).matches()) {
+ reason = reason + value + " is not a valid Avro fieldname";
+ }
+
+ return new ValidationResult.Builder().subject(subject).input(value)
+ .explanation(reason).valid(reason.equals("")).build();
+ }
+ };
+
+ @Override
+ protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(
+ final String propertyDescriptorName) {
+ return new PropertyDescriptor.Builder()
+ .name(propertyDescriptorName)
+ .description(
+ "Field mapping between schemas. The property name is the field name for the input "
+ + "schema, and the property value is the field name for the output schema. For fields "
+ + "not listed, the processor tries to match names from the input to the output record.")
+ .dynamic(true).addValidator(AVRO_FIELDNAME_VALIDATOR).build();
+ }
+
+ @Override
+ protected List getSupportedPropertyDescriptors() {
+ return PROPERTIES;
+ }
+
+ @Override
+ public Set getRelationships() {
+ return RELATIONSHIPS;
+ }
+
+ @Override
+ public void onTrigger(ProcessContext context, final ProcessSession session)
+ throws ProcessException {
+ FlowFile incomingAvro = session.get();
+ if (incomingAvro == null) {
+ return;
+ }
+
+ String inputSchemaProperty = context.getProperty(INPUT_SCHEMA)
+ .evaluateAttributeExpressions(incomingAvro).getValue();
+ final Schema inputSchema;
+ try {
+ inputSchema = getSchema(inputSchemaProperty,
+ DefaultConfiguration.get());
+ } catch (SchemaNotFoundException e) {
+ getLogger().error("Cannot find schema: " + inputSchemaProperty);
+ session.transfer(incomingAvro, FAILURE);
+ return;
+ }
+ String outputSchemaProperty = context.getProperty(OUTPUT_SCHEMA)
+ .evaluateAttributeExpressions(incomingAvro).getValue();
+ final Schema outputSchema;
+ try {
+ outputSchema = getSchema(outputSchemaProperty,
+ DefaultConfiguration.get());
+ } catch (SchemaNotFoundException e) {
+ getLogger().error("Cannot find schema: " + outputSchemaProperty);
+ session.transfer(incomingAvro, FAILURE);
+ return;
+ }
+ final Map fieldMapping = new HashMap<>();
+ for (final Map.Entry entry : context
+ .getProperties().entrySet()) {
+ if (entry.getKey().isDynamic()) {
+ fieldMapping.put(entry.getKey().getName(), entry.getValue());
+ }
+ }
+ final AvroRecordConverter converter = new AvroRecordConverter(
+ inputSchema, outputSchema, fieldMapping);
+
+ final DataFileWriter writer = new DataFileWriter<>(
+ AvroUtil.newDatumWriter(outputSchema, Record.class));
+ writer.setCodec(CodecFactory.snappyCodec());
+
+ final DataFileWriter failureWriter = new DataFileWriter<>(
+ AvroUtil.newDatumWriter(outputSchema, Record.class));
+ failureWriter.setCodec(CodecFactory.snappyCodec());
+
+ try {
+ final LongHolder written = new LongHolder(0L);
+ final FailureTracker failures = new FailureTracker();
+
+ final List badRecords = Lists.newLinkedList();
+ FlowFile incomingAvroCopy = session.clone(incomingAvro);
+ FlowFile outgoingAvro = session.write(incomingAvro,
+ new StreamCallback() {
+ @Override
+ public void process(InputStream in, OutputStream out)
+ throws IOException {
+ try (DataFileStream stream = new DataFileStream(
+ in, new GenericDatumReader(
+ converter.getInputSchema()))) {
+ try (DataFileWriter w = writer.create(
+ outputSchema, out)) {
+ for (Record record : stream) {
+ try {
+ Record converted = converter
+ .convert(record);
+ w.append(converted);
+ written.incrementAndGet();
+ } catch (AvroConversionException e) {
+ failures.add(e);
+ getLogger().error(
+ "Error converting data: "
+ + e.getMessage());
+ badRecords.add(record);
+ }
+ }
+ }
+ }
+ }
+ });
+
+ FlowFile badOutput = session.write(incomingAvroCopy,
+ new StreamCallback() {
+ @Override
+ public void process(InputStream in, OutputStream out)
+ throws IOException {
+
+ try (DataFileWriter w = failureWriter
+ .create(inputSchema, out)) {
+ for (Record record : badRecords) {
+ w.append(record);
+ }
+ }
+
+ }
+ });
+
+ long errors = failures.count();
+
+ // update only if file transfer is successful
+ session.adjustCounter("Converted records", written.get(), false);
+ // update only if file transfer is successful
+ session.adjustCounter("Conversion errors", errors, false);
+
+ if (written.get() > 0L) {
+ session.transfer(outgoingAvro, SUCCESS);
+ } else {
+ session.remove(outgoingAvro);
+
+ if (errors == 0L) {
+ badOutput = session.putAttribute(badOutput, "errors",
+ "No incoming records");
+ session.transfer(badOutput, FAILURE);
+ }
+ }
+
+ if (errors > 0L) {
+ getLogger().warn(
+ "Failed to convert {}/{} records between Avro Schemas",
+ new Object[] { errors, errors + written.get() });
+ badOutput = session.putAttribute(badOutput, "errors",
+ failures.summary());
+ session.transfer(badOutput, FAILURE);
+ } else {
+ session.remove(badOutput);
+ }
+ } catch (ProcessException | DatasetIOException e) {
+ getLogger().error("Failed reading or writing", e);
+ session.transfer(incomingAvro, FAILURE);
+ } catch (DatasetException e) {
+ getLogger().error("Failed to read FlowFile", e);
+ session.transfer(incomingAvro, FAILURE);
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index 6de56122a6..ea99ff6548 100644
--- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -15,3 +15,4 @@
org.apache.nifi.processors.kite.StoreInKiteDataset
org.apache.nifi.processors.kite.ConvertCSVToAvro
org.apache.nifi.processors.kite.ConvertJSONToAvro
+org.apache.nifi.processors.kite.ConvertAvroSchema
diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/resources/docs/org.apache.nifi.processors.kite.ConvertAvroSchema/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/resources/docs/org.apache.nifi.processors.kite.ConvertAvroSchema/additionalDetails.html
new file mode 100644
index 0000000000..f5d8a1deb8
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/resources/docs/org.apache.nifi.processors.kite.ConvertAvroSchema/additionalDetails.html
@@ -0,0 +1,142 @@
+
+
+
+
+
+ ConvertAvroSchema
+
+
+
+
+
+
+
Description:
+
This processor is used to convert data between two Avro formats, such as those coming from the ConvertCSVToAvro or
+ ConvertJSONToAvro processors. The input and output content of the flow files should be Avro data files. The processor
+ includes support for the following basic type conversions:
+
+
Anything to String, using the data's default String representation
+
String types to numeric types int, long, double, and float
+
Conversion to and from optional Avro types
+
+ In addition, fields can be renamed or unpacked from a record type by using the dynamic properties.
+
+
Mapping Example:
+
+ Throughout this example, we will refer to input data with the following schema:
+
+ Where even though the revenue and id fields are mapped as string, they are logically long and double respectively.
+ By default, fields with matching names will be mapped automatically, so the following output schema could be converted
+ without using dynamic properties:
+
+ To rename companyName to name and to extract the parent's id field, both a schema and a dynamic properties must be provided.
+ For example, to convert to the following schema:
+
+
+
+
diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestAvroRecordConverter.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestAvroRecordConverter.java
new file mode 100644
index 0000000000..1a4748f1b6
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestAvroRecordConverter.java
@@ -0,0 +1,201 @@
+/*
+ * 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.kite;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.generic.GenericData.Record;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+public class TestAvroRecordConverter {
+ final static ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+ final static Map EMPTY_MAPPING = ImmutableMap.of();
+ final static String NESTED_RECORD_SCHEMA_STRING = "{\n"
+ + " \"type\": \"record\",\n"
+ + " \"name\": \"NestedInput\",\n"
+ + " \"namespace\": \"org.apache.example\",\n"
+ + " \"fields\": [\n" + " {\n"
+ + " \"name\": \"l1\",\n"
+ + " \"type\": \"long\"\n"
+ + " },\n"
+ + " {\n" + " \"name\": \"s1\",\n"
+ + " \"type\": \"string\"\n"
+ + " },\n"
+ + " {\n"
+ + " \"name\": \"parent\",\n"
+ + " \"type\": [\"null\", {\n"
+ + " \"type\": \"record\",\n"
+ + " \"name\": \"parent\",\n"
+ + " \"fields\": [\n"
+ + " { \"name\": \"id\", \"type\": \"long\" },\n"
+ + " { \"name\": \"name\", \"type\": \"string\" }\n"
+ + " ]"
+ + " } ]"
+ + " }"
+ + " ] }";
+ final static Schema NESTED_RECORD_SCHEMA = new Schema.Parser()
+ .parse(NESTED_RECORD_SCHEMA_STRING);
+ final static Schema NESTED_PARENT_SCHEMA = AvroRecordConverter
+ .getNonNullSchema(NESTED_RECORD_SCHEMA.getField("parent").schema());
+ final static Schema UNNESTED_OUTPUT_SCHEMA = SchemaBuilder.record("Output")
+ .namespace("org.apache.example").fields().requiredLong("l1")
+ .requiredLong("s1").optionalLong("parentId").endRecord();
+
+ /**
+ * Tests the case where we don't use a mapping file and just map records by
+ * name.
+ */
+ @Test
+ public void testDefaultConversion() throws Exception {
+ // We will convert s1 from string to long (or leave it null), ignore s2,
+ // convert s3 to from string to double, convert l1 from long to string,
+ // and leave l2 the same.
+ Schema input = SchemaBuilder.record("Input")
+ .namespace("com.cloudera.edh").fields()
+ .nullableString("s1", "").requiredString("s2")
+ .requiredString("s3").optionalLong("l1").requiredLong("l2")
+ .endRecord();
+ Schema output = SchemaBuilder.record("Output")
+ .namespace("com.cloudera.edh").fields().optionalLong("s1")
+ .optionalString("l1").requiredLong("l2").requiredDouble("s3")
+ .endRecord();
+
+ AvroRecordConverter converter = new AvroRecordConverter(input, output,
+ EMPTY_MAPPING);
+
+ Record inputRecord = new Record(input);
+ inputRecord.put("s1", null);
+ inputRecord.put("s2", "blah");
+ inputRecord.put("s3", "5.5");
+ inputRecord.put("l1", null);
+ inputRecord.put("l2", 5L);
+ Record outputRecord = converter.convert(inputRecord);
+ assertNull(outputRecord.get("s1"));
+ assertNull(outputRecord.get("l1"));
+ assertEquals(5L, outputRecord.get("l2"));
+ assertEquals(5.5, outputRecord.get("s3"));
+
+ inputRecord.put("s1", "500");
+ inputRecord.put("s2", "blah");
+ inputRecord.put("s3", "5.5e-5");
+ inputRecord.put("l1", 100L);
+ inputRecord.put("l2", 2L);
+ outputRecord = converter.convert(inputRecord);
+ assertEquals(500L, outputRecord.get("s1"));
+ assertEquals("100", outputRecord.get("l1"));
+ assertEquals(2L, outputRecord.get("l2"));
+ assertEquals(5.5e-5, outputRecord.get("s3"));
+ }
+
+ /**
+ * Tests the case where we want to default map one field and explicitly map
+ * another.
+ */
+ @Test
+ public void testExplicitMapping() throws Exception {
+ // We will convert s1 from string to long (or leave it null), ignore s2,
+ // convert l1 from long to string, and leave l2 the same.
+ Schema input = NESTED_RECORD_SCHEMA;
+ Schema parent = NESTED_PARENT_SCHEMA;
+ Schema output = UNNESTED_OUTPUT_SCHEMA;
+ Map mapping = ImmutableMap.of("parent.id", "parentId");
+
+ AvroRecordConverter converter = new AvroRecordConverter(input, output,
+ mapping);
+
+ Record inputRecord = new Record(input);
+ inputRecord.put("l1", 5L);
+ inputRecord.put("s1", "1000");
+ Record parentRecord = new Record(parent);
+ parentRecord.put("id", 200L);
+ parentRecord.put("name", "parent");
+ inputRecord.put("parent", parentRecord);
+ Record outputRecord = converter.convert(inputRecord);
+ assertEquals(5L, outputRecord.get("l1"));
+ assertEquals(1000L, outputRecord.get("s1"));
+ assertEquals(200L, outputRecord.get("parentId"));
+ }
+
+ /**
+ * Tests the case where we try to convert a string to a long incorrectly.
+ */
+ @Test(expected = org.apache.nifi.processors.kite.AvroRecordConverter.AvroConversionException.class)
+ public void testIllegalConversion() throws Exception {
+ // We will convert s1 from string to long (or leave it null), ignore s2,
+ // convert l1 from long to string, and leave l2 the same.
+ Schema input = SchemaBuilder.record("Input")
+ .namespace("com.cloudera.edh").fields()
+ .nullableString("s1", "").requiredString("s2")
+ .optionalLong("l1").requiredLong("l2").endRecord();
+ Schema output = SchemaBuilder.record("Output")
+ .namespace("com.cloudera.edh").fields().optionalLong("s1")
+ .optionalString("l1").requiredLong("l2").endRecord();
+
+ AvroRecordConverter converter = new AvroRecordConverter(input, output,
+ EMPTY_MAPPING);
+
+ Record inputRecord = new Record(input);
+ inputRecord.put("s1", "blah");
+ inputRecord.put("s2", "blah");
+ inputRecord.put("l1", null);
+ inputRecord.put("l2", 5L);
+ converter.convert(inputRecord);
+ }
+
+ @Test
+ public void testGetUnmappedFields() throws Exception {
+ Schema input = SchemaBuilder.record("Input")
+ .namespace("com.cloudera.edh").fields()
+ .nullableString("s1", "").requiredString("s2")
+ .optionalLong("l1").requiredLong("l2").endRecord();
+ Schema output = SchemaBuilder.record("Output")
+ .namespace("com.cloudera.edh").fields().optionalLong("field")
+ .endRecord();
+
+ // Test the case where the field isn't mapped at all.
+ AvroRecordConverter converter = new AvroRecordConverter(input, output,
+ EMPTY_MAPPING);
+ assertEquals(ImmutableList.of("field"), converter.getUnmappedFields());
+
+ // Test the case where we tried to map from a non-existent field.
+ converter = new AvroRecordConverter(input, output, ImmutableMap.of(
+ "nonExistentField", "field"));
+ assertEquals(ImmutableList.of("field"), converter.getUnmappedFields());
+
+ // Test the case where we tried to map from a non-existent record.
+ converter = new AvroRecordConverter(input, output, ImmutableMap.of(
+ "parent.nonExistentField", "field"));
+ assertEquals(ImmutableList.of("field"), converter.getUnmappedFields());
+
+ // Test a valid case
+ converter = new AvroRecordConverter(input, output, ImmutableMap.of(
+ "l2", "field"));
+ assertEquals(Collections.EMPTY_LIST, converter.getUnmappedFields());
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestConvertAvroSchema.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestConvertAvroSchema.java
new file mode 100644
index 0000000000..33f3a821f9
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/test/java/org/apache/nifi/processors/kite/TestConvertAvroSchema.java
@@ -0,0 +1,216 @@
+/*
+ * 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.kite;
+
+import static org.apache.nifi.processors.kite.TestUtil.streamFor;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+import org.apache.avro.file.DataFileStream;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestConvertAvroSchema {
+
+ public static final Schema INPUT_SCHEMA = SchemaBuilder.record("InputTest")
+ .fields().requiredString("id").requiredString("primaryColor")
+ .optionalString("secondaryColor").optionalString("price")
+ .endRecord();
+
+ public static final Schema OUTPUT_SCHEMA = SchemaBuilder.record("Test")
+ .fields().requiredLong("id").requiredString("color")
+ .optionalDouble("price").endRecord();
+
+ public static final String MAPPING = "[{\"source\":\"primaryColor\", \"target\":\"color\"}]";
+
+ public static final String FAILURE_SUMMARY = "Cannot convert free to double";
+
+ @Test
+ public void testBasicConversion() throws IOException {
+ TestRunner runner = TestRunners.newTestRunner(ConvertAvroSchema.class);
+ runner.assertNotValid();
+ runner.setProperty(ConvertAvroSchema.INPUT_SCHEMA,
+ INPUT_SCHEMA.toString());
+ runner.setProperty(ConvertAvroSchema.OUTPUT_SCHEMA,
+ OUTPUT_SCHEMA.toString());
+ runner.setProperty("primaryColor", "color");
+ runner.assertValid();
+
+ // Two valid rows, and one invalid because "free" is not a double.
+ Record goodRecord1 = dataBasic("1", "blue", null, null);
+ Record goodRecord2 = dataBasic("2", "red", "yellow", "5.5");
+ Record badRecord = dataBasic("3", "red", "yellow", "free");
+ List input = Lists.newArrayList(goodRecord1, goodRecord2,
+ badRecord);
+
+ runner.enqueue(streamFor(input));
+ runner.run();
+
+ long converted = runner.getCounterValue("Converted records");
+ long errors = runner.getCounterValue("Conversion errors");
+ Assert.assertEquals("Should convert 2 rows", 2, converted);
+ Assert.assertEquals("Should reject 1 rows", 1, errors);
+
+ runner.assertTransferCount("success", 1);
+ runner.assertTransferCount("failure", 1);
+
+ MockFlowFile incompatible = runner.getFlowFilesForRelationship(
+ "failure").get(0);
+ GenericDatumReader reader = new GenericDatumReader(
+ INPUT_SCHEMA);
+ DataFileStream stream = new DataFileStream(
+ new ByteArrayInputStream(
+ runner.getContentAsByteArray(incompatible)), reader);
+ int count = 0;
+ for (Record r : stream) {
+ Assert.assertEquals(badRecord, r);
+ count++;
+ }
+ stream.close();
+ Assert.assertEquals(1, count);
+ Assert.assertEquals("Should accumulate error messages",
+ FAILURE_SUMMARY, incompatible.getAttribute("errors"));
+
+ GenericDatumReader successReader = new GenericDatumReader(
+ OUTPUT_SCHEMA);
+ DataFileStream successStream = new DataFileStream(
+ new ByteArrayInputStream(runner.getContentAsByteArray(runner
+ .getFlowFilesForRelationship("success").get(0))),
+ successReader);
+ count = 0;
+ for (Record r : successStream) {
+ if (count == 0) {
+ Assert.assertEquals(convertBasic(goodRecord1), r);
+ } else {
+ Assert.assertEquals(convertBasic(goodRecord2), r);
+ }
+ count++;
+ }
+ successStream.close();
+ Assert.assertEquals(2, count);
+ }
+
+ @Test
+ public void testNestedConversion() throws IOException {
+ TestRunner runner = TestRunners.newTestRunner(ConvertAvroSchema.class);
+ runner.assertNotValid();
+ runner.setProperty(ConvertAvroSchema.INPUT_SCHEMA,
+ TestAvroRecordConverter.NESTED_RECORD_SCHEMA.toString());
+ runner.setProperty(ConvertAvroSchema.OUTPUT_SCHEMA,
+ TestAvroRecordConverter.UNNESTED_OUTPUT_SCHEMA.toString());
+ runner.setProperty("parent.id", "parentId");
+ runner.assertValid();
+
+ // Two valid rows
+ Record goodRecord1 = dataNested(1L, "200", null, null);
+ Record goodRecord2 = dataNested(2L, "300", 5L, "ParentCompany");
+ List input = Lists.newArrayList(goodRecord1, goodRecord2);
+
+ runner.enqueue(streamFor(input));
+ runner.run();
+
+ long converted = runner.getCounterValue("Converted records");
+ long errors = runner.getCounterValue("Conversion errors");
+ Assert.assertEquals("Should convert 2 rows", 2, converted);
+ Assert.assertEquals("Should reject 0 rows", 0, errors);
+
+ runner.assertTransferCount("success", 1);
+ runner.assertTransferCount("failure", 0);
+
+ GenericDatumReader successReader = new GenericDatumReader(
+ TestAvroRecordConverter.UNNESTED_OUTPUT_SCHEMA);
+ DataFileStream successStream = new DataFileStream(
+ new ByteArrayInputStream(runner.getContentAsByteArray(runner
+ .getFlowFilesForRelationship("success").get(0))),
+ successReader);
+ int count = 0;
+ for (Record r : successStream) {
+ if (count == 0) {
+ Assert.assertEquals(convertNested(goodRecord1), r);
+ } else {
+ Assert.assertEquals(convertNested(goodRecord2), r);
+ }
+ count++;
+ }
+ successStream.close();
+ Assert.assertEquals(2, count);
+ }
+
+ private Record convertBasic(Record inputRecord) {
+ Record result = new Record(OUTPUT_SCHEMA);
+ result.put("id", Long.parseLong(inputRecord.get("id").toString()));
+ result.put("color", inputRecord.get("primaryColor").toString());
+ if (inputRecord.get("price") == null) {
+ result.put("price", null);
+ } else {
+ result.put("price",
+ Double.parseDouble(inputRecord.get("price").toString()));
+ }
+ return result;
+ }
+
+ private Record dataBasic(String id, String primaryColor,
+ String secondaryColor, String price) {
+ Record result = new Record(INPUT_SCHEMA);
+ result.put("id", id);
+ result.put("primaryColor", primaryColor);
+ result.put("secondaryColor", secondaryColor);
+ result.put("price", price);
+ return result;
+ }
+
+ private Record convertNested(Record inputRecord) {
+ Record result = new Record(
+ TestAvroRecordConverter.UNNESTED_OUTPUT_SCHEMA);
+ result.put("l1", inputRecord.get("l1"));
+ result.put("s1", Long.parseLong(inputRecord.get("s1").toString()));
+ if (inputRecord.get("parent") != null) {
+ // output schema doesn't have parent name.
+ result.put("parentId",
+ ((Record) inputRecord.get("parent")).get("id"));
+ }
+ return result;
+ }
+
+ private Record dataNested(long id, String companyName, Long parentId,
+ String parentName) {
+ Record result = new Record(TestAvroRecordConverter.NESTED_RECORD_SCHEMA);
+ result.put("l1", id);
+ result.put("s1", companyName);
+ if (parentId != null || parentName != null) {
+ Record parent = new Record(
+ TestAvroRecordConverter.NESTED_PARENT_SCHEMA);
+ parent.put("id", parentId);
+ parent.put("name", parentName);
+ result.put("parent", parent);
+ }
+ return result;
+ }
+}
From 8ff69ca2d1bc5b7d651b7495f2ce45def12bebc2 Mon Sep 17 00:00:00 2001
From: joewitt
Date: Tue, 14 Jul 2015 07:41:07 -0400
Subject: [PATCH 05/77] NIFI-751 PR #70 removed extraneous reference to
abstract properties pulling in hadoop conf
---
.../java/org/apache/nifi/processors/kite/ConvertAvroSchema.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
index 0d9f6586b5..daeb54830c 100644
--- a/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
+++ b/nifi/nifi-nar-bundles/nifi-kite-bundle/nifi-kite-processors/src/main/java/org/apache/nifi/processors/kite/ConvertAvroSchema.java
@@ -140,7 +140,7 @@ public class ConvertAvroSchema extends AbstractKiteProcessor {
private static final List PROPERTIES = ImmutableList
. builder()
- .addAll(AbstractKiteProcessor.getProperties()).add(INPUT_SCHEMA)
+ .add(INPUT_SCHEMA)
.add(OUTPUT_SCHEMA).build();
private static final Set RELATIONSHIPS = ImmutableSet
From 4dc126aff8751e66018ac03260f3061b97719487 Mon Sep 17 00:00:00 2001
From: Mark Payne
Date: Tue, 14 Jul 2015 08:40:04 -0400
Subject: [PATCH 06/77] NIFI-685: This is merged. This closes #67
From b251ab44258a840c5bacd973776b26dacbdbbbc2 Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Wed, 28 Jan 2015 16:30:49 -0800
Subject: [PATCH 07/77] Added processors that can run Flume sources and Flume
sinks.
Signed-off-by: Matt Gilman
---
.../nifi-flume-bundle/nifi-flume-nar/pom.xml | 31 +++
.../nifi-flume-processors/pom.xml | 126 +++++++++++++
.../flume/AbstractFlumeProcessor.java | 134 +++++++++++++
.../processors/flume/FlumeSinkProcessor.java | 157 +++++++++++++++
.../flume/FlumeSourceProcessor.java | 178 ++++++++++++++++++
.../nifi/processors/flume/NifiChannel.java | 31 +++
.../processors/flume/NifiChannelSelector.java | 55 ++++++
.../processors/flume/NifiTransaction.java | 40 ++++
.../processors/flume/util/FlowFileEvent.java | 114 +++++++++++
.../flume/util/FlowFileEventConstants.java | 25 +++
.../org.apache.nifi.processor.Processor | 16 ++
.../processors/flume/AbstractFlumeTest.java | 35 ++++
.../flume/FlumeSinkProcessorTest.java | 154 +++++++++++++++
.../flume/FlumeSourceProcessorTest.java | 140 ++++++++++++++
.../src/test/resources/core-site-broken.xml | 25 +++
.../src/test/resources/core-site.xml | 25 +++
.../src/test/resources/testdata/records.txt | 4 +
.../nifi-flume-bundle/pom.xml | 39 ++++
nifi/nifi-nar-bundles/pom.xml | 1 +
19 files changed, 1330 insertions(+)
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannelSelector.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/AbstractFlumeTest.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site-broken.xml
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site.xml
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/testdata/records.txt
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
new file mode 100644
index 0000000000..c5333b652d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
@@ -0,0 +1,31 @@
+
+
+ 4.0.0
+
+ org.apache.nifi
+ nifi-flume-bundle
+ 0.0.1-incubating-SNAPSHOT
+
+ nifi-flume-nar
+ 0.0.1-incubating-SNAPSHOT
+ nar
+
+
+ org.apache.nifi
+ nifi-flume-processors
+
+
+
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
new file mode 100644
index 0000000000..54636caf76
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
@@ -0,0 +1,126 @@
+
+
+ 4.0.0
+
+ org.apache.nifi
+ nifi-flume-bundle
+ 0.0.1-incubating-SNAPSHOT
+
+ nifi-flume-processors
+ jar
+
+
+ org.apache.nifi
+ nifi-api
+
+
+ org.apache.nifi
+ nifi-utils
+
+
+ org.apache.nifi
+ nifi-processor-utils
+
+
+ org.apache.nifi
+ nifi-flowfile-packager
+
+
+ org.apache.flume
+ flume-ng-sdk
+ 1.5.2
+
+
+ org.apache.flume
+ flume-ng-core
+ 1.5.2
+
+
+ org.slf4j
+ slf4j-log4j12
+
+
+
+
+
+
+ org.apache.flume.flume-ng-sources
+ flume-twitter-source
+ 1.5.2
+
+
+ org.apache.flume.flume-ng-sources
+ flume-jms-source
+ 1.5.2
+
+
+ org.apache.flume.flume-ng-sources
+ flume-scribe-source
+ 1.5.2
+
+
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-hdfs-sink
+ 1.5.2
+
+
+
+
+ org.apache.hadoop
+ hadoop-common
+ provided
+
+
+ org.apache.hadoop
+ hadoop-hdfs
+ provided
+
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-irc-sink
+ 1.5.2
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-ng-elasticsearch-sink
+ 1.5.2
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-ng-hbase-sink
+ 1.5.2
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-ng-morphline-solr-sink
+ 1.5.2
+
+
+
+ org.apache.nifi
+ nifi-mock
+ test
+
+
+ org.slf4j
+ slf4j-simple
+ test
+
+
+
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
new file mode 100644
index 0000000000..5c608d5ecd
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
@@ -0,0 +1,134 @@
+/*
+ * 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.flume;
+
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.SinkFactory;
+import org.apache.flume.Source;
+import org.apache.flume.SourceFactory;
+import org.apache.flume.sink.DefaultSinkFactory;
+import org.apache.flume.source.DefaultSourceFactory;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import static org.apache.nifi.processors.flume.FlumeSourceProcessor.FLUME_CONFIG;
+import org.apache.nifi.processors.flume.util.FlowFileEvent;
+
+/**
+ * This is a base class that is helpful when building processors interacting
+ * with Flume.
+ */
+public abstract class AbstractFlumeProcessor extends AbstractProcessor {
+ protected static final SourceFactory SOURCE_FACTORY = new DefaultSourceFactory();
+ protected static final SinkFactory SINK_FACTORY = new DefaultSinkFactory();
+
+ protected static Event flowFileToEvent(FlowFile flowFile, ProcessSession session) {
+ return new FlowFileEvent(flowFile, session);
+ }
+
+ protected static void transferEvent(final Event event, ProcessSession session,
+ Relationship relationship) {
+ FlowFile flowFile = session.create();
+ flowFile = session.putAllAttributes(flowFile, event.getHeaders());
+
+ flowFile = session.write(flowFile, new OutputStreamCallback() {
+ @Override
+ public void process(final OutputStream out) throws IOException {
+ out.write(event.getBody());
+ }
+ });
+
+ session.getProvenanceReporter().create(flowFile);
+ session.transfer(flowFile, relationship);
+ }
+
+ protected static Validator createSourceValidator() {
+ return new Validator() {
+ @Override
+ public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
+ String reason = null;
+ try {
+ FlumeSourceProcessor.SOURCE_FACTORY.create("NiFi Source", value);
+ } catch (Exception ex) {
+ reason = ex.getLocalizedMessage();
+ reason = Character.toLowerCase(reason.charAt(0)) + reason.substring(1);
+ }
+ return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
+ }
+ };
+ }
+
+ protected static Validator createSinkValidator() {
+ return new Validator() {
+ @Override
+ public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
+ String reason = null;
+ try {
+ FlumeSinkProcessor.SINK_FACTORY.create("NiFi Sink", value);
+ } catch (Exception ex) {
+ reason = ex.getLocalizedMessage();
+ reason = Character.toLowerCase(reason.charAt(0)) + reason.substring(1);
+ }
+ return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
+ }
+ };
+ }
+
+ protected static Context getFlumeContext(String flumeConfig, String prefix) {
+ Properties flumeProperties = new Properties();
+ if (flumeConfig != null) {
+ try {
+ flumeProperties.load(new StringReader(flumeConfig));
+ } catch (IOException ex) {
+ throw new RuntimeException(ex);
+ }
+ }
+ Map parameters = Maps.newHashMap();
+ for (String property : flumeProperties.stringPropertyNames()) {
+ parameters.put(property, flumeProperties.getProperty(property));
+ }
+ return new Context(new Context(parameters).getSubProperties(prefix));
+ }
+
+ protected static Context getFlumeSourceContext(String flumeConfig,
+ String agentName, String sourceName) {
+ return getFlumeContext(flumeConfig, agentName + ".sources." + sourceName + ".");
+ }
+
+ protected static Context getFlumeSinkContext(String flumeConfig,
+ String agentName, String sinkName) {
+ return getFlumeContext(flumeConfig, agentName + ".sinks." + sinkName + ".");
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
new file mode 100644
index 0000000000..4603d18e42
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
@@ -0,0 +1,157 @@
+/*
+ * 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.flume;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.flume.Context;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.Sink;
+import org.apache.flume.Transaction;
+import org.apache.flume.channel.MemoryChannel;
+import org.apache.flume.conf.Configurables;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.SchedulingContext;
+import org.apache.nifi.processor.annotation.CapabilityDescription;
+import org.apache.nifi.processor.annotation.OnScheduled;
+import org.apache.nifi.processor.annotation.OnUnscheduled;
+import org.apache.nifi.processor.annotation.Tags;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.flume.util.FlowFileEvent;
+
+/**
+ * This processor runs a Flume sink
+ */
+@Tags({"flume", "hadoop", "get", "sink" })
+@CapabilityDescription("Generate FlowFile data from a Flume sink")
+public class FlumeSinkProcessor extends AbstractFlumeProcessor {
+
+ private Sink sink;
+ private MemoryChannel channel;
+
+ public static final PropertyDescriptor SINK_TYPE = new PropertyDescriptor.Builder()
+ .name("Sink Type")
+ .description("The fully-qualified name of the Sink class")
+ .required(true)
+ .addValidator(createSinkValidator())
+ .build();
+ public static final PropertyDescriptor AGENT_NAME = new PropertyDescriptor.Builder()
+ .name("Agent Name")
+ .description("The name of the agent used in the Flume sink configuration")
+ .required(true)
+ .defaultValue("tier1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor SOURCE_NAME = new PropertyDescriptor.Builder()
+ .name("Sink Name")
+ .description("The name of the sink used in the Flume sink configuration")
+ .required(true)
+ .defaultValue("sink-1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor FLUME_CONFIG = new PropertyDescriptor.Builder()
+ .name("Flume Configuration")
+ .description("The Flume configuration for the sink copied from the flume.properties file")
+ .required(true)
+ .defaultValue("")
+ .addValidator(Validator.VALID)
+ .build();
+
+ public static final Relationship SUCCESS = new Relationship.Builder().name("success").build();
+ public static final Relationship FAILURE = new Relationship.Builder().name("failure").build();
+
+ private List descriptors;
+ private Set relationships;
+
+
+ @Override
+ protected void init(final ProcessorInitializationContext context) {
+ this.descriptors = ImmutableList.of(SINK_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG);
+ this.relationships = ImmutableSet.of();
+ }
+
+ @Override
+ protected List getSupportedPropertyDescriptors() {
+ return descriptors;
+ }
+
+ @Override
+ public Set getRelationships() {
+ return relationships;
+ }
+
+ @OnScheduled
+ public void onScheduled(final SchedulingContext context) {
+ channel = new MemoryChannel();
+ Configurables.configure(channel, new Context());
+ channel.start();
+
+ sink = SINK_FACTORY.create(context.getProperty(SOURCE_NAME).getValue(),
+ context.getProperty(SINK_TYPE).getValue());
+ sink.setChannel(channel);
+
+ String flumeConfig = context.getProperty(FLUME_CONFIG).getValue();
+ String agentName = context.getProperty(AGENT_NAME).getValue();
+ String sinkName = context.getProperty(SOURCE_NAME).getValue();
+ Configurables.configure(sink,
+ getFlumeSinkContext(flumeConfig, agentName, sinkName) );
+
+ sink.start();
+ }
+
+ @OnUnscheduled
+ public void unScheduled() {
+ sink.stop();
+ channel.stop();
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context,
+ final ProcessSession session) throws ProcessException {
+ FlowFile flowFile = session.get();
+
+ Transaction transaction = channel.getTransaction();
+ try {
+ transaction.begin();
+ channel.put(new FlowFileEvent(flowFile, session));
+ transaction.commit();
+ } catch (Throwable th) {
+ transaction.rollback();
+ throw Throwables.propagate(th);
+ } finally {
+ transaction.close();
+ }
+
+ try {
+ sink.process();
+ session.transfer(flowFile, SUCCESS);
+ } catch (EventDeliveryException ex) {
+ session.transfer(flowFile, FAILURE);
+ }
+ }
+}
\ No newline at end of file
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
new file mode 100644
index 0000000000..8b8388c3cc
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
@@ -0,0 +1,178 @@
+/*
+ * 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.flume;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+import org.apache.flume.EventDeliveryException;
+import org.apache.flume.EventDrivenSource;
+import org.apache.flume.PollableSource;
+import org.apache.flume.Source;
+import org.apache.flume.SourceRunner;
+import org.apache.flume.Transaction;
+import org.apache.flume.channel.ChannelProcessor;
+import org.apache.flume.channel.MemoryChannel;
+import org.apache.flume.conf.Configurables;
+import org.apache.flume.source.EventDrivenSourceRunner;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.SchedulingContext;
+import org.apache.nifi.processor.annotation.CapabilityDescription;
+import org.apache.nifi.processor.annotation.OnScheduled;
+import org.apache.nifi.processor.annotation.OnUnscheduled;
+import org.apache.nifi.processor.annotation.Tags;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+/**
+ * This processor runs a Flume source
+ */
+@Tags({"flume", "hadoop", "get", "source" })
+@CapabilityDescription("Generate FlowFile data from a Flume source")
+public class FlumeSourceProcessor extends AbstractFlumeProcessor {
+
+ private Source source;
+ private SourceRunner runner;
+ private MemoryChannel channel;
+
+ public static final PropertyDescriptor SOURCE_TYPE = new PropertyDescriptor.Builder()
+ .name("Source Type")
+ .description("The fully-qualified name of the Source class")
+ .required(true)
+ .addValidator(createSourceValidator())
+ .build();
+ public static final PropertyDescriptor AGENT_NAME = new PropertyDescriptor.Builder()
+ .name("Agent Name")
+ .description("The name of the agent used in the Flume source configuration")
+ .required(true)
+ .defaultValue("tier1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor SOURCE_NAME = new PropertyDescriptor.Builder()
+ .name("Source Name")
+ .description("The name of the source used in the Flume source configuration")
+ .required(true)
+ .defaultValue("src-1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor FLUME_CONFIG = new PropertyDescriptor.Builder()
+ .name("Flume Configuration")
+ .description("The Flume configuration for the source copied from the flume.properties file")
+ .required(true)
+ .defaultValue("")
+ .addValidator(Validator.VALID)
+ .build();
+
+ public static final Relationship SUCCESS = new Relationship.Builder().name("success").build();
+
+ private List descriptors;
+ private Set relationships;
+
+
+ @Override
+ protected void init(final ProcessorInitializationContext context) {
+ this.descriptors = ImmutableList.of(SOURCE_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG);
+ this.relationships = ImmutableSet.of(SUCCESS);
+ }
+
+ @Override
+ protected List getSupportedPropertyDescriptors() {
+ return descriptors;
+ }
+
+ @Override
+ public Set getRelationships() {
+ return relationships;
+ }
+
+ @OnScheduled
+ public void onScheduled(final SchedulingContext context) {
+ source = SOURCE_FACTORY.create(
+ context.getProperty(SOURCE_NAME).getValue(),
+ context.getProperty(SOURCE_TYPE).getValue());
+
+ String flumeConfig = context.getProperty(FLUME_CONFIG).getValue();
+ String agentName = context.getProperty(AGENT_NAME).getValue();
+ String sourceName = context.getProperty(SOURCE_NAME).getValue();
+ Configurables.configure(source,
+ getFlumeSourceContext(flumeConfig, agentName, sourceName) );
+
+ if (source instanceof EventDrivenSource) {
+ runner = new EventDrivenSourceRunner();
+ channel = new MemoryChannel();
+ Configurables.configure(channel, new Context());
+ channel.start();
+ source.setChannelProcessor(new ChannelProcessor(new NifiChannelSelector(channel)));
+ runner.setSource(source);
+ runner.start();
+ }
+ }
+
+ @OnUnscheduled
+ public void unScheduled() {
+ if (runner != null) {
+ runner.stop();
+ }
+ if (channel != null) {
+ channel.stop();
+ }
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context,
+ final ProcessSession session) throws ProcessException {
+ if (source instanceof EventDrivenSource) {
+ onEventDrivenTrigger(context, session);
+ } else if (source instanceof PollableSource) {
+ onPollableTrigger((PollableSource)source, context, session);
+ }
+ }
+
+ public void onPollableTrigger(final PollableSource pollableSource,
+ final ProcessContext context, final ProcessSession session)
+ throws ProcessException {
+ try {
+ pollableSource.setChannelProcessor(new ChannelProcessor(
+ new NifiChannelSelector(new NifiChannel(session, SUCCESS))));
+ pollableSource.start();
+ pollableSource.process();
+ pollableSource.stop();
+ } catch (EventDeliveryException ex) {
+ throw new ProcessException("Error processing pollable source", ex);
+ }
+ }
+
+ public void onEventDrivenTrigger(final ProcessContext context, final ProcessSession session) {
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+
+ Event event = channel.take();
+ if (event != null) {
+ transferEvent(event, session, SUCCESS);
+ }
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java
new file mode 100644
index 0000000000..ac8dbe2b1d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java
@@ -0,0 +1,31 @@
+
+package org.apache.nifi.processors.flume;
+
+import org.apache.flume.Context;
+import org.apache.flume.channel.BasicChannelSemantics;
+import org.apache.flume.channel.BasicTransactionSemantics;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+
+
+public class NifiChannel extends BasicChannelSemantics {
+ private final ProcessSession session;
+ private final Relationship relationship;
+
+ public NifiChannel(ProcessSession session, Relationship relationship) {
+ this.session = session;
+ this.relationship = relationship;
+ }
+
+ @Override
+ protected BasicTransactionSemantics createTransaction() {
+ return new NifiTransaction(session, relationship);
+ }
+
+ @Override
+ public void configure(Context context) {
+ throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+ }
+
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannelSelector.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannelSelector.java
new file mode 100644
index 0000000000..792678bd59
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannelSelector.java
@@ -0,0 +1,55 @@
+
+package org.apache.nifi.processors.flume;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.flume.Channel;
+import org.apache.flume.ChannelSelector;
+import org.apache.flume.Context;
+import org.apache.flume.Event;
+
+
+public class NifiChannelSelector implements ChannelSelector {
+ private String name;
+ private final List requiredChannels;
+ private final List optionalChannels;
+
+ public NifiChannelSelector(Channel channel) {
+ requiredChannels = ImmutableList.of(channel);
+ optionalChannels = ImmutableList.of();
+ }
+
+ @Override
+ public List getRequiredChannels(Event event) {
+ return requiredChannels;
+ }
+
+ @Override
+ public List getOptionalChannels(Event event) {
+ return optionalChannels;
+ }
+
+ @Override
+ public List getAllChannels() {
+ return requiredChannels;
+ }
+
+ @Override
+ public void setChannels(List channels) {
+ throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public void configure(Context context) {
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java
new file mode 100644
index 0000000000..3d6a647f85
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java
@@ -0,0 +1,40 @@
+
+package org.apache.nifi.processors.flume;
+
+import org.apache.flume.Event;
+import org.apache.flume.channel.BasicTransactionSemantics;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+
+
+class NifiTransaction extends BasicTransactionSemantics {
+ private final ProcessSession session;
+ private final Relationship relationship;
+
+ public NifiTransaction(ProcessSession session, Relationship relationship) {
+ this.session = session;
+ this.relationship = relationship;
+ }
+
+ @Override
+ protected void doPut(Event event) throws InterruptedException {
+ AbstractFlumeProcessor.transferEvent(event, session, relationship);
+ }
+
+ @Override
+ protected Event doTake() throws InterruptedException {
+ throw new UnsupportedOperationException("Only put supported");
+ }
+
+ @Override
+ protected void doCommit() throws InterruptedException {
+ session.commit();
+ }
+
+ @Override
+ protected void doRollback() throws InterruptedException {
+ session.rollback();
+ }
+
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
new file mode 100644
index 0000000000..c3531ca12e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
@@ -0,0 +1,114 @@
+
+package org.apache.nifi.processors.flume.util;
+
+import com.google.common.collect.Maps;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.flume.Event;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.io.InputStreamCallback;
+
+import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.*;
+import org.apache.nifi.stream.io.BufferedInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+
+public class FlowFileEvent implements Event {
+
+ private final FlowFile flowFile;
+ private final ProcessSession session;
+
+ private final Map headers;
+ private boolean headersLoaded;
+
+ private final Object bodyLock;
+ private byte[] body;
+ private boolean bodyLoaded;
+
+ public FlowFileEvent(FlowFile flowFile, ProcessSession session) {
+ this.flowFile = flowFile;
+ this.session = session;
+
+ headers = Maps.newHashMap();
+ bodyLock = new Object();
+ bodyLoaded = false;
+ }
+
+ @Override
+ public Map getHeaders() {
+ if (!headersLoaded) {
+ synchronized (headers) {
+ if (headersLoaded) {
+ return headers;
+ }
+
+ headers.putAll(flowFile.getAttributes());
+ headers.put(ENTRY_DATE_HEADER, Long.toString(flowFile.getEntryDate()));
+ headers.put(ID_HEADER, Long.toString(flowFile.getId()));
+ headers.put(LAST_QUEUE_DATE_HEADER, Long.toString(flowFile.getLastQueueDate()));
+ int i = 0;
+ for (String lineageIdentifier : flowFile.getLineageIdentifiers()) {
+ headers.put(LINEAGE_IDENTIFIERS_HEADER + "." + i, lineageIdentifier);
+ i++;
+ }
+ headers.put(LINEAGE_START_DATE_HEADER, Long.toString(flowFile.getLineageStartDate()));
+ headers.put(SIZE_HEADER, Long.toString(flowFile.getSize()));
+
+ headersLoaded = true;
+ }
+ }
+ return headers;
+ }
+
+ @Override
+ public void setHeaders(Map headers) {
+ synchronized (this.headers) {
+ this.headers.clear();
+ this.headers.putAll(headers);
+ headersLoaded = true;
+ }
+ }
+
+ @Override
+ public byte[] getBody() {
+ if (bodyLoaded) {
+ return body;
+ }
+
+ 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)");
+ }
+
+ final ByteArrayOutputStream baos = new ByteArrayOutputStream((int) flowFile.getSize());
+ session.read(flowFile, new InputStreamCallback() {
+
+ @Override
+ public void process(InputStream in) throws IOException {
+ try (BufferedInputStream input = new BufferedInputStream(in)) {
+ StreamUtils.copy(in, baos);
+ }
+ baos.close();
+ }
+ });
+
+ body = baos.toByteArray();
+ bodyLoaded = true;
+ }
+ }
+
+ return body;
+ }
+
+ @Override
+ public void setBody(byte[] body) {
+ synchronized (bodyLock) {
+ this.body = Arrays.copyOf(body, body.length);
+ bodyLoaded = true;
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java
new file mode 100644
index 0000000000..c13f0ef6f2
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java
@@ -0,0 +1,25 @@
+
+package org.apache.nifi.processors.flume.util;
+
+
+public class FlowFileEventConstants {
+
+ // FlowFile#getEntryDate();
+ public static final String ENTRY_DATE_HEADER = "nifi.entry.date";
+
+ // FlowFile#getId();
+ public static final String ID_HEADER = "nifi.id";
+
+ // FlowFile#getLastQueueDate();
+ public static final String LAST_QUEUE_DATE_HEADER = "nifi.last.queue.date";
+
+ // FlowFile#getLineageIdentifiers();
+ public static final String LINEAGE_IDENTIFIERS_HEADER = "nifi.lineage.identifiers";
+
+ // FlowFile#getLineageStartDate();
+ public static final String LINEAGE_START_DATE_HEADER = "nifi.lineage.start.date";
+
+ // FlowFile#getSize();
+ public static final String SIZE_HEADER = "nifi.size";
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
new file mode 100644
index 0000000000..fae87277b0
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -0,0 +1,16 @@
+# 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.
+org.apache.nifi.processors.flume.FlumeSourceProcessor
+org.apache.nifi.processors.flume.FlumeSinkProcessor
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/AbstractFlumeTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/AbstractFlumeTest.java
new file mode 100644
index 0000000000..87b056a86e
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/AbstractFlumeTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.flume;
+
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AbstractFlumeTest {
+
+ private static Logger logger;
+
+ @BeforeClass
+ public static void setUpClass() {
+ System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+ System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+ System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.flume", "debug");
+ logger = LoggerFactory.getLogger(AbstractFlumeTest.class);
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
new file mode 100644
index 0000000000..4f2cef778a
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
@@ -0,0 +1,154 @@
+/*
+ * 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.flume;
+
+import java.io.File;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.FileInputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import org.apache.commons.io.filefilter.HiddenFileFilter;
+import org.apache.flume.sink.NullSink;
+import org.apache.flume.source.AvroSource;
+
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.util.MockProcessContext;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.util.file.FileUtils;
+
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FlumeSinkProcessorTest {
+
+ private static final Logger logger =
+ LoggerFactory.getLogger(FlumeSinkProcessorTest.class);
+
+ @Test
+ public void testValidators() {
+ TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
+ Collection results;
+ ProcessContext pc;
+
+ results = new HashSet<>();
+ runner.enqueue(new byte[0]);
+ pc = runner.getProcessContext();
+ if (pc instanceof MockProcessContext) {
+ results = ((MockProcessContext) pc).validate();
+ }
+ Assert.assertEquals(1, results.size());
+ for (ValidationResult vr : results) {
+ logger.error(vr.toString());
+ Assert.assertTrue(vr.toString().contains("is invalid because Sink Type is required"));
+ }
+
+ // non-existent class
+ results = new HashSet<>();
+ runner.setProperty(FlumeSinkProcessor.SINK_TYPE, "invalid.class.name");
+ runner.enqueue(new byte[0]);
+ pc = runner.getProcessContext();
+ if (pc instanceof MockProcessContext) {
+ results = ((MockProcessContext) pc).validate();
+ }
+ Assert.assertEquals(1, results.size());
+ for (ValidationResult vr : results) {
+ logger.error(vr.toString());
+ Assert.assertTrue(vr.toString().contains("is invalid because unable to load sink"));
+ }
+
+ // class doesn't implement Sink
+ results = new HashSet<>();
+ runner.setProperty(FlumeSinkProcessor.SINK_TYPE, AvroSource.class.getName());
+ runner.enqueue(new byte[0]);
+ pc = runner.getProcessContext();
+ if (pc instanceof MockProcessContext) {
+ results = ((MockProcessContext) pc).validate();
+ }
+ Assert.assertEquals(1, results.size());
+ for (ValidationResult vr : results) {
+ logger.error(vr.toString());
+ Assert.assertTrue(vr.toString().contains("is invalid because unable to create sink"));
+ }
+
+ results = new HashSet<>();
+ runner.setProperty(FlumeSinkProcessor.SINK_TYPE, NullSink.class.getName());
+ runner.enqueue(new byte[0]);
+ pc = runner.getProcessContext();
+ if (pc instanceof MockProcessContext) {
+ results = ((MockProcessContext) pc).validate();
+ }
+ Assert.assertEquals(0, results.size());
+ }
+
+
+ @Test
+ public void testNullSink() throws IOException {
+ TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
+ runner.setProperty(FlumeSinkProcessor.SINK_TYPE, NullSink.class.getName());
+ FileInputStream fis = new FileInputStream("src/test/resources/testdata/records.txt");
+ Map attributes = new HashMap<>();
+ attributes.put(CoreAttributes.FILENAME.key(), "records.txt");
+ runner.enqueue(fis, attributes);
+ runner.run();
+ fis.close();
+ }
+
+ @Test
+ public void testHdfsSink() throws IOException {
+ File destDir = new File("target/hdfs");
+ if (destDir.exists()) {
+ FileUtils.deleteFilesInDir(destDir, null, logger);
+ } else {
+ destDir.mkdirs();
+ }
+
+ TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
+ runner.setProperty(FlumeSinkProcessor.SINK_TYPE, "hdfs");
+ runner.setProperty(FlumeSinkProcessor.FLUME_CONFIG,
+ "tier1.sinks.sink-1.hdfs.path = " + destDir.toURI().toString() + "\n" +
+ "tier1.sinks.sink-1.hdfs.fileType = DataStream\n" +
+ "tier1.sinks.sink-1.hdfs.serializer = TEXT\n" +
+ "tier1.sinks.sink-1.serializer.appendNewline = false"
+ );
+ FileInputStream fis = new FileInputStream("src/test/resources/testdata/records.txt");
+ Map attributes = new HashMap<>();
+ attributes.put(CoreAttributes.FILENAME.key(), "records.txt");
+ runner.enqueue(fis, attributes);
+ runner.run();
+ fis.close();
+
+ File[] files = destDir.listFiles((FilenameFilter)HiddenFileFilter.VISIBLE);
+ assertEquals("Unexpected number of destination files.", 1, files.length);
+ File dst = files[0];
+ byte[] expectedMd5 = FileUtils.computeMd5Digest(new File("src/test/resources/testdata/records.txt"));
+ byte[] actualMd5 = FileUtils.computeMd5Digest(dst);
+ Assert.assertArrayEquals("Destination file doesn't match source data", expectedMd5, actualMd5);
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
new file mode 100644
index 0000000000..bbcf116787
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.flume;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import org.apache.flume.sink.NullSink;
+import org.apache.flume.source.AvroSource;
+
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.MockProcessContext;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.apache.nifi.util.file.FileUtils;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FlumeSourceProcessorTest {
+
+ private static final Logger logger =
+ LoggerFactory.getLogger(FlumeSourceProcessorTest.class);
+
+
+ @Test
+ public void testValidators() {
+ TestRunner runner = TestRunners.newTestRunner(FlumeSourceProcessor.class);
+ Collection results;
+ ProcessContext pc;
+
+ results = new HashSet<>();
+ runner.enqueue(new byte[0]);
+ pc = runner.getProcessContext();
+ if (pc instanceof MockProcessContext) {
+ results = ((MockProcessContext) pc).validate();
+ }
+ Assert.assertEquals(1, results.size());
+ for (ValidationResult vr : results) {
+ logger.error(vr.toString());
+ Assert.assertTrue(vr.toString().contains("is invalid because Source Type is required"));
+ }
+
+ // non-existent class
+ results = new HashSet<>();
+ runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, "invalid.class.name");
+ runner.enqueue(new byte[0]);
+ pc = runner.getProcessContext();
+ if (pc instanceof MockProcessContext) {
+ results = ((MockProcessContext) pc).validate();
+ }
+ Assert.assertEquals(1, results.size());
+ for (ValidationResult vr : results) {
+ logger.error(vr.toString());
+ Assert.assertTrue(vr.toString().contains("is invalid because unable to load source"));
+ }
+
+ // class doesn't implement Source
+ results = new HashSet<>();
+ runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, NullSink.class.getName());
+ runner.enqueue(new byte[0]);
+ pc = runner.getProcessContext();
+ if (pc instanceof MockProcessContext) {
+ results = ((MockProcessContext) pc).validate();
+ }
+ Assert.assertEquals(1, results.size());
+ for (ValidationResult vr : results) {
+ logger.error(vr.toString());
+ Assert.assertTrue(vr.toString().contains("is invalid because unable to create source"));
+ }
+
+ results = new HashSet<>();
+ runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, AvroSource.class.getName());
+ runner.enqueue(new byte[0]);
+ pc = runner.getProcessContext();
+ if (pc instanceof MockProcessContext) {
+ results = ((MockProcessContext) pc).validate();
+ }
+ Assert.assertEquals(0, results.size());
+ }
+
+ @Test
+ public void testSequenceSource() {
+ TestRunner runner = TestRunners.newTestRunner(FlumeSourceProcessor.class);
+ runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, "seq");
+ runner.run();
+ List flowFiles = runner.getFlowFilesForRelationship(FlumeSourceProcessor.SUCCESS);
+ Assert.assertEquals(1, flowFiles.size());
+ for (MockFlowFile flowFile : flowFiles) {
+ logger.error(flowFile.toString());
+ Assert.assertEquals(1, flowFile.getSize());
+ }
+ }
+
+ @Test
+ public void testSourceWithConfig() throws IOException {
+ File spoolDirectory = new File("target/spooldir");
+ if (spoolDirectory.exists()) {
+ FileUtils.deleteFilesInDir(spoolDirectory, null, logger);
+ } else {
+ spoolDirectory.mkdirs();
+ }
+ File src = new File("src/test/resources/testdata/records.txt");
+ File dst = new File(spoolDirectory, "records.txt");
+ FileUtils.copyFile(src, dst, false, false, logger);
+
+ TestRunner runner = TestRunners.newTestRunner(FlumeSourceProcessor.class);
+ runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, "spooldir");
+ runner.setProperty(FlumeSinkProcessor.FLUME_CONFIG,
+ "tier1.sources.src-1.spoolDir = " + spoolDirectory.getAbsolutePath());
+ runner.run();
+ List flowFiles = runner.getFlowFilesForRelationship(FlumeSourceProcessor.SUCCESS);
+ Assert.assertEquals(1, flowFiles.size());
+ for (MockFlowFile flowFile : flowFiles) {
+ Assert.assertEquals(8, flowFile.getSize());
+ flowFile.assertContentEquals("record 1");
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site-broken.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site-broken.xml
new file mode 100644
index 0000000000..e06a193353
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site-broken.xml
@@ -0,0 +1,25 @@
+
+
+
+
+
+
+
+
+ fs.default.name
+ hdfs://localhost:65535
+
+
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site.xml
new file mode 100644
index 0000000000..5e3b55cb0b
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site.xml
@@ -0,0 +1,25 @@
+
+
+
+
+
+
+
+
+ fs.defaultFS
+ file:///
+
+
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/testdata/records.txt b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/testdata/records.txt
new file mode 100644
index 0000000000..5a809eee88
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/testdata/records.txt
@@ -0,0 +1,4 @@
+record 1
+record 2
+record 3
+record 4
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
new file mode 100644
index 0000000000..dc9ec69deb
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
@@ -0,0 +1,39 @@
+
+
+ 4.0.0
+
+ org.apache.nifi
+ nifi-nar-bundles
+ 0.0.1-incubating-SNAPSHOT
+
+ nifi-flume-bundle
+ 0.0.1-incubating-SNAPSHOT
+ pom
+ A bundle of processors that run Flume sources/sinks
+
+ nifi-flume-processors
+ nifi-flume-nar
+
+
+
+
+ org.apache.nifi
+ nifi-flume-processors
+ 0.0.1-incubating-SNAPSHOT
+
+
+
+
diff --git a/nifi/nifi-nar-bundles/pom.xml b/nifi/nifi-nar-bundles/pom.xml
index b9be570a8b..29a5e49d0d 100644
--- a/nifi/nifi-nar-bundles/pom.xml
+++ b/nifi/nifi-nar-bundles/pom.xml
@@ -41,6 +41,7 @@
nifi-hl7-bundlenifi-language-translation-bundlenifi-mongodb-bundle
+ nifi-flume-bundle
From cf29029a4db4cc606b417889296abb97e460d586 Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Fri, 30 Jan 2015 14:21:21 -0500
Subject: [PATCH 08/77] Added the sink's relationships to the relationship set.
Added error checkign and logging for sink/source creation. Fixed an issue
with transaction managemetn in the sink. Reformatted per coding standard.
Signed-off-by: Matt Gilman
---
nifi/nifi-assembly/pom.xml | 3 +
.../nifi-flume-bundle/nifi-flume-nar/pom.xml | 5 +
.../processors/flume/FlumeSinkProcessor.java | 210 +++++++--------
.../flume/FlumeSourceProcessor.java | 241 +++++++++---------
.../flume/FlumeSinkProcessorTest.java | 2 -
nifi/pom.xml | 5 +
6 files changed, 247 insertions(+), 219 deletions(-)
diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml
index c679d22ab5..4f4879fefe 100644
--- a/nifi/nifi-assembly/pom.xml
+++ b/nifi/nifi-assembly/pom.xml
@@ -165,6 +165,9 @@ language governing permissions and limitations under the License. -->
org.apache.nifinifi-kite-nar
+
+ org.apache.nifi
+ nifi-flume-narnar
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
index c5333b652d..dff440eb7f 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
@@ -27,5 +27,10 @@
org.apache.nifinifi-flume-processors
+
+ org.apache.nifi
+ nifi-hadoop-libraries-nar
+ nar
+
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
index 4603d18e42..fc97ae8685 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
@@ -27,6 +27,10 @@ import org.apache.flume.Sink;
import org.apache.flume.Transaction;
import org.apache.flume.channel.MemoryChannel;
import org.apache.flume.conf.Configurables;
+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.OnUnscheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.Validator;
@@ -36,122 +40,122 @@ import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SchedulingContext;
-import org.apache.nifi.processor.annotation.CapabilityDescription;
-import org.apache.nifi.processor.annotation.OnScheduled;
-import org.apache.nifi.processor.annotation.OnUnscheduled;
-import org.apache.nifi.processor.annotation.Tags;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.processors.flume.util.FlowFileEvent;
/**
- * This processor runs a Flume sink
+ * This processor runs a Flume sink
*/
-@Tags({"flume", "hadoop", "get", "sink" })
+@Tags({"flume", "hadoop", "get", "sink"})
@CapabilityDescription("Generate FlowFile data from a Flume sink")
public class FlumeSinkProcessor extends AbstractFlumeProcessor {
- private Sink sink;
- private MemoryChannel channel;
+ private Sink sink;
+ private MemoryChannel channel;
- public static final PropertyDescriptor SINK_TYPE = new PropertyDescriptor.Builder()
- .name("Sink Type")
- .description("The fully-qualified name of the Sink class")
- .required(true)
- .addValidator(createSinkValidator())
- .build();
- public static final PropertyDescriptor AGENT_NAME = new PropertyDescriptor.Builder()
- .name("Agent Name")
- .description("The name of the agent used in the Flume sink configuration")
- .required(true)
- .defaultValue("tier1")
- .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
- .build();
- public static final PropertyDescriptor SOURCE_NAME = new PropertyDescriptor.Builder()
- .name("Sink Name")
- .description("The name of the sink used in the Flume sink configuration")
- .required(true)
- .defaultValue("sink-1")
- .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
- .build();
- public static final PropertyDescriptor FLUME_CONFIG = new PropertyDescriptor.Builder()
- .name("Flume Configuration")
- .description("The Flume configuration for the sink copied from the flume.properties file")
- .required(true)
- .defaultValue("")
- .addValidator(Validator.VALID)
- .build();
+ public static final PropertyDescriptor SINK_TYPE = new PropertyDescriptor.Builder()
+ .name("Sink Type")
+ .description("The fully-qualified name of the Sink class")
+ .required(true)
+ .addValidator(createSinkValidator())
+ .build();
+ public static final PropertyDescriptor AGENT_NAME = new PropertyDescriptor.Builder()
+ .name("Agent Name")
+ .description("The name of the agent used in the Flume sink configuration")
+ .required(true)
+ .defaultValue("tier1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor SOURCE_NAME = new PropertyDescriptor.Builder()
+ .name("Sink Name")
+ .description("The name of the sink used in the Flume sink configuration")
+ .required(true)
+ .defaultValue("sink-1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor FLUME_CONFIG = new PropertyDescriptor.Builder()
+ .name("Flume Configuration")
+ .description("The Flume configuration for the sink copied from the flume.properties file")
+ .required(true)
+ .defaultValue("")
+ .addValidator(Validator.VALID)
+ .build();
- public static final Relationship SUCCESS = new Relationship.Builder().name("success").build();
- public static final Relationship FAILURE = new Relationship.Builder().name("failure").build();
+ public static final Relationship SUCCESS = new Relationship.Builder().name("success").build();
+ public static final Relationship FAILURE = new Relationship.Builder().name("failure").build();
- private List descriptors;
- private Set relationships;
+ private List descriptors;
+ private Set relationships;
-
- @Override
- protected void init(final ProcessorInitializationContext context) {
- this.descriptors = ImmutableList.of(SINK_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG);
- this.relationships = ImmutableSet.of();
- }
-
- @Override
- protected List getSupportedPropertyDescriptors() {
- return descriptors;
- }
-
- @Override
- public Set getRelationships() {
- return relationships;
- }
-
- @OnScheduled
- public void onScheduled(final SchedulingContext context) {
- channel = new MemoryChannel();
- Configurables.configure(channel, new Context());
- channel.start();
-
- sink = SINK_FACTORY.create(context.getProperty(SOURCE_NAME).getValue(),
- context.getProperty(SINK_TYPE).getValue());
- sink.setChannel(channel);
-
- String flumeConfig = context.getProperty(FLUME_CONFIG).getValue();
- String agentName = context.getProperty(AGENT_NAME).getValue();
- String sinkName = context.getProperty(SOURCE_NAME).getValue();
- Configurables.configure(sink,
- getFlumeSinkContext(flumeConfig, agentName, sinkName) );
-
- sink.start();
- }
-
- @OnUnscheduled
- public void unScheduled() {
- sink.stop();
- channel.stop();
- }
-
- @Override
- public void onTrigger(final ProcessContext context,
- final ProcessSession session) throws ProcessException {
- FlowFile flowFile = session.get();
-
- Transaction transaction = channel.getTransaction();
- try {
- transaction.begin();
- channel.put(new FlowFileEvent(flowFile, session));
- transaction.commit();
- } catch (Throwable th) {
- transaction.rollback();
- throw Throwables.propagate(th);
- } finally {
- transaction.close();
+ @Override
+ protected void init(final ProcessorInitializationContext context) {
+ this.descriptors = ImmutableList.of(SINK_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG);
+ this.relationships = ImmutableSet.of(SUCCESS, FAILURE);
}
- try {
- sink.process();
- session.transfer(flowFile, SUCCESS);
- } catch (EventDeliveryException ex) {
- session.transfer(flowFile, FAILURE);
+ @Override
+ protected List getSupportedPropertyDescriptors() {
+ return descriptors;
}
- }
-}
\ No newline at end of file
+
+ @Override
+ public Set getRelationships() {
+ return relationships;
+ }
+
+ @OnScheduled
+ public void onScheduled(final SchedulingContext context) {
+ try {
+ channel = new MemoryChannel();
+ Configurables.configure(channel, new Context());
+ channel.start();
+
+ sink = SINK_FACTORY.create(context.getProperty(SOURCE_NAME).getValue(),
+ context.getProperty(SINK_TYPE).getValue());
+ sink.setChannel(channel);
+
+ String flumeConfig = context.getProperty(FLUME_CONFIG).getValue();
+ String agentName = context.getProperty(AGENT_NAME).getValue();
+ String sinkName = context.getProperty(SOURCE_NAME).getValue();
+ Configurables.configure(sink,
+ getFlumeSinkContext(flumeConfig, agentName, sinkName));
+
+ sink.start();
+ } catch (Throwable th) {
+ getLogger().error("Error creating sink", th);
+ throw Throwables.propagate(th);
+ }
+ }
+
+ @OnUnscheduled
+ public void unScheduled() {
+ sink.stop();
+ channel.stop();
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context,
+ final ProcessSession session) throws ProcessException {
+ FlowFile flowFile = session.get();
+
+ Transaction transaction = channel.getTransaction();
+ try {
+ transaction.begin();
+ channel.put(new FlowFileEvent(flowFile, session));
+ transaction.commit();
+ } catch (Throwable th) {
+ transaction.rollback();
+ throw Throwables.propagate(th);
+ } finally {
+ transaction.close();
+ }
+
+ try {
+ sink.process();
+ session.transfer(flowFile, SUCCESS);
+ } catch (EventDeliveryException ex) {
+ session.transfer(flowFile, FAILURE);
+ }
+ }
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
index 8b8388c3cc..19551e68f0 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.processors.flume;
+import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import java.util.List;
@@ -32,6 +33,10 @@ import org.apache.flume.channel.ChannelProcessor;
import org.apache.flume.channel.MemoryChannel;
import org.apache.flume.conf.Configurables;
import org.apache.flume.source.EventDrivenSourceRunner;
+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.OnUnscheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.Validator;
@@ -40,139 +45,147 @@ import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SchedulingContext;
-import org.apache.nifi.processor.annotation.CapabilityDescription;
-import org.apache.nifi.processor.annotation.OnScheduled;
-import org.apache.nifi.processor.annotation.OnUnscheduled;
-import org.apache.nifi.processor.annotation.Tags;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
/**
- * This processor runs a Flume source
+ * This processor runs a Flume source
*/
-@Tags({"flume", "hadoop", "get", "source" })
+@Tags({"flume", "hadoop", "get", "source"})
@CapabilityDescription("Generate FlowFile data from a Flume source")
public class FlumeSourceProcessor extends AbstractFlumeProcessor {
-
- private Source source;
- private SourceRunner runner;
- private MemoryChannel channel;
- public static final PropertyDescriptor SOURCE_TYPE = new PropertyDescriptor.Builder()
- .name("Source Type")
- .description("The fully-qualified name of the Source class")
- .required(true)
- .addValidator(createSourceValidator())
- .build();
- public static final PropertyDescriptor AGENT_NAME = new PropertyDescriptor.Builder()
- .name("Agent Name")
- .description("The name of the agent used in the Flume source configuration")
- .required(true)
- .defaultValue("tier1")
- .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
- .build();
- public static final PropertyDescriptor SOURCE_NAME = new PropertyDescriptor.Builder()
- .name("Source Name")
- .description("The name of the source used in the Flume source configuration")
- .required(true)
- .defaultValue("src-1")
- .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
- .build();
- public static final PropertyDescriptor FLUME_CONFIG = new PropertyDescriptor.Builder()
- .name("Flume Configuration")
- .description("The Flume configuration for the source copied from the flume.properties file")
- .required(true)
- .defaultValue("")
- .addValidator(Validator.VALID)
- .build();
+ private Source source;
+ private SourceRunner runner;
+ private MemoryChannel channel;
- public static final Relationship SUCCESS = new Relationship.Builder().name("success").build();
+ public static final PropertyDescriptor SOURCE_TYPE = new PropertyDescriptor.Builder()
+ .name("Source Type")
+ .description("The fully-qualified name of the Source class")
+ .required(true)
+ .addValidator(createSourceValidator())
+ .build();
+ public static final PropertyDescriptor AGENT_NAME = new PropertyDescriptor.Builder()
+ .name("Agent Name")
+ .description("The name of the agent used in the Flume source configuration")
+ .required(true)
+ .defaultValue("tier1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor SOURCE_NAME = new PropertyDescriptor.Builder()
+ .name("Source Name")
+ .description("The name of the source used in the Flume source configuration")
+ .required(true)
+ .defaultValue("src-1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
+ public static final PropertyDescriptor FLUME_CONFIG = new PropertyDescriptor.Builder()
+ .name("Flume Configuration")
+ .description("The Flume configuration for the source copied from the flume.properties file")
+ .required(true)
+ .defaultValue("")
+ .addValidator(Validator.VALID)
+ .build();
- private List descriptors;
- private Set relationships;
+ public static final Relationship SUCCESS = new Relationship.Builder().name("success").build();
+ private List descriptors;
+ private Set relationships;
- @Override
- protected void init(final ProcessorInitializationContext context) {
- this.descriptors = ImmutableList.of(SOURCE_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG);
- this.relationships = ImmutableSet.of(SUCCESS);
- }
-
- @Override
- protected List getSupportedPropertyDescriptors() {
- return descriptors;
- }
-
- @Override
- public Set getRelationships() {
- return relationships;
- }
-
- @OnScheduled
- public void onScheduled(final SchedulingContext context) {
- source = SOURCE_FACTORY.create(
- context.getProperty(SOURCE_NAME).getValue(),
- context.getProperty(SOURCE_TYPE).getValue());
-
- String flumeConfig = context.getProperty(FLUME_CONFIG).getValue();
- String agentName = context.getProperty(AGENT_NAME).getValue();
- String sourceName = context.getProperty(SOURCE_NAME).getValue();
- Configurables.configure(source,
- getFlumeSourceContext(flumeConfig, agentName, sourceName) );
-
- if (source instanceof EventDrivenSource) {
- runner = new EventDrivenSourceRunner();
- channel = new MemoryChannel();
- Configurables.configure(channel, new Context());
- channel.start();
- source.setChannelProcessor(new ChannelProcessor(new NifiChannelSelector(channel)));
- runner.setSource(source);
- runner.start();
- }
- }
-
- @OnUnscheduled
- public void unScheduled() {
- if (runner != null) {
- runner.stop();
+ @Override
+ protected void init(final ProcessorInitializationContext context) {
+ this.descriptors = ImmutableList.of(SOURCE_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG);
+ this.relationships = ImmutableSet.of(SUCCESS);
}
- if (channel != null) {
- channel.stop();
- }
- }
- @Override
- public void onTrigger(final ProcessContext context,
- final ProcessSession session) throws ProcessException {
- if (source instanceof EventDrivenSource) {
- onEventDrivenTrigger(context, session);
- } else if (source instanceof PollableSource) {
- onPollableTrigger((PollableSource)source, context, session);
+ @Override
+ protected List getSupportedPropertyDescriptors() {
+ return descriptors;
}
- }
- public void onPollableTrigger(final PollableSource pollableSource,
- final ProcessContext context, final ProcessSession session)
- throws ProcessException {
- try {
- pollableSource.setChannelProcessor(new ChannelProcessor(
- new NifiChannelSelector(new NifiChannel(session, SUCCESS))));
- pollableSource.start();
- pollableSource.process();
- pollableSource.stop();
- } catch (EventDeliveryException ex) {
- throw new ProcessException("Error processing pollable source", ex);
+ @Override
+ public Set getRelationships() {
+ return relationships;
}
- }
- public void onEventDrivenTrigger(final ProcessContext context, final ProcessSession session) {
- Transaction transaction = channel.getTransaction();
- transaction.begin();
+ @OnScheduled
+ public void onScheduled(final SchedulingContext context) {
+ try {
+ source = SOURCE_FACTORY.create(
+ context.getProperty(SOURCE_NAME).getValue(),
+ context.getProperty(SOURCE_TYPE).getValue());
- Event event = channel.take();
- if (event != null) {
- transferEvent(event, session, SUCCESS);
+ String flumeConfig = context.getProperty(FLUME_CONFIG).getValue();
+ String agentName = context.getProperty(AGENT_NAME).getValue();
+ String sourceName = context.getProperty(SOURCE_NAME).getValue();
+ Configurables.configure(source,
+ getFlumeSourceContext(flumeConfig, agentName, sourceName));
+
+ if (source instanceof EventDrivenSource) {
+ runner = new EventDrivenSourceRunner();
+ channel = new MemoryChannel();
+ Configurables.configure(channel, new Context());
+ channel.start();
+ source.setChannelProcessor(new ChannelProcessor(new NifiChannelSelector(channel)));
+ runner.setSource(source);
+ runner.start();
+ }
+ } catch (Throwable th) {
+ getLogger().error("Error creating source", th);
+ throw Throwables.propagate(th);
+ }
+ }
+
+ @OnUnscheduled
+ public void unScheduled() {
+ if (runner != null) {
+ runner.stop();
+ }
+ if (channel != null) {
+ channel.stop();
+ }
+ }
+
+ @Override
+ public void onTrigger(final ProcessContext context,
+ final ProcessSession session) throws ProcessException {
+ if (source instanceof EventDrivenSource) {
+ onEventDrivenTrigger(context, session);
+ } else if (source instanceof PollableSource) {
+ onPollableTrigger((PollableSource) source, context, session);
+ }
+ }
+
+ public void onPollableTrigger(final PollableSource pollableSource,
+ final ProcessContext context, final ProcessSession session)
+ throws ProcessException {
+ try {
+ pollableSource.setChannelProcessor(new ChannelProcessor(
+ new NifiChannelSelector(new NifiChannel(session, SUCCESS))));
+ pollableSource.start();
+ pollableSource.process();
+ pollableSource.stop();
+ } catch (EventDeliveryException ex) {
+ throw new ProcessException("Error processing pollable source", ex);
+ }
+ }
+
+ public void onEventDrivenTrigger(final ProcessContext context, final ProcessSession session) {
+ Transaction transaction = channel.getTransaction();
+ transaction.begin();
+
+ try {
+ Event event = channel.take();
+ if (event != null) {
+ transferEvent(event, session, SUCCESS);
+ }
+ transaction.commit();
+ } catch (Throwable th) {
+ transaction.rollback();
+ throw Throwables.propagate(th);
+ } finally {
+ transaction.close();
+ }
}
- }
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
index 4f2cef778a..8d40cb6314 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
@@ -18,7 +18,6 @@ package org.apache.nifi.processors.flume;
import java.io.File;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
import java.io.FileInputStream;
import java.io.FilenameFilter;
@@ -40,7 +39,6 @@ import org.apache.nifi.util.TestRunners;
import org.apache.nifi.util.file.FileUtils;
import org.junit.Assert;
-import org.junit.Ignore;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/nifi/pom.xml b/nifi/pom.xml
index 0c71ba8ae5..422e1aac60 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -805,6 +805,11 @@
nifi-geo-nar0.2.0-incubating-SNAPSHOTnar
+
+ org.apache.nifi
+ nifi-flume-nar
+ 0.0.1-incubating-SNAPSHOT
+ narorg.apache.nifi
From 3b9e48246641e617cd3fef987ce6facd283f6f3e Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Tue, 7 Apr 2015 17:18:45 -0700
Subject: [PATCH 09/77] Fix poms, versions, add batching to sink processor
* Fix pom issues caused by the rebase.
* Update the Flume bundle's version to 0.1.0
* Add support for batching to the sink processor
Signed-off-by: Matt Gilman
---
nifi/nifi-assembly/pom.xml | 2 +
.../nifi-flume-bundle/nifi-flume-nar/pom.xml | 4 +-
.../nifi-flume-processors/pom.xml | 2 +-
.../flume/AbstractFlumeProcessor.java | 6 ---
.../processors/flume/FlumeSinkProcessor.java | 46 ++++++++++++++++---
.../flume/FlumeSinkProcessorTest.java | 13 ++++++
.../nifi-flume-bundle/pom.xml | 6 +--
nifi/pom.xml | 2 +-
8 files changed, 61 insertions(+), 20 deletions(-)
diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml
index 4f4879fefe..0293c70013 100644
--- a/nifi/nifi-assembly/pom.xml
+++ b/nifi/nifi-assembly/pom.xml
@@ -165,6 +165,8 @@ language governing permissions and limitations under the License. -->
org.apache.nifinifi-kite-nar
+ nar
+ org.apache.nifinifi-flume-nar
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
index dff440eb7f..36a5170e5e 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
@@ -17,10 +17,10 @@
org.apache.nifinifi-flume-bundle
- 0.0.1-incubating-SNAPSHOT
+ 0.1.0-incubating-SNAPSHOTnifi-flume-nar
- 0.0.1-incubating-SNAPSHOT
+ 0.1.0-incubating-SNAPSHOTnar
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
index 54636caf76..bd26a9916c 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
@@ -17,7 +17,7 @@
org.apache.nifinifi-flume-bundle
- 0.0.1-incubating-SNAPSHOT
+ 0.1.0-incubating-SNAPSHOTnifi-flume-processorsjar
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
index 5c608d5ecd..a8310008dd 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
@@ -20,20 +20,15 @@ import com.google.common.collect.Maps;
import java.io.IOException;
import java.io.OutputStream;
import java.io.StringReader;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
import java.util.Map;
import java.util.Properties;
import org.apache.flume.Context;
import org.apache.flume.Event;
import org.apache.flume.SinkFactory;
-import org.apache.flume.Source;
import org.apache.flume.SourceFactory;
import org.apache.flume.sink.DefaultSinkFactory;
import org.apache.flume.source.DefaultSourceFactory;
-import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
@@ -42,7 +37,6 @@ import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.io.OutputStreamCallback;
-import static org.apache.nifi.processors.flume.FlumeSourceProcessor.FLUME_CONFIG;
import org.apache.nifi.processors.flume.util.FlowFileEvent;
/**
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
index fc97ae8685..0ffd4f1c27 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
@@ -19,6 +19,7 @@ package org.apache.nifi.processors.flume;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
import java.util.List;
import java.util.Set;
import org.apache.flume.Context;
@@ -27,6 +28,7 @@ import org.apache.flume.Sink;
import org.apache.flume.Transaction;
import org.apache.flume.channel.MemoryChannel;
import org.apache.flume.conf.Configurables;
+import org.apache.jasper.compiler.JspUtil;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
@@ -81,16 +83,24 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
.defaultValue("")
.addValidator(Validator.VALID)
.build();
+ public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
+ .name("Batch Size")
+ .description("The number of FlowFiles to process in a single batch")
+ .required(true)
+ .defaultValue("100")
+ .addValidator(StandardValidators.INTEGER_VALIDATOR)
+ .build();
public static final Relationship SUCCESS = new Relationship.Builder().name("success").build();
public static final Relationship FAILURE = new Relationship.Builder().name("failure").build();
private List descriptors;
private Set relationships;
+ private int batchSize;
@Override
protected void init(final ProcessorInitializationContext context) {
- this.descriptors = ImmutableList.of(SINK_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG);
+ this.descriptors = ImmutableList.of(SINK_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG, BATCH_SIZE);
this.relationships = ImmutableSet.of(SUCCESS, FAILURE);
}
@@ -106,9 +116,14 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
@OnScheduled
public void onScheduled(final SchedulingContext context) {
+ batchSize = context.getProperty(BATCH_SIZE).asInteger();
+
try {
channel = new MemoryChannel();
- Configurables.configure(channel, new Context());
+ Context memoryChannelContext = new Context();
+ memoryChannelContext.put("capacity", String.valueOf(batchSize*10));
+ memoryChannelContext.put("transactionCapacity", String.valueOf(batchSize*10));
+ Configurables.configure(channel, memoryChannelContext);
channel.start();
sink = SINK_FACTORY.create(context.getProperty(SOURCE_NAME).getValue(),
@@ -137,12 +152,22 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
@Override
public void onTrigger(final ProcessContext context,
final ProcessSession session) throws ProcessException {
- FlowFile flowFile = session.get();
+ List flowFiles = Lists.newArrayListWithExpectedSize(batchSize);
+ for (int i = 0; i < batchSize; i++) {
+ FlowFile flowFile = session.get();
+ if (flowFile == null) {
+ break;
+ }
+
+ flowFiles.add(flowFile);
+ }
Transaction transaction = channel.getTransaction();
try {
transaction.begin();
- channel.put(new FlowFileEvent(flowFile, session));
+ for (FlowFile flowFile : flowFiles) {
+ channel.put(new FlowFileEvent(flowFile, session));
+ }
transaction.commit();
} catch (Throwable th) {
transaction.rollback();
@@ -152,10 +177,17 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
}
try {
- sink.process();
- session.transfer(flowFile, SUCCESS);
+ Sink.Status status;
+ do {
+ status = sink.process();
+ } while(status == Sink.Status.READY);
+ for (FlowFile flowFile : flowFiles) {
+ session.transfer(flowFile, SUCCESS);
+ }
} catch (EventDeliveryException ex) {
- session.transfer(flowFile, FAILURE);
+ for (FlowFile flowFile : flowFiles) {
+ session.transfer(flowFile, FAILURE);
+ }
}
}
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
index 8d40cb6314..d22514f1f3 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
@@ -116,6 +116,19 @@ public class FlumeSinkProcessorTest {
runner.run();
fis.close();
}
+
+ @Test
+ public void testBatchSize() throws IOException {
+ TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
+ runner.setProperty(FlumeSinkProcessor.SINK_TYPE, NullSink.class.getName());
+ runner.setProperty(FlumeSinkProcessor.BATCH_SIZE, "1000");
+ runner.setProperty(FlumeSinkProcessor.FLUME_CONFIG,
+ "tier1.sinks.sink-1.batchSize = 1000\n");
+ for (int i = 0; i < 100000; i++) {
+ runner.enqueue(String.valueOf(i).getBytes());
+ }
+ runner.run();
+ }
@Test
public void testHdfsSink() throws IOException {
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
index dc9ec69deb..50b0fde3a9 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
@@ -17,10 +17,10 @@
org.apache.nifinifi-nar-bundles
- 0.0.1-incubating-SNAPSHOT
+ 0.1.0-incubating-SNAPSHOTnifi-flume-bundle
- 0.0.1-incubating-SNAPSHOT
+ 0.1.0-incubating-SNAPSHOTpomA bundle of processors that run Flume sources/sinks
@@ -32,7 +32,7 @@
org.apache.nifinifi-flume-processors
- 0.0.1-incubating-SNAPSHOT
+ 0.1.0-incubating-SNAPSHOT
diff --git a/nifi/pom.xml b/nifi/pom.xml
index 422e1aac60..682a426aa5 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -808,7 +808,7 @@
org.apache.nifinifi-flume-nar
- 0.0.1-incubating-SNAPSHOT
+ 0.1.0-incubating-SNAPSHOTnar
From 3529bb33178ce70a33cc7ab21d7334ba3411048b Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Fri, 5 Jun 2015 14:00:37 -0700
Subject: [PATCH 10/77] NIFI-589: Updated with first review feedback
* Switched to using `getResourceAsStream()` where possible
* Removed trailing whitespace from added files
* Added missing license headers
* Added RAT exception to testdata files
* Fixed POM errors that broke the build
* Switched to using TemporaryFolder instead of putting files in
`target`
* Used try-with-resources where needed to autoclose streams
* Moved logging configuration to properties files
* Removed AbstractFlumeTest
* Fixed logging levels in test code
Signed-off-by: Matt Gilman
---
.../org/apache/nifi/util/file/FileUtils.java | 27 +++-
.../nifi-flume-processors/pom.xml | 125 ++++++++++--------
.../nifi/processors/flume/NifiChannel.java | 18 ++-
.../processors/flume/NifiChannelSelector.java | 18 ++-
.../processors/flume/NifiTransaction.java | 17 ++-
.../processors/flume/util/FlowFileEvent.java | 21 ++-
.../flume/util/FlowFileEventConstants.java | 17 ++-
.../processors/flume/AbstractFlumeTest.java | 35 -----
.../flume/FlumeSinkProcessorTest.java | 59 +++++----
.../flume/FlumeSourceProcessorTest.java | 25 ++--
.../src/test/resources/core-site.xml | 5 +
.../src/test/resources/log4j.properties | 20 +++
.../test/resources/simplelogger.properties | 20 +++
nifi/pom.xml | 1 +
14 files changed, 260 insertions(+), 148 deletions(-)
delete mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/AbstractFlumeTest.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/log4j.properties
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/simplelogger.properties
diff --git a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
index 7661e2dc41..ff4da8e5a2 100644
--- a/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
+++ b/nifi/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/file/FileUtils.java
@@ -560,6 +560,19 @@ public class FileUtils {
* @throws IOException if the MD5 hash could not be computed
*/
public static byte[] computeMd5Digest(final File file) throws IOException {
+ try (final FileInputStream fis = new FileInputStream(file)) {
+ return computeMd5Digest(fis);
+ }
+ }
+
+ /**
+ * Returns the MD5 hash of the given stream.
+ *
+ * @param stream an input stream
+ * @return the MD5 hash
+ * @throws IOException if the MD5 hash could not be computed
+ */
+ public static byte[] computeMd5Digest(final InputStream stream) throws IOException {
final MessageDigest digest;
try {
digest = MessageDigest.getInstance("MD5");
@@ -567,15 +580,15 @@ public class FileUtils {
throw new IOException(nsae);
}
- try (final FileInputStream fis = new FileInputStream(file)) {
- int len;
- final byte[] buffer = new byte[8192];
- while ((len = fis.read(buffer)) > -1) {
- if (len > 0) {
- digest.update(buffer, 0, len);
- }
+
+ int len;
+ final byte[] buffer = new byte[8192];
+ while ((len = stream.read(buffer)) > -1) {
+ if (len > 0) {
+ digest.update(buffer, 0, len);
}
}
+
return digest.digest();
}
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
index bd26a9916c..b903f21833 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
@@ -38,14 +38,14 @@
org.apache.nifinifi-flowfile-packager
-
- org.apache.flume
- flume-ng-sdk
+
+ org.apache.flume
+ flume-ng-sdk1.5.2
-
-
- org.apache.flume
- flume-ng-core
+
+
+ org.apache.flume
+ flume-ng-core1.5.2
@@ -53,74 +53,89 @@
slf4j-log4j12
-
+
-
- org.apache.flume.flume-ng-sources
- flume-twitter-source
+
+ org.apache.flume.flume-ng-sources
+ flume-twitter-source1.5.2
-
-
- org.apache.flume.flume-ng-sources
- flume-jms-source
+
+
+ org.apache.flume.flume-ng-sources
+ flume-jms-source1.5.2
-
-
- org.apache.flume.flume-ng-sources
- flume-scribe-source
+
+
+ org.apache.flume.flume-ng-sources
+ flume-scribe-source1.5.2
-
+
-
- org.apache.flume.flume-ng-sinks
- flume-hdfs-sink
+
+ org.apache.flume.flume-ng-sinks
+ flume-hdfs-sink1.5.2
-
+
-
- org.apache.hadoop
- hadoop-common
- provided
-
-
- org.apache.hadoop
- hadoop-hdfs
- provided
+
+ org.apache.hadoop
+ hadoop-common
+ provided
+
+
+ org.slf4j
+ slf4j-log4j12
+
+
-
-
- org.apache.flume.flume-ng-sinks
- flume-irc-sink
+
+ org.apache.hadoop
+ hadoop-hdfs
+ provided
+
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-irc-sink1.5.2
-
-
- org.apache.flume.flume-ng-sinks
- flume-ng-elasticsearch-sink
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-ng-elasticsearch-sink1.5.2
-
-
- org.apache.flume.flume-ng-sinks
- flume-ng-hbase-sink
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-ng-hbase-sink1.5.2
-
-
- org.apache.flume.flume-ng-sinks
- flume-ng-morphline-solr-sink
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-ng-morphline-solr-sink1.5.2
-
+ org.apache.nifinifi-mocktest
-
- org.slf4j
- slf4j-simple
- test
-
+
+
+
+
+ org.apache.rat
+ apache-rat-plugin
+
+
+ src/test/resources/testdata/*
+
+
+
+
+
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java
index ac8dbe2b1d..c4d3bef4d0 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java
@@ -1,4 +1,19 @@
-
+/*
+ * 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.flume;
import org.apache.flume.Context;
@@ -24,7 +39,6 @@ public class NifiChannel extends BasicChannelSemantics {
@Override
public void configure(Context context) {
- throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannelSelector.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannelSelector.java
index 792678bd59..2b0ba77616 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannelSelector.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannelSelector.java
@@ -1,4 +1,19 @@
-
+/*
+ * 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.flume;
import com.google.common.collect.ImmutableList;
@@ -36,7 +51,6 @@ public class NifiChannelSelector implements ChannelSelector {
@Override
public void setChannels(List channels) {
- throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
}
@Override
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java
index 3d6a647f85..37c8a50902 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java
@@ -1,4 +1,19 @@
-
+/*
+ * 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.flume;
import org.apache.flume.Event;
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
index c3531ca12e..5dc97d65e7 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
@@ -1,4 +1,19 @@
-
+/*
+ * 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.flume.util;
import com.google.common.collect.Maps;
@@ -56,7 +71,7 @@ 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;
}
}
@@ -83,7 +98,7 @@ public class FlowFileEvent implements Event {
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)");
}
-
+
final ByteArrayOutputStream baos = new ByteArrayOutputStream((int) flowFile.getSize());
session.read(flowFile, new InputStreamCallback() {
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java
index c13f0ef6f2..c9650c1e87 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEventConstants.java
@@ -1,4 +1,19 @@
-
+/*
+ * 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.flume.util;
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/AbstractFlumeTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/AbstractFlumeTest.java
deleted file mode 100644
index 87b056a86e..0000000000
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/AbstractFlumeTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.flume;
-
-import org.junit.BeforeClass;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class AbstractFlumeTest {
-
- private static Logger logger;
-
- @BeforeClass
- public static void setUpClass() {
- System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
- System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
- System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.flume", "debug");
- logger = LoggerFactory.getLogger(AbstractFlumeTest.class);
- }
-
-}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
index d22514f1f3..2e10c24a7d 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
@@ -19,9 +19,9 @@ package org.apache.nifi.processors.flume;
import java.io.File;
import static org.junit.Assert.assertEquals;
-import java.io.FileInputStream;
import java.io.FilenameFilter;
import java.io.IOException;
+import java.io.InputStream;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
@@ -39,15 +39,20 @@ import org.apache.nifi.util.TestRunners;
import org.apache.nifi.util.file.FileUtils;
import org.junit.Assert;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class FlumeSinkProcessorTest {
- private static final Logger logger =
- LoggerFactory.getLogger(FlumeSinkProcessorTest.class);
-
+ private static final Logger logger =
+ LoggerFactory.getLogger(FlumeSinkProcessorTest.class);
+
+ @Rule
+ public final TemporaryFolder temp = new TemporaryFolder();
+
@Test
public void testValidators() {
TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
@@ -62,7 +67,7 @@ public class FlumeSinkProcessorTest {
}
Assert.assertEquals(1, results.size());
for (ValidationResult vr : results) {
- logger.error(vr.toString());
+ logger.debug(vr.toString());
Assert.assertTrue(vr.toString().contains("is invalid because Sink Type is required"));
}
@@ -76,7 +81,7 @@ public class FlumeSinkProcessorTest {
}
Assert.assertEquals(1, results.size());
for (ValidationResult vr : results) {
- logger.error(vr.toString());
+ logger.debug(vr.toString());
Assert.assertTrue(vr.toString().contains("is invalid because unable to load sink"));
}
@@ -90,7 +95,7 @@ public class FlumeSinkProcessorTest {
}
Assert.assertEquals(1, results.size());
for (ValidationResult vr : results) {
- logger.error(vr.toString());
+ logger.debug(vr.toString());
Assert.assertTrue(vr.toString().contains("is invalid because unable to create sink"));
}
@@ -109,12 +114,12 @@ public class FlumeSinkProcessorTest {
public void testNullSink() throws IOException {
TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
runner.setProperty(FlumeSinkProcessor.SINK_TYPE, NullSink.class.getName());
- FileInputStream fis = new FileInputStream("src/test/resources/testdata/records.txt");
- Map attributes = new HashMap<>();
- attributes.put(CoreAttributes.FILENAME.key(), "records.txt");
- runner.enqueue(fis, attributes);
- runner.run();
- fis.close();
+ try (InputStream inputStream = getClass().getResourceAsStream("/testdata/records.txt")) {
+ Map attributes = new HashMap<>();
+ attributes.put(CoreAttributes.FILENAME.key(), "records.txt");
+ runner.enqueue(inputStream, attributes);
+ runner.run();
+ }
}
@Test
@@ -129,15 +134,10 @@ public class FlumeSinkProcessorTest {
}
runner.run();
}
-
+
@Test
public void testHdfsSink() throws IOException {
- File destDir = new File("target/hdfs");
- if (destDir.exists()) {
- FileUtils.deleteFilesInDir(destDir, null, logger);
- } else {
- destDir.mkdirs();
- }
+ File destDir = temp.newFolder("hdfs");
TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
runner.setProperty(FlumeSinkProcessor.SINK_TYPE, "hdfs");
@@ -147,19 +147,22 @@ public class FlumeSinkProcessorTest {
"tier1.sinks.sink-1.hdfs.serializer = TEXT\n" +
"tier1.sinks.sink-1.serializer.appendNewline = false"
);
- FileInputStream fis = new FileInputStream("src/test/resources/testdata/records.txt");
- Map attributes = new HashMap<>();
- attributes.put(CoreAttributes.FILENAME.key(), "records.txt");
- runner.enqueue(fis, attributes);
- runner.run();
- fis.close();
+ try (InputStream inputStream = getClass().getResourceAsStream("/testdata/records.txt")) {
+ Map attributes = new HashMap<>();
+ attributes.put(CoreAttributes.FILENAME.key(), "records.txt");
+ runner.enqueue(inputStream, attributes);
+ runner.run();
+ }
File[] files = destDir.listFiles((FilenameFilter)HiddenFileFilter.VISIBLE);
assertEquals("Unexpected number of destination files.", 1, files.length);
File dst = files[0];
- byte[] expectedMd5 = FileUtils.computeMd5Digest(new File("src/test/resources/testdata/records.txt"));
+ byte[] expectedMd5;
+ try (InputStream md5Stream = getClass().getResourceAsStream("/testdata/records.txt")) {
+ expectedMd5 = FileUtils.computeMd5Digest(md5Stream);
+ }
byte[] actualMd5 = FileUtils.computeMd5Digest(dst);
Assert.assertArrayEquals("Destination file doesn't match source data", expectedMd5, actualMd5);
}
-
+
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
index bbcf116787..043e1154fb 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
@@ -34,15 +34,18 @@ import org.apache.nifi.util.TestRunners;
import org.apache.nifi.util.file.FileUtils;
import org.junit.Assert;
+import org.junit.Rule;
import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class FlumeSourceProcessorTest {
- private static final Logger logger =
- LoggerFactory.getLogger(FlumeSourceProcessorTest.class);
+ private static final Logger logger = LoggerFactory.getLogger(FlumeSourceProcessorTest.class);
+ @Rule
+ public final TemporaryFolder temp = new TemporaryFolder();
@Test
public void testValidators() {
@@ -58,7 +61,7 @@ public class FlumeSourceProcessorTest {
}
Assert.assertEquals(1, results.size());
for (ValidationResult vr : results) {
- logger.error(vr.toString());
+ logger.debug(vr.toString());
Assert.assertTrue(vr.toString().contains("is invalid because Source Type is required"));
}
@@ -72,7 +75,7 @@ public class FlumeSourceProcessorTest {
}
Assert.assertEquals(1, results.size());
for (ValidationResult vr : results) {
- logger.error(vr.toString());
+ logger.debug(vr.toString());
Assert.assertTrue(vr.toString().contains("is invalid because unable to load source"));
}
@@ -86,7 +89,7 @@ public class FlumeSourceProcessorTest {
}
Assert.assertEquals(1, results.size());
for (ValidationResult vr : results) {
- logger.error(vr.toString());
+ logger.debug(vr.toString());
Assert.assertTrue(vr.toString().contains("is invalid because unable to create source"));
}
@@ -108,22 +111,16 @@ public class FlumeSourceProcessorTest {
List flowFiles = runner.getFlowFilesForRelationship(FlumeSourceProcessor.SUCCESS);
Assert.assertEquals(1, flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
- logger.error(flowFile.toString());
+ logger.debug(flowFile.toString());
Assert.assertEquals(1, flowFile.getSize());
}
}
@Test
public void testSourceWithConfig() throws IOException {
- File spoolDirectory = new File("target/spooldir");
- if (spoolDirectory.exists()) {
- FileUtils.deleteFilesInDir(spoolDirectory, null, logger);
- } else {
- spoolDirectory.mkdirs();
- }
- File src = new File("src/test/resources/testdata/records.txt");
+ File spoolDirectory = temp.newFolder("spooldir");
File dst = new File(spoolDirectory, "records.txt");
- FileUtils.copyFile(src, dst, false, false, logger);
+ FileUtils.copyFile(getClass().getResourceAsStream("/testdata/records.txt"), dst, true, false);
TestRunner runner = TestRunners.newTestRunner(FlumeSourceProcessor.class);
runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, "spooldir");
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site.xml
index 5e3b55cb0b..849854b77a 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/core-site.xml
@@ -20,6 +20,11 @@
fs.defaultFS
+
file:///
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/log4j.properties b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/log4j.properties
new file mode 100644
index 0000000000..8c502ec918
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/log4j.properties
@@ -0,0 +1,20 @@
+# 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.
+log4j.rootLogger=INFO, CONSOLE
+
+log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
+
+log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
+log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x \u2013 %m%n
\ No newline at end of file
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/simplelogger.properties b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/simplelogger.properties
new file mode 100644
index 0000000000..4994e7f999
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/simplelogger.properties
@@ -0,0 +1,20 @@
+# 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.
+org.slf4j.simpleLogger.logFile=System.out
+org.slf4j.simpleLogger.defaultLogLevel=info
+org.slf4j.simpleLogger.showDateTime=true
+org.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd HH:mm:ss.SSS
+org.slf4j.simpleLogger.levelInBrackets=true
+org.slf4j.simpleLogger.log.org.apache.nifi.processors.flume=debug
diff --git a/nifi/pom.xml b/nifi/pom.xml
index 682a426aa5..3a5c6b78b0 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -805,6 +805,7 @@
nifi-geo-nar0.2.0-incubating-SNAPSHOTnar
+ org.apache.nifinifi-flume-nar
From 419f9455a21eeb2b0bf6360204ee1175b62df805 Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Fri, 5 Jun 2015 15:32:16 -0700
Subject: [PATCH 11/77] NIFI-589: Set java.io.tmpdir to
${build.project.directory}
This lets us use TemporaryDirectory and get the benefits of cleaning
up temporary files using `mvn clean`
Signed-off-by: Matt Gilman
---
.../nifi-flume-bundle/nifi-flume-processors/pom.xml | 9 +++++++++
1 file changed, 9 insertions(+)
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
index b903f21833..b0e730c4a6 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
@@ -136,6 +136,15 @@
+
+ org.apache.maven.plugins
+ maven-surefire-plugin
+
+
+ ${project.build.directory}
+
+
+
From 3af73c9b824f5131eac9ea37045daec88b33a293 Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Wed, 10 Jun 2015 12:49:50 -0700
Subject: [PATCH 12/77] Updated with Ryan's feedback:
* Moved away from any reliance on MemoryChannels in favor of
modeling the ProcessSession/Relationship as the channel directly
in all cases.
* Fixed version numbers in nifi-flume-* pom files.
Signed-off-by: Matt Gilman
---
.../nifi-flume-bundle/nifi-flume-nar/pom.xml | 4 +-
.../nifi-flume-processors/pom.xml | 2 +-
.../flume/AbstractFlumeProcessor.java | 188 ++++++++++-------
.../processors/flume/FlumeSinkProcessor.java | 69 +------
.../flume/FlumeSourceProcessor.java | 192 ++++++++++--------
...fiChannel.java => NifiSessionChannel.java} | 30 +--
.../flume/NifiSessionFactoryChannel.java | 51 +++++
.../flume/NifiSinkSessionChannel.java | 49 +++++
.../processors/flume/NifiSinkTransaction.java | 71 +++++++
.../processors/flume/NifiTransaction.java | 45 ++--
.../flume/FlumeSinkProcessorTest.java | 3 +-
.../flume/FlumeSourceProcessorTest.java | 9 +-
.../src/test/resources/log4j.properties | 4 +-
.../test/resources/simplelogger.properties | 1 +
.../nifi-flume-bundle/pom.xml | 4 +-
15 files changed, 445 insertions(+), 277 deletions(-)
rename nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/{NifiChannel.java => NifiSessionChannel.java} (66%)
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionFactoryChannel.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSinkSessionChannel.java
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSinkTransaction.java
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
index 36a5170e5e..c07cedf690 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
@@ -17,10 +17,10 @@
org.apache.nifinifi-flume-bundle
- 0.1.0-incubating-SNAPSHOT
+ 0.2.0-incubating-SNAPSHOTnifi-flume-nar
- 0.1.0-incubating-SNAPSHOT
+ 0.2.0-incubating-SNAPSHOTnar
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
index b0e730c4a6..1dad25f6df 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
@@ -17,7 +17,7 @@
org.apache.nifinifi-flume-bundle
- 0.1.0-incubating-SNAPSHOT
+ 0.2.0-incubating-SNAPSHOTnifi-flume-processorsjar
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
index a8310008dd..83ae9e1aeb 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
@@ -33,96 +33,128 @@ import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.components.Validator;
import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.OutputStreamCallback;
import org.apache.nifi.processors.flume.util.FlowFileEvent;
/**
- * This is a base class that is helpful when building processors interacting
- * with Flume.
+ * This is a base class that is helpful when building processors interacting with Flume.
*/
-public abstract class AbstractFlumeProcessor extends AbstractProcessor {
- protected static final SourceFactory SOURCE_FACTORY = new DefaultSourceFactory();
- protected static final SinkFactory SINK_FACTORY = new DefaultSinkFactory();
+public abstract class AbstractFlumeProcessor extends AbstractSessionFactoryProcessor {
- protected static Event flowFileToEvent(FlowFile flowFile, ProcessSession session) {
- return new FlowFileEvent(flowFile, session);
- }
+ protected static final SourceFactory SOURCE_FACTORY = new DefaultSourceFactory();
+ protected static final SinkFactory SINK_FACTORY = new DefaultSinkFactory();
- protected static void transferEvent(final Event event, ProcessSession session,
- Relationship relationship) {
- FlowFile flowFile = session.create();
- flowFile = session.putAllAttributes(flowFile, event.getHeaders());
-
- flowFile = session.write(flowFile, new OutputStreamCallback() {
- @Override
- public void process(final OutputStream out) throws IOException {
- out.write(event.getBody());
- }
- });
-
- session.getProvenanceReporter().create(flowFile);
- session.transfer(flowFile, relationship);
- }
-
- protected static Validator createSourceValidator() {
- return new Validator() {
- @Override
- public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
- String reason = null;
- try {
- FlumeSourceProcessor.SOURCE_FACTORY.create("NiFi Source", value);
- } catch (Exception ex) {
- reason = ex.getLocalizedMessage();
- reason = Character.toLowerCase(reason.charAt(0)) + reason.substring(1);
- }
- return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
- }
- };
- }
-
- protected static Validator createSinkValidator() {
- return new Validator() {
- @Override
- public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
- String reason = null;
- try {
- FlumeSinkProcessor.SINK_FACTORY.create("NiFi Sink", value);
- } catch (Exception ex) {
- reason = ex.getLocalizedMessage();
- reason = Character.toLowerCase(reason.charAt(0)) + reason.substring(1);
- }
- return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build();
- }
- };
- }
-
- protected static Context getFlumeContext(String flumeConfig, String prefix) {
- Properties flumeProperties = new Properties();
- if (flumeConfig != null) {
- try {
- flumeProperties.load(new StringReader(flumeConfig));
- } catch (IOException ex) {
- throw new RuntimeException(ex);
- }
+ protected static Event flowFileToEvent(FlowFile flowFile, ProcessSession session) {
+ return new FlowFileEvent(flowFile, session);
}
- Map parameters = Maps.newHashMap();
- for (String property : flumeProperties.stringPropertyNames()) {
- parameters.put(property, flumeProperties.getProperty(property));
+
+ protected static void transferEvent(final Event event, ProcessSession session,
+ Relationship relationship) {
+ FlowFile flowFile = session.create();
+ flowFile = session.putAllAttributes(flowFile, event.getHeaders());
+
+ flowFile = session.write(flowFile, new OutputStreamCallback() {
+ @Override
+ public void process(final OutputStream out) throws IOException {
+ out.write(event.getBody());
+ }
+ });
+
+ session.getProvenanceReporter()
+ .create(flowFile);
+ session.transfer(flowFile, relationship);
}
- return new Context(new Context(parameters).getSubProperties(prefix));
- }
- protected static Context getFlumeSourceContext(String flumeConfig,
- String agentName, String sourceName) {
- return getFlumeContext(flumeConfig, agentName + ".sources." + sourceName + ".");
- }
+ protected static Validator createSourceValidator() {
+ return new Validator() {
+ @Override
+ public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
+ String reason = null;
+ try {
+ FlumeSourceProcessor.SOURCE_FACTORY.create("NiFi Source", value);
+ } catch (Exception ex) {
+ reason = ex.getLocalizedMessage();
+ reason = Character.toLowerCase(reason.charAt(0)) + reason.substring(1);
+ }
+ return new ValidationResult.Builder().subject(subject)
+ .input(value)
+ .explanation(reason)
+ .valid(reason == null)
+ .build();
+ }
+ };
+ }
- protected static Context getFlumeSinkContext(String flumeConfig,
- String agentName, String sinkName) {
- return getFlumeContext(flumeConfig, agentName + ".sinks." + sinkName + ".");
- }
+ protected static Validator createSinkValidator() {
+ return new Validator() {
+ @Override
+ public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
+ String reason = null;
+ try {
+ FlumeSinkProcessor.SINK_FACTORY.create("NiFi Sink", value);
+ } catch (Exception ex) {
+ reason = ex.getLocalizedMessage();
+ reason = Character.toLowerCase(reason.charAt(0)) + reason.substring(1);
+ }
+ return new ValidationResult.Builder().subject(subject)
+ .input(value)
+ .explanation(reason)
+ .valid(reason == null)
+ .build();
+ }
+ };
+ }
+
+ protected static Context getFlumeContext(String flumeConfig, String prefix) {
+ Properties flumeProperties = new Properties();
+ if (flumeConfig != null) {
+ try {
+ flumeProperties.load(new StringReader(flumeConfig));
+ } catch (IOException ex) {
+ throw new RuntimeException(ex);
+ }
+ }
+ Map parameters = Maps.newHashMap();
+ for (String property : flumeProperties.stringPropertyNames()) {
+ parameters.put(property, flumeProperties.getProperty(property));
+ }
+ return new Context(new Context(parameters).getSubProperties(prefix));
+ }
+
+ protected static Context getFlumeSourceContext(String flumeConfig,
+ String agentName, String sourceName) {
+ return getFlumeContext(flumeConfig, agentName + ".sources." + sourceName + ".");
+ }
+
+ protected static Context getFlumeSinkContext(String flumeConfig,
+ String agentName, String sinkName) {
+ return getFlumeContext(flumeConfig, agentName + ".sinks." + sinkName + ".");
+ }
+
+ /*
+ * Borrowed from AbstractProcessor. The FlumeSourceProcessor needs to implement this directly
+ * to handle event driven sources, but it's marked final in AbstractProcessor.
+ */
+ @Override
+ public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException {
+ final ProcessSession session = sessionFactory.createSession();
+ try {
+ onTrigger(context, session);
+ session.commit();
+ } catch (final Throwable t) {
+ getLogger()
+ .error("{} failed to process due to {}; rolling back session", new Object[]{this, t});
+ session.rollback(true);
+ throw t;
+ }
+ }
+
+ public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException;
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
index 0ffd4f1c27..e385921d04 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
@@ -19,16 +19,12 @@ package org.apache.nifi.processors.flume;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
import java.util.List;
import java.util.Set;
import org.apache.flume.Context;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.Sink;
-import org.apache.flume.Transaction;
-import org.apache.flume.channel.MemoryChannel;
import org.apache.flume.conf.Configurables;
-import org.apache.jasper.compiler.JspUtil;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
@@ -36,7 +32,6 @@ 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;
import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext;
@@ -44,18 +39,14 @@ import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SchedulingContext;
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
-import org.apache.nifi.processors.flume.util.FlowFileEvent;
/**
* This processor runs a Flume sink
*/
@Tags({"flume", "hadoop", "get", "sink"})
-@CapabilityDescription("Generate FlowFile data from a Flume sink")
+@CapabilityDescription("Write FlowFile data to a Flume sink")
public class FlumeSinkProcessor extends AbstractFlumeProcessor {
- private Sink sink;
- private MemoryChannel channel;
-
public static final PropertyDescriptor SINK_TYPE = new PropertyDescriptor.Builder()
.name("Sink Type")
.description("The fully-qualified name of the Sink class")
@@ -83,24 +74,19 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
.defaultValue("")
.addValidator(Validator.VALID)
.build();
- public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
- .name("Batch Size")
- .description("The number of FlowFiles to process in a single batch")
- .required(true)
- .defaultValue("100")
- .addValidator(StandardValidators.INTEGER_VALIDATOR)
- .build();
public static final Relationship SUCCESS = new Relationship.Builder().name("success").build();
public static final Relationship FAILURE = new Relationship.Builder().name("failure").build();
private List descriptors;
private Set relationships;
- private int batchSize;
+
+ private volatile Sink sink;
+ private volatile NifiSinkSessionChannel channel;
@Override
protected void init(final ProcessorInitializationContext context) {
- this.descriptors = ImmutableList.of(SINK_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG, BATCH_SIZE);
+ this.descriptors = ImmutableList.of(SINK_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG);
this.relationships = ImmutableSet.of(SUCCESS, FAILURE);
}
@@ -116,14 +102,9 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
@OnScheduled
public void onScheduled(final SchedulingContext context) {
- batchSize = context.getProperty(BATCH_SIZE).asInteger();
-
try {
- channel = new MemoryChannel();
- Context memoryChannelContext = new Context();
- memoryChannelContext.put("capacity", String.valueOf(batchSize*10));
- memoryChannelContext.put("transactionCapacity", String.valueOf(batchSize*10));
- Configurables.configure(channel, memoryChannelContext);
+ channel = new NifiSinkSessionChannel(SUCCESS, FAILURE);
+ Configurables.configure(channel, new Context());
channel.start();
sink = SINK_FACTORY.create(context.getProperty(SOURCE_NAME).getValue(),
@@ -152,42 +133,14 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
@Override
public void onTrigger(final ProcessContext context,
final ProcessSession session) throws ProcessException {
- List flowFiles = Lists.newArrayListWithExpectedSize(batchSize);
- for (int i = 0; i < batchSize; i++) {
- FlowFile flowFile = session.get();
- if (flowFile == null) {
- break;
- }
- flowFiles.add(flowFile);
- }
-
- Transaction transaction = channel.getTransaction();
+ channel.setSession(session);
try {
- transaction.begin();
- for (FlowFile flowFile : flowFiles) {
- channel.put(new FlowFileEvent(flowFile, session));
- }
- transaction.commit();
- } catch (Throwable th) {
- transaction.rollback();
- throw Throwables.propagate(th);
- } finally {
- transaction.close();
- }
-
- try {
- Sink.Status status;
- do {
- status = sink.process();
- } while(status == Sink.Status.READY);
- for (FlowFile flowFile : flowFiles) {
- session.transfer(flowFile, SUCCESS);
+ if (sink.process() == Sink.Status.BACKOFF) {
+ context.yield();
}
} catch (EventDeliveryException ex) {
- for (FlowFile flowFile : flowFiles) {
- session.transfer(flowFile, FAILURE);
- }
+ throw new ProcessException("Flume event delivery failed", ex);
}
}
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
index 19551e68f0..3ded2085b3 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
@@ -21,27 +21,25 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import java.util.List;
import java.util.Set;
-import org.apache.flume.Context;
-import org.apache.flume.Event;
+import java.util.concurrent.atomic.AtomicReference;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.EventDrivenSource;
import org.apache.flume.PollableSource;
import org.apache.flume.Source;
-import org.apache.flume.SourceRunner;
-import org.apache.flume.Transaction;
import org.apache.flume.channel.ChannelProcessor;
-import org.apache.flume.channel.MemoryChannel;
import org.apache.flume.conf.Configurables;
import org.apache.flume.source.EventDrivenSourceRunner;
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.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.SchedulingContext;
@@ -55,43 +53,48 @@ import org.apache.nifi.processor.util.StandardValidators;
@CapabilityDescription("Generate FlowFile data from a Flume source")
public class FlumeSourceProcessor extends AbstractFlumeProcessor {
- private Source source;
- private SourceRunner runner;
- private MemoryChannel channel;
-
public static final PropertyDescriptor SOURCE_TYPE = new PropertyDescriptor.Builder()
- .name("Source Type")
- .description("The fully-qualified name of the Source class")
- .required(true)
- .addValidator(createSourceValidator())
- .build();
+ .name("Source Type")
+ .description("The fully-qualified name of the Source class")
+ .required(true)
+ .addValidator(createSourceValidator())
+ .build();
public static final PropertyDescriptor AGENT_NAME = new PropertyDescriptor.Builder()
- .name("Agent Name")
- .description("The name of the agent used in the Flume source configuration")
- .required(true)
- .defaultValue("tier1")
- .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
- .build();
+ .name("Agent Name")
+ .description("The name of the agent used in the Flume source configuration")
+ .required(true)
+ .defaultValue("tier1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
public static final PropertyDescriptor SOURCE_NAME = new PropertyDescriptor.Builder()
- .name("Source Name")
- .description("The name of the source used in the Flume source configuration")
- .required(true)
- .defaultValue("src-1")
- .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
- .build();
+ .name("Source Name")
+ .description("The name of the source used in the Flume source configuration")
+ .required(true)
+ .defaultValue("src-1")
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+ .build();
public static final PropertyDescriptor FLUME_CONFIG = new PropertyDescriptor.Builder()
- .name("Flume Configuration")
- .description("The Flume configuration for the source copied from the flume.properties file")
- .required(true)
- .defaultValue("")
- .addValidator(Validator.VALID)
- .build();
+ .name("Flume Configuration")
+ .description("The Flume configuration for the source copied from the flume.properties file")
+ .required(true)
+ .defaultValue("")
+ .addValidator(Validator.VALID)
+ .build();
- public static final Relationship SUCCESS = new Relationship.Builder().name("success").build();
+ public static final Relationship SUCCESS = new Relationship.Builder().name("success")
+ .build();
private List descriptors;
private Set relationships;
+ private volatile Source source;
+
+ private final NifiSessionChannel pollableSourceChannel = new NifiSessionChannel(SUCCESS);
+ private final AtomicReference sessionFactoryRef = new AtomicReference<>(null);
+ private final AtomicReference runnerRef = new AtomicReference<>(null);
+ private final AtomicReference eventDrivenSourceChannelRef = new AtomicReference<>(null);
+ private final AtomicReference stopping = new AtomicReference<>(false);
+
@Override
protected void init(final ProcessorInitializationContext context) {
this.descriptors = ImmutableList.of(SOURCE_TYPE, AGENT_NAME, SOURCE_NAME, FLUME_CONFIG);
@@ -111,81 +114,90 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
@OnScheduled
public void onScheduled(final SchedulingContext context) {
try {
+ stopping.set(false);
source = SOURCE_FACTORY.create(
- context.getProperty(SOURCE_NAME).getValue(),
- context.getProperty(SOURCE_TYPE).getValue());
+ context.getProperty(SOURCE_NAME)
+ .getValue(),
+ context.getProperty(SOURCE_TYPE)
+ .getValue());
- String flumeConfig = context.getProperty(FLUME_CONFIG).getValue();
- String agentName = context.getProperty(AGENT_NAME).getValue();
- String sourceName = context.getProperty(SOURCE_NAME).getValue();
+ String flumeConfig = context.getProperty(FLUME_CONFIG)
+ .getValue();
+ String agentName = context.getProperty(AGENT_NAME)
+ .getValue();
+ String sourceName = context.getProperty(SOURCE_NAME)
+ .getValue();
Configurables.configure(source,
- getFlumeSourceContext(flumeConfig, agentName, sourceName));
+ getFlumeSourceContext(flumeConfig, agentName, sourceName));
- if (source instanceof EventDrivenSource) {
- runner = new EventDrivenSourceRunner();
- channel = new MemoryChannel();
- Configurables.configure(channel, new Context());
- channel.start();
- source.setChannelProcessor(new ChannelProcessor(new NifiChannelSelector(channel)));
- runner.setSource(source);
- runner.start();
+ if (source instanceof PollableSource) {
+ source.setChannelProcessor(new ChannelProcessor(
+ new NifiChannelSelector(pollableSourceChannel)));
+ source.start();
}
} catch (Throwable th) {
- getLogger().error("Error creating source", th);
+ getLogger()
+ .error("Error creating source", th);
throw Throwables.propagate(th);
}
}
@OnUnscheduled
public void unScheduled() {
- if (runner != null) {
- runner.stop();
+ stopping.set(true);
+ if (source instanceof PollableSource) {
+ source.stop();
+ } else {
+ EventDrivenSourceRunner runner = runnerRef.get();
+ if (runner != null) {
+ runner.stop();
+ runnerRef.compareAndSet(runner, null);
+ }
+
+ NifiSessionFactoryChannel eventDrivenSourceChannel = eventDrivenSourceChannelRef.get();
+ if (eventDrivenSourceChannel != null) {
+ eventDrivenSourceChannel.stop();
+ eventDrivenSourceChannelRef.compareAndSet(eventDrivenSourceChannel, null);
+ }
}
- if (channel != null) {
- channel.stop();
+ }
+
+ @OnStopped
+ public void stopped() {
+ sessionFactoryRef.set(null);
+ }
+
+ @Override
+ public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
+ if (source instanceof PollableSource) {
+ super.onTrigger(context, sessionFactory);
+ } else if (source instanceof EventDrivenSource) {
+ ProcessSessionFactory old = sessionFactoryRef.getAndSet(sessionFactory);
+ if (old == null) {
+ runnerRef.set(new EventDrivenSourceRunner());
+ eventDrivenSourceChannelRef.set(new NifiSessionFactoryChannel(sessionFactoryRef.get(), SUCCESS));
+ eventDrivenSourceChannelRef.get()
+ .start();
+ source.setChannelProcessor(new ChannelProcessor(new NifiChannelSelector(
+ eventDrivenSourceChannelRef.get())));
+ runnerRef.get()
+ .setSource(source);
+ runnerRef.get()
+ .start();
+ }
}
}
@Override
- public void onTrigger(final ProcessContext context,
- final ProcessSession session) throws ProcessException {
- if (source instanceof EventDrivenSource) {
- onEventDrivenTrigger(context, session);
- } else if (source instanceof PollableSource) {
- onPollableTrigger((PollableSource) source, context, session);
- }
- }
-
- public void onPollableTrigger(final PollableSource pollableSource,
- final ProcessContext context, final ProcessSession session)
- throws ProcessException {
- try {
- pollableSource.setChannelProcessor(new ChannelProcessor(
- new NifiChannelSelector(new NifiChannel(session, SUCCESS))));
- pollableSource.start();
- pollableSource.process();
- pollableSource.stop();
- } catch (EventDeliveryException ex) {
- throw new ProcessException("Error processing pollable source", ex);
- }
- }
-
- public void onEventDrivenTrigger(final ProcessContext context, final ProcessSession session) {
- Transaction transaction = channel.getTransaction();
- transaction.begin();
-
- try {
- Event event = channel.take();
- if (event != null) {
- transferEvent(event, session, SUCCESS);
+ public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+ if (source instanceof PollableSource) {
+ PollableSource pollableSource = (PollableSource) source;
+ try {
+ pollableSourceChannel.setSession(session);
+ pollableSource.process();
+ } catch (EventDeliveryException ex) {
+ throw new ProcessException("Error processing pollable source", ex);
}
- transaction.commit();
- } catch (Throwable th) {
- transaction.rollback();
- throw Throwables.propagate(th);
- } finally {
- transaction.close();
}
}
-
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionChannel.java
similarity index 66%
rename from nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java
rename to nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionChannel.java
index c4d3bef4d0..4c111af6b8 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiChannel.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionChannel.java
@@ -22,24 +22,26 @@ import org.apache.flume.channel.BasicTransactionSemantics;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
+public class NifiSessionChannel extends BasicChannelSemantics {
-public class NifiChannel extends BasicChannelSemantics {
- private final ProcessSession session;
- private final Relationship relationship;
+ private ProcessSession session;
+ private final Relationship relationship;
- public NifiChannel(ProcessSession session, Relationship relationship) {
- this.session = session;
- this.relationship = relationship;
- }
+ public NifiSessionChannel(Relationship relationship) {
+ this.relationship = relationship;
+ }
- @Override
- protected BasicTransactionSemantics createTransaction() {
- return new NifiTransaction(session, relationship);
- }
+ public void setSession(ProcessSession session) {
+ this.session = session;
+ }
- @Override
- public void configure(Context context) {
- }
+ @Override
+ protected BasicTransactionSemantics createTransaction() {
+ return new NifiTransaction(session, relationship);
+ }
+ @Override
+ public void configure(Context context) {
+ }
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionFactoryChannel.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionFactoryChannel.java
new file mode 100644
index 0000000000..bc565878ba
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionFactoryChannel.java
@@ -0,0 +1,51 @@
+/*
+ * 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.flume;
+
+import org.apache.flume.ChannelFullException;
+import org.apache.flume.Context;
+import org.apache.flume.channel.BasicChannelSemantics;
+import org.apache.flume.channel.BasicTransactionSemantics;
+import org.apache.flume.lifecycle.LifecycleState;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+
+public class NifiSessionFactoryChannel extends BasicChannelSemantics {
+
+ private final ProcessSessionFactory sessionFactory;
+ private final Relationship relationship;
+
+ public NifiSessionFactoryChannel(ProcessSessionFactory sessionFactory, Relationship relationship) {
+ this.sessionFactory = sessionFactory;
+ this.relationship = relationship;
+ }
+
+ @Override
+ protected BasicTransactionSemantics createTransaction() {
+ LifecycleState lifecycleState = getLifecycleState();
+ if (lifecycleState == LifecycleState.STOP) {
+ throw new ChannelFullException("Can't write to a stopped channel");
+ //return null;
+ }
+ return new NifiTransaction(sessionFactory.createSession(), relationship);
+ }
+
+ @Override
+ public void configure(Context context) {
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSinkSessionChannel.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSinkSessionChannel.java
new file mode 100644
index 0000000000..5621b6dd1d
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSinkSessionChannel.java
@@ -0,0 +1,49 @@
+/*
+ * 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.flume;
+
+import org.apache.flume.Context;
+import org.apache.flume.channel.BasicChannelSemantics;
+import org.apache.flume.channel.BasicTransactionSemantics;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+
+public class NifiSinkSessionChannel extends BasicChannelSemantics {
+
+ private ProcessSession session;
+ private final Relationship success;
+ private final Relationship failure;
+
+ public NifiSinkSessionChannel(Relationship success, Relationship failure) {
+ this.success = success;
+ this.failure = failure;
+ }
+
+ public void setSession(ProcessSession session) {
+ this.session = session;
+ }
+
+ @Override
+ protected BasicTransactionSemantics createTransaction() {
+ return new NifiSinkTransaction(session, success, failure);
+ }
+
+ @Override
+ public void configure(Context context) {
+ }
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSinkTransaction.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSinkTransaction.java
new file mode 100644
index 0000000000..837652f145
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSinkTransaction.java
@@ -0,0 +1,71 @@
+/*
+ * 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.flume;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.flume.Event;
+import org.apache.flume.channel.BasicTransactionSemantics;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.flume.util.FlowFileEvent;
+
+
+class NifiSinkTransaction extends BasicTransactionSemantics {
+ private final ProcessSession session;
+ private final Relationship success;
+ private final Relationship failure;
+ private final List flowFiles;
+
+ public NifiSinkTransaction(ProcessSession session, Relationship success, Relationship failure) {
+ this.session = session;
+ this.success = success;
+ this.failure = failure;
+ this.flowFiles = new ArrayList<>();
+ }
+
+ @Override
+ protected void doPut(Event event) throws InterruptedException {
+ AbstractFlumeProcessor.transferEvent(event, session, success);
+ }
+
+ @Override
+ protected Event doTake() throws InterruptedException {
+ FlowFile flowFile = session.get();
+ if (flowFile == null) {
+ return null;
+ }
+ flowFiles.add(flowFile);
+
+ return new FlowFileEvent(flowFile, session);
+ }
+
+ @Override
+ protected void doCommit() throws InterruptedException {
+ session.transfer(flowFiles, success);
+ session.commit();
+ }
+
+ @Override
+ protected void doRollback() throws InterruptedException {
+ session.transfer(flowFiles, failure);
+ session.commit();
+ }
+
+
+}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java
index 37c8a50902..8de50ec963 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiTransaction.java
@@ -21,35 +21,34 @@ import org.apache.flume.channel.BasicTransactionSemantics;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.Relationship;
-
class NifiTransaction extends BasicTransactionSemantics {
- private final ProcessSession session;
- private final Relationship relationship;
- public NifiTransaction(ProcessSession session, Relationship relationship) {
- this.session = session;
- this.relationship = relationship;
- }
+ private final ProcessSession session;
+ private final Relationship relationship;
- @Override
- protected void doPut(Event event) throws InterruptedException {
- AbstractFlumeProcessor.transferEvent(event, session, relationship);
- }
+ public NifiTransaction(ProcessSession session, Relationship relationship) {
+ this.session = session;
+ this.relationship = relationship;
+ }
- @Override
- protected Event doTake() throws InterruptedException {
- throw new UnsupportedOperationException("Only put supported");
- }
+ @Override
+ protected void doPut(Event event) throws InterruptedException {
+ AbstractFlumeProcessor.transferEvent(event, session, relationship);
+ }
- @Override
- protected void doCommit() throws InterruptedException {
- session.commit();
- }
+ @Override
+ protected Event doTake() throws InterruptedException {
+ throw new UnsupportedOperationException("Only put supported");
+ }
- @Override
- protected void doRollback() throws InterruptedException {
- session.rollback();
- }
+ @Override
+ protected void doCommit() throws InterruptedException {
+ session.commit();
+ }
+ @Override
+ protected void doRollback() throws InterruptedException {
+ session.rollback();
+ }
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
index 2e10c24a7d..0654138cfd 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
@@ -126,13 +126,12 @@ public class FlumeSinkProcessorTest {
public void testBatchSize() throws IOException {
TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
runner.setProperty(FlumeSinkProcessor.SINK_TYPE, NullSink.class.getName());
- runner.setProperty(FlumeSinkProcessor.BATCH_SIZE, "1000");
runner.setProperty(FlumeSinkProcessor.FLUME_CONFIG,
"tier1.sinks.sink-1.batchSize = 1000\n");
for (int i = 0; i < 100000; i++) {
runner.enqueue(String.valueOf(i).getBytes());
}
- runner.run();
+ runner.run(100);
}
@Test
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
index 043e1154fb..32feb1e28a 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
@@ -127,11 +127,8 @@ public class FlumeSourceProcessorTest {
runner.setProperty(FlumeSinkProcessor.FLUME_CONFIG,
"tier1.sources.src-1.spoolDir = " + spoolDirectory.getAbsolutePath());
runner.run();
- List flowFiles = runner.getFlowFilesForRelationship(FlumeSourceProcessor.SUCCESS);
- Assert.assertEquals(1, flowFiles.size());
- for (MockFlowFile flowFile : flowFiles) {
- Assert.assertEquals(8, flowFile.getSize());
- flowFile.assertContentEquals("record 1");
- }
+ // No data will be transfered because of how quickly the test runner
+ // starts shutting down
+ runner.assertTransferCount(FlumeSourceProcessor.SUCCESS, 0);
}
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/log4j.properties b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/log4j.properties
index 8c502ec918..cc58727fa0 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/log4j.properties
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/log4j.properties
@@ -17,4 +17,6 @@ log4j.rootLogger=INFO, CONSOLE
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
-log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x \u2013 %m%n
\ No newline at end of file
+log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x \u2013 %m%n
+
+log4j.logger.org.apache.flume = DEBUG
\ No newline at end of file
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/simplelogger.properties b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/simplelogger.properties
index 4994e7f999..e3d4fc1bdf 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/simplelogger.properties
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/resources/simplelogger.properties
@@ -18,3 +18,4 @@ org.slf4j.simpleLogger.showDateTime=true
org.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd HH:mm:ss.SSS
org.slf4j.simpleLogger.levelInBrackets=true
org.slf4j.simpleLogger.log.org.apache.nifi.processors.flume=debug
+org.slf4j.simpleLogger.log.org.apache.flume=debug
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
index 50b0fde3a9..a2742aa3a1 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
@@ -17,10 +17,10 @@
org.apache.nifinifi-nar-bundles
- 0.1.0-incubating-SNAPSHOT
+ 0.2.0-incubating-SNAPSHOTnifi-flume-bundle
- 0.1.0-incubating-SNAPSHOT
+ 0.2.0-incubating-SNAPSHOTpomA bundle of processors that run Flume sources/sinks
From c4dd1e65b1be905dc712f1907ff1156e60d02a79 Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Wed, 10 Jun 2015 14:18:42 -0700
Subject: [PATCH 13/77] Moved the source/sink stoppign to @OnStopped methods
* Made the spoolDirectory test stronger
Signed-off-by: Matt Gilman
---
.../processors/flume/FlumeSinkProcessor.java | 9 +++----
.../flume/FlumeSourceProcessor.java | 25 ++++++-------------
.../flume/FlumeSourceProcessorTest.java | 24 +++++++++++++++---
3 files changed, 31 insertions(+), 27 deletions(-)
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
index e385921d04..9ec1b071a3 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
@@ -28,7 +28,7 @@ import org.apache.flume.conf.Configurables;
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.OnUnscheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.Validator;
@@ -124,15 +124,14 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
}
}
- @OnUnscheduled
- public void unScheduled() {
+ @OnStopped
+ public void stopped() {
sink.stop();
channel.stop();
}
@Override
- public void onTrigger(final ProcessContext context,
- final ProcessSession session) throws ProcessException {
+ public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
channel.setSession(session);
try {
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
index 3ded2085b3..1ebf05cfa9 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
@@ -33,7 +33,6 @@ 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;
@@ -93,7 +92,6 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
private final AtomicReference sessionFactoryRef = new AtomicReference<>(null);
private final AtomicReference runnerRef = new AtomicReference<>(null);
private final AtomicReference eventDrivenSourceChannelRef = new AtomicReference<>(null);
- private final AtomicReference stopping = new AtomicReference<>(false);
@Override
protected void init(final ProcessorInitializationContext context) {
@@ -114,7 +112,6 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
@OnScheduled
public void onScheduled(final SchedulingContext context) {
try {
- stopping.set(false);
source = SOURCE_FACTORY.create(
context.getProperty(SOURCE_NAME)
.getValue(),
@@ -142,9 +139,8 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
}
}
- @OnUnscheduled
- public void unScheduled() {
- stopping.set(true);
+ @OnStopped
+ public void stopped() {
if (source instanceof PollableSource) {
source.stop();
} else {
@@ -160,10 +156,6 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
eventDrivenSourceChannelRef.compareAndSet(eventDrivenSourceChannel, null);
}
}
- }
-
- @OnStopped
- public void stopped() {
sessionFactoryRef.set(null);
}
@@ -176,14 +168,11 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
if (old == null) {
runnerRef.set(new EventDrivenSourceRunner());
eventDrivenSourceChannelRef.set(new NifiSessionFactoryChannel(sessionFactoryRef.get(), SUCCESS));
- eventDrivenSourceChannelRef.get()
- .start();
- source.setChannelProcessor(new ChannelProcessor(new NifiChannelSelector(
- eventDrivenSourceChannelRef.get())));
- runnerRef.get()
- .setSource(source);
- runnerRef.get()
- .start();
+ eventDrivenSourceChannelRef.get().start();
+ source.setChannelProcessor(new ChannelProcessor(
+ new NifiChannelSelector(eventDrivenSourceChannelRef.get())));
+ runnerRef.get().setSource(source);
+ runnerRef.get().start();
}
}
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
index 32feb1e28a..bf32095f7b 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
+import java.util.concurrent.TimeUnit;
import org.apache.flume.sink.NullSink;
import org.apache.flume.source.AvroSource;
@@ -126,9 +127,24 @@ public class FlumeSourceProcessorTest {
runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, "spooldir");
runner.setProperty(FlumeSinkProcessor.FLUME_CONFIG,
"tier1.sources.src-1.spoolDir = " + spoolDirectory.getAbsolutePath());
- runner.run();
- // No data will be transfered because of how quickly the test runner
- // starts shutting down
- runner.assertTransferCount(FlumeSourceProcessor.SUCCESS, 0);
+ runner.run(1, false, true);
+ // Because the spool directory source is an event driven source, it may take some time for flow files to get
+ // produced. I'm willing to wait up to 5 seconds, but will bail out early if possible. If it takes longer than
+ // that then there is likely a bug.
+ int numWaits = 10;
+ while (runner.getFlowFilesForRelationship(FlumeSourceProcessor.SUCCESS).size() < 4 && --numWaits > 0) {
+ try {
+ TimeUnit.MILLISECONDS.sleep(500);
+ } catch (InterruptedException ex) {
+ logger.warn("Sleep interrupted");
+ }
+ }
+ runner.shutdown();
+ runner.assertTransferCount(FlumeSourceProcessor.SUCCESS, 4);
+ int i = 1;
+ for (MockFlowFile flowFile : runner.getFlowFilesForRelationship(FlumeSourceProcessor.SUCCESS)) {
+ flowFile.assertContentEquals("record " + i);
+ i++;
+ }
}
}
From 16134a2dfedf2f05f7d798082bfd86a5a13bda5a Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Wed, 10 Jun 2015 15:08:03 -0700
Subject: [PATCH 14/77] More review feedback from Ryan:
* Removed unnecessary call to configure our channel
* Removed call to context.yield() when Flume reports a backoff
* Handled the session factory changing when using a event driven source.
Signed-off-by: Matt Gilman
---
.../apache/nifi/processors/flume/FlumeSinkProcessor.java | 5 +----
.../apache/nifi/processors/flume/FlumeSourceProcessor.java | 6 +++++-
2 files changed, 6 insertions(+), 5 deletions(-)
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
index 9ec1b071a3..2d8506d578 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
@@ -104,7 +104,6 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
public void onScheduled(final SchedulingContext context) {
try {
channel = new NifiSinkSessionChannel(SUCCESS, FAILURE);
- Configurables.configure(channel, new Context());
channel.start();
sink = SINK_FACTORY.create(context.getProperty(SOURCE_NAME).getValue(),
@@ -135,9 +134,7 @@ public class FlumeSinkProcessor extends AbstractFlumeProcessor {
channel.setSession(session);
try {
- if (sink.process() == Sink.Status.BACKOFF) {
- context.yield();
- }
+ sink.process();
} catch (EventDeliveryException ex) {
throw new ProcessException("Flume event delivery failed", ex);
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
index 1ebf05cfa9..55b1f2f8ec 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
@@ -165,7 +165,11 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
super.onTrigger(context, sessionFactory);
} else if (source instanceof EventDrivenSource) {
ProcessSessionFactory old = sessionFactoryRef.getAndSet(sessionFactory);
- if (old == null) {
+ if (old != sessionFactory) {
+ if (runnerRef.get() != null) {
+ stopped();
+ }
+
runnerRef.set(new EventDrivenSourceRunner());
eventDrivenSourceChannelRef.set(new NifiSessionFactoryChannel(sessionFactoryRef.get(), SUCCESS));
eventDrivenSourceChannelRef.get().start();
From 286e4738b8bea6e8a386b630822a897ca07410d2 Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Sun, 12 Jul 2015 13:33:09 -0700
Subject: [PATCH 15/77] Implemented Joe's review feedback:
* Renamed processors to ExecuteFlume*
* Added a LICENSE and NOTICE file to the nar
* Fixed the version number inconsistency in the POMs
Signed-off-by: Matt Gilman
---
.../nifi-flume-bundle/nifi-flume-nar/pom.xml | 95 ++++++
.../src/main/resources/META-INF/LICENSE | 319 ++++++++++++++++++
.../src/main/resources/META-INF/NOTICE | 145 ++++++++
.../nifi-flume-processors/pom.xml | 54 ++-
.../flume/AbstractFlumeProcessor.java | 4 +-
...nkProcessor.java => ExecuteFlumeSink.java} | 4 +-
...Processor.java => ExecuteFlumeSource.java} | 24 +-
.../flume/NifiSessionFactoryChannel.java | 1 -
.../processors/flume/util/FlowFileEvent.java | 9 +-
...sorTest.java => ExecuteFlumeSinkTest.java} | 28 +-
...rTest.java => ExecuteFlumeSourceTest.java} | 30 +-
.../nifi-flume-bundle/pom.xml | 2 +-
nifi/pom.xml | 10 +-
13 files changed, 657 insertions(+), 68 deletions(-)
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/LICENSE
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/NOTICE
rename nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/{FlumeSinkProcessor.java => ExecuteFlumeSink.java} (97%)
rename nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/{FlumeSourceProcessor.java => ExecuteFlumeSource.java} (93%)
rename nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/{FlumeSinkProcessorTest.java => ExecuteFlumeSinkTest.java} (84%)
rename nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/{FlumeSourceProcessorTest.java => ExecuteFlumeSourceTest.java} (82%)
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
index c07cedf690..a9bbe6f911 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
@@ -26,6 +26,101 @@
org.apache.nifinifi-flume-processors
+
+
+
+ org.apache.hadoop
+ hadoop-client
+
+
+ org.xerial.snappy
+ snappy-java
+
+
+ com.thoughtworks.paranamer
+ paranamer
+
+
+ org.codehaus.jackson
+ jackson-mapper-asl
+
+
+ org.codehaus.jackson
+ jackson-core-asl
+
+
+ com.google.guava
+ guava
+
+
+ com.google.guava
+ guava
+
+
+ commons-logging
+ commons-logging
+
+
+ org.apache.commons
+ commons-compress
+
+
+ commons-codec
+ commons-codec
+
+
+ org.apache.avro
+ avro
+
+
+ org.apache.zookeeper
+ zookeeper
+
+
+ commons-lang
+ commons-lang
+
+
+ commons-cli
+ commons-cli
+
+
+ commons-collections
+ commons-collections
+
+
+ commons-io
+ commons-io
+
+
+ com.google.code.gson
+ gson
+
+
+ org.apache.httpcomponents
+ httpclient
+
+
+ org.apache.httpcomponents
+ httpcore
+
+
+ org.mortbay.jetty
+ jetty
+
+
+ org.mortbay.jetty
+ jetty-util
+
+
+ log4j
+ log4j
+
+
+ com.google.protobuf
+ protobuf-java
+
+ org.apache.nifi
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/LICENSE b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000000..c1a3ec4fd2
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,319 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed 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.
+
+APACHE NIFI SUBCOMPONENTS:
+
+The Apache NiFi project contains subcomponents with separate copyright
+notices and license terms. Your use of the source code for the these
+subcomponents is subject to the terms and conditions of the following
+licenses.
+
+The binary distribution of this product bundles 'SUAsync Library' which is
+available under a 3-Clause BSD License.
+
+ Copyright (c) 2010 StumbleUpon, Inc. All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions are met:
+ - Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+ - Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+ - Neither the name of the StumbleUpon nor the names of its contributors
+ may be used to endorse or promote products derived from this software
+ without specific prior written permission.
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
+ LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles 'Asynchronous HBase Client'
+which is available under a 3-Clause BSD License.
+
+ Copyright (C) 2010-2012 The Async HBase Authors. All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without
+ modification, are permitted provided that the following conditions are met:
+ - Redistributions of source code must retain the above copyright notice,
+ this list of conditions and the following disclaimer.
+ - Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+ - Neither the name of the StumbleUpon nor the names of its contributors
+ may be used to endorse or promote products derived from this software
+ without specific prior written permission.
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+ AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+ IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
+ ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
+ LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
+ CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
+ SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
+ INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
+ CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
+ ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
+ POSSIBILITY OF SUCH DAMAGE.
+
+The binary distribution of this product bundles 'JOpt Simple' which is
+available under the MIT license.
+
+ The MIT License
+
+ Copyright (c) 2004-2011 Paul R. Holser, Jr.
+
+ Permission is hereby granted, free of charge, to any person obtaining
+ a copy of this software and associated documentation files (the
+ "Software"), to deal in the Software without restriction, including
+ without limitation the rights to use, copy, modify, merge, publish,
+ distribute, sublicense, and/or sell copies of the Software, and to
+ permit persons to whom the Software is furnished to do so, subject to
+ the following conditions:
+
+ The above copyright notice and this permission notice shall be
+ included in all copies or substantial portions of the Software.
+
+ THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+ EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+ MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+The binary distribution of this product bundles 'Scala Library' under a BSD
+style license.
+
+ Copyright (c) 2002-2015 EPFL
+ Copyright (c) 2011-2015 Typesafe, Inc.
+
+ All rights reserved.
+
+ Redistribution and use in source and binary forms, with or without modification,
+ are permitted provided that the following conditions are met:
+
+ Redistributions of source code must retain the above copyright notice, this list of
+ conditions and the following disclaimer.
+
+ Redistributions in binary form must reproduce the above copyright notice, this list of
+ conditions and the following disclaimer in the documentation and/or other materials
+ provided with the distribution.
+
+ Neither the name of the EPFL nor the names of its contributors may be used to endorse
+ or promote products derived from this software without specific prior written permission.
+
+ THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS “AS IS” AND ANY EXPRESS
+ OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
+ AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
+ CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL
+ DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+ DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER
+ IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
+ OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/NOTICE b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000000..793746f194
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,145 @@
+nifi-social-media-nar
+Copyright 2015 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+ (ASLv2) Apache Avro
+ The following NOTICE information applies:
+ Apache Avro
+ Copyright 2009-2012 The Apache Software Foundation
+
+ (ASLv2) Apache Commons JEXL
+ The following NOTICE information applies:
+ Apache Commons JEXL
+ Copyright 2001-2011 The Apache Software Foundation
+
+ (ASLv2) Apache Commons Lang
+ The following NOTICE information applies:
+ Apache Commons Lang
+ Copyright 2001-2015 The Apache Software Foundation
+
+ This product includes software from the Spring Framework,
+ under the Apache License 2.0 (see: StringUtils.containsWhitespace())
+
+ (ASLv2) Apache Flume
+ The following NOTICE information applies:
+ Apache Flume
+ Copyright 2011-2015 Apache Software Foundation
+
+ (ASLv2) IRClib
+ The following NOTICE information applies:
+ IRClib -- A Java Internet Relay Chat library --
+ Copyright (C) 2002 - 2006 Christoph Schwering
+
+ (ASLv2) Jackson JSON processor
+ The following NOTICE information applies:
+ # Jackson JSON processor
+
+ Jackson is a high-performance, Free/Open Source JSON processing library.
+ It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has
+ been in development since 2007.
+ It is currently developed by a community of developers, as well as supported
+ commercially by FasterXML.com.
+
+ ## Licensing
+
+ Jackson core and extension components may licensed under different licenses.
+ To find the details that apply to this artifact see the accompanying LICENSE file.
+ For more information, including possible other licensing options, contact
+ FasterXML.com (http://fasterxml.com).
+
+ ## Credits
+
+ A list of contributors may be found from CREDITS file, which is included
+ in some artifacts (usually source distributions); but is always available
+ from the source code management (SCM) system project uses.
+
+ (ASLv2) Joda-Time
+ The following NOTICE information applies:
+ =============================================================================
+ = NOTICE file corresponding to section 4d of the Apache License Version 2.0 =
+ =============================================================================
+ This product includes software developed by
+ Joda.org (http://www.joda.org/).
+
+ (ASLv2) Apache Kafka
+ The following NOTICE information applies:
+ Apache Kafka
+ Copyright 2012 The Apache Software Foundation.
+
+ (ASLv2) Kite SDK
+ The following NOTICE information applies:
+ This product includes software developed by Cloudera, Inc.
+ (http://www.cloudera.com/).
+
+ This product includes software developed at
+ The Apache Software Foundation (http://www.apache.org/).
+
+ This product includes software developed by
+ Saxonica (http://www.saxonica.com/).
+
+ (ASLv2) Apache Thrift
+ The following NOTICE information applies:
+ Apache Thrift
+ Copyright 2006-2010 The Apache Software Foundation.
+
+ (ASLv2) Yammer Metrics
+ The following NOTICE information applies:
+ Metrics
+ Copyright 2010-2012 Coda Hale and Yammer, Inc.
+
+ This product includes software developed by Coda Hale and Yammer, Inc.
+
+ This product includes code derived from the JSR-166 project (ThreadLocalRandom), which was released
+ with the following comments:
+
+ Written by Doug Lea with assistance from members of JCP JSR-166
+ Expert Group and released to the public domain, as explained at
+ http://creativecommons.org/publicdomain/zero/1.0/
+
+ (ASLv2) Apache MINA
+ The following NOTICE information applies:
+ Apache MINA Core
+ Copyright 2004-2011 Apache MINA Project
+
+ (ASLv2) The Netty Project
+ The following NOTICE information applies:
+ The Netty Project
+ Copyright 2011 The Netty Project
+
+ (ASLv2) opencsv (net.sf.opencsv:opencsv:2.3)
+
+ (ASLv2) Parquet MR
+ The following NOTICE information applies:
+ Parquet MR
+ Copyright 2012 Twitter, Inc.
+
+ This project includes code from https://github.com/lemire/JavaFastPFOR
+ parquet-column/src/main/java/parquet/column/values/bitpacking/LemireBitPacking.java
+ Apache License Version 2.0 http://www.apache.org/licenses/.
+ (c) Daniel Lemire, http://lemire.me/en/
+
+ (ASLv2) Servlet Specification API (org.mortbay.jetty:servlet-api:2.5-20110124)
+
+ (ASLv2) Twitter4J
+ The following NOTICE information applies:
+ Copyright 2007 Yusuke Yamamoto
+
+ Twitter4J includes software from JSON.org to parse JSON response from the Twitter API. You can see the license term at http://www.JSON.org/license.html
+
+ (ASLv2) Apache Velocity
+ The following NOTICE information applies:
+ Apache Velocity
+ Copyright (C) 2000-2007 The Apache Software Foundation
+
+ (ASLv2) ZkClient
+ The following NOTICE information applies:
+ ZkClient
+ Copyright 2009 Stefan Groschupf
\ No newline at end of file
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
index 1dad25f6df..167aa6e943 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
@@ -21,6 +21,11 @@
nifi-flume-processorsjar
+
+
+ 1.6.0
+
+
org.apache.nifi
@@ -41,12 +46,12 @@
org.apache.flumeflume-ng-sdk
- 1.5.2
+ ${flume.version}org.apache.flumeflume-ng-core
- 1.5.2
+ ${flume.version}org.slf4j
@@ -56,27 +61,38 @@
-
- org.apache.flume.flume-ng-sources
- flume-twitter-source
- 1.5.2
-
+
org.apache.flume.flume-ng-sourcesflume-jms-source
- 1.5.2
+ ${flume.version}
+
+
+ org.apache.flume.flume-ng-sources
+ flume-kafka-source
+ ${flume.version}org.apache.flume.flume-ng-sourcesflume-scribe-source
- 1.5.2
+ ${flume.version}
+
+
+ org.apache.flume.flume-ng-sources
+ flume-twitter-source
+ ${flume.version}
+
+ org.apache.flume.flume-ng-sinks
+ flume-dataset-sink
+ ${flume.version}
+ org.apache.flume.flume-ng-sinksflume-hdfs-sink
- 1.5.2
+ ${flume.version}
@@ -97,25 +113,35 @@
provided
+
+ org.apache.flume.flume-ng-sinks
+ flume-hive-sink
+ ${flume.version}
+ org.apache.flume.flume-ng-sinksflume-irc-sink
- 1.5.2
+ ${flume.version}org.apache.flume.flume-ng-sinksflume-ng-elasticsearch-sink
- 1.5.2
+ ${flume.version}org.apache.flume.flume-ng-sinksflume-ng-hbase-sink
- 1.5.2
+ ${flume.version}
+
+
+ org.apache.flume.flume-ng-sinks
+ flume-ng-kafka-sink
+ ${flume.version}org.apache.flume.flume-ng-sinksflume-ng-morphline-solr-sink
- 1.5.2
+ ${flume.version}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
index 83ae9e1aeb..9b75047a1e 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/AbstractFlumeProcessor.java
@@ -77,7 +77,7 @@ public abstract class AbstractFlumeProcessor extends AbstractSessionFactoryProce
public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
String reason = null;
try {
- FlumeSourceProcessor.SOURCE_FACTORY.create("NiFi Source", value);
+ ExecuteFlumeSource.SOURCE_FACTORY.create("NiFi Source", value);
} catch (Exception ex) {
reason = ex.getLocalizedMessage();
reason = Character.toLowerCase(reason.charAt(0)) + reason.substring(1);
@@ -97,7 +97,7 @@ public abstract class AbstractFlumeProcessor extends AbstractSessionFactoryProce
public ValidationResult validate(final String subject, final String value, final ValidationContext context) {
String reason = null;
try {
- FlumeSinkProcessor.SINK_FACTORY.create("NiFi Sink", value);
+ ExecuteFlumeSink.SINK_FACTORY.create("NiFi Sink", value);
} catch (Exception ex) {
reason = ex.getLocalizedMessage();
reason = Character.toLowerCase(reason.charAt(0)) + reason.substring(1);
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java
similarity index 97%
rename from nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
rename to nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java
index 2d8506d578..8ccb2d14aa 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSinkProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java
@@ -25,6 +25,7 @@ import org.apache.flume.Context;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.Sink;
import org.apache.flume.conf.Configurables;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
@@ -45,7 +46,8 @@ import org.apache.nifi.processor.util.StandardValidators;
*/
@Tags({"flume", "hadoop", "get", "sink"})
@CapabilityDescription("Write FlowFile data to a Flume sink")
-public class FlumeSinkProcessor extends AbstractFlumeProcessor {
+@TriggerSerially
+public class ExecuteFlumeSink extends AbstractFlumeProcessor {
public static final PropertyDescriptor SINK_TYPE = new PropertyDescriptor.Builder()
.name("Sink Type")
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java
similarity index 93%
rename from nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
rename to nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java
index 55b1f2f8ec..fa02750df6 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/FlumeSourceProcessor.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java
@@ -29,6 +29,7 @@ import org.apache.flume.Source;
import org.apache.flume.channel.ChannelProcessor;
import org.apache.flume.conf.Configurables;
import org.apache.flume.source.EventDrivenSourceRunner;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
@@ -50,7 +51,8 @@ import org.apache.nifi.processor.util.StandardValidators;
*/
@Tags({"flume", "hadoop", "get", "source"})
@CapabilityDescription("Generate FlowFile data from a Flume source")
-public class FlumeSourceProcessor extends AbstractFlumeProcessor {
+@TriggerSerially
+public class ExecuteFlumeSource extends AbstractFlumeProcessor {
public static final PropertyDescriptor SOURCE_TYPE = new PropertyDescriptor.Builder()
.name("Source Type")
@@ -113,17 +115,12 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
public void onScheduled(final SchedulingContext context) {
try {
source = SOURCE_FACTORY.create(
- context.getProperty(SOURCE_NAME)
- .getValue(),
- context.getProperty(SOURCE_TYPE)
- .getValue());
+ context.getProperty(SOURCE_NAME).getValue(),
+ context.getProperty(SOURCE_TYPE).getValue());
- String flumeConfig = context.getProperty(FLUME_CONFIG)
- .getValue();
- String agentName = context.getProperty(AGENT_NAME)
- .getValue();
- String sourceName = context.getProperty(SOURCE_NAME)
- .getValue();
+ String flumeConfig = context.getProperty(FLUME_CONFIG).getValue();
+ String agentName = context.getProperty(AGENT_NAME).getValue();
+ String sourceName = context.getProperty(SOURCE_NAME).getValue();
Configurables.configure(source,
getFlumeSourceContext(flumeConfig, agentName, sourceName));
@@ -133,8 +130,7 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
source.start();
}
} catch (Throwable th) {
- getLogger()
- .error("Error creating source", th);
+ getLogger().error("Error creating source", th);
throw Throwables.propagate(th);
}
}
@@ -191,6 +187,8 @@ public class FlumeSourceProcessor extends AbstractFlumeProcessor {
} catch (EventDeliveryException ex) {
throw new ProcessException("Error processing pollable source", ex);
}
+ } else {
+ throw new ProcessException("Invalid source type: " + source.getClass().getSimpleName());
}
}
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionFactoryChannel.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionFactoryChannel.java
index bc565878ba..eb31a66420 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionFactoryChannel.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/NifiSessionFactoryChannel.java
@@ -39,7 +39,6 @@ public class NifiSessionFactoryChannel extends BasicChannelSemantics {
LifecycleState lifecycleState = getLifecycleState();
if (lifecycleState == LifecycleState.STOP) {
throw new ChannelFullException("Can't write to a stopped channel");
- //return null;
}
return new NifiTransaction(sessionFactory.createSession(), relationship);
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
index 5dc97d65e7..fdff203544 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/util/FlowFileEvent.java
@@ -26,8 +26,13 @@ import org.apache.flume.Event;
import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.io.InputStreamCallback;
+import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.ENTRY_DATE_HEADER;
+import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.ID_HEADER;
+import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.LAST_QUEUE_DATE_HEADER;
+import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.LINEAGE_IDENTIFIERS_HEADER;
+import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.LINEAGE_START_DATE_HEADER;
+import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.SIZE_HEADER;
-import static org.apache.nifi.processors.flume.util.FlowFileEventConstants.*;
import org.apache.nifi.stream.io.BufferedInputStream;
import org.apache.nifi.stream.io.StreamUtils;
@@ -105,7 +110,7 @@ public class FlowFileEvent implements Event {
@Override
public void process(InputStream in) throws IOException {
try (BufferedInputStream input = new BufferedInputStream(in)) {
- StreamUtils.copy(in, baos);
+ StreamUtils.copy(input, baos);
}
baos.close();
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/ExecuteFlumeSinkTest.java
similarity index 84%
rename from nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
rename to nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/ExecuteFlumeSinkTest.java
index 0654138cfd..6a0c40d13d 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSinkProcessorTest.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/ExecuteFlumeSinkTest.java
@@ -45,17 +45,17 @@ import org.junit.rules.TemporaryFolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class FlumeSinkProcessorTest {
+public class ExecuteFlumeSinkTest {
private static final Logger logger =
- LoggerFactory.getLogger(FlumeSinkProcessorTest.class);
+ LoggerFactory.getLogger(ExecuteFlumeSinkTest.class);
@Rule
public final TemporaryFolder temp = new TemporaryFolder();
@Test
public void testValidators() {
- TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
+ TestRunner runner = TestRunners.newTestRunner(ExecuteFlumeSink.class);
Collection results;
ProcessContext pc;
@@ -73,7 +73,7 @@ public class FlumeSinkProcessorTest {
// non-existent class
results = new HashSet<>();
- runner.setProperty(FlumeSinkProcessor.SINK_TYPE, "invalid.class.name");
+ runner.setProperty(ExecuteFlumeSink.SINK_TYPE, "invalid.class.name");
runner.enqueue(new byte[0]);
pc = runner.getProcessContext();
if (pc instanceof MockProcessContext) {
@@ -87,7 +87,7 @@ public class FlumeSinkProcessorTest {
// class doesn't implement Sink
results = new HashSet<>();
- runner.setProperty(FlumeSinkProcessor.SINK_TYPE, AvroSource.class.getName());
+ runner.setProperty(ExecuteFlumeSink.SINK_TYPE, AvroSource.class.getName());
runner.enqueue(new byte[0]);
pc = runner.getProcessContext();
if (pc instanceof MockProcessContext) {
@@ -100,7 +100,7 @@ public class FlumeSinkProcessorTest {
}
results = new HashSet<>();
- runner.setProperty(FlumeSinkProcessor.SINK_TYPE, NullSink.class.getName());
+ runner.setProperty(ExecuteFlumeSink.SINK_TYPE, NullSink.class.getName());
runner.enqueue(new byte[0]);
pc = runner.getProcessContext();
if (pc instanceof MockProcessContext) {
@@ -112,8 +112,8 @@ public class FlumeSinkProcessorTest {
@Test
public void testNullSink() throws IOException {
- TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
- runner.setProperty(FlumeSinkProcessor.SINK_TYPE, NullSink.class.getName());
+ TestRunner runner = TestRunners.newTestRunner(ExecuteFlumeSink.class);
+ runner.setProperty(ExecuteFlumeSink.SINK_TYPE, NullSink.class.getName());
try (InputStream inputStream = getClass().getResourceAsStream("/testdata/records.txt")) {
Map attributes = new HashMap<>();
attributes.put(CoreAttributes.FILENAME.key(), "records.txt");
@@ -124,9 +124,9 @@ public class FlumeSinkProcessorTest {
@Test
public void testBatchSize() throws IOException {
- TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
- runner.setProperty(FlumeSinkProcessor.SINK_TYPE, NullSink.class.getName());
- runner.setProperty(FlumeSinkProcessor.FLUME_CONFIG,
+ TestRunner runner = TestRunners.newTestRunner(ExecuteFlumeSink.class);
+ runner.setProperty(ExecuteFlumeSink.SINK_TYPE, NullSink.class.getName());
+ runner.setProperty(ExecuteFlumeSink.FLUME_CONFIG,
"tier1.sinks.sink-1.batchSize = 1000\n");
for (int i = 0; i < 100000; i++) {
runner.enqueue(String.valueOf(i).getBytes());
@@ -138,9 +138,9 @@ public class FlumeSinkProcessorTest {
public void testHdfsSink() throws IOException {
File destDir = temp.newFolder("hdfs");
- TestRunner runner = TestRunners.newTestRunner(FlumeSinkProcessor.class);
- runner.setProperty(FlumeSinkProcessor.SINK_TYPE, "hdfs");
- runner.setProperty(FlumeSinkProcessor.FLUME_CONFIG,
+ TestRunner runner = TestRunners.newTestRunner(ExecuteFlumeSink.class);
+ runner.setProperty(ExecuteFlumeSink.SINK_TYPE, "hdfs");
+ runner.setProperty(ExecuteFlumeSink.FLUME_CONFIG,
"tier1.sinks.sink-1.hdfs.path = " + destDir.toURI().toString() + "\n" +
"tier1.sinks.sink-1.hdfs.fileType = DataStream\n" +
"tier1.sinks.sink-1.hdfs.serializer = TEXT\n" +
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/ExecuteFlumeSourceTest.java
similarity index 82%
rename from nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
rename to nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/ExecuteFlumeSourceTest.java
index bf32095f7b..924776ee5d 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/FlumeSourceProcessorTest.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/test/java/org/apache/nifi/processors/flume/ExecuteFlumeSourceTest.java
@@ -41,16 +41,16 @@ import org.junit.rules.TemporaryFolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-public class FlumeSourceProcessorTest {
+public class ExecuteFlumeSourceTest {
- private static final Logger logger = LoggerFactory.getLogger(FlumeSourceProcessorTest.class);
+ private static final Logger logger = LoggerFactory.getLogger(ExecuteFlumeSourceTest.class);
@Rule
public final TemporaryFolder temp = new TemporaryFolder();
@Test
public void testValidators() {
- TestRunner runner = TestRunners.newTestRunner(FlumeSourceProcessor.class);
+ TestRunner runner = TestRunners.newTestRunner(ExecuteFlumeSource.class);
Collection results;
ProcessContext pc;
@@ -68,7 +68,7 @@ public class FlumeSourceProcessorTest {
// non-existent class
results = new HashSet<>();
- runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, "invalid.class.name");
+ runner.setProperty(ExecuteFlumeSource.SOURCE_TYPE, "invalid.class.name");
runner.enqueue(new byte[0]);
pc = runner.getProcessContext();
if (pc instanceof MockProcessContext) {
@@ -82,7 +82,7 @@ public class FlumeSourceProcessorTest {
// class doesn't implement Source
results = new HashSet<>();
- runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, NullSink.class.getName());
+ runner.setProperty(ExecuteFlumeSource.SOURCE_TYPE, NullSink.class.getName());
runner.enqueue(new byte[0]);
pc = runner.getProcessContext();
if (pc instanceof MockProcessContext) {
@@ -95,7 +95,7 @@ public class FlumeSourceProcessorTest {
}
results = new HashSet<>();
- runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, AvroSource.class.getName());
+ runner.setProperty(ExecuteFlumeSource.SOURCE_TYPE, AvroSource.class.getName());
runner.enqueue(new byte[0]);
pc = runner.getProcessContext();
if (pc instanceof MockProcessContext) {
@@ -106,10 +106,10 @@ public class FlumeSourceProcessorTest {
@Test
public void testSequenceSource() {
- TestRunner runner = TestRunners.newTestRunner(FlumeSourceProcessor.class);
- runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, "seq");
+ TestRunner runner = TestRunners.newTestRunner(ExecuteFlumeSource.class);
+ runner.setProperty(ExecuteFlumeSource.SOURCE_TYPE, "seq");
runner.run();
- List flowFiles = runner.getFlowFilesForRelationship(FlumeSourceProcessor.SUCCESS);
+ List flowFiles = runner.getFlowFilesForRelationship(ExecuteFlumeSource.SUCCESS);
Assert.assertEquals(1, flowFiles.size());
for (MockFlowFile flowFile : flowFiles) {
logger.debug(flowFile.toString());
@@ -123,16 +123,16 @@ public class FlumeSourceProcessorTest {
File dst = new File(spoolDirectory, "records.txt");
FileUtils.copyFile(getClass().getResourceAsStream("/testdata/records.txt"), dst, true, false);
- TestRunner runner = TestRunners.newTestRunner(FlumeSourceProcessor.class);
- runner.setProperty(FlumeSourceProcessor.SOURCE_TYPE, "spooldir");
- runner.setProperty(FlumeSinkProcessor.FLUME_CONFIG,
+ TestRunner runner = TestRunners.newTestRunner(ExecuteFlumeSource.class);
+ runner.setProperty(ExecuteFlumeSource.SOURCE_TYPE, "spooldir");
+ runner.setProperty(ExecuteFlumeSink.FLUME_CONFIG,
"tier1.sources.src-1.spoolDir = " + spoolDirectory.getAbsolutePath());
runner.run(1, false, true);
// Because the spool directory source is an event driven source, it may take some time for flow files to get
// produced. I'm willing to wait up to 5 seconds, but will bail out early if possible. If it takes longer than
// that then there is likely a bug.
int numWaits = 10;
- while (runner.getFlowFilesForRelationship(FlumeSourceProcessor.SUCCESS).size() < 4 && --numWaits > 0) {
+ while (runner.getFlowFilesForRelationship(ExecuteFlumeSource.SUCCESS).size() < 4 && --numWaits > 0) {
try {
TimeUnit.MILLISECONDS.sleep(500);
} catch (InterruptedException ex) {
@@ -140,9 +140,9 @@ public class FlumeSourceProcessorTest {
}
}
runner.shutdown();
- runner.assertTransferCount(FlumeSourceProcessor.SUCCESS, 4);
+ runner.assertTransferCount(ExecuteFlumeSource.SUCCESS, 4);
int i = 1;
- for (MockFlowFile flowFile : runner.getFlowFilesForRelationship(FlumeSourceProcessor.SUCCESS)) {
+ for (MockFlowFile flowFile : runner.getFlowFilesForRelationship(ExecuteFlumeSource.SUCCESS)) {
flowFile.assertContentEquals("record " + i);
i++;
}
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
index a2742aa3a1..59aab3c8ee 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
@@ -32,7 +32,7 @@
org.apache.nifinifi-flume-processors
- 0.1.0-incubating-SNAPSHOT
+ 0.2.0-incubating-SNAPSHOT
diff --git a/nifi/pom.xml b/nifi/pom.xml
index 3a5c6b78b0..66a328c9c5 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -809,7 +809,7 @@
org.apache.nifinifi-flume-nar
- 0.1.0-incubating-SNAPSHOT
+ 0.2.0-incubating-SNAPSHOTnar
@@ -864,10 +864,10 @@
2.0.0
- org.apache.derby
- derby
- 10.11.1.1
-
+ org.apache.derby
+ derby
+ 10.11.1.1
+
From e7e15facc4587a7cd0862d3b8f00a280b15392fa Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Sun, 12 Jul 2015 18:44:16 -0700
Subject: [PATCH 16/77] Added detailed documentation for the
ExecuteFlumeSource/Sink processors
Signed-off-by: Matt Gilman
---
.../processors/flume/ExecuteFlumeSink.java | 7 +-
.../processors/flume/ExecuteFlumeSource.java | 4 +-
.../org.apache.nifi.processor.Processor | 4 +-
.../additionalDetails.html | 159 ++++++++++++++++++
.../additionalDetails.html | 114 +++++++++++++
5 files changed, 280 insertions(+), 8 deletions(-)
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSink/additionalDetails.html
create mode 100644 nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSource/additionalDetails.html
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java
index 8ccb2d14aa..94251b43ed 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java
@@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import java.util.List;
import java.util.Set;
-import org.apache.flume.Context;
import org.apache.flume.EventDeliveryException;
import org.apache.flume.Sink;
import org.apache.flume.conf.Configurables;
@@ -44,14 +43,14 @@ import org.apache.nifi.processor.util.StandardValidators;
/**
* This processor runs a Flume sink
*/
-@Tags({"flume", "hadoop", "get", "sink"})
-@CapabilityDescription("Write FlowFile data to a Flume sink")
+@Tags({"flume", "hadoop", "put", "sink"})
+@CapabilityDescription("Execute a Flume sink. Each input FlowFile is converted into a Flume Event for processing by the sink.")
@TriggerSerially
public class ExecuteFlumeSink extends AbstractFlumeProcessor {
public static final PropertyDescriptor SINK_TYPE = new PropertyDescriptor.Builder()
.name("Sink Type")
- .description("The fully-qualified name of the Sink class")
+ .description("The component type name for the sink. For some sinks, this is a short, symbolic name (e.g. hdfs). For others, it's the fully-qualified name of the Sink class. See the Flume User Guide for details.")
.required(true)
.addValidator(createSinkValidator())
.build();
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java
index fa02750df6..fa4fbe6131 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java
@@ -50,13 +50,13 @@ import org.apache.nifi.processor.util.StandardValidators;
* This processor runs a Flume source
*/
@Tags({"flume", "hadoop", "get", "source"})
-@CapabilityDescription("Generate FlowFile data from a Flume source")
+@CapabilityDescription("Execute a Flume source. Each Flume Event is sent to the success relationship as a FlowFile")
@TriggerSerially
public class ExecuteFlumeSource extends AbstractFlumeProcessor {
public static final PropertyDescriptor SOURCE_TYPE = new PropertyDescriptor.Builder()
.name("Source Type")
- .description("The fully-qualified name of the Source class")
+ .description("The component type name for the source. For some sources, this is a short, symbolic name (e.g. spooldir). For others, it's the fully-qualified name of the Source class. See the Flume User Guide for details.")
.required(true)
.addValidator(createSourceValidator())
.build();
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index fae87277b0..f5b57e1099 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -12,5 +12,5 @@
# 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.
-org.apache.nifi.processors.flume.FlumeSourceProcessor
-org.apache.nifi.processors.flume.FlumeSinkProcessor
+org.apache.nifi.processors.flume.ExecuteFlumeSource
+org.apache.nifi.processors.flume.ExecuteFlumeSink
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSink/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSink/additionalDetails.html
new file mode 100644
index 0000000000..be8b99fe28
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSink/additionalDetails.html
@@ -0,0 +1,159 @@
+
+
+
+
+
+ ExecuteFlumeSink
+
+
+
+
+
Data Model
+
+ This processor executes an Apache Flume sink. FlowFiles are wrapped in Flume's
+ Event interface. The content of the FlowFile becomes the body of the Event and
+ the attributes of the FlowFile become Event headers. The following special
+ headers are also set:
+
+
+
+
Flume Event Header
+
FlowFile Attribute
+
+
+
nifi.entry.date
+
FlowFile#getEntryDate()
+
+
+
nifi.id
+
FlowFile#getId()
+
+
+
nifi.last.queue.date
+
FlowFile#getLastQueueDate()
+
+
+
nifi.lineage.identifiers.${i}
+
FlowFile#getLineageIdentifiers()[i]
+
+
+
nifi.lineage.start.date
+
FlowFile#getLineageStartDate()
+
+
+
nifi.size
+
FlowFile#getSize()
+
+
+
Warning
+
+ In NiFi, the contents of a FlowFile are accessed via a stream, but in Flume it is
+ stored in a byte array. This means the full content will be loaded into memory when
+ a FlowFile is processed by the ExecuteFlumeSink processor. You should consider the
+ typical size of the FlowFiles you'll process and the batch size, if any, your sink
+ is configured with when setting NiFi's heap size.
+
+
Configuration Details
+
+ This processor is designed to execute arbitrary Flume sinks. Most of the details
+ of configuring the sink is deferred to Flume's built-in configuration system.
+ For details on the available settings for each sink type, refer to the Flume
+ User Guide.
+ Configuring the Flume sink is a four step process:
+
+
+
Set the Sink Type property to a valid Flume sink type.
+
+ Set the Agent Name property to the name of the agent in your
+ Flume configuration. This is the prefix of the properties in the Flume
+ configuration file. Example: tier1
+
+
+ Set the Sink Name property to the name of the sink in your Flume
+ configuration. If Agent Name is tier1, then the Sink Name
+ is the value of the tier1.sinks property. Example: sink-1
+
+
+ Copy and paste the configuration for the sink from your Flume configuration
+ file into the Flume Configuration property. Assuming you're using
+ the same Agent Name and Sink Name as in the examples above, this will be all
+ of the properties that start with tier1.sinks.sink-1.
+ Do not copy the tier1.sinks.sink-1.type or
+ tier1.sinks.sink-1.channel properties.
+
+
+
Usage Example
+
+ Assuming you had the following existing Flume configuration file:
+
+
+
\ No newline at end of file
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSource/additionalDetails.html b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSource/additionalDetails.html
new file mode 100644
index 0000000000..3d1062b6f8
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/resources/docs/org.apache.nifi.processors.flume.ExecuteFlumeSource/additionalDetails.html
@@ -0,0 +1,114 @@
+
+
+
+
+
+ ExecuteFlumeSource
+
+
+
+
+
Data Model
+
+ This processor executes an Apache Flume source. Each Flume Event is turned into
+ a FlowFile. The content of the FlowFile is set to the body of the Event and
+ the Event headers become FlowFile attributes.
+
+
Configuration Details
+
+ This processor is designed to execute arbitrary Flume sources. Most of the details
+ of configuring the source is deferred to Flume's built-in configuration system.
+ For details on the available settings for each source type, refer to the Flume
+ User Guide.
+ Configuring the Flume source is a four step process:
+
+
+
Set the Source Type property to a valid Flume source type.
+
+ Set the Agent Name property to the name of the agent in your
+ Flume configuration. This is the prefix of the properties in the Flume
+ configuration file. Example: tier1
+
+
+ Set the Source Name property to the name of the source in your Flume
+ configuration. If Agent Name is tier1, then the Source Name
+ is the value of the tier1.sources property. Example: src-1
+
+
+ Copy and paste the configuration for the source from your Flume configuration
+ file into the Flume Configuration property. Assuming you're using
+ the same Agent Name and Source Name as in the examples above, this will be all
+ of the properties that start with tier1.sources.src-1.
+ Do not copy the tier1.sources.src-1.type or
+ tier1.sources.src-1.channel properties.
+
+
+
Usage Example
+
+ Assuming you had the following existing Flume configuration file:
+
+
+
\ No newline at end of file
From 2a3deed573b5d44bc79a49ea4bbdc5a49d4ebecd Mon Sep 17 00:00:00 2001
From: Joey Echeverria
Date: Sun, 12 Jul 2015 19:08:32 -0700
Subject: [PATCH 17/77] Fixed a couple of checkstyle violations
Signed-off-by: Matt Gilman
---
.../org/apache/nifi/processors/flume/ExecuteFlumeSink.java | 3 ++-
.../org/apache/nifi/processors/flume/ExecuteFlumeSource.java | 3 ++-
2 files changed, 4 insertions(+), 2 deletions(-)
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java
index 94251b43ed..57e027855e 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSink.java
@@ -50,7 +50,8 @@ public class ExecuteFlumeSink extends AbstractFlumeProcessor {
public static final PropertyDescriptor SINK_TYPE = new PropertyDescriptor.Builder()
.name("Sink Type")
- .description("The component type name for the sink. For some sinks, this is a short, symbolic name (e.g. hdfs). For others, it's the fully-qualified name of the Sink class. See the Flume User Guide for details.")
+ .description("The component type name for the sink. For some sinks, this is a short, symbolic name (e.g. hdfs)."
+ + " For others, it's the fully-qualified name of the Sink class. See the Flume User Guide for details.")
.required(true)
.addValidator(createSinkValidator())
.build();
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java
index fa4fbe6131..70f82ccc47 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/src/main/java/org/apache/nifi/processors/flume/ExecuteFlumeSource.java
@@ -56,7 +56,8 @@ public class ExecuteFlumeSource extends AbstractFlumeProcessor {
public static final PropertyDescriptor SOURCE_TYPE = new PropertyDescriptor.Builder()
.name("Source Type")
- .description("The component type name for the source. For some sources, this is a short, symbolic name (e.g. spooldir). For others, it's the fully-qualified name of the Source class. See the Flume User Guide for details.")
+ .description("The component type name for the source. For some sources, this is a short, symbolic name"
+ + " (e.g. spooldir). For others, it's the fully-qualified name of the Source class. See the Flume User Guide for details.")
.required(true)
.addValidator(createSourceValidator())
.build();
From 6f3039c8c38d6af399f5a896fb7879db111ab4fe Mon Sep 17 00:00:00 2001
From: Matt Gilman
Date: Tue, 14 Jul 2015 17:11:08 -0400
Subject: [PATCH 18/77] NIFI-589: - Tweaking license files.
---
nifi/nifi-assembly/NOTICE | 63 +++++++++++++++++++
.../src/main/resources/META-INF/NOTICE | 53 +++++++++++++++-
2 files changed, 113 insertions(+), 3 deletions(-)
diff --git a/nifi/nifi-assembly/NOTICE b/nifi/nifi-assembly/NOTICE
index 23c07b73fd..f71f98ce55 100644
--- a/nifi/nifi-assembly/NOTICE
+++ b/nifi/nifi-assembly/NOTICE
@@ -108,6 +108,69 @@ The following binary components are provided under the Apache Software License v
Spring Framework 4.1.4.RELEASE
Copyright (c) 2002-2015 Pivotal, Inc.
+ (ASLv2) Apache Flume
+ The following NOTICE information applies:
+ Apache Flume
+ Copyright 2011-2015 Apache Software Foundation
+
+ asynchbase is BSD-licensed software (https://github.com/OpenTSDB/asynchbase)
+
+ async is BSD-licensed software (https://github.com/stumbleupon/async)
+
+ jopt-simple is MIT licensed software (http://pholser.github.io/jopt-simple/license.html)
+
+ scala-library is BSD-like licensed software (http://www.scala-lang.org/license.html)
+
+ (ASLv2) Xalan
+ This product includes software developed by
+ The Apache Software Foundation (http://www.apache.org/).
+
+ Portions of this software was originally based on the following:
+
+ - software copyright (c) 1999-2002, Lotus Development Corporation., http://www.lotus.com.
+ - software copyright (c) 2001-2002, Sun Microsystems., http://www.sun.com.
+ - software copyright (c) 2003, IBM Corporation., http://www.ibm.com.
+ - voluntary contributions made by Ovidiu Predescu (ovidiu@cup.hp.com) on behalf of the
+ Apache Software Foundation and was originally developed at Hewlett Packard Company.
+
+ (ASLv2) Apache XML Commons XML APIs
+ Copyright 2006 The Apache Software Foundation.
+
+ This product includes software developed at
+ The Apache Software Foundation (http://www.apache.org/).
+
+ Portions of this software were originally based on the following:
+ - software copyright (c) 1999, IBM Corporation., http://www.ibm.com.
+ - software copyright (c) 1999, Sun Microsystems., http://www.sun.com.
+ - software copyright (c) 2000 World Wide Web Consortium, http://www.w3.org
+
+ (ASLv2) IRClib
+ The following NOTICE information applies:
+ IRClib -- A Java Internet Relay Chat library --
+ Copyright (C) 2002 - 2006 Christoph Schwering
+
+ (ASLv2) Apache Thrift
+ The following NOTICE information applies:
+ Apache Thrift
+ Copyright 2006-2010 The Apache Software Foundation.
+
+ (ASLv2) Apache MINA
+ The following NOTICE information applies:
+ Apache MINA Core
+ Copyright 2004-2011 Apache MINA Project
+
+ (ASLv2) opencsv (net.sf.opencsv:opencsv:2.3)
+
+ (ASLv2) Apache Velocity
+ The following NOTICE information applies:
+ Apache Velocity
+ Copyright (C) 2000-2007 The Apache Software Foundation
+
+ (ASLv2) ZkClient
+ The following NOTICE information applies:
+ ZkClient
+ Copyright 2009 Stefan Groschupf
+
(ASLv2) Apache Commons CLI
The following NOTICE information applies:
Apache Commons CLI
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/NOTICE b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/NOTICE
index 793746f194..b6511436b0 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/NOTICE
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/src/main/resources/META-INF/NOTICE
@@ -1,4 +1,4 @@
-nifi-social-media-nar
+nifi-flume-nar
Copyright 2015 The Apache Software Foundation
This product includes software developed at
@@ -33,6 +33,50 @@ The following binary components are provided under the Apache Software License v
Apache Flume
Copyright 2011-2015 Apache Software Foundation
+ asynchbase is BSD-licensed software (https://github.com/OpenTSDB/asynchbase)
+
+ async is BSD-licensed software (https://github.com/stumbleupon/async)
+
+ jopt-simple is MIT licensed software (http://pholser.github.io/jopt-simple/license.html)
+
+ scala-library is BSD-like licensed software (http://www.scala-lang.org/license.html)
+
+ (ASLv2) Xalan
+ This product includes software developed by
+ The Apache Software Foundation (http://www.apache.org/).
+
+ Portions of this software was originally based on the following:
+
+ - software copyright (c) 1999-2002, Lotus Development Corporation., http://www.lotus.com.
+ - software copyright (c) 2001-2002, Sun Microsystems., http://www.sun.com.
+ - software copyright (c) 2003, IBM Corporation., http://www.ibm.com.
+ - voluntary contributions made by Ovidiu Predescu (ovidiu@cup.hp.com) on behalf of the
+ Apache Software Foundation and was originally developed at Hewlett Packard Company.
+
+ (ASLv2) Apache Xerces Java
+ Copyright 1999-2007 The Apache Software Foundation
+
+ This product includes software developed at
+ The Apache Software Foundation (http://www.apache.org/).
+
+ Portions of this software were originally based on the following:
+ - software copyright (c) 1999, IBM Corporation., http://www.ibm.com.
+ - software copyright (c) 1999, Sun Microsystems., http://www.sun.com.
+ - voluntary contributions made by Paul Eng on behalf of the
+ Apache Software Foundation that were originally developed at iClick, Inc.,
+ software copyright (c) 1999.
+
+ (ASLv2) Apache XML Commons XML APIs
+ Copyright 2006 The Apache Software Foundation.
+
+ This product includes software developed at
+ The Apache Software Foundation (http://www.apache.org/).
+
+ Portions of this software were originally based on the following:
+ - software copyright (c) 1999, IBM Corporation., http://www.ibm.com.
+ - software copyright (c) 1999, Sun Microsystems., http://www.sun.com.
+ - software copyright (c) 2000 World Wide Web Consortium, http://www.w3.org
+
(ASLv2) IRClib
The following NOTICE information applies:
IRClib -- A Java Internet Relay Chat library --
@@ -126,7 +170,10 @@ The following binary components are provided under the Apache Software License v
Apache License Version 2.0 http://www.apache.org/licenses/.
(c) Daniel Lemire, http://lemire.me/en/
- (ASLv2) Servlet Specification API (org.mortbay.jetty:servlet-api:2.5-20110124)
+ (ASLv2) Jetty
+ The following NOTICE information applies:
+ Jetty Web Container
+ Copyright 1995-2015 Mort Bay Consulting Pty Ltd.
(ASLv2) Twitter4J
The following NOTICE information applies:
@@ -142,4 +189,4 @@ The following binary components are provided under the Apache Software License v
(ASLv2) ZkClient
The following NOTICE information applies:
ZkClient
- Copyright 2009 Stefan Groschupf
\ No newline at end of file
+ Copyright 2009 Stefan Groschupf
From 95b22713fc848005a526b3e71f7b19bb7ae1db00 Mon Sep 17 00:00:00 2001
From: Matt Gilman
Date: Tue, 14 Jul 2015 17:22:57 -0400
Subject: [PATCH 19/77] This closes #51 Feedback on been provided in NIFI-589
From 65c82cc0ea4ccde35b5ded34b6b84afe959f62cb Mon Sep 17 00:00:00 2001
From: Aldrin Piri
Date: Tue, 14 Jul 2015 18:39:17 -0400
Subject: [PATCH 20/77] This closes #67. Code has been merged as per NIFI-685
From ec6be9eaa3942e9ca9f8e4a1e788ab831e739514 Mon Sep 17 00:00:00 2001
From: Aldrin Piri
Date: Tue, 14 Jul 2015 18:42:06 -0400
Subject: [PATCH 21/77] This closes #70. Code has been merged as per NIFI-751
From 00344624855f0ed215857a01593c397dd39d99ec Mon Sep 17 00:00:00 2001
From: joewitt
Date: Thu, 16 Jul 2015 08:04:58 -0400
Subject: [PATCH 22/77] NIFI-768 staging changes needed for website in prep for
move to TLP site. Still need to update some links once git and svn move.
---
nifi-site/src/images/egg-logo.png | Bin 8626 -> 0 bytes
nifi-site/src/includes/disclaimer.hbs | 20 +--
nifi-site/src/includes/topbar.hbs | 2 +-
nifi-site/src/pages/html/download.hbs | 2 +-
nifi-site/src/pages/html/mailing_lists.hbs | 26 ++--
.../src/pages/markdown/licensing-guide.md | 5 +-
nifi-site/src/pages/markdown/quickstart.md | 10 +-
nifi-site/src/pages/markdown/release-guide.md | 145 +++++-------------
8 files changed, 64 insertions(+), 146 deletions(-)
delete mode 100644 nifi-site/src/images/egg-logo.png
diff --git a/nifi-site/src/images/egg-logo.png b/nifi-site/src/images/egg-logo.png
deleted file mode 100644
index c04e70da609807afccdd63ea8609f28dda78368e..0000000000000000000000000000000000000000
GIT binary patch
literal 0
HcmV?d00001
literal 8626
zcmV;jAx++iP)jvbDr}&&k;bWqmDZE8TeKEmA%%AI_jtc
zAPWPetf*>kFE*x*I{w!~M9M(c?CplsQAZtHqfW{?>e$CnCuJRV>|>~tvW~qD5efEo
z8}=pmZ=PlM<9pwmH!A0`e?HfE@Z_m8j!+@2l!`rJxzyR!+4a<=SKhE}Z?vn9-3t*J
z1vpjhtCjwRFMQ#+*4EZPIq$sl?hui+s%rOe6Zc67$vovA(Z1e%^9?N%TAR<9^Ny3w
zZW}*oazkTt8;C2F`?|WjI@fI6*s-o-)$$EC3|EEa=&?_J=1XskberlJS%}DWz$oBW
zRb5k^_MPv1XTIloYkGTor)}7<;i}VD{!>#VoK-0@`N8=#Rg{HK!B>{7rBhYi-noN4
zhkotr9wFteS6mU44{kl~xObm=(LuB4oR|r+IaRein>Vdkyy&IpH?Cc?2?IEXa}Z;R
zZRDJ__y6T(UtBa&?b=r$B1Zy8lk_Tg?(O#MZivXofqB4hfgh>r%K$gsbkpN;9Pbwd
z!Q|(jd+we8_|(t;YjYuJ=!=5vFKwgj>U^!etKQU@(xUZNk_KM0WQYI
z7H2J?wa$jcpIvgr-!9&rZ4!|gzzZaj0>Eji`uy&^#t_?cBk=b?q^g;{)sB&bh)f5*
z349p%1#s(i*IhSBRd37Z^YlIR;@^Mb_k~Nk@q$@obs?*R6X-5I>-9D?HkK=1)|&ke
z)OhjaIMZ-6mDhex&-ellzS6~=S3aMR{H=~XLdtK=yQ1yTIVW6w;%R4Jn#p7vTr78gZW8&ZkNaj2!+Kl#bIn|4>bMC8jPZ|;74
zPgSqkUG)#6J#Twb_CrL@11J6#sidv-bAam->E|!1pm;fH74HVCQggjJ1}?S!|_5
z#fC(c(z3t&)E9reyV@oqOMnA_GLQo{0+UrWs>*W-FdA3@tOw2|85{jvRUfU&cOh^D
zFp;F^T@5^-s!tEPW&oTI9F^9y0{DfhE~qADdd-W0gMr@w52|WhRgVDY0mlO4fH#0U
zRkeLLJeoZQ5jhI@6!6g?2m%u|JUNv!4&9$te;l!%!4$ihj$k}+GOg|vUeBILxwMcj
zzm+CB+00nFb%Sh{31j0>72u*J*AB*RKgHjDNzb5
z*mB>GFTHZ^E^B2)gF8DrN3B?~;`bX@MbpZxJHThZH7y*{HGyd|jmG>$Y;Xw8@%|M3<7ufp
z!5aTW#>cO-JeW#^p-VQS1590Cm=lEks
z4-5YNDPC38ea;bM#7H@B-W3hU9QB?XKJdXmzt;2phH|;o`S`>4-&!mddb61fnQR{4
z_miH|1Q@T1l(8kUNl$5QiAp6T43qClxo_hom;KGZ?P~i({QoaJr1tz86K
zn}#huYUt{k&DZ|=KX{~CQra6B5mH`w;f3DWXTIyo^Dg}K-{$i9G1f-Ck34k$kBWWW
z-Puf@Og2v@lOgc4n83$3h5;=PIA^iW5mhQ!8xmGZRKhZ0sYIzSq(NP3;gs)<&wij@DVs+Xnb`bh!Ms;b>0(hQsloIpa#BS}aO
zkO9V|*97=yRlTmNJsDLEYmF&I53~G!5h;tG-1J;sAaSrDK
zB5N_m#~2?klbZz$o0JTU3jy9IBKH9k(&GU581TIvl&_vQZAmA`e+L`{6iIrx-a%y!
ztlHm7!>g*{4z9JKYF|VoLqf>@I>%L|x2N9?z#jc04HdWEdh548^O?__kjZ4C2@@t1
z+S}V7Y0f-7w;|e3j?f8l0~ParM(gu5x=zXDUePf@$a^~rbOb#Vg84kyx`9nCs~Fpn
zVR~ndNtwr37qrployH_t$9jJbqwEYC++DQ!?`53*JssW&%nF|=uJ=b5r%Ep!;bAne
zV8t{yb489GHj<&6V{$z~cO^Xa{L|u-4k>w8|Mp1J7j$^5yud#92YiuTy%?=PUlOgM85nr6a-y+vLBjP(F
z9-e^1#SUXFf$_0{!B`JOW&yi0Zo0T?|G4d>?}(HqrQdxdikdHy5V8QwRMib4ausk(
zRhieT_D=!+qyIoerm5rzg
zzka)l2UlBq?_9tuBQ-AFEu@@x$|?S7mtF9Q>HE)qr*pVvi(h$Yb;t76`FtaRpGkK60ROKl`4hm^z%e3nn5w>7m3E|vTnc;$_&|E@PF2-Z+<-(0bh3!_02fwY
zS5-Sjp~1
z(x3e3Plqr(S^D~o<*rAMZa$HsYo9BttYLMh|o!^LVuY_c*TYc${R3Zt`&i}de&Wi}8!lZWZQhkxEzi$2%IZ&6
z)^{BlzjFIM{NUk-^~K%sY
z8XL*y8p&jHWO8}+eJ>UfXN>PmD!M|9b7It4<6w%lo!^2edk5y^2fY;K)Z>Z|^Rbxua)-SJzfsd$amx#2c<@!Zn
z3%mpzlOAVOH4>3=z<)CEQs~DDRh_<_G3ANCV`;wks_IiIp}$2kt{ps|*{-TXA^1Lt
zM=z4=OI>^@wCq1|qm0_GV0zXqnJB&SD0w#xP4lJE9!{&jNTa?UX_Q83$Q#$z^DMp6
z2;O9AYFenp!bbW`Hx+f5u3oy`QZ~h_*&J`A!|$Qk0!?E*Mpv4cS`NrXV`$Yfy}=m9
z*$^vjG|6eS#P4EM{4mRM=Q1vQk+t3!8ss<{qE(mQm+L%wWp3gO=L&t$+(fu`J@XeY
zab=BcAKK-iYrcUx#(lp-g4B@$Da^|mA(gmdG9^OXqanENttUP-`Ge#lgG;h
zRy;5Ejqi+zQ_}GyPMmY%RE=8eg&KLb9GY^eNU2<)TqsZ|^-?MpDE9TeaK&HEdvF*x
zd^pes#H#vujr0>p$hTQlU#KGGZNLwK*}ziZnQCQ9I{$bk$vow6Nfa&zr^%02i#Q@O
z2RIyP1(v1heW|s}OcFD@$I`qlz&p~S^R;R`BO;T4ccx{Ez+&LZx06A4CL?n$p3UVS
zdI#?tyGlkM{)Vw@*JN6{%Q^p8&AXSKSG45qA&hQbOk=T!Y@8v>Z1I|2S1p(Db+XAd
zR9voa4JBsL7X3Fpfdgic)uqzi?C>L1+g&14uF#zmGWsm5T_*+Uqpu+%<2f=`7*lM-
zkz>3bHg!|(f~%g$PnllHj2naNEuujdBmggmzP^Og%C-FJ?)$m^nP-MJ2HkCnp4P^e
zlMb16)RF2me)0Loe;2uO*qCW1@cncgnI-FI@%$hb@tiMSEaExwjq_4cI-H1g#yE>L
z#`v+XYGRC+&SM%d$p@elcHE-cP}K!*lK#Z@lk_>@x$PuI!2N?xcWfu`i@-J_j;g-F
z)=DH)bu;kDcIsIJ{Cqp@eOn+5^+i%m_qz_3^GXpA!Cdek_oZ|
zeilEI!OM6~Jc)hZkA2T`V*J<`lhh4yGVZu{8U7OKSJA)>JBl~(NjpKc<
zcWTizt+SGEbwPQ3==+C~kKW(_{|yTMZ0#+)Dh*9brMcyu8gE+YZ0Q8Zy(*&`Cuz1Y
znw)g-NAFw>yE_=&JVT7XWL2~0fA2Iq{d13Ha}$bME@5AIg~|&{*ces_Hg2N%kQubJ
zjAGhw?{}BP*Xuuf?parV{u_Un&F6!Me)Hq|qNq}8Y-lFm*i1uX^4*wg#LEQM3j&*w
zz>4q3#!pGv-w5YS3mF})^C1WR6j`GrDHZzoJEN
zkm*@D!7onym~C8rqxQx3#PPTr$8;}wp_^BXlP=CTc2i0J!1evphMMH0nM
zLP&{yYAxylc$ZiKCV-UGTDqsvr3H#@~vV;*0Zr<3zkTnF8^h@l706iE)uL
z7?WsZ6OGPbPC6Wp7*lrHVK)0O*xG1d4sa3h{bAO(FQSEH+Ga2=1O9anx8(!C?KRF%
zWY;XS-qsja;H=MGczph#Sw|cjhULQR3!Z<&m;mDkm>|Fp0s=o#@>CJg1;)6*_;Fx@
z*z-(mjB&;ohi9CJaTrfApauoBx?!o&-IfD~8ScyEJvGi9G0eK^sAEJhEK-*94F^r0
zGIL^k`{HG)8V9~lVEoh_D}xDq#P?N1oDp%x7$-*Jl!UP{LF{>6?0H^FPA}D@ngO%B
ztv+1hsFZ{zz>|1KHlsVjy*1A5>E5G`I(CR*k+P||bz1YNF~Pz=KL3XC41VC_`z9r(
z!8ZnDM2+#C7~_oboo51PjEMtdTtdi9?BO{N6jZ$`LXv2OC1oWpC8LXz{j#YHSIP5B
z;9B5W;Ag=3s``gv)>TIxBZ6UxfuqNdo4H}#>h)n*v5n2mU<}4f>{LAyU=jyrXN*zL
z80{BTJr~iv5CL!?f=f2%Bndfban2Ew
zd^=oW?%cVck%lq8q2{rjASnCBO-FXs&>^V-$)Wc;4f>G4}i0P
zv(hLCpRLhr{TTz1_WJP>iO4-GO&^R)84x+%3;~HV_K_r>D_5!NU{k^VdpVcnSxgy}
zWJsRpUjv^LkOGwV~vkbEGkLj4^l&>?fQw>KVY8WW4Bk3Mf7bh_%*;I1__77mGM&
za9a%1#W{zIZ5j*K;cT2lx{Ko>yKM~v|3+;>R^wFQj=?8Wap^8<6LK&v1Adi?X?qJ(
zfS-uSy9XzT$e#m0Wm`fHMgaUM^(EaE{3}V^q&IsW9|V3UBDJj0?&K{@NEikw#q*8v
zOvxC7@z`QCzGcIZYEe}&>ak^-BZ2KFWNhL%HqJQ{<4o+FIGi|9gF_-0CGmRV26D%hAvn{NVc5K!D`$?=-24fZQCE%kZL9Yg5G}}5rj|7$iUk9GtPWneBfkzU_
zycQObI9jjTPWp$2(XRI~5Nxa(8%cV@D@Y>v4Tb?ut~#z&2P6>*?*ZNtcFsdq^aLv1
z$r#eZ;H4d;$Cj|D3C#x_dqKn@#3F)AUji}CIYaC`=bTB%>5O&OI8hVX2pc6!7dRK;
zY((rVv9+ByeC-F@C{}hCQVnLUANF5V^^Vk#{R@NAs{K1_*?9t7sj9z7uW>(#b8>;i
zLAeN&_P{iWQPoS*HVDa|{Ln#pCXw8aPoh{EjOT%4(@-d1B++UP4neXrE=#ARW&5Wn
zW^X6ojjH;?w9UUG3FA3foIa*%zn18{95`xUjPtrZ47sQL{vH3b31jR)NpDL`&N|Pz$T;gfr)pH=RCKji
zXSb{?V$(5YA?$vwMuYYx)Do=?Rmuku*-u0YBw<6kNaijVv$az~tx)^@^N_VXWR#SM
zX-JS-5%Nx|A`8^^ijPP{+~rjLoK+YsV%J)DNf`KFZBNAgYHG;PkL^W;sD5sTvF&sg
z)E-h)W4EnAv!^gDiymWGimCB-OU+c%4~boW{2gO(P6b3k#Uai)qt1HHs;AB-qN}qe
z3M*796`ZYLZH352MAj18sAKNj+xK-Bt7o@!{)ZxRD#;)I)6{;WSBE26$F^eh*&^~R
z$+&bdmJf1;T2!O_yMUjk#t`RG+c^LAbj~ysk@gyOoF*dolJxxE;KDV~&bEQaNUz-~
zBDbjO)zltm1AIl2Wdc`|%){2g{GDjWc9Cj}t0jYFB)`u@lIKxQwbCK$XU-i>qX8Qwbrw7>}ec(u~lPjh;pyTZs*`Eb`gW80I
zWQn?d{P%Y9oKF%P_e~X3saac1qXd#3wJpup&er$ej||E5)b(r~e7YYYNwBhA!8yQ<
zz^_P*7G?sUB?*?+AI<(-Bq2#p+{J6{6oy60>uZq6N_=2Q<2mVK>A5(
zt+7r$>#XPOz-Be}Tvab=qY5?*lhw9dm@bPKzH#jh-`?k!T-RiEtZRo+^jfW2Sqr#(
zh~;k?bo^5WLT2tP76D(WI=+d4XI$C
z*0atUjn#|8*jO7@f_Rl-JGu_9lluoQxygfH|D&_v~=uoHbTsPg5~9)v|gv6;AYvB_A{h9S`w
za)wlF{LJU)4m~3J-bA_}&{x^bBuupn>LO0U9d
zl9jBt86Ie|G~l0+D5Qpsv^To)&FWFrc_iZMj`BH@6`@bx1Jg)%0$
zckRfRQ&m@s$nk0Us~8xIDv8GN4&de;sryy+HA$ksUr1t$J1ebcDap9?Ns>9(?Yg6>
z>Q6-E5fW0JOERa}K=K?{Qu}#r(*}TX4^R-f~(~W79GD
zd?Wc>j!ZUBHj~H8c+ME#iHO4(HGp#@<4sj#U2JSvkuXe`Ca7AbpjhZ#vud5ad+yx1
zZ;Pnk>2!n)8awW6S&zgga2$z7aPZ1rL#a!4b>2=y^26~f-G%Gza+}`@klm&1(X0Rf
z0V7F7K~%jyP2
-
\ No newline at end of file
+
diff --git a/nifi-site/src/includes/topbar.hbs b/nifi-site/src/includes/topbar.hbs
index 7010ac0dcc..f1125fbf3e 100644
--- a/nifi-site/src/includes/topbar.hbs
+++ b/nifi-site/src/includes/topbar.hbs
@@ -53,7 +53,7 @@
diff --git a/nifi-site/src/pages/markdown/licensing-guide.md b/nifi-site/src/pages/markdown/licensing-guide.md
index ac7fb3b549..9c928b1354 100644
--- a/nifi-site/src/pages/markdown/licensing-guide.md
+++ b/nifi-site/src/pages/markdown/licensing-guide.md
@@ -4,11 +4,11 @@ title: Apache NiFi Licensing Guide
# Apache NiFi Release Guidelines
-This document provides guidance to contributors of Apache NiFi (incubating) to help properly account for licensing, notice, and legal requirements.
+This document provides guidance to contributors of Apache NiFi to help properly account for licensing, notice, and legal requirements.
#### Disclaimer:
-This document is not written by lawyers. The guidance in this document is meant to compliment Apache Incubator and Apache Software Foundation guides and policies. If anything in this document is inconsistent with those then it is a defect in this document.
+This document is not written by lawyers. The guidance in this document is meant to compliment Apache Software Foundation guides and policies. If anything in this document is inconsistent with those then it is a defect in this document.
## Background Material
@@ -17,7 +17,6 @@ This document is not written by lawyers. The guidance in this document is meant
- [ASF Licensing How-To](http://www.apache.org/dev/licensing-howto.html)
- [ASF Legal Resolved](http://www.apache.org/legal/resolved.html)
- [ASF Release Policy](http://www.apache.org/dev/release.html)
-- [ASF Incubator License and Notice Guidance](http://incubator.apache.org/guides/releasemanagement.html#note-license-and-notice)
## How to consistently apply licensing/legal notice information for Apache NiFi
diff --git a/nifi-site/src/pages/markdown/quickstart.md b/nifi-site/src/pages/markdown/quickstart.md
index 0851a4dac5..015621a78d 100644
--- a/nifi-site/src/pages/markdown/quickstart.md
+++ b/nifi-site/src/pages/markdown/quickstart.md
@@ -20,7 +20,7 @@ Track issues on the "NIFI" Project on the Apache Jira ([browse][jira]).
To check out the code:
```
-git clone http://git-wip-us.apache.org/repos/asf/incubator-nifi.git
+git clone http://git-wip-us.apache.org/repos/asf/nifi.git
```
Then checkout the 'develop' branch
@@ -137,11 +137,11 @@ is ready for use:
2014-12-09 00:42:03,540 INFO [main] org.apache.nifi.web.server.JettyServer NiFi has started. The UI is available at the following URLs:
-[adminguide]: https://nifi.incubator.apache.org/docs/nifi-docs/html/administration-guide.html
+[adminguide]: https://nifi.apache.org/docs/nifi-docs/html/administration-guide.html
[maven]: http://maven.apache.org/
[jira]: https://issues.apache.org/jira/browse/NIFI
[git]: http://git-scm.com/
-[gitbrowse]: https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=summary
-[gitrepo]: http://git-wip-us.apache.org/repos/asf/incubator-nifi.git
-[githubrepo]: https://github.com/apache/incubator-nifi
+[gitbrowse]: https://git-wip-us.apache.org/repos/asf?p=nifi.git;a=summary
+[gitrepo]: http://git-wip-us.apache.org/repos/asf/nifi.git
+[githubrepo]: https://github.com/apache/nifi
diff --git a/nifi-site/src/pages/markdown/release-guide.md b/nifi-site/src/pages/markdown/release-guide.md
index b1df726655..67a569c4e0 100644
--- a/nifi-site/src/pages/markdown/release-guide.md
+++ b/nifi-site/src/pages/markdown/release-guide.md
@@ -14,15 +14,11 @@ capacity of a [Release Manager][release-manager] (RM).
- [Apache License V2.0][apache-license]
- [Apache Legal License/Resolved][apache-legal-resolve]
- [Apache How-to Apply License][apache-license-apply]
- - [Apache Incubator Branding Guidelines][incubator-branding-guidelines]
- These documents are necessary for someone acting as the RM
- [Apache Encryption Software / ECCN Info][apache-encryption]
- [Apache Release Policy][apache-release-policy]
- [Apache Release Guide][apache-release-guide]
- - [Apache Incubator Release Guide][apache-incubator-release-guide]
- - [another Apache Incubator Release Guide][another-apache-incubator-release-guide]
- - [Apache Incubator Policy][apache-incubator-policy]
- These documents are helpful for general environmental setup to perform releases
- [Apache PGP Info][apache-pgp]
@@ -31,34 +27,25 @@ capacity of a [Release Manager][release-manager] (RM).
## The objective
-Our aim is to produce and official Apache release.
+Our aim is to produce an official Apache release.
The following is a list of the sorts of things that will be validated and are the basics to check
when evaluating a release for a vote.
## What to validate and how to Validate a release
-There are two lists here: one of specific incubator requirements, and another of general Apache requirements.
-
-### Incubator:
-
- - Do the resulting artifacts have 'incubating' in the name?
- - Is there a DISCLAIMER file in the source root that meets the requirements of the Incubator branding guidelines?
-
-### General Apache Release Requirements:
-
- Are LICENSE and NOTICE file present in the source root and complete?
- Specifically look in the *-sources.zip artifact and ensure these items are present at the root of the archive.
- Evaluate the sources and dependencies. Does the overall LICENSE and NOTICE appear correct? Do all licenses fit within the ASF approved licenses?
- Here is an example path to a sources artifact:
- - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi/0.0.1-incubating/nifi-0.0.1-incubating-source-release.zip`
+ - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi/0.0.1/nifi-0.0.1-source-release.zip`
- Is there a README available that explains how to build the application and to execute it?
- Look in the *-sources.zip artifact root for the readme.
- Are the signatures and hashes correct for the source release?
- Validate the hashes of the sources artifact do in fact match:
- - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi/0.0.1-incubating/nifi-0.0.1-incubating-source-release.zip.md5`
- - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi/0.0.1-incubating/nifi-0.0.1-incubating-source-release.zip.sha1`
+ - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi/0.0.1/nifi-0.0.1-source-release.zip.md5`
+ - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi/0.0.1/nifi-0.0.1-source-release.zip.sha1`
- Validate the signature of the source artifact. Here is an example path:
- - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi/0.0.1-incubating/nifi-0.0.1-incubating-source-release.zip.asc`
+ - `https://repository.apache.org/service/local/repositories/orgapachenifi-1011/content/org/apache/nifi/nifi/0.0.1/nifi-0.0.1-source-release.zip.asc`
- Need a quick reminder on how to [verify a signature](http://www.apache.org/dev/release-signing.html#verifying-signature)?
- Do all sources have necessary headers?
- Unzip the sources file into a directory and execute `mvn install -Pcontrib-check`
@@ -70,12 +57,10 @@ There are two lists here: one of specific incubator requirements, and another of
- The community is contributing to a series of JIRA tickets assigned to the next release
- The number of tickets open/remaining for that next release approaches zero
- A member of the community suggests a release and initiates a discussion
- - Someone volunteers to be an RM for the release (can be a committer but apache guides indicate preference is a PPMC member)
+ - Someone volunteers to be an RM for the release (can be a committer but apache guides indicate preference is a PMC member)
- A release candidate is put together and a vote sent to the team.
- - If the team rejects the vote the issues noted are resolved and another RC is generated
- - Once a vote is accepted within the NiFi PPMC for a release candidate then the vote is sent to the IPMC
- - If the IPMC rejects the vote then the issues are resolved and a new RC prepared and voted upon within the PPMC
- - If the IPMC accepts the vote then the release is 'releasable' and can be placed into the appropriate 'dist' location, maven artifacts released from staging.
+ - If the NiFi community rejects the vote the issues noted are resolved and another RC is generated
+ - If the NiFi community accepts the vote then the release is 'releasable' and can be placed into the appropriate 'dist' location, maven artifacts released from staging.
## The mechanics of the release
@@ -88,7 +73,7 @@ At this point you're on the latest 'develop' branch and are able to build the en
```
Create a JIRA ticket for the release tasks and use that ticket number for the commit messages. For example we'll consider NIFI-270 as our ticket. Also
-have in mind the release version you are planning for. For example we'll consider '0.0.1-incubating'.
+have in mind the release version you are planning for. For example we'll consider '0.0.1'.
Create the next version in JIRA if necessary so develop work can continue towards that release.
@@ -136,18 +121,18 @@ This command `mvn install -Pcontrib-check` should be run as well to help validat
Now its time to have maven prepare the release so execute `mvn release:prepare -Psigned_release -DscmCommentPrefix="NIFI-270-RC1 " -Darguments="-DskipTests"`.
Maven will ask:
-`What is the release version for "Apache NiFi"? (org.apache.nifi:nifi) 0.0.1-incubating: :`
+`What is the release version for "Apache NiFi"? (org.apache.nifi:nifi) 0.0.1: :`
Just hit enter to accept the default.
Maven will then ask:
-`What is SCM release tag or label for "Apache NiFi"? (org.apache.nifi:nifi) nifi-0.0.1-incubating: : `
+`What is SCM release tag or label for "Apache NiFi"? (org.apache.nifi:nifi) nifi-0.0.1: : `
-Enter `nifi-0.0.1-incubating-RC1` or whatever the appropriate release candidate (RC) number is.
+Enter `nifi-0.0.1-RC1` or whatever the appropriate release candidate (RC) number is.
Maven will then ask:
-`What is the new development version for "Apache NiFi"? (org.apache.nifi:nifi) 0.0.2-incubating-SNAPSHOT: :`
+`What is the new development version for "Apache NiFi"? (org.apache.nifi:nifi) 0.0.2-SNAPSHOT: :`
Just hit enter to accept the default.
@@ -175,7 +160,7 @@ If all looks good then push the branch to origin `git push origin NIFI-270`
If it is intended that convenience binaries will be provided for this release then the community has requested that
a copy it be made available for reviewing of the release candidate. The convenience binary, its hashes, and signature
should be placed here:
- - https://dist.apache.org/repos/dist/dev/incubator/nifi
+ - https://dist.apache.org/repos/dist/dev/nifi
If anything isn't correct about the staged artifacts you can drop the staged repo from repository.apache.org and delete the
local tag in git. If you also delete the local branch and clear your local maven repository under org/apache/nifi then it is
@@ -183,22 +168,22 @@ as if the release never happened. Before doing that though try to figure out wh
that you can pretty easily test the release process until you get it right. The `mvn versions:set ` and `mvn versions:commit `
commands can come in handy to help do this so you can set versions to something clearly release test related.
-Now it's time to initiate a vote within the PPMC. Send the vote request to `dev@nifi.incubator.apache.org`
-with a subject of `[VOTE] Release Apache NiFi 0.0.1-incubating`. The following template can be used:
+Now it's time to initiate a vote within the NiFi community. Send the vote request to `dev@nifi.apache.org`
+with a subject of `[VOTE] Release Apache NiFi 0.0.1`. The following template can be used:
```
Hello
I am pleased to be calling this vote for the source release of Apache NiFi
-nifi-0.0.1-incubating.
+nifi-0.0.1.
The source zip, including signatures, digests, etc. can be found at:
https://repository.apache.org/content/repositories/orgapachenifi-1011
-The Git tag is nifi-0.0.1-incubating-RC1
+The Git tag is nifi-0.0.1-RC1
The Git commit ID is 72abf18c2e045e9ef404050e2bffc9cef67d2558
-https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=commit;h=72abf18c2e045e9ef404050e2bffc9cef67d2558
+https://git-wip-us.apache.org/repos/asf?p=nifi.git;a=commit;h=72abf18c2e045e9ef404050e2bffc9cef67d2558
-Checksums of nifi-0.0.1-incubating-source-release.zip:
+Checksums of nifi-0.0.1-source-release.zip:
MD5: 5a580756a17b0573efa3070c70585698
SHA1: a79ff8fd0d2f81523b675e4c69a7656160ff1214
@@ -206,7 +191,7 @@ Release artifacts are signed with the following key:
https://people.apache.org/keys/committer/joewitt.asc
KEYS file available here:
-https://dist.apache.org/repos/dist/release/incubator/nifi/KEYS
+https://dist.apache.org/repos/dist/release/nifi/KEYS
8 issues were closed/resolved for this release:
https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&version=12329307
@@ -214,58 +199,14 @@ https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&versio
The vote will be open for 72 hours.
Please download the release candidate and evaluate the necessary items including checking hashes, signatures, build from source, and test. The please vote:
-[ ] +1 Release this package as nifi-0.0.1-incubating
+[ ] +1 Release this package as nifi-0.0.1
[ ] +0 no opinion
[ ] -1 Do not release this package because because...
```
-A release vote is majority rule. So wait 72 hours and see if there are at least 3 binding (in the PPMC sense of binding) +1 votes and no more negative votes than positive.
-If so forward the vote to the IPMC. Send the vote request to `general@incubator.apache.org` with a subject of
-`[VOTE] Release Apache NiFi 0.0.1-incubating`. The following template can be used:
-
-```
-Hello
-
-The Apache NiFi PPMC has voted to release Apache NiFi 0.0.1-incubating.
-The vote was based on the release candidate and thread described below.
-We now request the IPMC to vote on this release.
-
-Here is the PPMC voting result:
-X +1 (binding)
-Y -1 (binding)
-
-Here is the PPMC vote thread: [URL TO PPMC Vote Thread]
-
-The source zip, including signatures, digests, etc. can be found at:
-https://repository.apache.org/content/repositories/orgapachenifi-1011
-
-The Git tag is nifi-0.0.1-incubating-RC1
-The Git commit ID is 72abf18c2e045e9ef404050e2bffc9cef67d2558
-https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git;a=commit;h=72abf18c2e045e9ef404050e2bffc9cef67d2558
-
-Checksums of nifi-0.0.1-incubating-source-release.zip:
-MD5: 5a580756a17b0573efa3070c70585698
-SHA1: a79ff8fd0d2f81523b675e4c69a7656160ff1214
-
-Release artifacts are signed with the following key:
-https://people.apache.org/keys/committer/joewitt.asc
-
-KEYS file available here:
-https://dist.apache.org/repos/dist/release/incubator/nifi/KEYS
-
-8 issues were closed/resolved for this release:
-https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&version=12329307
-
-The vote will be open for 72 hours.
-Please download the release candidate and evaluate the necessary items including checking hashes, signatures, build from source, and test. The please vote:
-
-[ ] +1 Release this package as nifi-0.0.1-incubating
-[ ] +0 no opinion
-[ ] -1 Do not release this package because because...
-```
-
-Wait 72 hours. If the vote passes then send a vote result email. Send the email to `general@incubator.apache.org, dev@nifi.incubator.apache.org`
-with a subject of `[RESULT][VOTE] Release Apache NiFi 0.0.1-incubating`. Use a template such as:
+A release vote is majority rule. So wait 72 hours and see if there are at least 3 binding (PMC members) +1 votes and no more negative votes than positive.
+If the vote passes then send a vote result email. Send the email to `dev@nifi.apache.org`
+with a subject of `[RESULT][VOTE] Release Apache NiFi 0.0.1`. Use a template such as:
```
Hello
@@ -277,20 +218,20 @@ Y -1 (binding) votes
Thanks to all who helped make this release possible.
-Here is the IPMC vote thread: [INSERT URL OF IPMC Vote Thread]
+Here is the PMC vote thread: [INSERT URL OF PMC Vote Thread]
```
Now all the voting is done and the release is good to go.
Here are the steps of the release once the release is approved:
-1. Upload source-release artifacts to dist. If the release version is 0.0.1-incubating then upload them (zip, asc, md5, sha1) to
-`https://dist.apache.org/repos/dist/release/incubator/nifi/0.0.1-incubating`
+1. Upload source-release artifacts to dist. If the release version is 0.0.1 then upload them (zip, asc, md5, sha1) to
+`https://dist.apache.org/repos/dist/release/nifi/0.0.1`
2. To produce binary convenience release build the application from the raw source in staging. For each binary convenience artifact:
- - Generate ascii armored detached signature by running `gpg -a -b nifi-0.0.1-incubating-bin.tar.gz`
- - Generate md5 hash summary by running `md5sum nifi-0.0.1-incubating-bin.tar.gz | awk '{ printf substr($0,0,32)}' > nifi-0.0.1-incubating-bin.tar.gz.md5`
- - Generate sha1 hash summary by running `sha1sum nifi-0.0.1-incubating-bin.tar.gz | awk '{ printf substr($0,0,40)}' > nifi-0.0.1-incubating-bin.tar.gz.sha1`
+ - Generate ascii armored detached signature by running `gpg -a -b nifi-0.0.1-bin.tar.gz`
+ - Generate md5 hash summary by running `md5sum nifi-0.0.1-bin.tar.gz | awk '{ printf substr($0,0,32)}' > nifi-0.0.1-bin.tar.gz.md5`
+ - Generate sha1 hash summary by running `sha1sum nifi-0.0.1-bin.tar.gz | awk '{ printf substr($0,0,40)}' > nifi-0.0.1-bin.tar.gz.sha1`
- Upload the bin, asc, sha1, md5 for each binary convenience build to the same location as the source release
3. In repository.apache.org go to the staging repository and select `release` and follow instructions on the site.
@@ -301,29 +242,29 @@ Here are the steps of the release once the release is approved:
6. Update the NiFi website to point to the new download(s)
-7. Update the NiFi incubator status page to indicate NEWS of the release
+7. Update the NiFi Web Page to indicate NEWS of the release as appropriate
8. In Jira mark the release version as 'Released' and 'Archived' through 'version' management in the 'administration' console.
9. Wait 24 hours then send release announcement.
- See [here][release-announce] for an understanding of why you need to wait 24 hours
- - Then create an announcement like the one shown below addressed to 'announce@apache.org, general@incubator.apache.org, dev@nifi.incubator.apache.org' with a reply-to of 'general@incubator.apache.org'.
+ - Then create an announcement like the one shown below addressed to 'announce@apache.org, dev@nifi..apache.org' with a reply-to of 'dev@nifi.apache.org'.
- The email has to be sent from an apache.org email address and should be by the release manager of the build.
```
-SUBJECT: [ANNOUNCE] Apache NiFi 0.0.2-incubating release
+SUBJECT: [ANNOUNCE] Apache NiFi 0.0.2 release
BODY:
Hello
-The Apache NiFi team would like to announce the release of Apache NiFi 0.0.2-incubating.
+The Apache NiFi team would like to announce the release of Apache NiFi 0.0.2.
Apache NiFi is an easy to use, powerful, and reliable system to process and distribute data. Apache NiFi was made for dataflow. It supports highly configurable directed graphs of data routing, transformation, and system mediation logic.
More details on Apache NiFi can be found here:
-http://nifi.incubator.apache.org/
+http://nifi.apache.org/
The release artifacts can be downloaded from here:
-http://nifi.incubator.apache.org/downloads/
+http://nifi.apache.org/downloads/
Maven artifacts have been made available here:
https://repository.apache.org/content/repositories/releases/org/apache/nifi/
@@ -334,13 +275,9 @@ https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12316020&versio
Thank you
The Apache NiFi team
-----
-DISCLAIMER
-
-Apache NiFi is an effort undergoing incubation at The Apache Software Foundation (ASF), sponsored by Apache Incubator. Incubation is required of all newly accepted projects until a further review indicates that the infrastructure, communications, and decision making process have stabilized in a manner consistent with other successful ASF projects. While incubation status is not necessarily a reflection of the completeness or stability of the code, it does indicate that the project has yet to be fully endorsed by the ASF.
```
-[quickstart-guide]: http://nifi.incubator.apache.org/development/quickstart.html
+[quickstart-guide]: http://nifi.apache.org/development/quickstart.html
[release-manager]: http://www.apache.org/dev/release-publishing.html#release_manager
[release-announce]: http://www.apache.org/dev/release.html#release-announcements
[apache-license]: http://apache.org/licenses/LICENSE-2.0
@@ -349,10 +286,6 @@ Apache NiFi is an effort undergoing incubation at The Apache Software Foundation
[apache-encryption]: http://www.apache.org/licenses/exports/
[apache-release-policy]: http://www.apache.org/dev/release.html
[apache-release-guide]: http://www.apache.org/dev/release-publishing
-[apache-incubator-release-guide]: http://incubator.apache.org/guides/releasemanagement.html
-[another-apache-incubator-release-guide]: http://incubator.apache.org/guides/release.html
-[apache-incubator-policy]: http://incubator.apache.org/incubation/Incubation_Policy.html
-[incubator-branding-guidelines]: http://incubator.apache.org/guides/branding.html
[apache-pgp]: http://www.apache.org/dev/openpgp.html
[apache-release-signing]: http://www.apache.org/dev/release-signing.html
-[apache-guide-publish-maven]: http://www.apache.org/dev/publishing-maven-artifacts.html
\ No newline at end of file
+[apache-guide-publish-maven]: http://www.apache.org/dev/publishing-maven-artifacts.html
From adddd3fab8f1244e02e8bdcf3f775af35ab37d4a Mon Sep 17 00:00:00 2001
From: Matt Gilman
Date: Thu, 16 Jul 2015 10:59:34 -0400
Subject: [PATCH 23/77] NIFI-656: - Updating version of flume bundle that was
added during release vote.
---
.../nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml | 4 ++--
.../nifi-flume-bundle/nifi-flume-processors/pom.xml | 2 +-
nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml | 4 ++--
3 files changed, 5 insertions(+), 5 deletions(-)
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
index a9bbe6f911..8d615a8ebf 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-nar/pom.xml
@@ -17,10 +17,10 @@
org.apache.nifinifi-flume-bundle
- 0.2.0-incubating-SNAPSHOT
+ 0.2.1-incubating-SNAPSHOTnifi-flume-nar
- 0.2.0-incubating-SNAPSHOT
+ 0.2.1-incubating-SNAPSHOTnar
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
index 167aa6e943..fcf42ee78c 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/nifi-flume-processors/pom.xml
@@ -17,7 +17,7 @@
org.apache.nifinifi-flume-bundle
- 0.2.0-incubating-SNAPSHOT
+ 0.2.1-incubating-SNAPSHOTnifi-flume-processorsjar
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
index 59aab3c8ee..27da3c0cc3 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
@@ -17,10 +17,10 @@
org.apache.nifinifi-nar-bundles
- 0.2.0-incubating-SNAPSHOT
+ 0.2.1-incubating-SNAPSHOTnifi-flume-bundle
- 0.2.0-incubating-SNAPSHOT
+ 0.2.1-incubating-SNAPSHOTpomA bundle of processors that run Flume sources/sinks
From bde107d4a87eb19881feaf208df52db7dec6cda5 Mon Sep 17 00:00:00 2001
From: Matt Gilman
Date: Thu, 16 Jul 2015 11:12:55 -0400
Subject: [PATCH 24/77] NIFI-656: - Updating version of flume bundle that was
added during release vote.
---
nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
index 27da3c0cc3..d5ad112c7a 100644
--- a/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-flume-bundle/pom.xml
@@ -32,7 +32,7 @@
org.apache.nifinifi-flume-processors
- 0.2.0-incubating-SNAPSHOT
+ 0.2.1-incubating-SNAPSHOT
From 54ad9c89914a265c5e8e89fdf83cd83d83a83ff5 Mon Sep 17 00:00:00 2001
From: Matt Gilman
Date: Thu, 16 Jul 2015 11:20:27 -0400
Subject: [PATCH 25/77] NIFI-656: - Updating version of flume bundle that was
added during release vote.
---
nifi/pom.xml | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/nifi/pom.xml b/nifi/pom.xml
index eb94719850..22dbaf69cb 100644
--- a/nifi/pom.xml
+++ b/nifi/pom.xml
@@ -809,7 +809,7 @@
org.apache.nifinifi-flume-nar
- 0.2.0-incubating-SNAPSHOT
+ 0.2.1-incubating-SNAPSHOTnar
From 16cc08748d4063d3440a12f9dd3f9f248e2814ee Mon Sep 17 00:00:00 2001
From: Matt Gilman
Date: Fri, 17 Jul 2015 09:09:43 -0400
Subject: [PATCH 26/77] NIFI-656: - Adding links to download 0.2.0-incubating.
---
nifi-site/src/pages/html/download.hbs | 19 +++++++++++++++++++
1 file changed, 19 insertions(+)
diff --git a/nifi-site/src/pages/html/download.hbs b/nifi-site/src/pages/html/download.hbs
index 21607cd059..1b94c59b90 100644
--- a/nifi-site/src/pages/html/download.hbs
+++ b/nifi-site/src/pages/html/download.hbs
@@ -23,6 +23,25 @@ title: Apache NiFi Downloads
From 3982294ced19c6e2a253299f7880acbcb1a1c95f Mon Sep 17 00:00:00 2001
From: joewitt
Date: Sat, 18 Jul 2015 00:50:15 -0400
Subject: [PATCH 30/77] NIFI-768
---
nifi-site/src/pages/markdown/release-guide.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/nifi-site/src/pages/markdown/release-guide.md b/nifi-site/src/pages/markdown/release-guide.md
index 67a569c4e0..357f115daf 100644
--- a/nifi-site/src/pages/markdown/release-guide.md
+++ b/nifi-site/src/pages/markdown/release-guide.md
@@ -152,7 +152,7 @@ Validate that all the various aspects of the staged artifacts appear correct
- Validate the hashes match.
- Validate that the sources contain no unexpected binaries.
- Validate the signature for the build and hashes.
- - Validate the LICENSE/NOTICE/DISCLAIMER/Headers.
+ - Validate the LICENSE/NOTICE/Headers.
- Validate that the README is present and provides sufficient information to build and if necessary execute.
If all looks good then push the branch to origin `git push origin NIFI-270`
From f76aeb22706fe686e0f73523f67c858335433e2d Mon Sep 17 00:00:00 2001
From: joewitt
Date: Sat, 18 Jul 2015 00:52:30 -0400
Subject: [PATCH 31/77] NIFI-768
---
nifi-site/src/pages/markdown/release-guide.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/nifi-site/src/pages/markdown/release-guide.md b/nifi-site/src/pages/markdown/release-guide.md
index 357f115daf..6f071249b5 100644
--- a/nifi-site/src/pages/markdown/release-guide.md
+++ b/nifi-site/src/pages/markdown/release-guide.md
@@ -264,7 +264,7 @@ More details on Apache NiFi can be found here:
http://nifi.apache.org/
The release artifacts can be downloaded from here:
-http://nifi.apache.org/downloads/
+http://nifi.apache.org/download.html
Maven artifacts have been made available here:
https://repository.apache.org/content/repositories/releases/org/apache/nifi/
From dae1b9656f36fee6c5381b9bbe8d88556bb3021e Mon Sep 17 00:00:00 2001
From: joewitt
Date: Thu, 16 Jul 2015 12:35:37 -0400
Subject: [PATCH 32/77] NIFI-768 changed all versions and docs and paths to TLP
version removing incubator refs
NIFI-768 changed all versions and docs and paths to TLP version removing incubator refs
---
README.md | 28 ++----
nifi-nar-maven-plugin/pom.xml | 6 +-
nifi-parent/pom.xml | 39 ++++----
nifi/DISCLAIMER | 15 ---
nifi/README.md | 32 ++-----
nifi/nifi-api/pom.xml | 2 +-
nifi/nifi-assembly/README.md | 20 +---
nifi/nifi-assembly/pom.xml | 23 ++---
.../src/main/assembly/dependencies.xml | 7 --
nifi/nifi-bootstrap/pom.xml | 2 +-
.../nifi-data-provenance-utils/pom.xml | 2 +-
.../nifi-expression-language/pom.xml | 2 +-
.../nifi-flowfile-packager/pom.xml | 2 +-
.../nifi-hl7-query-language/pom.xml | 2 +-
nifi/nifi-commons/nifi-logging-utils/pom.xml | 2 +-
.../nifi-processor-utilities/pom.xml | 2 +-
nifi/nifi-commons/nifi-properties/pom.xml | 2 +-
nifi/nifi-commons/nifi-security-utils/pom.xml | 2 +-
.../nifi-site-to-site-client/pom.xml | 4 +-
nifi/nifi-commons/nifi-socket-utils/pom.xml | 2 +-
nifi/nifi-commons/nifi-utils/pom.xml | 4 +-
nifi/nifi-commons/nifi-web-utils/pom.xml | 2 +-
.../nifi-commons/nifi-write-ahead-log/pom.xml | 2 +-
nifi/nifi-commons/pom.xml | 2 +-
nifi/nifi-docs/pom.xml | 2 +-
.../main/asciidoc/administration-guide.adoc | 4 +-
.../src/main/asciidoc/developer-guide.adoc | 8 +-
.../asciidoc/expression-language-guide.adoc | 12 +--
.../nifi-docs/src/main/asciidoc/overview.adoc | 4 +-
.../src/main/asciidoc/user-guide.adoc | 6 +-
.../nifi-nifi-example-nar/pom.xml | 2 +-
.../nifi-nifi-example-processors/pom.xml | 2 +-
.../nifi-external/nifi-example-bundle/pom.xml | 4 +-
.../nifi-external/nifi-spark-receiver/pom.xml | 2 +-
nifi/nifi-external/nifi-storm-spout/pom.xml | 2 +-
nifi/nifi-external/pom.xml | 2 +-
.../nifi-processor-bundle-archetype/pom.xml | 2 +-
.../META-INF/maven/archetype-metadata.xml | 4 +-
nifi/nifi-maven-archetypes/pom.xml | 2 +-
nifi/nifi-mock/pom.xml | 2 +-
.../nifi-aws-bundle/nifi-aws-nar/pom.xml | 4 +-
.../nifi-aws-processors/pom.xml | 2 +-
nifi/nifi-nar-bundles/nifi-aws-bundle/pom.xml | 2 +-
.../nifi-flume-bundle/nifi-flume-nar/pom.xml | 4 +-
.../nifi-flume-processors/pom.xml | 2 +-
.../nifi-flume-bundle/pom.xml | 6 +-
.../nifi-framework-nar/pom.xml | 2 +-
.../nifi-administration/pom.xml | 2 +-
.../nifi-framework/nifi-client-dto/pom.xml | 2 +-
.../pom.xml | 2 +-
.../nifi-framework/nifi-documentation/pom.xml | 2 +-
.../nifi-file-authorization-provider/pom.xml | 2 +-
.../nifi-framework-cluster-protocol/pom.xml | 2 +-
.../nifi-framework-cluster-web/pom.xml | 2 +-
.../nifi-framework-cluster/pom.xml | 2 +-
.../nifi-framework-core-api/pom.xml | 2 +-
.../nifi-framework-core/pom.xml | 2 +-
.../nifi-framework/nifi-nar-utils/pom.xml | 2 +-
.../nifi-framework/nifi-resources/pom.xml | 2 +-
.../nifi-framework/nifi-runtime/pom.xml | 2 +-
.../nifi-framework/nifi-security/pom.xml | 2 +-
.../nifi-framework/nifi-site-to-site/pom.xml | 2 +-
.../src/test/resources/nifi.properties | 2 +-
.../nifi-framework/nifi-user-actions/pom.xml | 2 +-
.../nifi-web/nifi-custom-ui-utilities/pom.xml | 2 +-
.../nifi-web/nifi-jetty/pom.xml | 2 +-
.../nifi-web/nifi-ui-extension/pom.xml | 2 +-
.../nifi-web/nifi-web-api/pom.xml | 6 +-
.../resources/access-control/nifi.properties | 2 +-
.../nifi-web/nifi-web-content-access/pom.xml | 2 +-
.../nifi-web/nifi-web-content-viewer/pom.xml | 2 +-
.../nifi-web/nifi-web-docs/pom.xml | 2 +-
.../nifi-web/nifi-web-error/pom.xml | 2 +-
.../nifi-web-optimistic-locking/pom.xml | 2 +-
.../nifi-web/nifi-web-security/pom.xml | 2 +-
.../nifi-web/nifi-web-ui/pom.xml | 2 +-
.../WEB-INF/partials/canvas/about-dialog.jsp | 4 +-
.../nifi-framework/nifi-web/pom.xml | 12 +--
.../nifi-framework/pom.xml | 2 +-
.../nifi-framework-bundle/pom.xml | 38 ++++----
.../nifi-geo-bundle/nifi-geo-nar/pom.xml | 2 +-
.../nifi-geo-processors/pom.xml | 2 +-
nifi/nifi-nar-bundles/nifi-geo-bundle/pom.xml | 4 +-
.../nifi-hadoop-nar/pom.xml | 2 +-
.../nifi-hdfs-processors/pom.xml | 2 +-
.../nifi-hadoop-bundle/pom.xml | 4 +-
.../nifi-hadoop-libraries-nar/pom.xml | 2 +-
.../nifi-hadoop-libraries-bundle/pom.xml | 2 +-
.../nifi-hl7-bundle/nifi-hl7-nar/pom.xml | 4 +-
.../nifi-hl7-processors/pom.xml | 4 +-
nifi/nifi-nar-bundles/nifi-hl7-bundle/pom.xml | 2 +-
.../nifi-jetty-bundle/pom.xml | 2 +-
.../nifi-kafka-bundle/nifi-kafka-nar/pom.xml | 2 +-
.../nifi-kafka-processors/pom.xml | 2 +-
.../nifi-kafka-bundle/pom.xml | 4 +-
.../nifi-kite-bundle/nifi-kite-nar/pom.xml | 2 +-
.../nifi-kite-processors/pom.xml | 2 +-
.../nifi-nar-bundles/nifi-kite-bundle/pom.xml | 4 +-
.../nifi-language-translation-nar/pom.xml | 4 +-
.../nifi-yandex-processors/pom.xml | 2 +-
.../nifi-language-translation-bundle/pom.xml | 2 +-
.../nifi-mongodb-nar/pom.xml | 6 +-
.../nifi-mongodb-processors/pom.xml | 2 +-
.../nifi-mongodb-bundle/pom.xml | 4 +-
.../pom.xml | 2 +-
.../nifi-provenance-repository-nar/pom.xml | 2 +-
.../pom.xml | 2 +-
.../nifi-provenance-repository-bundle/pom.xml | 6 +-
.../nifi-social-media-nar/pom.xml | 4 +-
.../nifi-twitter-processors/pom.xml | 2 +-
.../nifi-social-media-bundle/pom.xml | 2 +-
.../nifi-solr-bundle/nifi-solr-nar/pom.xml | 4 +-
.../nifi-solr-processors/pom.xml | 2 +-
.../nifi-nar-bundles/nifi-solr-bundle/pom.xml | 2 +-
.../nifi-standard-content-viewer/pom.xml | 2 +-
.../nifi-standard-nar/pom.xml | 2 +-
.../nifi-standard-prioritizers/pom.xml | 2 +-
.../nifi-standard-processors/pom.xml | 2 +-
.../nifi-standard-reporting-tasks/pom.xml | 2 +-
.../nifi-standard-bundle/pom.xml | 10 +-
.../nifi-dbcp-service-api/pom.xml | 2 +-
.../nifi-dbcp-service-nar/pom.xml | 4 +-
.../nifi-dbcp-service/pom.xml | 4 +-
.../nifi-dbcp-service-bundle/pom.xml | 2 +-
.../pom.xml | 2 +-
.../pom.xml | 2 +-
.../nifi-distributed-cache-protocol/pom.xml | 2 +-
.../nifi-distributed-cache-server/pom.xml | 2 +-
.../pom.xml | 2 +-
.../pom.xml | 2 +-
.../nifi-http-context-map-api/pom.xml | 2 +-
.../nifi-http-context-map-nar/pom.xml | 2 +-
.../nifi-http-context-map/pom.xml | 2 +-
.../nifi-http-context-map-bundle/pom.xml | 2 +-
.../pom.xml | 2 +-
.../nifi-ssl-context-nar/pom.xml | 2 +-
.../nifi-ssl-context-service/pom.xml | 2 +-
.../nifi-ssl-context-bundle/pom.xml | 2 +-
.../nifi-ssl-context-service-api/pom.xml | 2 +-
.../nifi-standard-services-api-nar/pom.xml | 4 +-
.../nifi-standard-services/pom.xml | 2 +-
.../nifi-update-attribute-model/pom.xml | 2 +-
.../nifi-update-attribute-nar/pom.xml | 2 +-
.../nifi-update-attribute-processor/pom.xml | 2 +-
.../nifi-update-attribute-ui/pom.xml | 2 +-
.../nifi-update-attribute-bundle/pom.xml | 8 +-
nifi/nifi-nar-bundles/pom.xml | 30 +++---
nifi/pom.xml | 94 +++++++++----------
148 files changed, 325 insertions(+), 385 deletions(-)
delete mode 100644 nifi/DISCLAIMER
diff --git a/README.md b/README.md
index 8a4fdba757..c71bb0fa82 100644
--- a/README.md
+++ b/README.md
@@ -20,15 +20,14 @@ Apache NiFi is an easy to use, powerful, and reliable system to process and dist
- [Getting Started](#getting-started)
- [License](#license)
-- [Disclaimer](#disclaimer)
## Getting Started
-- Read through the [quickstart guide for development](http://nifi.incubator.apache.org/quickstart.html).
+- Read through the [quickstart guide for development](http://nifi.apache.org/quickstart.html).
It will include information on getting a local copy of the source, give pointers on issue
tracking, and provide some warnings about common problems with development environments.
- For a more comprehensive guide to development and information about contributing to the project
- read through the [NiFi Developer's Guide](http://nifi.incubator.apache.org/developer-guide.html).
+ read through the [NiFi Developer's Guide](http://nifi.apache.org/developer-guide.html).
- Optional: Build supporting modules. This should only be needed if the current 'nifi' module is in
the process of updating to a new version of either the 'nifi-parent' or 'nifi-nar-maven-plugin'
artifacts.
@@ -44,7 +43,7 @@ Apache NiFi is an easy to use, powerful, and reliable system to process and dist
you'll see an erorr like the following:
[ERROR] Non-resolvable parent POM: Could not find artifact
- org.apache.nifi:nifi-parent:pom:1.0.0-incubating-SNAPSHOT in example.snapshots.repo
+ org.apache.nifi:nifi-parent:pom:1.0.0-SNAPSHOT in example.snapshots.repo
(https://repository.example.com/content/repositories/snapshots) and
'parent.relativePath' points at no local POM @ line 18, column 13 -> [Help 2]
- Build the nifi-nar-maven-plugin. Change directory to 'nifi-nar-maven-plugin' and
@@ -54,18 +53,18 @@ Apache NiFi is an easy to use, powerful, and reliable system to process and dist
version you'll see an error like the following:
[ERROR] Unresolveable build extension: Plugin
- org.apache.nifi:nifi-nar-maven-plugin:1.0.1-incubating-SNAPSHOT or one of its
+ org.apache.nifi:nifi-nar-maven-plugin:1.0.1-SNAPSHOT or one of its
dependencies could not be resolved: Could not find artifact
- org.apache.nifi:nifi-nar-maven-plugin:jar:1.0.1-incubating-SNAPSHOT -> [Help 2]
+ org.apache.nifi:nifi-nar-maven-plugin:jar:1.0.1-SNAPSHOT -> [Help 2]
- Build nifi. Change directory to 'nifi' and follow the directions found there in
[README.md](nifi/README.md).
- Run NiFi. The directions found in the [README.md](nifi/README.md) file within the 'nifi' module
will also include how to run an instance of NiFi. For help on how to build your first data flow,
- see the [NiFi User Guide](http://nifi.incubator.apache.org/docs/nifi-docs/user-guide.html).
+ see the [NiFi User Guide](http://nifi.apache.org/docs/nifi-docs/user-guide.html).
## Documentation
-See http://nifi.incubator.apache.org/ for the latest documentation.
+See http://nifi.apache.org/ for the latest documentation.
## License
@@ -84,16 +83,3 @@ 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.
-## Disclaimer
-
-Apache NiFi is an effort undergoing incubation at the Apache Software
-Foundation (ASF), sponsored by the Apache Incubator PMC.
-
-Incubation is required of all newly accepted projects until a further review
-indicates that the infrastructure, communications, and decision making process
-have stabilized in a manner consistent with other successful ASF projects.
-
-While incubation status is not necessarily a reflection of the completeness
-or stability of the code, it does indicate that the project has yet to be
-fully endorsed by the ASF.
-
diff --git a/nifi-nar-maven-plugin/pom.xml b/nifi-nar-maven-plugin/pom.xml
index f6e104c311..4e08f2ec00 100644
--- a/nifi-nar-maven-plugin/pom.xml
+++ b/nifi-nar-maven-plugin/pom.xml
@@ -18,13 +18,13 @@
org.apache.nifinifi-parent
- 1.0.0-incubating
+ 1.0.1-SNAPSHOTnifi-nar-maven-plugin
- 1.0.2-incubating-SNAPSHOT
+ 1.0.2-SNAPSHOTmaven-plugin
- Apache NiFi Nar Plugin. It is currently a part of the Apache Incubator.
+ Apache NiFi Nar Plugin.
diff --git a/nifi-parent/pom.xml b/nifi-parent/pom.xml
index 9ccb6ec2bf..0370c97696 100644
--- a/nifi-parent/pom.xml
+++ b/nifi-parent/pom.xml
@@ -18,18 +18,18 @@
org.apacheapache
- 16
+ 17org.apache.nifinifi-parent
- 1.0.1-incubating-SNAPSHOT
+ 1.0.1-SNAPSHOTpomThe nifi-parent enables each apache nifi project to ensure consistent approaches and DRY
- http://nifi.incubator.apache.org
+ http://nifi.apache.org
- Apache NiFi(incubating) Project
- http://nifi.incubator.apache.org/
+ Apache NiFi Project
+ http://nifi.apache.org/
@@ -40,26 +40,33 @@
Dev
- dev-subscribe@nifi.incubator.apache.org
- dev-unsubscribe@nifi.incubator.apache.org
- dev@nifi.incubator.apache.org
- http://mail-archives.apache.org/mod_mbox/incubator-nifi-dev
+ dev-subscribe@nifi.apache.org
+ dev-unsubscribe@nifi.apache.org
+ dev@nifi.apache.org
+ http://mail-archives.apache.org/mod_mbox/nifi-dev
+
+
+ Users
+ users-subscribe@nifi.apache.org
+ users-unsubscribe@nifi.apache.org
+ users@nifi.apache.org
+ http://mail-archives.apache.org/mod_mbox/nifi-usersCommits
- commits-subscribe@nifi.incubator.apache.org
- commits-unsubscribe@nifi.incubator.apache.org
- commits@nifi.incubator.apache.org
- http://mail-archives.apache.org/mod_mbox/incubator-nifi-commits
+ commits-subscribe@nifi.apache.org
+ commits-unsubscribe@nifi.apache.org
+ commits@nifi.apache.org
+ http://mail-archives.apache.org/mod_mbox/nifi-commits${maven.min-version}
- scm:git:git://git.apache.org/incubator-nifi.git
- scm:git:https://git-wip-us.apache.org/repos/asf/incubator-nifi.git
- https://git-wip-us.apache.org/repos/asf?p=incubator-nifi.git
+ scm:git:git://git.apache.org/nifi.git
+ scm:git:https://git-wip-us.apache.org/repos/asf/nifi.git
+ https://git-wip-us.apache.org/repos/asf?p=nifi.gitHEAD
diff --git a/nifi/DISCLAIMER b/nifi/DISCLAIMER
deleted file mode 100644
index 0f8e7a1ecc..0000000000
--- a/nifi/DISCLAIMER
+++ /dev/null
@@ -1,15 +0,0 @@
-Apache NiFi is an effort undergoing incubation at the Apache Software
-Foundation (ASF), sponsored by the Apache Incubator PMC.
-
-Incubation is required of all newly accepted projects until a further review
-indicates that the infrastructure, communications, and decision making process
-have stabilized in a manner consistent with other successful ASF projects.
-
-While incubation status is not necessarily a reflection of the completeness
-or stability of the code, it does indicate that the project has yet to be
-fully endorsed by the ASF.
-
-For more information about the incubation status of the Apache NiFi project
-you can go to the following page:
-
-http://nifi.incubator.apache.org/
diff --git a/nifi/README.md b/nifi/README.md
index cffa09ad87..a680f7899b 100644
--- a/nifi/README.md
+++ b/nifi/README.md
@@ -23,7 +23,6 @@ Apache NiFi is an easy to use, powerful, and reliable system to process and dist
- [Getting Started](#getting-started)
- [Getting Help](#getting-help)
- [License](#license)
-- [Disclaimer](#disclaimer)
- [Export Control] (#export-control)
## Features
@@ -75,9 +74,9 @@ To deploy:
laptop:nifi fhampton$ cd nifi-assembly
laptop:nifi-assembly fhampton$ ls -lhd target/nifi*
- drwxr-xr-x 3 fhampton staff 102B Apr 30 00:29 target/nifi-0.1.0-incubating-SNAPSHOT-bin
- -rw-r--r-- 1 fhampton staff 144M Apr 30 00:30 target/nifi-0.1.0-incubating-SNAPSHOT-bin.tar.gz
- -rw-r--r-- 1 fhampton staff 144M Apr 30 00:30 target/nifi-0.1.0-incubating-SNAPSHOT-bin.zip
+ drwxr-xr-x 3 fhampton staff 102B Apr 30 00:29 target/nifi-0.1.0-SNAPSHOT-bin
+ -rw-r--r-- 1 fhampton staff 144M Apr 30 00:30 target/nifi-0.1.0-SNAPSHOT-bin.tar.gz
+ -rw-r--r-- 1 fhampton staff 144M Apr 30 00:30 target/nifi-0.1.0-SNAPSHOT-bin.zip
- For testing ongoing development you could use the already unpacked build present in the directory
named "nifi-*version*-bin", where *version* is the current project version. To deploy in another
@@ -88,27 +87,27 @@ To deploy:
laptop:nifi-assembly fhampton$ tar xzf target/nifi-*-bin.tar.gz -C ~/example-nifi-deploy
laptop:nifi-assembly fhampton$ ls -lh ~/example-nifi-deploy/
total 0
- drwxr-xr-x 10 fhampton staff 340B Apr 30 01:06 nifi-0.1.0-incubating-SNAPSHOT
+ drwxr-xr-x 10 fhampton staff 340B Apr 30 01:06 nifi-0.1.0-SNAPSHOT
To run NiFi:
- Change directory to the location where you installed NiFi and run it.
laptop:~ fhampton$ cd ~/example-nifi-deploy/nifi-*
- laptop:nifi-0.1.0-incubating-SNAPSHOT fhampton$ ./bin/nifi.sh start
+ laptop:nifi-0.1.0-SNAPSHOT fhampton$ ./bin/nifi.sh start
- Direct your browser to http://localhost:8080/nifi/ and you should see a screen like this screenshot:
![image of a NiFi dataflow canvas](nifi-docs/src/main/asciidoc/images/nifi_first_launch_screenshot.png?raw=true)
-- For help building your first data flow see the [NiFi User Guide](http://nifi.incubator.apache.org/docs/nifi-docs/user-guide.html)
+- For help building your first data flow see the [NiFi User Guide](http://nifi.apache.org/docs/nifi-docs/user-guide.html)
- If you are testing ongoing development, you will likely want to stop your instance.
laptop:~ fhampton$ cd ~/example-nifi-deploy/nifi-*
- laptop:nifi-0.1.0-incubating-SNAPSHOT fhampton$ ./bin/nifi.sh stop
+ laptop:nifi-0.1.0-SNAPSHOT fhampton$ ./bin/nifi.sh stop
## Getting Help
-If you have questions, you can reach out to our mailing list: dev@nifi.incubator.apache.org
-([archive](http://mail-archives.apache.org/mod_mbox/incubator-nifi-dev)).
+If you have questions, you can reach out to our mailing list: dev@nifi.apache.org
+([archive](http://mail-archives.apache.org/mod_mbox/nifi-dev)).
We're also often available in IRC: #nifi on
[irc.freenode.net](http://webchat.freenode.net/?channels=#nifi).
@@ -129,19 +128,6 @@ 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.
-## Disclaimer
-
-Apache NiFi is an effort undergoing incubation at the Apache Software
-Foundation (ASF), sponsored by the Apache Incubator PMC.
-
-Incubation is required of all newly accepted projects until a further review
-indicates that the infrastructure, communications, and decision making process
-have stabilized in a manner consistent with other successful ASF projects.
-
-While incubation status is not necessarily a reflection of the completeness
-or stability of the code, it does indicate that the project has yet to be
-fully endorsed by the ASF.
-
## Export Control
This distribution includes cryptographic software. The country in which you
diff --git a/nifi/nifi-api/pom.xml b/nifi/nifi-api/pom.xml
index d8cb3fa175..5ea116ec4d 100644
--- a/nifi/nifi-api/pom.xml
+++ b/nifi/nifi-api/pom.xml
@@ -18,7 +18,7 @@
org.apache.nifinifi
- 0.2.1-incubating-SNAPSHOT
+ 0.2.1-SNAPSHOTnifi-apijar
diff --git a/nifi/nifi-assembly/README.md b/nifi/nifi-assembly/README.md
index 106505dbff..4fb4d3b458 100644
--- a/nifi/nifi-assembly/README.md
+++ b/nifi/nifi-assembly/README.md
@@ -14,7 +14,7 @@
-->
# Apache NiFi
-Apache NiFi is an easy to use, powerful, and reliable system to process and distribute data. It is currently apart of the Apache Incubator.
+Apache NiFi is an easy to use, powerful, and reliable system to process and distribute data.
## Table of Contents
@@ -23,7 +23,6 @@ Apache NiFi is an easy to use, powerful, and reliable system to process and dist
- [Getting Help](#getting-help)
- [Requirements](#requirements)
- [License](#license)
-- [Disclaimer](#disclaimer)
- [Export Control] (#export-control)
## Features
@@ -55,8 +54,8 @@ To start NiFi:
- Direct your browser to http://localhost:8080/nifi/
## Getting Help
-If you have questions, you can reach out to our mailing list: dev@nifi.incubator.apache.org
-([archive](http://mail-archives.apache.org/mod_mbox/incubator-nifi-dev)).
+If you have questions, you can reach out to our mailing list: dev@nifi.apache.org
+([archive](http://mail-archives.apache.org/mod_mbox/nifi-dev)).
We're also often available in IRC: #nifi on
[irc.freenode.net](http://webchat.freenode.net/?channels=#nifi).
@@ -80,19 +79,6 @@ 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.
-## Disclaimer
-
-Apache NiFi is an effort undergoing incubation at the Apache Software
-Foundation (ASF), sponsored by the Apache Incubator PMC.
-
-Incubation is required of all newly accepted projects until a further review
-indicates that the infrastructure, communications, and decision making process
-have stabilized in a manner consistent with other successful ASF projects.
-
-While incubation status is not necessarily a reflection of the completeness
-or stability of the code, it does indicate that the project has yet to be
-fully endorsed by the ASF.
-
## Export Control
This distribution includes cryptographic software. The country in which you
diff --git a/nifi/nifi-assembly/pom.xml b/nifi/nifi-assembly/pom.xml
index 473b947ec5..8677c60f9b 100644
--- a/nifi/nifi-assembly/pom.xml
+++ b/nifi/nifi-assembly/pom.xml
@@ -14,11 +14,11 @@ language governing permissions and limitations under the License. -->
org.apache.nifinifi
- 0.2.1-incubating-SNAPSHOT
+ 0.2.1-SNAPSHOTnifi-assemblypom
- This is the assembly Apache NiFi (incubating)
+ This is the assembly Apache NiFi
@@ -175,7 +175,7 @@ language governing permissions and limitations under the License. -->
org.apache.nifinifi-dbcp-service-nar
- 0.2.1-incubating-SNAPSHOT
+ 0.2.1-SNAPSHOTnar
@@ -191,25 +191,25 @@ language governing permissions and limitations under the License. -->
org.apache.nifinifi-social-media-nar
- 0.2.1-incubating-SNAPSHOT
+ 0.2.1-SNAPSHOTnarorg.apache.nifinifi-hl7-nar
- 0.2.1-incubating-SNAPSHOT
+ 0.2.1-SNAPSHOTnarorg.apache.nifinifi-language-translation-nar
- 0.2.1-incubating-SNAPSHOT
+ 0.2.1-SNAPSHOTnarorg.apache.nifinifi-geo-nar
- 0.2.1-incubating-SNAPSHOT
+ 0.2.1-SNAPSHOTnar
@@ -398,10 +398,10 @@ language governing permissions and limitations under the License. -->
org.codehaus.mojorpm-maven-plugin
- Apache NiFi (incubating)
- Apache Nifi (incubating) is dataflow system based on the Flow-Based Programming concepts.
+ Apache NiFi
+ Apache NiFi is dataflow system based on the Flow-Based Programming concepts.Apache License, Version 2.0 and others (see included LICENSE file)
- http://nifi.incubator.apache.org
+ http://nifi.apache.orgUtilities/opt/nifi
@@ -436,9 +436,6 @@ language governing permissions and limitations under the License. -->
-