mirror of https://github.com/apache/nifi.git
Merge branch 'develop' into NIFI-632
This commit is contained in:
commit
28bfb70e46
|
@ -121,6 +121,11 @@ title: Apache NiFi Team
|
|||
<td>Dan Bress</td>
|
||||
<td></td>
|
||||
</tr>
|
||||
<tr>
|
||||
<td>bbende</td>
|
||||
<td>Bryan Bende</td>
|
||||
<td></td>
|
||||
</tr>
|
||||
</table>
|
||||
</div>
|
||||
</div>
|
|
@ -49,6 +49,8 @@
|
|||
<useTransitiveFiltering>true</useTransitiveFiltering>
|
||||
<includes>
|
||||
<include>nifi-bootstrap</include>
|
||||
<include>slf4j-api</include>
|
||||
<include>logback-classic</include>
|
||||
</includes>
|
||||
</dependencySet>
|
||||
|
||||
|
|
|
@ -21,4 +21,11 @@
|
|||
</parent>
|
||||
<artifactId>nifi-bootstrap</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.slf4j</groupId>
|
||||
<artifactId>slf4j-api</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.net.ServerSocket;
|
|||
import java.net.Socket;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.bootstrap.util.LimitingInputStream;
|
||||
|
@ -40,6 +41,7 @@ public class NiFiListener {
|
|||
listener = new Listener(serverSocket, runner);
|
||||
final Thread listenThread = new Thread(listener);
|
||||
listenThread.setName("Listen to NiFi");
|
||||
listenThread.setDaemon(true);
|
||||
listenThread.start();
|
||||
return localPort;
|
||||
}
|
||||
|
@ -62,7 +64,16 @@ public class NiFiListener {
|
|||
|
||||
public Listener(final ServerSocket serverSocket, final RunNiFi runner) {
|
||||
this.serverSocket = serverSocket;
|
||||
this.executor = Executors.newFixedThreadPool(2);
|
||||
this.executor = Executors.newFixedThreadPool(2, new ThreadFactory() {
|
||||
@Override
|
||||
public Thread newThread(final Runnable runnable) {
|
||||
final Thread t = Executors.defaultThreadFactory().newThread(runnable);
|
||||
t.setDaemon(true);
|
||||
t.setName("NiFi Bootstrap Command Listener");
|
||||
return t;
|
||||
}
|
||||
});
|
||||
|
||||
this.runner = runner;
|
||||
}
|
||||
|
||||
|
|
|
@ -41,13 +41,17 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.ThreadFactory;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.logging.ConsoleHandler;
|
||||
import java.util.logging.Handler;
|
||||
import java.util.logging.Level;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -94,19 +98,27 @@ public class RunNiFi {
|
|||
|
||||
private final File bootstrapConfigFile;
|
||||
|
||||
private final java.util.logging.Logger logger;
|
||||
// used for logging initial info; these will be logged to console by default when the app is started
|
||||
private final Logger cmdLogger = LoggerFactory.getLogger("org.apache.nifi.bootstrap.Command");
|
||||
// used for logging all info. These by default will be written to the log file
|
||||
private final Logger defaultLogger = LoggerFactory.getLogger(RunNiFi.class);
|
||||
|
||||
|
||||
private final ExecutorService loggingExecutor;
|
||||
private volatile Set<Future<?>> loggingFutures = new HashSet<>(2);
|
||||
|
||||
public RunNiFi(final File bootstrapConfigFile, final boolean verbose) {
|
||||
this.bootstrapConfigFile = bootstrapConfigFile;
|
||||
logger = java.util.logging.Logger.getLogger("Bootstrap");
|
||||
if (verbose) {
|
||||
logger.info("Enabling Verbose Output");
|
||||
|
||||
logger.setLevel(Level.FINE);
|
||||
final Handler handler = new ConsoleHandler();
|
||||
handler.setLevel(Level.FINE);
|
||||
logger.addHandler(handler);
|
||||
}
|
||||
loggingExecutor = Executors.newFixedThreadPool(2, new ThreadFactory() {
|
||||
@Override
|
||||
public Thread newThread(final Runnable runnable) {
|
||||
final Thread t = Executors.defaultThreadFactory().newThread(runnable);
|
||||
t.setDaemon(true);
|
||||
t.setName("NiFi logging handler");
|
||||
return t;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private static void printUsage() {
|
||||
|
@ -185,10 +197,10 @@ public class RunNiFi {
|
|||
|
||||
switch (cmd.toLowerCase()) {
|
||||
case "start":
|
||||
runNiFi.start(false);
|
||||
runNiFi.start();
|
||||
break;
|
||||
case "run":
|
||||
runNiFi.start(true);
|
||||
runNiFi.start();
|
||||
break;
|
||||
case "stop":
|
||||
runNiFi.stop();
|
||||
|
@ -198,7 +210,7 @@ public class RunNiFi {
|
|||
break;
|
||||
case "restart":
|
||||
runNiFi.stop();
|
||||
runNiFi.start(false);
|
||||
runNiFi.start();
|
||||
break;
|
||||
case "dump":
|
||||
runNiFi.dump(dumpFile);
|
||||
|
@ -206,40 +218,44 @@ public class RunNiFi {
|
|||
}
|
||||
}
|
||||
|
||||
public File getStatusFile() {
|
||||
File getStatusFile() {
|
||||
return getStatusFile(defaultLogger);
|
||||
}
|
||||
|
||||
public File getStatusFile(final Logger logger) {
|
||||
final File confDir = bootstrapConfigFile.getParentFile();
|
||||
final File nifiHome = confDir.getParentFile();
|
||||
final File bin = new File(nifiHome, "bin");
|
||||
final File statusFile = new File(bin, "nifi.pid");
|
||||
|
||||
logger.log(Level.FINE, "Status File: {0}", statusFile);
|
||||
logger.debug("Status File: {}", statusFile);
|
||||
|
||||
return statusFile;
|
||||
}
|
||||
|
||||
private Properties loadProperties() throws IOException {
|
||||
private Properties loadProperties(final Logger logger) throws IOException {
|
||||
final Properties props = new Properties();
|
||||
final File statusFile = getStatusFile();
|
||||
final File statusFile = getStatusFile(logger);
|
||||
if (statusFile == null || !statusFile.exists()) {
|
||||
logger.fine("No status file to load properties from");
|
||||
logger.debug("No status file to load properties from");
|
||||
return props;
|
||||
}
|
||||
|
||||
try (final FileInputStream fis = new FileInputStream(getStatusFile())) {
|
||||
try (final FileInputStream fis = new FileInputStream(getStatusFile(logger))) {
|
||||
props.load(fis);
|
||||
}
|
||||
|
||||
final Map<Object, Object> modified = new HashMap<>(props);
|
||||
modified.remove("secret.key");
|
||||
logger.log(Level.FINE, "Properties: {0}", modified);
|
||||
logger.debug("Properties: {}", modified);
|
||||
|
||||
return props;
|
||||
}
|
||||
|
||||
private synchronized void saveProperties(final Properties nifiProps) throws IOException {
|
||||
final File statusFile = getStatusFile();
|
||||
private synchronized void saveProperties(final Properties nifiProps, final Logger logger) throws IOException {
|
||||
final File statusFile = getStatusFile(logger);
|
||||
if (statusFile.exists() && !statusFile.delete()) {
|
||||
logger.log(Level.WARNING, "Failed to delete {0}", statusFile);
|
||||
logger.warn("Failed to delete {}", statusFile);
|
||||
}
|
||||
|
||||
if (!statusFile.createNewFile()) {
|
||||
|
@ -252,7 +268,7 @@ public class RunNiFi {
|
|||
perms.add(PosixFilePermission.OWNER_WRITE);
|
||||
Files.setPosixFilePermissions(statusFile.toPath(), perms);
|
||||
} catch (final Exception e) {
|
||||
logger.log(Level.WARNING, "Failed to set permissions so that only the owner can read status file {0}; "
|
||||
logger.warn("Failed to set permissions so that only the owner can read status file {}; "
|
||||
+ "this may allows others to have access to the key needed to communicate with NiFi. "
|
||||
+ "Permissions should be changed so that only the owner can read this file", statusFile);
|
||||
}
|
||||
|
@ -262,23 +278,23 @@ public class RunNiFi {
|
|||
fos.getFD().sync();
|
||||
}
|
||||
|
||||
logger.log(Level.FINE, "Saved Properties {0} to {1}", new Object[]{nifiProps, statusFile});
|
||||
logger.debug("Saved Properties {} to {}", new Object[]{nifiProps, statusFile});
|
||||
}
|
||||
|
||||
private boolean isPingSuccessful(final int port, final String secretKey) {
|
||||
logger.log(Level.FINE, "Pinging {0}", port);
|
||||
private boolean isPingSuccessful(final int port, final String secretKey, final Logger logger) {
|
||||
logger.debug("Pinging {}", port);
|
||||
|
||||
try (final Socket socket = new Socket("localhost", port)) {
|
||||
final OutputStream out = socket.getOutputStream();
|
||||
out.write((PING_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
|
||||
out.flush();
|
||||
|
||||
logger.fine("Sent PING command");
|
||||
logger.debug("Sent PING command");
|
||||
socket.setSoTimeout(5000);
|
||||
final InputStream in = socket.getInputStream();
|
||||
final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
|
||||
final String response = reader.readLine();
|
||||
logger.log(Level.FINE, "PING response: {0}", response);
|
||||
logger.debug("PING response: {}", response);
|
||||
out.close();
|
||||
reader.close();
|
||||
|
||||
|
@ -288,27 +304,27 @@ public class RunNiFi {
|
|||
}
|
||||
}
|
||||
|
||||
private Integer getCurrentPort() throws IOException {
|
||||
final Properties props = loadProperties();
|
||||
private Integer getCurrentPort(final Logger logger) throws IOException {
|
||||
final Properties props = loadProperties(logger);
|
||||
final String portVal = props.getProperty("port");
|
||||
if (portVal == null) {
|
||||
logger.fine("No Port found in status file");
|
||||
logger.debug("No Port found in status file");
|
||||
return null;
|
||||
} else {
|
||||
logger.log(Level.FINE, "Port defined in status file: {0}", portVal);
|
||||
logger.debug("Port defined in status file: {}", portVal);
|
||||
}
|
||||
|
||||
final int port = Integer.parseInt(portVal);
|
||||
final boolean success = isPingSuccessful(port, props.getProperty("secret.key"));
|
||||
final boolean success = isPingSuccessful(port, props.getProperty("secret.key"), logger);
|
||||
if (success) {
|
||||
logger.log(Level.FINE, "Successful PING on port {0}", port);
|
||||
logger.debug("Successful PING on port {}", port);
|
||||
return port;
|
||||
}
|
||||
|
||||
final String pid = props.getProperty("pid");
|
||||
logger.log(Level.FINE, "PID in status file is {0}", pid);
|
||||
logger.debug("PID in status file is {}", pid);
|
||||
if (pid != null) {
|
||||
final boolean procRunning = isProcessRunning(pid);
|
||||
final boolean procRunning = isProcessRunning(pid, logger);
|
||||
if (procRunning) {
|
||||
return port;
|
||||
} else {
|
||||
|
@ -319,7 +335,7 @@ public class RunNiFi {
|
|||
return null;
|
||||
}
|
||||
|
||||
private boolean isProcessRunning(final String pid) {
|
||||
private boolean isProcessRunning(final String pid, final Logger logger) {
|
||||
try {
|
||||
// We use the "ps" command to check if the process is still running.
|
||||
final ProcessBuilder builder = new ProcessBuilder();
|
||||
|
@ -343,9 +359,9 @@ public class RunNiFi {
|
|||
|
||||
// If output of the ps command had our PID, the process is running.
|
||||
if (running) {
|
||||
logger.log(Level.FINE, "Process with PID {0} is running", pid);
|
||||
logger.debug("Process with PID {} is running", pid);
|
||||
} else {
|
||||
logger.log(Level.FINE, "Process with PID {0} is not running", pid);
|
||||
logger.debug("Process with PID {} is not running", pid);
|
||||
}
|
||||
|
||||
return running;
|
||||
|
@ -355,10 +371,10 @@ public class RunNiFi {
|
|||
}
|
||||
}
|
||||
|
||||
private Status getStatus() {
|
||||
private Status getStatus(final Logger logger) {
|
||||
final Properties props;
|
||||
try {
|
||||
props = loadProperties();
|
||||
props = loadProperties(logger);
|
||||
} catch (final IOException ioe) {
|
||||
return new Status(null, null, false, false);
|
||||
}
|
||||
|
@ -380,7 +396,7 @@ public class RunNiFi {
|
|||
if (portValue != null) {
|
||||
try {
|
||||
port = Integer.parseInt(portValue);
|
||||
pingSuccess = isPingSuccessful(port, secretKey);
|
||||
pingSuccess = isPingSuccessful(port, secretKey, logger);
|
||||
} catch (final NumberFormatException nfe) {
|
||||
return new Status(null, null, false, false);
|
||||
}
|
||||
|
@ -390,20 +406,21 @@ public class RunNiFi {
|
|||
return new Status(port, pid, true, true);
|
||||
}
|
||||
|
||||
final boolean alive = (pid == null) ? false : isProcessRunning(pid);
|
||||
final boolean alive = (pid == null) ? false : isProcessRunning(pid, logger);
|
||||
return new Status(port, pid, pingSuccess, alive);
|
||||
}
|
||||
|
||||
public void status() throws IOException {
|
||||
final Status status = getStatus();
|
||||
final Logger logger = cmdLogger;
|
||||
final Status status = getStatus(logger);
|
||||
if (status.isRespondingToPing()) {
|
||||
logger.log(Level.INFO, "Apache NiFi is currently running, listening to Bootstrap on port {0}, PID={1}",
|
||||
logger.info("Apache NiFi is currently running, listening to Bootstrap on port {}, PID={}",
|
||||
new Object[]{status.getPort(), status.getPid() == null ? "unknkown" : status.getPid()});
|
||||
return;
|
||||
}
|
||||
|
||||
if (status.isProcessRunning()) {
|
||||
logger.log(Level.INFO, "Apache NiFi is running at PID {0} but is not responding to ping requests", status.getPid());
|
||||
logger.info("Apache NiFi is running at PID {} but is not responding to ping requests", status.getPid());
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -427,36 +444,36 @@ public class RunNiFi {
|
|||
* @throws IOException if any issues occur while writing the dump file
|
||||
*/
|
||||
public void dump(final File dumpFile) throws IOException {
|
||||
final Integer port = getCurrentPort();
|
||||
final Logger logger = defaultLogger; // dump to bootstrap log file by default
|
||||
final Integer port = getCurrentPort(logger);
|
||||
if (port == null) {
|
||||
System.out.println("Apache NiFi is not currently running");
|
||||
logger.info("Apache NiFi is not currently running");
|
||||
return;
|
||||
}
|
||||
|
||||
final Properties nifiProps = loadProperties();
|
||||
final Properties nifiProps = loadProperties(logger);
|
||||
final String secretKey = nifiProps.getProperty("secret.key");
|
||||
|
||||
final StringBuilder sb = new StringBuilder();
|
||||
try (final Socket socket = new Socket()) {
|
||||
logger.fine("Connecting to NiFi instance");
|
||||
logger.debug("Connecting to NiFi instance");
|
||||
socket.setSoTimeout(60000);
|
||||
socket.connect(new InetSocketAddress("localhost", port));
|
||||
logger.fine("Established connection to NiFi instance.");
|
||||
logger.debug("Established connection to NiFi instance.");
|
||||
socket.setSoTimeout(60000);
|
||||
|
||||
logger.log(Level.FINE, "Sending DUMP Command to port {0}", port);
|
||||
logger.debug("Sending DUMP Command to port {}", port);
|
||||
final OutputStream out = socket.getOutputStream();
|
||||
out.write((DUMP_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
|
||||
out.flush();
|
||||
out.close();
|
||||
|
||||
final InputStream in = socket.getInputStream();
|
||||
final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
sb.append(line).append("\n");
|
||||
try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
sb.append(line).append("\n");
|
||||
}
|
||||
}
|
||||
reader.close();
|
||||
}
|
||||
|
||||
final String dump = sb.toString();
|
||||
|
@ -466,28 +483,35 @@ public class RunNiFi {
|
|||
try (final FileOutputStream fos = new FileOutputStream(dumpFile)) {
|
||||
fos.write(dump.getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
logger.log(Level.INFO, "Successfully wrote thread dump to {0}", dumpFile.getAbsolutePath());
|
||||
// we want to log to the console (by default) that we wrote the thread dump to the specified file
|
||||
cmdLogger.info("Successfully wrote thread dump to {}", dumpFile.getAbsolutePath());
|
||||
}
|
||||
}
|
||||
|
||||
public void stop() throws IOException {
|
||||
final Integer port = getCurrentPort();
|
||||
final Logger logger = cmdLogger;
|
||||
final Integer port = getCurrentPort(logger);
|
||||
if (port == null) {
|
||||
System.out.println("Apache NiFi is not currently running");
|
||||
logger.info("Apache NiFi is not currently running");
|
||||
return;
|
||||
}
|
||||
|
||||
final Properties nifiProps = loadProperties();
|
||||
final Properties nifiProps = loadProperties(logger);
|
||||
final String secretKey = nifiProps.getProperty("secret.key");
|
||||
|
||||
final File statusFile = getStatusFile(logger);
|
||||
if (statusFile.exists() && !statusFile.delete()) {
|
||||
logger.error("Failed to delete status file {}; this file should be cleaned up manually", statusFile);
|
||||
}
|
||||
|
||||
try (final Socket socket = new Socket()) {
|
||||
logger.fine("Connecting to NiFi instance");
|
||||
logger.debug("Connecting to NiFi instance");
|
||||
socket.setSoTimeout(60000);
|
||||
socket.connect(new InetSocketAddress("localhost", port));
|
||||
logger.fine("Established connection to NiFi instance.");
|
||||
logger.debug("Established connection to NiFi instance.");
|
||||
socket.setSoTimeout(60000);
|
||||
|
||||
logger.log(Level.FINE, "Sending SHUTDOWN Command to port {0}", port);
|
||||
logger.debug("Sending SHUTDOWN Command to port {}", port);
|
||||
final OutputStream out = socket.getOutputStream();
|
||||
out.write((SHUTDOWN_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
|
||||
out.flush();
|
||||
|
@ -501,7 +525,7 @@ public class RunNiFi {
|
|||
}
|
||||
final String response = sb.toString().trim();
|
||||
|
||||
logger.log(Level.FINE, "Received response to SHUTDOWN command: {0}", response);
|
||||
logger.debug("Received response to SHUTDOWN command: {}", response);
|
||||
|
||||
if (SHUTDOWN_CMD.equals(response)) {
|
||||
logger.info("Apache NiFi has accepted the Shutdown Command and is shutting down now");
|
||||
|
@ -522,17 +546,17 @@ public class RunNiFi {
|
|||
}
|
||||
|
||||
final long startWait = System.nanoTime();
|
||||
while (isProcessRunning(pid)) {
|
||||
while (isProcessRunning(pid, logger)) {
|
||||
logger.info("Waiting for Apache NiFi to finish shutting down...");
|
||||
final long waitNanos = System.nanoTime() - startWait;
|
||||
final long waitSeconds = TimeUnit.NANOSECONDS.toSeconds(waitNanos);
|
||||
if (waitSeconds >= gracefulShutdownSeconds && gracefulShutdownSeconds > 0) {
|
||||
if (isProcessRunning(pid)) {
|
||||
logger.log(Level.WARNING, "NiFi has not finished shutting down after {0} seconds. Killing process.", gracefulShutdownSeconds);
|
||||
if (isProcessRunning(pid, logger)) {
|
||||
logger.warn("NiFi has not finished shutting down after {} seconds. Killing process.", gracefulShutdownSeconds);
|
||||
try {
|
||||
killProcessTree(pid);
|
||||
killProcessTree(pid, logger);
|
||||
} catch (final IOException ioe) {
|
||||
logger.log(Level.SEVERE, "Failed to kill Process with PID {0}", pid);
|
||||
logger.error("Failed to kill Process with PID {}", pid);
|
||||
}
|
||||
}
|
||||
break;
|
||||
|
@ -546,16 +570,11 @@ public class RunNiFi {
|
|||
|
||||
logger.info("NiFi has finished shutting down.");
|
||||
}
|
||||
|
||||
final File statusFile = getStatusFile();
|
||||
if (!statusFile.delete()) {
|
||||
logger.log(Level.SEVERE, "Failed to delete status file {0}; this file should be cleaned up manually", statusFile);
|
||||
}
|
||||
} else {
|
||||
logger.log(Level.SEVERE, "When sending SHUTDOWN command to NiFi, got unexpected response {0}", response);
|
||||
logger.error("When sending SHUTDOWN command to NiFi, got unexpected response {}", response);
|
||||
}
|
||||
} catch (final IOException ioe) {
|
||||
logger.log(Level.SEVERE, "Failed to send shutdown command to port {0} due to {1}", new Object[]{port, ioe});
|
||||
logger.error("Failed to send shutdown command to port {} due to {}", new Object[]{port, ioe.toString(), ioe});
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -574,14 +593,14 @@ public class RunNiFi {
|
|||
return childPids;
|
||||
}
|
||||
|
||||
private void killProcessTree(final String pid) throws IOException {
|
||||
logger.log(Level.FINE, "Killing Process Tree for PID {0}", pid);
|
||||
private void killProcessTree(final String pid, final Logger logger) throws IOException {
|
||||
logger.debug("Killing Process Tree for PID {}", pid);
|
||||
|
||||
final List<String> children = getChildProcesses(pid);
|
||||
logger.log(Level.FINE, "Children of PID {0}: {1}", new Object[]{pid, children});
|
||||
logger.debug("Children of PID {}: {}", new Object[]{pid, children});
|
||||
|
||||
for (final String childPid : children) {
|
||||
killProcessTree(childPid);
|
||||
killProcessTree(childPid, logger);
|
||||
}
|
||||
|
||||
Runtime.getRuntime().exec(new String[]{"kill", "-9", pid});
|
||||
|
@ -597,10 +616,10 @@ public class RunNiFi {
|
|||
}
|
||||
|
||||
@SuppressWarnings({"rawtypes", "unchecked"})
|
||||
public void start(final boolean monitor) throws IOException, InterruptedException {
|
||||
final Integer port = getCurrentPort();
|
||||
public void start() throws IOException, InterruptedException {
|
||||
final Integer port = getCurrentPort(cmdLogger);
|
||||
if (port != null) {
|
||||
System.out.println("Apache NiFi is already running, listening to Bootstrap on port " + port);
|
||||
cmdLogger.info("Apache NiFi is already running, listening to Bootstrap on port " + port);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -739,122 +758,154 @@ public class RunNiFi {
|
|||
cmdBuilder.append(s).append(" ");
|
||||
}
|
||||
|
||||
logger.info("Starting Apache NiFi...");
|
||||
logger.log(Level.INFO, "Working Directory: {0}", workingDir.getAbsolutePath());
|
||||
logger.log(Level.INFO, "Command: {0}", cmdBuilder.toString());
|
||||
cmdLogger.info("Starting Apache NiFi...");
|
||||
cmdLogger.info("Working Directory: {}", workingDir.getAbsolutePath());
|
||||
cmdLogger.info("Command: {}", cmdBuilder.toString());
|
||||
|
||||
if (monitor) {
|
||||
String gracefulShutdown = props.get(GRACEFUL_SHUTDOWN_PROP);
|
||||
if (gracefulShutdown == null) {
|
||||
gracefulShutdown = DEFAULT_GRACEFUL_SHUTDOWN_VALUE;
|
||||
}
|
||||
String gracefulShutdown = props.get(GRACEFUL_SHUTDOWN_PROP);
|
||||
if (gracefulShutdown == null) {
|
||||
gracefulShutdown = DEFAULT_GRACEFUL_SHUTDOWN_VALUE;
|
||||
}
|
||||
|
||||
final int gracefulShutdownSeconds;
|
||||
try {
|
||||
gracefulShutdownSeconds = Integer.parseInt(gracefulShutdown);
|
||||
} catch (final NumberFormatException nfe) {
|
||||
throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File "
|
||||
+ bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer");
|
||||
}
|
||||
final int gracefulShutdownSeconds;
|
||||
try {
|
||||
gracefulShutdownSeconds = Integer.parseInt(gracefulShutdown);
|
||||
} catch (final NumberFormatException nfe) {
|
||||
throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File "
|
||||
+ bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer");
|
||||
}
|
||||
|
||||
if (gracefulShutdownSeconds < 0) {
|
||||
throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File "
|
||||
+ bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer");
|
||||
}
|
||||
if (gracefulShutdownSeconds < 0) {
|
||||
throw new NumberFormatException("The '" + GRACEFUL_SHUTDOWN_PROP + "' property in Bootstrap Config File "
|
||||
+ bootstrapConfigAbsoluteFile.getAbsolutePath() + " has an invalid value. Must be a non-negative integer");
|
||||
}
|
||||
|
||||
Process process = builder.start();
|
||||
Long pid = getPid(process);
|
||||
if (pid != null) {
|
||||
nifiPid = pid;
|
||||
final Properties nifiProps = new Properties();
|
||||
nifiProps.setProperty("pid", String.valueOf(nifiPid));
|
||||
saveProperties(nifiProps);
|
||||
}
|
||||
Process process = builder.start();
|
||||
handleLogging(process);
|
||||
Long pid = getPid(process, cmdLogger);
|
||||
if (pid != null) {
|
||||
nifiPid = pid;
|
||||
final Properties nifiProps = new Properties();
|
||||
nifiProps.setProperty("pid", String.valueOf(nifiPid));
|
||||
saveProperties(nifiProps, cmdLogger);
|
||||
}
|
||||
|
||||
shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds);
|
||||
final Runtime runtime = Runtime.getRuntime();
|
||||
runtime.addShutdownHook(shutdownHook);
|
||||
shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor);
|
||||
final Runtime runtime = Runtime.getRuntime();
|
||||
runtime.addShutdownHook(shutdownHook);
|
||||
|
||||
while (true) {
|
||||
final boolean alive = isAlive(process);
|
||||
while (true) {
|
||||
final boolean alive = isAlive(process);
|
||||
|
||||
if (alive) {
|
||||
try {
|
||||
Thread.sleep(1000L);
|
||||
} catch (final InterruptedException ie) {
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
runtime.removeShutdownHook(shutdownHook);
|
||||
} catch (final IllegalStateException ise) {
|
||||
// happens when already shutting down
|
||||
}
|
||||
if (alive) {
|
||||
try {
|
||||
Thread.sleep(1000L);
|
||||
} catch (final InterruptedException ie) {
|
||||
}
|
||||
} else {
|
||||
try {
|
||||
runtime.removeShutdownHook(shutdownHook);
|
||||
} catch (final IllegalStateException ise) {
|
||||
// happens when already shutting down
|
||||
}
|
||||
|
||||
if (autoRestartNiFi) {
|
||||
logger.warning("Apache NiFi appears to have died. Restarting...");
|
||||
process = builder.start();
|
||||
|
||||
pid = getPid(process);
|
||||
if (pid != null) {
|
||||
nifiPid = pid;
|
||||
final Properties nifiProps = new Properties();
|
||||
nifiProps.setProperty("pid", String.valueOf(nifiPid));
|
||||
saveProperties(nifiProps);
|
||||
}
|
||||
|
||||
shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds);
|
||||
runtime.addShutdownHook(shutdownHook);
|
||||
|
||||
final boolean started = waitForStart();
|
||||
|
||||
if (started) {
|
||||
logger.log(Level.INFO, "Successfully started Apache NiFi{0}", (pid == null ? "" : " with PID " + pid));
|
||||
} else {
|
||||
logger.severe("Apache NiFi does not appear to have started");
|
||||
}
|
||||
} else {
|
||||
if (autoRestartNiFi) {
|
||||
final File statusFile = getStatusFile(defaultLogger);
|
||||
if (!statusFile.exists()) {
|
||||
defaultLogger.debug("Status File no longer exists. Will not restart NiFi");
|
||||
return;
|
||||
}
|
||||
|
||||
defaultLogger.warn("Apache NiFi appears to have died. Restarting...");
|
||||
process = builder.start();
|
||||
handleLogging(process);
|
||||
|
||||
pid = getPid(process, defaultLogger);
|
||||
if (pid != null) {
|
||||
nifiPid = pid;
|
||||
final Properties nifiProps = new Properties();
|
||||
nifiProps.setProperty("pid", String.valueOf(nifiPid));
|
||||
saveProperties(nifiProps, defaultLogger);
|
||||
}
|
||||
|
||||
shutdownHook = new ShutdownHook(process, this, secretKey, gracefulShutdownSeconds, loggingExecutor);
|
||||
runtime.addShutdownHook(shutdownHook);
|
||||
|
||||
final boolean started = waitForStart();
|
||||
|
||||
if (started) {
|
||||
defaultLogger.info("Successfully started Apache NiFi{}", (pid == null ? "" : " with PID " + pid));
|
||||
} else {
|
||||
defaultLogger.error("Apache NiFi does not appear to have started");
|
||||
}
|
||||
} else {
|
||||
return;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
final Process process = builder.start();
|
||||
final Long pid = getPid(process);
|
||||
|
||||
if (pid != null) {
|
||||
nifiPid = pid;
|
||||
final Properties nifiProps = new Properties();
|
||||
nifiProps.setProperty("pid", String.valueOf(nifiPid));
|
||||
saveProperties(nifiProps);
|
||||
}
|
||||
|
||||
boolean started = waitForStart();
|
||||
|
||||
if (started) {
|
||||
logger.log(Level.INFO, "Successfully started Apache NiFi{0}", (pid == null ? "" : " with PID " + pid));
|
||||
} else {
|
||||
logger.severe("Apache NiFi does not appear to have started");
|
||||
}
|
||||
|
||||
listener.stop();
|
||||
}
|
||||
}
|
||||
|
||||
private Long getPid(final Process process) {
|
||||
private void handleLogging(final Process process) {
|
||||
final Set<Future<?>> existingFutures = loggingFutures;
|
||||
if (existingFutures != null) {
|
||||
for (final Future<?> future : existingFutures) {
|
||||
future.cancel(false);
|
||||
}
|
||||
}
|
||||
|
||||
final Future<?> stdOutFuture = loggingExecutor.submit(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
final Logger stdOutLogger = LoggerFactory.getLogger("org.apache.nifi.StdOut");
|
||||
final InputStream in = process.getInputStream();
|
||||
try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
stdOutLogger.info(line);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
defaultLogger.error("Failed to read from NiFi's Standard Out stream", e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
final Future<?> stdErrFuture = loggingExecutor.submit(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
final Logger stdErrLogger = LoggerFactory.getLogger("org.apache.nifi.StdErr");
|
||||
final InputStream in = process.getErrorStream();
|
||||
try (final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
|
||||
String line;
|
||||
while ((line = reader.readLine()) != null) {
|
||||
stdErrLogger.error(line);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
defaultLogger.error("Failed to read from NiFi's Standard Error stream", e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
final Set<Future<?>> futures = new HashSet<>();
|
||||
futures.add(stdOutFuture);
|
||||
futures.add(stdErrFuture);
|
||||
this.loggingFutures = futures;
|
||||
}
|
||||
|
||||
private Long getPid(final Process process, final Logger logger) {
|
||||
try {
|
||||
final Class<?> procClass = process.getClass();
|
||||
final Field pidField = procClass.getDeclaredField("pid");
|
||||
pidField.setAccessible(true);
|
||||
final Object pidObject = pidField.get(process);
|
||||
|
||||
logger.log(Level.FINE, "PID Object = {0}", pidObject);
|
||||
logger.debug("PID Object = {}", pidObject);
|
||||
|
||||
if (pidObject instanceof Number) {
|
||||
return ((Number) pidObject).longValue();
|
||||
}
|
||||
return null;
|
||||
} catch (final IllegalAccessException | NoSuchFieldException nsfe) {
|
||||
logger.log(Level.FINE, "Could not find PID for child process due to {0}", nsfe);
|
||||
logger.debug("Could not find PID for child process due to {}", nsfe);
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -913,7 +964,7 @@ public class RunNiFi {
|
|||
shutdownHook.setSecretKey(secretKey);
|
||||
}
|
||||
|
||||
final File statusFile = getStatusFile();
|
||||
final File statusFile = getStatusFile(defaultLogger);
|
||||
|
||||
final Properties nifiProps = new Properties();
|
||||
if (nifiPid != -1) {
|
||||
|
@ -923,12 +974,12 @@ public class RunNiFi {
|
|||
nifiProps.setProperty("secret.key", secretKey);
|
||||
|
||||
try {
|
||||
saveProperties(nifiProps);
|
||||
saveProperties(nifiProps, defaultLogger);
|
||||
} catch (final IOException ioe) {
|
||||
logger.log(Level.WARNING, "Apache NiFi has started but failed to persist NiFi Port information to {0} due to {1}", new Object[]{statusFile.getAbsolutePath(), ioe});
|
||||
defaultLogger.warn("Apache NiFi has started but failed to persist NiFi Port information to {} due to {}", new Object[]{statusFile.getAbsolutePath(), ioe});
|
||||
}
|
||||
|
||||
logger.log(Level.INFO, "Apache NiFi now running and listening for Bootstrap requests on port {0}", port);
|
||||
defaultLogger.info("Apache NiFi now running and listening for Bootstrap requests on port {}", port);
|
||||
}
|
||||
|
||||
int getNiFiCommandControlPort() {
|
||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
import java.io.OutputStream;
|
||||
import java.net.Socket;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class ShutdownHook extends Thread {
|
||||
|
@ -28,14 +29,16 @@ public class ShutdownHook extends Thread {
|
|||
private final Process nifiProcess;
|
||||
private final RunNiFi runner;
|
||||
private final int gracefulShutdownSeconds;
|
||||
private final ExecutorService executor;
|
||||
|
||||
private volatile String secretKey;
|
||||
|
||||
public ShutdownHook(final Process nifiProcess, final RunNiFi runner, final String secretKey, final int gracefulShutdownSeconds) {
|
||||
public ShutdownHook(final Process nifiProcess, final RunNiFi runner, final String secretKey, final int gracefulShutdownSeconds, final ExecutorService executor) {
|
||||
this.nifiProcess = nifiProcess;
|
||||
this.runner = runner;
|
||||
this.secretKey = secretKey;
|
||||
this.gracefulShutdownSeconds = gracefulShutdownSeconds;
|
||||
this.executor = executor;
|
||||
}
|
||||
|
||||
void setSecretKey(final String secretKey) {
|
||||
|
@ -44,6 +47,7 @@ public class ShutdownHook extends Thread {
|
|||
|
||||
@Override
|
||||
public void run() {
|
||||
executor.shutdown();
|
||||
runner.setAutoRestartNiFi(false);
|
||||
final int ccPort = runner.getNiFiCommandControlPort();
|
||||
if (ccPort > 0) {
|
||||
|
|
|
@ -163,6 +163,8 @@ public class SSLProperties {
|
|||
KEYSTORE, TRUSTSTORE
|
||||
}
|
||||
|
||||
private static final String DEFAULT_SSL_PROTOCOL_ALGORITHM = "TLS";
|
||||
|
||||
public static List<PropertyDescriptor> getKeystoreDescriptors(final boolean required) {
|
||||
final List<PropertyDescriptor> descriptors = new ArrayList<>();
|
||||
for (final PropertyDescriptor descriptor : KEYSTORE_DESCRIPTORS) {
|
||||
|
@ -196,14 +198,15 @@ public class SSLProperties {
|
|||
return SslContextFactory.createTrustSslContext(
|
||||
context.getProperty(TRUSTSTORE).getValue(),
|
||||
context.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
|
||||
context.getProperty(TRUSTSTORE_TYPE).getValue());
|
||||
context.getProperty(TRUSTSTORE_TYPE).getValue(),
|
||||
DEFAULT_SSL_PROTOCOL_ALGORITHM);
|
||||
} else {
|
||||
final String truststoreFile = context.getProperty(TRUSTSTORE).getValue();
|
||||
if (truststoreFile == null) {
|
||||
return SslContextFactory.createSslContext(
|
||||
context.getProperty(KEYSTORE).getValue(),
|
||||
context.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(),
|
||||
context.getProperty(KEYSTORE_TYPE).getValue());
|
||||
context.getProperty(KEYSTORE_TYPE).getValue(), DEFAULT_SSL_PROTOCOL_ALGORITHM);
|
||||
} else {
|
||||
return SslContextFactory.createSslContext(
|
||||
context.getProperty(KEYSTORE).getValue(),
|
||||
|
@ -212,7 +215,8 @@ public class SSLProperties {
|
|||
context.getProperty(TRUSTSTORE).getValue(),
|
||||
context.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
|
||||
context.getProperty(TRUSTSTORE_TYPE).getValue(),
|
||||
clientAuth);
|
||||
clientAuth,
|
||||
DEFAULT_SSL_PROTOCOL_ALGORITHM);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.io.BufferedInputStream;
|
|||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.InputStream;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.file.Path;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -84,9 +85,15 @@ public class NiFiPropertiesTest {
|
|||
|
||||
private NiFiProperties loadSpecifiedProperties(String propertiesFile) {
|
||||
|
||||
String file = NiFiPropertiesTest.class.getResource(propertiesFile).getFile();
|
||||
String filePath;
|
||||
try {
|
||||
filePath = NiFiPropertiesTest.class.getResource(propertiesFile).toURI().getPath();
|
||||
} catch (URISyntaxException ex) {
|
||||
throw new RuntimeException("Cannot load properties file due to "
|
||||
+ ex.getLocalizedMessage(), ex);
|
||||
}
|
||||
|
||||
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, file);
|
||||
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, filePath);
|
||||
|
||||
NiFiProperties properties = NiFiProperties.getInstance();
|
||||
|
||||
|
@ -97,7 +104,7 @@ public class NiFiPropertiesTest {
|
|||
|
||||
InputStream inStream = null;
|
||||
try {
|
||||
inStream = new BufferedInputStream(new FileInputStream(file));
|
||||
inStream = new BufferedInputStream(new FileInputStream(filePath));
|
||||
properties.load(inStream);
|
||||
} catch (final Exception ex) {
|
||||
throw new RuntimeException("Cannot load properties file due to "
|
||||
|
|
|
@ -57,6 +57,7 @@ public final class SslContextFactory {
|
|||
* @param truststorePasswd the truststore password
|
||||
* @param truststoreType the type of truststore (e.g., PKCS12, JKS)
|
||||
* @param clientAuth the type of client authentication
|
||||
* @param protocol the protocol to use for the SSL connection
|
||||
*
|
||||
* @return a SSLContext instance
|
||||
* @throws java.security.KeyStoreException if any issues accessing the keystore
|
||||
|
@ -69,7 +70,7 @@ public final class SslContextFactory {
|
|||
public static SSLContext createSslContext(
|
||||
final String keystore, final char[] keystorePasswd, final String keystoreType,
|
||||
final String truststore, final char[] truststorePasswd, final String truststoreType,
|
||||
final ClientAuth clientAuth)
|
||||
final ClientAuth clientAuth, final String protocol)
|
||||
throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
|
||||
UnrecoverableKeyException, KeyManagementException {
|
||||
|
||||
|
@ -90,7 +91,7 @@ public final class SslContextFactory {
|
|||
trustManagerFactory.init(trustStore);
|
||||
|
||||
// initialize the ssl context
|
||||
final SSLContext sslContext = SSLContext.getInstance("TLS");
|
||||
final SSLContext sslContext = SSLContext.getInstance(protocol);
|
||||
sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
|
||||
if (ClientAuth.REQUIRED == clientAuth) {
|
||||
sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
|
||||
|
@ -110,6 +111,7 @@ public final class SslContextFactory {
|
|||
* @param keystore the full path to the keystore
|
||||
* @param keystorePasswd the keystore password
|
||||
* @param keystoreType the type of keystore (e.g., PKCS12, JKS)
|
||||
* @param protocol the protocol to use for the SSL connection
|
||||
*
|
||||
* @return a SSLContext instance
|
||||
* @throws java.security.KeyStoreException if any issues accessing the keystore
|
||||
|
@ -120,7 +122,7 @@ public final class SslContextFactory {
|
|||
* @throws java.security.KeyManagementException if unable to manage the key
|
||||
*/
|
||||
public static SSLContext createSslContext(
|
||||
final String keystore, final char[] keystorePasswd, final String keystoreType)
|
||||
final String keystore, final char[] keystorePasswd, final String keystoreType, final String protocol)
|
||||
throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
|
||||
UnrecoverableKeyException, KeyManagementException {
|
||||
|
||||
|
@ -133,7 +135,7 @@ public final class SslContextFactory {
|
|||
keyManagerFactory.init(keyStore, keystorePasswd);
|
||||
|
||||
// initialize the ssl context
|
||||
final SSLContext ctx = SSLContext.getInstance("TLS");
|
||||
final SSLContext ctx = SSLContext.getInstance(protocol);
|
||||
ctx.init(keyManagerFactory.getKeyManagers(), new TrustManager[0], new SecureRandom());
|
||||
|
||||
return ctx;
|
||||
|
@ -146,6 +148,7 @@ public final class SslContextFactory {
|
|||
* @param truststore the full path to the truststore
|
||||
* @param truststorePasswd the truststore password
|
||||
* @param truststoreType the type of truststore (e.g., PKCS12, JKS)
|
||||
* @param protocol the protocol to use for the SSL connection
|
||||
*
|
||||
* @return a SSLContext instance
|
||||
* @throws java.security.KeyStoreException if any issues accessing the keystore
|
||||
|
@ -156,7 +159,7 @@ public final class SslContextFactory {
|
|||
* @throws java.security.KeyManagementException if unable to manage the key
|
||||
*/
|
||||
public static SSLContext createTrustSslContext(
|
||||
final String truststore, final char[] truststorePasswd, final String truststoreType)
|
||||
final String truststore, final char[] truststorePasswd, final String truststoreType, final String protocol)
|
||||
throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
|
||||
UnrecoverableKeyException, KeyManagementException {
|
||||
|
||||
|
@ -169,7 +172,7 @@ public final class SslContextFactory {
|
|||
trustManagerFactory.init(trustStore);
|
||||
|
||||
// initialize the ssl context
|
||||
final SSLContext ctx = SSLContext.getInstance("TLS");
|
||||
final SSLContext ctx = SSLContext.getInstance(protocol);
|
||||
ctx.init(new KeyManager[0], trustManagerFactory.getTrustManagers(), new SecureRandom());
|
||||
|
||||
return ctx;
|
||||
|
|
|
@ -40,11 +40,11 @@ import java.util.Set;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.processor.FlowFileFilter;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.QueueSize;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.FlowFileAccessException;
|
||||
|
@ -54,6 +54,7 @@ import org.apache.nifi.processor.io.InputStreamCallback;
|
|||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.io.StreamCallback;
|
||||
import org.apache.nifi.provenance.ProvenanceReporter;
|
||||
import org.junit.Assert;
|
||||
|
||||
public class MockProcessSession implements ProcessSession {
|
||||
|
||||
|
@ -65,14 +66,16 @@ public class MockProcessSession implements ProcessSession {
|
|||
private final Map<Long, MockFlowFile> originalVersions = new HashMap<>();
|
||||
private final SharedSessionState sharedState;
|
||||
private final Map<String, Long> counterMap = new HashMap<>();
|
||||
private final ProvenanceReporter provenanceReporter;
|
||||
|
||||
private boolean committed = false;
|
||||
private boolean rolledback = false;
|
||||
private int removedCount = 0;
|
||||
|
||||
public MockProcessSession(final SharedSessionState sharedState) {
|
||||
public MockProcessSession(final SharedSessionState sharedState, final Processor processor) {
|
||||
this.sharedState = sharedState;
|
||||
this.processorQueue = sharedState.getFlowFileQueue();
|
||||
provenanceReporter = new MockProvenanceReporter(this, sharedState, processor.getIdentifier(), processor.getClass().getSimpleName());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -194,7 +197,7 @@ public class MockProcessSession implements ProcessSession {
|
|||
|
||||
try {
|
||||
out.write(mock.getData());
|
||||
} catch (IOException e) {
|
||||
} catch (final IOException e) {
|
||||
throw new FlowFileAccessException(e.toString(), e);
|
||||
}
|
||||
}
|
||||
|
@ -409,7 +412,7 @@ public class MockProcessSession implements ProcessSession {
|
|||
final ByteArrayInputStream bais = new ByteArrayInputStream(mock.getData());
|
||||
try {
|
||||
callback.process(bais);
|
||||
} catch (IOException e) {
|
||||
} catch (final IOException e) {
|
||||
throw new ProcessException(e.toString(), e);
|
||||
}
|
||||
}
|
||||
|
@ -766,7 +769,7 @@ public class MockProcessSession implements ProcessSession {
|
|||
if (source == null || destination == null || source == destination) {
|
||||
return destination; //don't need to inherit from ourselves
|
||||
}
|
||||
FlowFile updated = putAllAttributes(destination, source.getAttributes());
|
||||
final FlowFile updated = putAllAttributes(destination, source.getAttributes());
|
||||
getProvenanceReporter().fork(source, Collections.singletonList(updated));
|
||||
return updated;
|
||||
}
|
||||
|
@ -803,7 +806,7 @@ public class MockProcessSession implements ProcessSession {
|
|||
}
|
||||
}
|
||||
|
||||
FlowFile updated = putAllAttributes(destination, intersectAttributes(sources));
|
||||
final FlowFile updated = putAllAttributes(destination, intersectAttributes(sources));
|
||||
getProvenanceReporter().join(sources, updated);
|
||||
return updated;
|
||||
}
|
||||
|
@ -982,7 +985,7 @@ public class MockProcessSession implements ProcessSession {
|
|||
|
||||
@Override
|
||||
public ProvenanceReporter getProvenanceReporter() {
|
||||
return sharedState.getProvenanceReporter();
|
||||
return provenanceReporter;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -997,4 +1000,27 @@ public class MockProcessSession implements ProcessSession {
|
|||
validateState(flowFile);
|
||||
return flowFile.getData();
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if a FlowFile is known in this session.
|
||||
*
|
||||
* @param flowFile
|
||||
* the FlowFile to check
|
||||
* @return <code>true</code> if the FlowFile is known in this session,
|
||||
* <code>false</code> otherwise.
|
||||
*/
|
||||
boolean isFlowFileKnown(final FlowFile flowFile) {
|
||||
final FlowFile curFlowFile = currentVersions.get(flowFile.getId());
|
||||
if (curFlowFile == null) {
|
||||
return false;
|
||||
}
|
||||
|
||||
final String curUuid = curFlowFile.getAttribute(CoreAttributes.UUID.key());
|
||||
final String providedUuid = curFlowFile.getAttribute(CoreAttributes.UUID.key());
|
||||
if (!curUuid.equals(providedUuid)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,186 +17,437 @@
|
|||
package org.apache.nifi.util;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.FlowFileHandlingException;
|
||||
import org.apache.nifi.provenance.ProvenanceEventBuilder;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.provenance.ProvenanceReporter;
|
||||
import org.apache.nifi.provenance.StandardProvenanceEventRecord;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class MockProvenanceReporter implements ProvenanceReporter {
|
||||
private static final Logger logger = LoggerFactory.getLogger(MockProvenanceReporter.class);
|
||||
private final MockProcessSession session;
|
||||
private final String processorId;
|
||||
private final String processorType;
|
||||
private final SharedSessionState sharedSessionState;
|
||||
private final Set<ProvenanceEventRecord> events = new LinkedHashSet<>();
|
||||
|
||||
@Override
|
||||
public void receive(FlowFile flowFile, String sourceSystemUri) {
|
||||
public MockProvenanceReporter(final MockProcessSession session, final SharedSessionState sharedState, final String processorId, final String processorType) {
|
||||
this.session = session;
|
||||
this.sharedSessionState = sharedState;
|
||||
this.processorId = processorId;
|
||||
this.processorType = processorType;
|
||||
}
|
||||
|
||||
private void verifyFlowFileKnown(final FlowFile flowFile) {
|
||||
if (session != null && !session.isFlowFileKnown(flowFile)) {
|
||||
throw new FlowFileHandlingException(flowFile + " is not known to " + session);
|
||||
}
|
||||
}
|
||||
|
||||
Set<ProvenanceEventRecord> getEvents() {
|
||||
return Collections.unmodifiableSet(events);
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the given event from the reporter
|
||||
*
|
||||
* @param event
|
||||
* event
|
||||
*/
|
||||
void remove(final ProvenanceEventRecord event) {
|
||||
events.remove(event);
|
||||
}
|
||||
|
||||
void clear() {
|
||||
events.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a Fork event for the given child and parents but does not
|
||||
* register the event. This is useful so that a ProcessSession has the
|
||||
* ability to de-dupe events, since one or more events may be created by the
|
||||
* session itself, as well as by the Processor
|
||||
*
|
||||
* @param parents
|
||||
* parents
|
||||
* @param child
|
||||
* child
|
||||
* @return record
|
||||
*/
|
||||
ProvenanceEventRecord generateJoinEvent(final Collection<FlowFile> parents, final FlowFile child) {
|
||||
final ProvenanceEventBuilder eventBuilder = build(child, ProvenanceEventType.JOIN);
|
||||
eventBuilder.addChildFlowFile(child);
|
||||
|
||||
for (final FlowFile parent : parents) {
|
||||
eventBuilder.addParentFlowFile(parent);
|
||||
}
|
||||
|
||||
return eventBuilder.build();
|
||||
}
|
||||
|
||||
ProvenanceEventRecord generateDropEvent(final FlowFile flowFile, final String details) {
|
||||
return build(flowFile, ProvenanceEventType.DROP).setDetails(details).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void send(FlowFile flowFile, String destinationSystemUri) {
|
||||
|
||||
public void receive(final FlowFile flowFile, final String transitUri) {
|
||||
receive(flowFile, transitUri, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void send(FlowFile flowFile, String destinationSystemUri, boolean force) {
|
||||
|
||||
public void receive(FlowFile flowFile, String transitUri, String sourceSystemFlowFileIdentifier) {
|
||||
receive(flowFile, transitUri, sourceSystemFlowFileIdentifier, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receive(FlowFile flowFile, String sourceSystemUri, long transmissionMillis) {
|
||||
|
||||
public void receive(final FlowFile flowFile, final String transitUri, final long transmissionMillis) {
|
||||
receive(flowFile, transitUri, null, transmissionMillis);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receive(FlowFile flowFile, String sourceSystemUri, String sourceSystemFlowFileIdentifier) {
|
||||
|
||||
public void receive(final FlowFile flowFile, final String transitUri, final String sourceSystemFlowFileIdentifier, final long transmissionMillis) {
|
||||
receive(flowFile, transitUri, sourceSystemFlowFileIdentifier, null, transmissionMillis);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receive(FlowFile flowFile, String sourceSystemUri, String sourceSystemFlowFileIdentifier, long transmissionMillis) {
|
||||
public void receive(final FlowFile flowFile, final String transitUri, final String sourceSystemFlowFileIdentifier, final String details, final long transmissionMillis) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.RECEIVE)
|
||||
.setTransitUri(transitUri).setSourceSystemFlowFileIdentifier(sourceSystemFlowFileIdentifier).setEventDuration(transmissionMillis).setDetails(details).build();
|
||||
events.add(record);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void send(FlowFile flowFile, String destinationSystemUri, long transmissionMillis) {
|
||||
|
||||
public void send(final FlowFile flowFile, final String transitUri, final long transmissionMillis) {
|
||||
send(flowFile, transitUri, transmissionMillis, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void send(FlowFile flowFile, String destinationSystemUri, long transmissionMillis, boolean force) {
|
||||
|
||||
public void send(final FlowFile flowFile, final String transitUri) {
|
||||
send(flowFile, transitUri, null, -1L, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void associate(FlowFile flowFile, String alternateIdentifierNamespace, String alternateIdentifier) {
|
||||
|
||||
public void send(final FlowFile flowFile, final String transitUri, final String details) {
|
||||
send(flowFile, transitUri, details, -1L, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void fork(FlowFile parent, Collection<FlowFile> children) {
|
||||
|
||||
public void send(final FlowFile flowFile, final String transitUri, final long transmissionMillis, final boolean force) {
|
||||
send(flowFile, transitUri, null, transmissionMillis, force);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void fork(FlowFile parent, Collection<FlowFile> children, long forkDuration) {
|
||||
|
||||
public void send(final FlowFile flowFile, final String transitUri, final String details, final boolean force) {
|
||||
send(flowFile, transitUri, details, -1L, force);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void fork(FlowFile parent, Collection<FlowFile> children, String details) {
|
||||
|
||||
public void send(final FlowFile flowFile, final String transitUri, final String details, final long transmissionMillis) {
|
||||
send(flowFile, transitUri, details, transmissionMillis, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void fork(FlowFile parent, java.util.Collection<FlowFile> children, String details, long forkDuration) {
|
||||
|
||||
public void send(final FlowFile flowFile, final String transitUri, final String details, final long transmissionMillis, final boolean force) {
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.SEND).setTransitUri(transitUri).setEventDuration(transmissionMillis).setDetails(details).build();
|
||||
if (force) {
|
||||
sharedSessionState.addProvenanceEvents(Collections.singleton(record));
|
||||
} else {
|
||||
events.add(record);
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void join(Collection<FlowFile> parents, FlowFile child) {
|
||||
|
||||
public void send(final FlowFile flowFile, final String transitUri, final boolean force) {
|
||||
send(flowFile, transitUri, -1L, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void join(Collection<FlowFile> parents, FlowFile child, long joinDuration) {
|
||||
public void associate(final FlowFile flowFile, final String alternateIdentifierNamespace, final String alternateIdentifier) {
|
||||
try {
|
||||
String trimmedNamespace = alternateIdentifierNamespace.trim();
|
||||
if (trimmedNamespace.endsWith(":")) {
|
||||
trimmedNamespace = trimmedNamespace.substring(0, trimmedNamespace.length() - 1);
|
||||
}
|
||||
|
||||
String trimmedIdentifier = alternateIdentifier.trim();
|
||||
if (trimmedIdentifier.startsWith(":")) {
|
||||
if (trimmedIdentifier.length() == 1) {
|
||||
throw new IllegalArgumentException("Illegal alternateIdentifier: " + alternateIdentifier);
|
||||
}
|
||||
trimmedIdentifier = trimmedIdentifier.substring(1);
|
||||
}
|
||||
|
||||
final String alternateIdentifierUri = trimmedNamespace + ":" + trimmedIdentifier;
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.ADDINFO).setAlternateIdentifierUri(alternateIdentifierUri).build();
|
||||
events.add(record);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ProvenanceEventRecord drop(final FlowFile flowFile, final String reason) {
|
||||
try {
|
||||
final ProvenanceEventBuilder builder = build(flowFile, ProvenanceEventType.DROP);
|
||||
if (reason != null) {
|
||||
builder.setDetails("Discard reason: " + reason);
|
||||
}
|
||||
final ProvenanceEventRecord record = builder.build();
|
||||
events.add(record);
|
||||
return record;
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
void expire(final FlowFile flowFile, final String details) {
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.EXPIRE).setDetails(details).build();
|
||||
events.add(record);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void join(Collection<FlowFile> parents, FlowFile child, String details) {
|
||||
|
||||
public void fork(final FlowFile parent, final Collection<FlowFile> children) {
|
||||
fork(parent, children, null, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void join(java.util.Collection<FlowFile> parents, FlowFile child, String details, long joinDuration) {
|
||||
|
||||
public void fork(final FlowFile parent, final Collection<FlowFile> children, final long forkDuration) {
|
||||
fork(parent, children, null, forkDuration);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clone(FlowFile parent, FlowFile child) {
|
||||
|
||||
public void fork(final FlowFile parent, final Collection<FlowFile> children, final String details) {
|
||||
fork(parent, children, details, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyContent(FlowFile flowFile) {
|
||||
public void fork(final FlowFile parent, final Collection<FlowFile> children, final String details, final long forkDuration) {
|
||||
verifyFlowFileKnown(parent);
|
||||
|
||||
try {
|
||||
final ProvenanceEventBuilder eventBuilder = build(parent, ProvenanceEventType.FORK);
|
||||
eventBuilder.addParentFlowFile(parent);
|
||||
for (final FlowFile child : children) {
|
||||
eventBuilder.addChildFlowFile(child);
|
||||
}
|
||||
|
||||
if (forkDuration > -1L) {
|
||||
eventBuilder.setEventDuration(forkDuration);
|
||||
}
|
||||
|
||||
if (details != null) {
|
||||
eventBuilder.setDetails(details);
|
||||
}
|
||||
|
||||
events.add(eventBuilder.build());
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyContent(FlowFile flowFile, String details) {
|
||||
|
||||
public void join(final Collection<FlowFile> parents, final FlowFile child) {
|
||||
join(parents, child, null, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyContent(FlowFile flowFile, long processingMillis) {
|
||||
|
||||
public void join(final Collection<FlowFile> parents, final FlowFile child, final long joinDuration) {
|
||||
join(parents, child, null, joinDuration);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyContent(FlowFile flowFile, String details, long processingMillis) {
|
||||
|
||||
public void join(final Collection<FlowFile> parents, final FlowFile child, final String details) {
|
||||
join(parents, child, details, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyAttributes(FlowFile flowFile) {
|
||||
public void join(final Collection<FlowFile> parents, final FlowFile child, final String details, final long joinDuration) {
|
||||
verifyFlowFileKnown(child);
|
||||
|
||||
try {
|
||||
final ProvenanceEventBuilder eventBuilder = build(child, ProvenanceEventType.JOIN);
|
||||
eventBuilder.addChildFlowFile(child);
|
||||
eventBuilder.setDetails(details);
|
||||
|
||||
for (final FlowFile parent : parents) {
|
||||
eventBuilder.addParentFlowFile(parent);
|
||||
}
|
||||
|
||||
events.add(eventBuilder.build());
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void modifyAttributes(FlowFile flowFile, String details) {
|
||||
public void clone(final FlowFile parent, final FlowFile child) {
|
||||
verifyFlowFileKnown(child);
|
||||
|
||||
try {
|
||||
final ProvenanceEventBuilder eventBuilder = build(parent, ProvenanceEventType.CLONE);
|
||||
eventBuilder.addChildFlowFile(child);
|
||||
eventBuilder.addParentFlowFile(parent);
|
||||
events.add(eventBuilder.build());
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void route(FlowFile flowFile, Relationship relationship) {
|
||||
|
||||
public void modifyContent(final FlowFile flowFile) {
|
||||
modifyContent(flowFile, null, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void route(FlowFile flowFile, Relationship relationship, String details) {
|
||||
|
||||
public void modifyContent(final FlowFile flowFile, final String details) {
|
||||
modifyContent(flowFile, details, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void route(FlowFile flowFile, Relationship relationship, long processingDuration) {
|
||||
|
||||
public void modifyContent(final FlowFile flowFile, final long processingMillis) {
|
||||
modifyContent(flowFile, null, processingMillis);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void route(FlowFile flowFile, Relationship relationship, String details, long processingDuration) {
|
||||
public void modifyContent(final FlowFile flowFile, final String details, final long processingMillis) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.CONTENT_MODIFIED).setEventDuration(processingMillis).setDetails(details).build();
|
||||
events.add(record);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void create(FlowFile flowFile) {
|
||||
|
||||
public void modifyAttributes(final FlowFile flowFile) {
|
||||
modifyAttributes(flowFile, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void create(FlowFile flowFile, String details) {
|
||||
public void modifyAttributes(final FlowFile flowFile, final String details) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.ATTRIBUTES_MODIFIED).setDetails(details).build();
|
||||
events.add(record);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receive(FlowFile flowFile, String sourceSystemUri, String sourceSystemFlowFileIdentifier, String details, long transmissionMillis) {
|
||||
|
||||
public void route(final FlowFile flowFile, final Relationship relationship) {
|
||||
route(flowFile, relationship, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void send(FlowFile flowFile, String destinationSystemUri, String details) {
|
||||
|
||||
public void route(final FlowFile flowFile, final Relationship relationship, final long processingDuration) {
|
||||
route(flowFile, relationship, null, processingDuration);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void send(FlowFile flowFile, String destinationSystemUri, String details, long transmissionMillis) {
|
||||
|
||||
public void route(final FlowFile flowFile, final Relationship relationship, final String details) {
|
||||
route(flowFile, relationship, details, -1L);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void send(FlowFile flowFile, String destinationSystemUri, String details, boolean force) {
|
||||
public void route(final FlowFile flowFile, final Relationship relationship, final String details, final long processingDuration) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.ROUTE).setRelationship(relationship).setDetails(details).setEventDuration(processingDuration).build();
|
||||
events.add(record);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void send(FlowFile flowFile, String destinationSystemUri, String details, long transmissionMillis, boolean force) {
|
||||
public void create(final FlowFile flowFile) {
|
||||
create(flowFile, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void create(final FlowFile flowFile, final String details) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.CREATE).setDetails(details).build();
|
||||
events.add(record);
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to generate Provenance Event due to " + e);
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.error("", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ProvenanceEventBuilder build(final FlowFile flowFile, final ProvenanceEventType eventType) {
|
||||
final ProvenanceEventBuilder builder = new StandardProvenanceEventRecord.Builder();
|
||||
builder.setEventType(eventType);
|
||||
builder.fromFlowFile(flowFile);
|
||||
builder.setLineageStartDate(flowFile.getLineageStartDate());
|
||||
builder.setComponentId(processorId);
|
||||
builder.setComponentType(processorType);
|
||||
return builder;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -22,20 +22,22 @@ import java.util.concurrent.CopyOnWriteArraySet;
|
|||
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
|
||||
public class MockSessionFactory implements ProcessSessionFactory {
|
||||
|
||||
private final Processor processor;
|
||||
private final SharedSessionState sharedState;
|
||||
|
||||
private final Set<MockProcessSession> createdSessions = new CopyOnWriteArraySet<>();
|
||||
|
||||
MockSessionFactory(final SharedSessionState sharedState) {
|
||||
MockSessionFactory(final SharedSessionState sharedState, final Processor processor) {
|
||||
this.sharedState = sharedState;
|
||||
this.processor = processor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProcessSession createSession() {
|
||||
final MockProcessSession session = new MockProcessSession(sharedState);
|
||||
final MockProcessSession session = new MockProcessSession(sharedState, processor);
|
||||
createdSessions.add(session);
|
||||
return session;
|
||||
}
|
||||
|
|
|
@ -16,11 +16,18 @@
|
|||
*/
|
||||
package org.apache.nifi.util;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceReporter;
|
||||
|
||||
public class SharedSessionState {
|
||||
|
@ -31,14 +38,27 @@ public class SharedSessionState {
|
|||
private final Processor processor;
|
||||
private final AtomicLong flowFileIdGenerator;
|
||||
private final ConcurrentMap<String, AtomicLong> counterMap = new ConcurrentHashMap<>();
|
||||
private final Set<ProvenanceEventRecord> events = new LinkedHashSet<>();
|
||||
|
||||
public SharedSessionState(final Processor processor, final AtomicLong flowFileIdGenerator) {
|
||||
flowFileQueue = new MockFlowFileQueue();
|
||||
provenanceReporter = new MockProvenanceReporter();
|
||||
provenanceReporter = new MockProvenanceReporter(null, this, UUID.randomUUID().toString(), "N/A");
|
||||
this.flowFileIdGenerator = flowFileIdGenerator;
|
||||
this.processor = processor;
|
||||
}
|
||||
|
||||
void addProvenanceEvents(final Collection<ProvenanceEventRecord> events) {
|
||||
this.events.addAll(events);
|
||||
}
|
||||
|
||||
void clearProvenanceEvents() {
|
||||
this.events.clear();
|
||||
}
|
||||
|
||||
public List<ProvenanceEventRecord> getProvenanceEvents() {
|
||||
return new ArrayList<>(this.events);
|
||||
}
|
||||
|
||||
public MockFlowFileQueue getFlowFileQueue() {
|
||||
return flowFileQueue;
|
||||
}
|
||||
|
@ -55,7 +75,7 @@ public class SharedSessionState {
|
|||
AtomicLong counter = counterMap.get(name);
|
||||
if (counter == null) {
|
||||
counter = new AtomicLong(0L);
|
||||
AtomicLong existingCounter = counterMap.putIfAbsent(name, counter);
|
||||
final AtomicLong existingCounter = counterMap.putIfAbsent(name, counter);
|
||||
if (existingCounter != null) {
|
||||
counter = existingCounter;
|
||||
}
|
||||
|
@ -66,6 +86,6 @@ public class SharedSessionState {
|
|||
|
||||
public Long getCounterValue(final String name) {
|
||||
final AtomicLong counterValue = counterMap.get(name);
|
||||
return (counterValue == null) ? null : counterValue.get();
|
||||
return counterValue == null ? null : counterValue.get();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -64,11 +64,10 @@ import org.apache.nifi.processor.ProcessSessionFactory;
|
|||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.QueueSize;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceReporter;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.junit.Assert;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class StandardProcessorTestRunner implements TestRunner {
|
||||
|
||||
|
@ -83,7 +82,6 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
private int numThreads = 1;
|
||||
private final AtomicInteger invocations = new AtomicInteger(0);
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(StandardProcessorTestRunner.class);
|
||||
private static final Set<Class<? extends Annotation>> deprecatedTypeAnnotations = new HashSet<>();
|
||||
private static final Set<Class<? extends Annotation>> deprecatedMethodAnnotations = new HashSet<>();
|
||||
|
||||
|
@ -99,7 +97,7 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
this.idGenerator = new AtomicLong(0L);
|
||||
this.sharedState = new SharedSessionState(processor, idGenerator);
|
||||
this.flowFileQueue = sharedState.getFlowFileQueue();
|
||||
this.sessionFactory = new MockSessionFactory(sharedState);
|
||||
this.sessionFactory = new MockSessionFactory(sharedState, processor);
|
||||
this.context = new MockProcessContext(processor);
|
||||
|
||||
detectDeprecatedAnnotations(processor);
|
||||
|
@ -109,7 +107,7 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
|
||||
try {
|
||||
ReflectionUtils.invokeMethodsWithAnnotation(OnAdded.class, processor);
|
||||
} catch (Exception e) {
|
||||
} catch (final Exception e) {
|
||||
Assert.fail("Could not invoke methods annotated with @OnAdded annotation due to: " + e);
|
||||
}
|
||||
|
||||
|
@ -194,7 +192,7 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
if (initialize) {
|
||||
try {
|
||||
ReflectionUtils.invokeMethodsWithAnnotation(OnScheduled.class, processor, context);
|
||||
} catch (Exception e) {
|
||||
} catch (final Exception e) {
|
||||
e.printStackTrace();
|
||||
Assert.fail("Could not invoke methods annotated with @OnScheduled annotation due to: " + e);
|
||||
}
|
||||
|
@ -223,7 +221,7 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
unscheduledRun = true;
|
||||
try {
|
||||
ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, processor, context);
|
||||
} catch (Exception e) {
|
||||
} catch (final Exception e) {
|
||||
Assert.fail("Could not invoke methods annotated with @OnUnscheduled annotation due to: " + e);
|
||||
}
|
||||
}
|
||||
|
@ -234,7 +232,7 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
if (!unscheduledRun) {
|
||||
try {
|
||||
ReflectionUtils.invokeMethodsWithAnnotation(OnUnscheduled.class, processor, context);
|
||||
} catch (Exception e) {
|
||||
} catch (final Exception e) {
|
||||
Assert.fail("Could not invoke methods annotated with @OnUnscheduled annotation due to: " + e);
|
||||
}
|
||||
}
|
||||
|
@ -242,7 +240,7 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
if (stopOnFinish) {
|
||||
try {
|
||||
ReflectionUtils.invokeMethodsWithAnnotation(OnStopped.class, processor);
|
||||
} catch (Exception e) {
|
||||
} catch (final Exception e) {
|
||||
Assert.fail("Could not invoke methods annotated with @OnStopped annotation due to: " + e);
|
||||
}
|
||||
}
|
||||
|
@ -255,7 +253,7 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
public void shutdown() {
|
||||
try {
|
||||
ReflectionUtils.invokeMethodsWithAnnotation(OnShutdown.class, processor);
|
||||
} catch (Exception e) {
|
||||
} catch (final Exception e) {
|
||||
Assert.fail("Could not invoke methods annotated with @OnShutdown annotation due to: " + e);
|
||||
}
|
||||
}
|
||||
|
@ -388,7 +386,7 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
|
||||
@Override
|
||||
public void enqueue(final InputStream data, final Map<String, String> attributes) {
|
||||
final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator));
|
||||
final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), processor);
|
||||
MockFlowFile flowFile = session.create();
|
||||
flowFile = session.importFrom(data, flowFile);
|
||||
flowFile = session.putAllAttributes(flowFile, attributes);
|
||||
|
@ -423,7 +421,11 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
return flowFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated The ProvenanceReporter should not be accessed through the test runner, as it does not expose the events that were emitted.
|
||||
*/
|
||||
@Override
|
||||
@Deprecated
|
||||
public ProvenanceReporter getProvenanceReporter() {
|
||||
return sharedState.getProvenanceReporter();
|
||||
}
|
||||
|
@ -703,4 +705,14 @@ public class StandardProcessorTestRunner implements TestRunner {
|
|||
return context.removeProperty(descriptor);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ProvenanceEventRecord> getProvenanceEvents() {
|
||||
return sharedState.getProvenanceEvents();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearProvenanceEvents() {
|
||||
sharedState.clearProvenanceEvents();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.nifi.processor.ProcessSessionFactory;
|
|||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.QueueSize;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceReporter;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
|
||||
|
@ -702,4 +703,18 @@ public interface TestRunner {
|
|||
* @return true if removed
|
||||
*/
|
||||
boolean removeProperty(PropertyDescriptor descriptor);
|
||||
|
||||
/**
|
||||
* Returns a {@link List} of all {@link ProvenanceEventRecord}s that were
|
||||
* emitted by the Processor
|
||||
*
|
||||
* @return a List of all Provenance Events that were emitted by the
|
||||
* Processor
|
||||
*/
|
||||
List<ProvenanceEventRecord> getProvenanceEvents();
|
||||
|
||||
/**
|
||||
* Clears the Provenance Events that have been emitted by the Processor
|
||||
*/
|
||||
void clearProvenanceEvents();
|
||||
}
|
||||
|
|
|
@ -363,7 +363,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
|
|||
return true;
|
||||
}
|
||||
|
||||
if (maxBytes > 0 && (queueSize.getByteCount() >= maxBytes)) {
|
||||
if (maxBytes > 0 && queueSize.getByteCount() >= maxBytes) {
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -437,7 +437,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
|
|||
final List<FlowFileRecord> swapRecords = new ArrayList<>(Math.min(SWAP_RECORD_POLL_SIZE, swapQueue.size()));
|
||||
final Iterator<FlowFileRecord> itr = swapQueue.iterator();
|
||||
while (itr.hasNext() && swapRecords.size() < SWAP_RECORD_POLL_SIZE) {
|
||||
FlowFileRecord record = itr.next();
|
||||
final FlowFileRecord record = itr.next();
|
||||
swapRecords.add(record);
|
||||
itr.remove();
|
||||
}
|
||||
|
@ -606,7 +606,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
|
|||
boolean isExpired;
|
||||
|
||||
migrateSwapToActive();
|
||||
boolean queueFullAtStart = queueFullRef.get();
|
||||
final boolean queueFullAtStart = queueFullRef.get();
|
||||
|
||||
do {
|
||||
flowFile = this.activeQueue.poll();
|
||||
|
@ -794,9 +794,6 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
|
|||
writeLock.lock();
|
||||
try {
|
||||
migrateSwapToActive();
|
||||
if (activeQueue.isEmpty()) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
|
||||
final long expirationMillis = this.flowFileExpirationMillis.get();
|
||||
final boolean queueFullAtStart = queueFullRef.get();
|
||||
|
@ -804,6 +801,13 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
|
|||
final List<FlowFileRecord> selectedFlowFiles = new ArrayList<>();
|
||||
final List<FlowFileRecord> unselected = new ArrayList<>();
|
||||
|
||||
// the prefetch doesn't allow us to add records back. So when this method is used,
|
||||
// if there are prefetched records, we have to requeue them into the active queue first.
|
||||
final PreFetch prefetch = preFetchRef.get();
|
||||
if (prefetch != null) {
|
||||
requeueExpiredPrefetch(prefetch);
|
||||
}
|
||||
|
||||
while (true) {
|
||||
FlowFileRecord flowFile = this.activeQueue.poll();
|
||||
if (flowFile == null) {
|
||||
|
@ -970,7 +974,7 @@ public final class StandardFlowFileQueue implements FlowFileQueue {
|
|||
boolean updated = false;
|
||||
|
||||
do {
|
||||
QueueSize queueSize = unacknowledgedSizeRef.get();
|
||||
final QueueSize queueSize = unacknowledgedSizeRef.get();
|
||||
final QueueSize newSize = new QueueSize(queueSize.getObjectCount() + addToCount, queueSize.getByteCount() + addToSize);
|
||||
updated = unacknowledgedSizeRef.compareAndSet(queueSize, newSize);
|
||||
} while (!updated);
|
||||
|
|
|
@ -178,7 +178,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
|
|||
throw new AssertionError("Connectable type is " + connectable.getConnectableType());
|
||||
}
|
||||
|
||||
this.provenanceReporter = new StandardProvenanceReporter(connectable.getIdentifier(), componentType, context.getProvenanceRepository(), this);
|
||||
this.provenanceReporter = new StandardProvenanceReporter(this, connectable.getIdentifier(), componentType,
|
||||
context.getProvenanceRepository(), this);
|
||||
this.sessionId = idGenerator.getAndIncrement();
|
||||
this.connectableDescription = description;
|
||||
|
||||
|
@ -324,7 +325,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
|
|||
}
|
||||
final long flowFileLife = System.currentTimeMillis() - flowFile.getEntryDate();
|
||||
final Connectable connectable = context.getConnectable();
|
||||
final Object terminator = (connectable instanceof ProcessorNode) ? ((ProcessorNode) connectable).getProcessor() : connectable;
|
||||
final Object terminator = connectable instanceof ProcessorNode ? ((ProcessorNode) connectable).getProcessor() : connectable;
|
||||
LOG.info("{} terminated by {}; life of FlowFile = {} ms", new Object[]{flowFile, terminator, flowFileLife});
|
||||
} else if (record.isWorking() && record.getWorkingClaim() != record.getOriginalClaim()) {
|
||||
//records which have been updated - remove original if exists
|
||||
|
@ -651,7 +652,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
|
|||
return new Iterator<ProvenanceEventRecord>() {
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return recordsToSubmitIterator.hasNext() || (autoTermIterator != null && autoTermIterator.hasNext());
|
||||
return recordsToSubmitIterator.hasNext() || autoTermIterator != null && autoTermIterator.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1056,8 +1057,8 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
|
|||
}
|
||||
|
||||
private void formatNanos(final long nanos, final StringBuilder sb) {
|
||||
final long seconds = (nanos > 1000000000L) ? (nanos / 1000000000L) : 0L;
|
||||
long millis = (nanos > 1000000L) ? (nanos / 1000000L) : 0L;;
|
||||
final long seconds = nanos > 1000000000L ? nanos / 1000000000L : 0L;
|
||||
long millis = nanos > 1000000L ? nanos / 1000000L : 0L;;
|
||||
final long nanosLeft = nanos % 1000000L;
|
||||
|
||||
if (seconds > 0) {
|
||||
|
@ -1609,7 +1610,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
|
|||
processorType = connectable.getClass().getSimpleName();
|
||||
}
|
||||
|
||||
final StandardProvenanceReporter expiredReporter = new StandardProvenanceReporter(connectable.getIdentifier(),
|
||||
final StandardProvenanceReporter expiredReporter = new StandardProvenanceReporter(this, connectable.getIdentifier(),
|
||||
processorType, context.getProvenanceRepository(), this);
|
||||
|
||||
final Map<String, FlowFileRecord> recordIdMap = new HashMap<>();
|
||||
|
@ -1623,7 +1624,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
|
|||
removeContent(flowFile.getContentClaim());
|
||||
|
||||
final long flowFileLife = System.currentTimeMillis() - flowFile.getEntryDate();
|
||||
final Object terminator = (connectable instanceof ProcessorNode) ? ((ProcessorNode) connectable).getProcessor() : connectable;
|
||||
final Object terminator = connectable instanceof ProcessorNode ? ((ProcessorNode) connectable).getProcessor() : connectable;
|
||||
LOG.info("{} terminated by {} due to FlowFile expiration; life of FlowFile = {} ms", new Object[]{flowFile, terminator, flowFileLife});
|
||||
}
|
||||
|
||||
|
@ -1828,7 +1829,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
|
|||
readCount += copied;
|
||||
|
||||
// don't add demarcator after the last claim
|
||||
if (useDemarcator && (++objectIndex < numSources)) {
|
||||
if (useDemarcator && ++objectIndex < numSources) {
|
||||
out.write(demarcator);
|
||||
writtenCount += demarcator.length;
|
||||
}
|
||||
|
@ -2488,6 +2489,16 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if a FlowFile is known in this session.
|
||||
*
|
||||
* @param flowFile the FlowFile to check
|
||||
* @return <code>true</code> if the FlowFile is known in this session, <code>false</code> otherwise.
|
||||
*/
|
||||
boolean isFlowFileKnown(final FlowFile flowFile) {
|
||||
return records.containsKey(flowFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FlowFile create(final FlowFile parent) {
|
||||
final Map<String, String> newAttributes = new HashMap<>(3);
|
||||
|
|
|
@ -23,12 +23,12 @@ import java.util.Set;
|
|||
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.FlowFileHandlingException;
|
||||
import org.apache.nifi.provenance.ProvenanceEventBuilder;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRepository;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.provenance.ProvenanceReporter;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -41,8 +41,11 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
private final Set<ProvenanceEventRecord> events = new LinkedHashSet<>();
|
||||
private final ProvenanceEventRepository repository;
|
||||
private final ProvenanceEventEnricher eventEnricher;
|
||||
private final StandardProcessSession session;
|
||||
|
||||
public StandardProvenanceReporter(final String processorId, final String processorType, final ProvenanceEventRepository repository, final ProvenanceEventEnricher enricher) {
|
||||
public StandardProvenanceReporter(final StandardProcessSession session, final String processorId, final String processorType,
|
||||
final ProvenanceEventRepository repository, final ProvenanceEventEnricher enricher) {
|
||||
this.session = session;
|
||||
this.processorId = processorId;
|
||||
this.processorType = processorType;
|
||||
this.repository = repository;
|
||||
|
@ -89,6 +92,12 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
return build(flowFile, ProvenanceEventType.DROP).setDetails(details).build();
|
||||
}
|
||||
|
||||
private void verifyFlowFileKnown(final FlowFile flowFile) {
|
||||
if (session != null && !session.isFlowFileKnown(flowFile)) {
|
||||
throw new FlowFileHandlingException(flowFile + " is not known to " + session);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receive(final FlowFile flowFile, final String transitUri) {
|
||||
receive(flowFile, transitUri, -1L);
|
||||
|
@ -111,6 +120,8 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
|
||||
@Override
|
||||
public void receive(final FlowFile flowFile, final String transitUri, final String sourceSystemFlowFileIdentifier, final String details, final long transmissionMillis) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.RECEIVE)
|
||||
.setTransitUri(transitUri).setSourceSystemFlowFileIdentifier(sourceSystemFlowFileIdentifier).setEventDuration(transmissionMillis).setDetails(details).build();
|
||||
|
@ -157,8 +168,7 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
public void send(final FlowFile flowFile, final String transitUri, final String details, final long transmissionMillis, final boolean force) {
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.SEND).setTransitUri(transitUri).setEventDuration(transmissionMillis).setDetails(details).build();
|
||||
|
||||
final ProvenanceEventRecord enriched = eventEnricher.enrich(record, flowFile);
|
||||
final ProvenanceEventRecord enriched = eventEnricher == null ? record : eventEnricher.enrich(record, flowFile);
|
||||
|
||||
if (force) {
|
||||
repository.registerEvent(enriched);
|
||||
|
@ -252,6 +262,8 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
|
||||
@Override
|
||||
public void fork(final FlowFile parent, final Collection<FlowFile> children, final String details, final long forkDuration) {
|
||||
verifyFlowFileKnown(parent);
|
||||
|
||||
try {
|
||||
final ProvenanceEventBuilder eventBuilder = build(parent, ProvenanceEventType.FORK);
|
||||
eventBuilder.addParentFlowFile(parent);
|
||||
|
@ -293,6 +305,8 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
|
||||
@Override
|
||||
public void join(final Collection<FlowFile> parents, final FlowFile child, final String details, final long joinDuration) {
|
||||
verifyFlowFileKnown(child);
|
||||
|
||||
try {
|
||||
final ProvenanceEventBuilder eventBuilder = build(child, ProvenanceEventType.JOIN);
|
||||
eventBuilder.addChildFlowFile(child);
|
||||
|
@ -313,6 +327,8 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
|
||||
@Override
|
||||
public void clone(final FlowFile parent, final FlowFile child) {
|
||||
verifyFlowFileKnown(child);
|
||||
|
||||
try {
|
||||
final ProvenanceEventBuilder eventBuilder = build(parent, ProvenanceEventType.CLONE);
|
||||
eventBuilder.addChildFlowFile(child);
|
||||
|
@ -343,6 +359,8 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
|
||||
@Override
|
||||
public void modifyContent(final FlowFile flowFile, final String details, final long processingMillis) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.CONTENT_MODIFIED).setEventDuration(processingMillis).setDetails(details).build();
|
||||
events.add(record);
|
||||
|
@ -361,6 +379,8 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
|
||||
@Override
|
||||
public void modifyAttributes(final FlowFile flowFile, final String details) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.ATTRIBUTES_MODIFIED).setDetails(details).build();
|
||||
events.add(record);
|
||||
|
@ -389,6 +409,8 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
|
||||
@Override
|
||||
public void route(final FlowFile flowFile, final Relationship relationship, final String details, final long processingDuration) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.ROUTE).setRelationship(relationship).setDetails(details).setEventDuration(processingDuration).build();
|
||||
events.add(record);
|
||||
|
@ -407,6 +429,8 @@ public class StandardProvenanceReporter implements ProvenanceReporter {
|
|||
|
||||
@Override
|
||||
public void create(final FlowFile flowFile, final String details) {
|
||||
verifyFlowFileKnown(flowFile);
|
||||
|
||||
try {
|
||||
final ProvenanceEventRecord record = build(flowFile, ProvenanceEventType.CREATE).setDetails(details).build();
|
||||
events.add(record);
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.io.InputStream;
|
|||
import java.io.OutputStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.StandardCopyOption;
|
||||
import java.nio.file.StandardOpenOption;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
|
@ -44,11 +43,10 @@ import org.apache.nifi.controller.repository.claim.StandardContentClaim;
|
|||
import org.apache.nifi.controller.repository.io.ArrayManagedOutputStream;
|
||||
import org.apache.nifi.controller.repository.io.MemoryManager;
|
||||
import org.apache.nifi.engine.FlowEngine;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.stream.io.ByteArrayInputStream;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.util.NiFiProperties;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -372,20 +370,16 @@ public class VolatileContentRepository implements ContentRepository {
|
|||
return 0L;
|
||||
}
|
||||
|
||||
if (append) {
|
||||
try (final InputStream in = read(claim);
|
||||
final OutputStream destinationStream = Files.newOutputStream(destination, StandardOpenOption.APPEND)) {
|
||||
final StandardOpenOption openOption = append ? StandardOpenOption.APPEND : StandardOpenOption.CREATE;
|
||||
try (final InputStream in = read(claim);
|
||||
final OutputStream destinationStream = Files.newOutputStream(destination, openOption)) {
|
||||
|
||||
if (offset > 0) {
|
||||
StreamUtils.skip(in, offset);
|
||||
}
|
||||
|
||||
StreamUtils.copy(in, destinationStream, length);
|
||||
return length;
|
||||
if (offset > 0) {
|
||||
StreamUtils.skip(in, offset);
|
||||
}
|
||||
} else {
|
||||
Files.copy(read(claim), destination, StandardCopyOption.REPLACE_EXISTING);
|
||||
return Files.size(destination);
|
||||
|
||||
StreamUtils.copy(in, destinationStream, length);
|
||||
return length;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -419,7 +413,7 @@ public class VolatileContentRepository implements ContentRepository {
|
|||
}
|
||||
|
||||
final ContentClaim backupClaim = getBackupClaim(claim);
|
||||
return (backupClaim == null) ? getContent(claim).getSize() : getBackupRepository().size(claim);
|
||||
return backupClaim == null ? getContent(claim).getSize() : getBackupRepository().size(claim);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -429,13 +423,13 @@ public class VolatileContentRepository implements ContentRepository {
|
|||
}
|
||||
|
||||
final ContentClaim backupClaim = getBackupClaim(claim);
|
||||
return (backupClaim == null) ? getContent(claim).read() : getBackupRepository().read(backupClaim);
|
||||
return backupClaim == null ? getContent(claim).read() : getBackupRepository().read(backupClaim);
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream write(final ContentClaim claim) throws IOException {
|
||||
final ContentClaim backupClaim = getBackupClaim(claim);
|
||||
return (backupClaim == null) ? getContent(claim).write() : getBackupRepository().write(backupClaim);
|
||||
return backupClaim == null ? getContent(claim).write() : getBackupRepository().write(backupClaim);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -481,8 +475,13 @@ public class VolatileContentRepository implements ContentRepository {
|
|||
@Override
|
||||
public void write(int b) throws IOException {
|
||||
try {
|
||||
final long bufferLengthBefore = getBufferLength();
|
||||
super.write(b);
|
||||
repoSizeCounter.incrementAndGet();
|
||||
final long bufferLengthAfter = getBufferLength();
|
||||
final long bufferSpaceAdded = bufferLengthAfter - bufferLengthBefore;
|
||||
if (bufferSpaceAdded > 0) {
|
||||
repoSizeCounter.addAndGet(bufferSpaceAdded);
|
||||
}
|
||||
} catch (final IOException e) {
|
||||
final byte[] buff = new byte[1];
|
||||
buff[0] = (byte) (b & 0xFF);
|
||||
|
@ -498,8 +497,13 @@ public class VolatileContentRepository implements ContentRepository {
|
|||
@Override
|
||||
public void write(byte[] b, int off, int len) throws IOException {
|
||||
try {
|
||||
final long bufferLengthBefore = getBufferLength();
|
||||
super.write(b, off, len);
|
||||
repoSizeCounter.addAndGet(len);
|
||||
final long bufferLengthAfter = getBufferLength();
|
||||
final long bufferSpaceAdded = bufferLengthAfter - bufferLengthBefore;
|
||||
if (bufferSpaceAdded > 0) {
|
||||
repoSizeCounter.addAndGet(bufferSpaceAdded);
|
||||
}
|
||||
} catch (final IOException e) {
|
||||
redirect(b, off, len);
|
||||
}
|
||||
|
|
|
@ -1685,7 +1685,11 @@ public final class StandardProcessGroup implements ProcessGroup {
|
|||
}
|
||||
|
||||
if (isRootGroup() && (!snippet.getInputPorts().isEmpty() || !snippet.getOutputPorts().isEmpty())) {
|
||||
throw new IllegalStateException("Cannot move Ports from the Root Group to a Non-Root Group");
|
||||
throw new IllegalStateException("Cannot move Ports out of the root group");
|
||||
}
|
||||
|
||||
if (destination.isRootGroup() && (!snippet.getInputPorts().isEmpty() || !snippet.getOutputPorts().isEmpty())) {
|
||||
throw new IllegalStateException("Cannot move Ports into the root group");
|
||||
}
|
||||
|
||||
for (final String id : replaceNullWithEmptySet(snippet.getInputPorts())) {
|
||||
|
|
|
@ -16,8 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.controller.repository;
|
||||
|
||||
import org.apache.nifi.controller.repository.StandardProvenanceReporter;
|
||||
import org.apache.nifi.controller.repository.StandardFlowFileRecord;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -27,7 +25,6 @@ import java.util.Set;
|
|||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRepository;
|
||||
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
@ -38,7 +35,7 @@ public class TestStandardProvenanceReporter {
|
|||
@Ignore
|
||||
public void testDuplicatesIgnored() {
|
||||
final ProvenanceEventRepository mockRepo = Mockito.mock(ProvenanceEventRepository.class);
|
||||
final StandardProvenanceReporter reporter = new StandardProvenanceReporter("1234", "TestProc", mockRepo, null);
|
||||
final StandardProvenanceReporter reporter = new StandardProvenanceReporter(null, "1234", "TestProc", mockRepo, null);
|
||||
|
||||
final List<FlowFile> parents = new ArrayList<>();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
|
@ -51,7 +48,7 @@ public class TestStandardProvenanceReporter {
|
|||
reporter.fork(flowFile, parents);
|
||||
reporter.fork(flowFile, parents);
|
||||
|
||||
Set<ProvenanceEventRecord> records = reporter.getEvents();
|
||||
final Set<ProvenanceEventRecord> records = reporter.getEvents();
|
||||
assertEquals(11, records.size()); // 1 for each parent in the spawn and 1 for the spawn itself
|
||||
|
||||
final FlowFile firstParent = parents.get(0);
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.io.File;
|
|||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.file.FileVisitResult;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
@ -173,9 +174,14 @@ public class NarUnpackerTest {
|
|||
}
|
||||
|
||||
private NiFiProperties loadSpecifiedProperties(String propertiesFile) {
|
||||
String file = NarUnpackerTest.class.getResource(propertiesFile).getFile();
|
||||
|
||||
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, file);
|
||||
String filePath;
|
||||
try {
|
||||
filePath = NarUnpackerTest.class.getResource(propertiesFile).toURI().getPath();
|
||||
} catch (URISyntaxException ex) {
|
||||
throw new RuntimeException("Cannot load properties file due to "
|
||||
+ ex.getLocalizedMessage(), ex);
|
||||
}
|
||||
System.setProperty(NiFiProperties.PROPERTIES_FILE_PATH, filePath);
|
||||
|
||||
NiFiProperties properties = NiFiProperties.getInstance();
|
||||
|
||||
|
@ -186,7 +192,7 @@ public class NarUnpackerTest {
|
|||
|
||||
InputStream inStream = null;
|
||||
try {
|
||||
inStream = new BufferedInputStream(new FileInputStream(file));
|
||||
inStream = new BufferedInputStream(new FileInputStream(filePath));
|
||||
properties.load(inStream);
|
||||
} catch (final Exception ex) {
|
||||
throw new RuntimeException("Cannot load properties file due to "
|
||||
|
|
|
@ -39,9 +39,9 @@ set CONF_DIR=conf
|
|||
set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
|
||||
set JAVA_ARGS=-Dorg.apache.nifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
|
||||
|
||||
set JAVA_PARAMS=-cp %LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.bootstrap.RunNiFi
|
||||
SET JAVA_PARAMS=-cp %CONF_DIR%;%LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.bootstrap.RunNiFi
|
||||
set BOOTSTRAP_ACTION=dump
|
||||
|
||||
cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
|
||||
|
||||
popd
|
||||
popd
|
||||
|
|
|
@ -159,7 +159,19 @@ run() {
|
|||
echo "Bootstrap Config File: $BOOTSTRAP_CONF"
|
||||
echo
|
||||
|
||||
exec "$JAVA" -cp "$NIFI_HOME"/lib/bootstrap/* -Xms12m -Xmx24m -Dorg.apache.nifi.bootstrap.config.file="$BOOTSTRAP_CONF" org.apache.nifi.bootstrap.RunNiFi $@
|
||||
# run 'start' in the background because the process will continue to run, monitoring NiFi.
|
||||
# all other commands will terminate quickly so want to just wait for them
|
||||
if [ "$1" = "start" ]; then
|
||||
("$JAVA" -cp "$NIFI_HOME"/conf/:"$NIFI_HOME"/lib/bootstrap/* -Xms12m -Xmx24m -Dorg.apache.nifi.bootstrap.config.file="$BOOTSTRAP_CONF" org.apache.nifi.bootstrap.RunNiFi $@ &)
|
||||
else
|
||||
"$JAVA" -cp "$NIFI_HOME"/conf/:"$NIFI_HOME"/lib/bootstrap/* -Xms12m -Xmx24m -Dorg.apache.nifi.bootstrap.config.file="$BOOTSTRAP_CONF" org.apache.nifi.bootstrap.RunNiFi $@
|
||||
fi
|
||||
|
||||
# Wait just a bit (3 secs) to wait for the logging to finish and then echo a new-line.
|
||||
# We do this to avoid having logs spewed on the console after running the command and then not giving
|
||||
# control back to the user
|
||||
sleep 3
|
||||
echo
|
||||
}
|
||||
|
||||
main() {
|
||||
|
@ -172,9 +184,14 @@ case "$1" in
|
|||
install)
|
||||
install "$@"
|
||||
;;
|
||||
start|stop|run|restart|status|dump)
|
||||
start|stop|run|status|dump)
|
||||
main "$@"
|
||||
;;
|
||||
restart)
|
||||
init
|
||||
run "stop"
|
||||
run "start"
|
||||
;;
|
||||
*)
|
||||
echo "Usage nifi {start|stop|run|restart|status|dump|install}"
|
||||
;;
|
||||
|
|
|
@ -39,9 +39,9 @@ set CONF_DIR=conf
|
|||
set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
|
||||
set JAVA_ARGS=-Dorg.apache.nifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
|
||||
|
||||
set JAVA_PARAMS=-cp %LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.bootstrap.RunNiFi
|
||||
SET JAVA_PARAMS=-cp %CONF_DIR%;%LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.bootstrap.RunNiFi
|
||||
set BOOTSTRAP_ACTION=run
|
||||
|
||||
cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
|
||||
|
||||
popd
|
||||
popd
|
||||
|
|
|
@ -1,47 +0,0 @@
|
|||
@echo off
|
||||
rem
|
||||
rem Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
rem contributor license agreements. See the NOTICE file distributed with
|
||||
rem this work for additional information regarding copyright ownership.
|
||||
rem The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
rem (the "License"); you may not use this file except in compliance with
|
||||
rem the License. You may obtain a copy of the License at
|
||||
rem
|
||||
rem http://www.apache.org/licenses/LICENSE-2.0
|
||||
rem
|
||||
rem Unless required by applicable law or agreed to in writing, software
|
||||
rem distributed under the License is distributed on an "AS IS" BASIS,
|
||||
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
rem See the License for the specific language governing permissions and
|
||||
rem limitations under the License.
|
||||
rem
|
||||
|
||||
rem Use JAVA_HOME if it's set; otherwise, just use java
|
||||
|
||||
if "%JAVA_HOME%" == "" goto noJavaHome
|
||||
if not exist "%JAVA_HOME%\bin\java.exe" goto noJavaHome
|
||||
set JAVA_EXE=%JAVA_HOME%\bin\java.exe
|
||||
goto startNifi
|
||||
|
||||
:noJavaHome
|
||||
echo The JAVA_HOME environment variable is not defined correctly.
|
||||
echo Instead the PATH will be used to find the java executable.
|
||||
echo.
|
||||
set JAVA_EXE=java
|
||||
goto startNifi
|
||||
|
||||
:startNifi
|
||||
set NIFI_ROOT=%~dp0..\
|
||||
pushd "%NIFI_ROOT%"
|
||||
set LIB_DIR=lib\bootstrap
|
||||
set CONF_DIR=conf
|
||||
|
||||
set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
|
||||
set JAVA_ARGS=-Dorg.apache.nifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
|
||||
|
||||
set JAVA_PARAMS=-cp %LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.bootstrap.RunNiFi
|
||||
set BOOTSTRAP_ACTION=start
|
||||
|
||||
cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
|
||||
|
||||
popd
|
|
@ -1,47 +0,0 @@
|
|||
@echo off
|
||||
rem
|
||||
rem Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
rem contributor license agreements. See the NOTICE file distributed with
|
||||
rem this work for additional information regarding copyright ownership.
|
||||
rem The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
rem (the "License"); you may not use this file except in compliance with
|
||||
rem the License. You may obtain a copy of the License at
|
||||
rem
|
||||
rem http://www.apache.org/licenses/LICENSE-2.0
|
||||
rem
|
||||
rem Unless required by applicable law or agreed to in writing, software
|
||||
rem distributed under the License is distributed on an "AS IS" BASIS,
|
||||
rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
rem See the License for the specific language governing permissions and
|
||||
rem limitations under the License.
|
||||
rem
|
||||
|
||||
rem Use JAVA_HOME if it's set; otherwise, just use java
|
||||
|
||||
if "%JAVA_HOME%" == "" goto noJavaHome
|
||||
if not exist "%JAVA_HOME%\bin\java.exe" goto noJavaHome
|
||||
set JAVA_EXE=%JAVA_HOME%\bin\java.exe
|
||||
goto startNifi
|
||||
|
||||
:noJavaHome
|
||||
echo The JAVA_HOME environment variable is not defined correctly.
|
||||
echo Instead the PATH will be used to find the java executable.
|
||||
echo.
|
||||
set JAVA_EXE=java
|
||||
goto startNifi
|
||||
|
||||
:startNifi
|
||||
set NIFI_ROOT=%~dp0..\
|
||||
pushd "%NIFI_ROOT%"
|
||||
set LIB_DIR=lib\bootstrap
|
||||
set CONF_DIR=conf
|
||||
|
||||
set BOOTSTRAP_CONF_FILE=%CONF_DIR%\bootstrap.conf
|
||||
set JAVA_ARGS=-Dorg.apache.nifi.bootstrap.config.file=%BOOTSTRAP_CONF_FILE%
|
||||
|
||||
set JAVA_PARAMS=-cp %LIB_DIR%\* -Xms12m -Xmx24m %JAVA_ARGS% org.apache.nifi.bootstrap.RunNiFi
|
||||
set BOOTSTRAP_ACTION=stop
|
||||
|
||||
cmd.exe /C "%JAVA_EXE%" %JAVA_PARAMS% %BOOTSTRAP_ACTION%
|
||||
|
||||
popd
|
|
@ -58,10 +58,35 @@
|
|||
<pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
|
||||
</encoder>
|
||||
</appender>
|
||||
|
||||
<appender name="BOOTSTRAP_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
|
||||
<file>logs/nifi-bootstrap.log</file>
|
||||
<rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
|
||||
<!--
|
||||
For daily rollover, use 'user_%d.log'.
|
||||
For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
|
||||
To GZIP rolled files, replace '.log' with '.log.gz'.
|
||||
To ZIP rolled files, replace '.log' with '.log.zip'.
|
||||
-->
|
||||
<fileNamePattern>./logs/nifi-bootstrap_%d.log</fileNamePattern>
|
||||
<!-- keep 5 log files worth of history -->
|
||||
<maxHistory>5</maxHistory>
|
||||
</rollingPolicy>
|
||||
<encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
|
||||
<pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
|
||||
</encoder>
|
||||
</appender>
|
||||
|
||||
<appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender">
|
||||
<encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
|
||||
<pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
|
||||
</encoder>
|
||||
</appender>
|
||||
|
||||
<!-- valid logging levels: TRACE, DEBUG, INFO, WARN, ERROR -->
|
||||
|
||||
<logger name="org.apache.nifi" level="INFO"/>
|
||||
<logger name="org.apache.nifi.controller.repository.StandardProcessSession" level="WARN" />
|
||||
|
||||
<!-- Logger for managing logging statements for nifi clusters. -->
|
||||
<logger name="org.apache.nifi.cluster" level="INFO"/>
|
||||
|
@ -101,6 +126,29 @@
|
|||
<appender-ref ref="USER_FILE"/>
|
||||
</logger>
|
||||
|
||||
|
||||
<!--
|
||||
Logger for capturing Bootstrap logs and NiFi's standard error and standard out.
|
||||
-->
|
||||
<logger name="org.apache.nifi.bootstrap" level="INFO" additivity="false">
|
||||
<appender-ref ref="BOOTSTRAP_FILE" />
|
||||
</logger>
|
||||
<logger name="org.apache.nifi.bootstrap.Command" level="INFO" additivity="false">
|
||||
<appender-ref ref="CONSOLE" />
|
||||
<appender-ref ref="BOOTSTRAP_FILE" />
|
||||
</logger>
|
||||
|
||||
<!-- Everything written to NiFi's Standard Out will be logged with the logger org.apache.nifi.StdOut at INFO level -->
|
||||
<logger name="org.apache.nifi.StdOut" level="INFO" additivity="false">
|
||||
<appender-ref ref="BOOTSTRAP_FILE" />
|
||||
</logger>
|
||||
|
||||
<!-- Everything written to NiFi's Standard Error will be logged with the logger org.apache.nifi.StdErr at ERROR level -->
|
||||
<logger name="org.apache.nifi.StdErr" level="ERROR" additivity="false">
|
||||
<appender-ref ref="BOOTSTRAP_FILE" />
|
||||
</logger>
|
||||
|
||||
|
||||
<root level="INFO">
|
||||
<appender-ref ref="APP_FILE"/>
|
||||
</root>
|
||||
|
|
|
@ -252,7 +252,7 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
|
|||
|
||||
@Override
|
||||
public boolean isValid() {
|
||||
return (getConnectableType() == ConnectableType.INPUT_PORT) ? !getConnections(Relationship.ANONYMOUS).isEmpty() : true;
|
||||
return getConnectableType() == ConnectableType.INPUT_PORT ? !getConnections(Relationship.ANONYMOUS).isEmpty() : true;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -275,14 +275,10 @@ public class StandardRootGroupPort extends AbstractPort implements RootGroupPort
|
|||
return false;
|
||||
}
|
||||
|
||||
if (processScheduler.getActiveThreadCount(this) > 0) {
|
||||
if (!requestQueue.isEmpty()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (requestQueue.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
requestLock.lock();
|
||||
try {
|
||||
return !activeRequests.isEmpty();
|
||||
|
|
|
@ -107,7 +107,7 @@ public class StandardNiFiContentAccess implements ContentAccess {
|
|||
|
||||
// get the file name
|
||||
final String contentDisposition = responseHeaders.getFirst("Content-Disposition");
|
||||
final String filename = StringUtils.substringAfterLast(contentDisposition, "filename=");
|
||||
final String filename = StringUtils.substringBetween(contentDisposition, "filename=\"", "\"");
|
||||
|
||||
// get the content type
|
||||
final String contentType = responseHeaders.getFirst("Content-Type");
|
||||
|
|
|
@ -472,7 +472,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
|
||||
// build the snippet dto
|
||||
final SnippetDTO responseSnippetDto = dtoFactory.createSnippetDto(snippet);
|
||||
responseSnippetDto.setContents(snippetUtils.populateFlowSnippet(snippet, false));
|
||||
responseSnippetDto.setContents(snippetUtils.populateFlowSnippet(snippet, false, false));
|
||||
|
||||
// save updated controller if applicable
|
||||
if (snippetDto.getParentGroupId() != null && snippet.isLinked()) {
|
||||
|
@ -995,7 +995,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
// add the snippet
|
||||
final Snippet snippet = snippetDAO.createSnippet(snippetDTO);
|
||||
final SnippetDTO responseSnippetDTO = dtoFactory.createSnippetDto(snippet);
|
||||
responseSnippetDTO.setContents(snippetUtils.populateFlowSnippet(snippet, false));
|
||||
responseSnippetDTO.setContents(snippetUtils.populateFlowSnippet(snippet, false, false));
|
||||
|
||||
return responseSnippetDTO;
|
||||
}
|
||||
|
@ -1092,7 +1092,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
templateDTO.setName(name);
|
||||
templateDTO.setDescription(description);
|
||||
templateDTO.setTimestamp(new Date());
|
||||
templateDTO.setSnippet(snippetUtils.populateFlowSnippet(snippet, true));
|
||||
templateDTO.setSnippet(snippetUtils.populateFlowSnippet(snippet, true, true));
|
||||
|
||||
// set the id based on the specified seed
|
||||
final ClusterContext clusterContext = ClusterContextThreadLocal.getContext();
|
||||
|
@ -1972,7 +1972,7 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
|
|||
public SnippetDTO getSnippet(String snippetId) {
|
||||
final Snippet snippet = snippetDAO.getSnippet(snippetId);
|
||||
final SnippetDTO snippetDTO = dtoFactory.createSnippetDto(snippet);
|
||||
snippetDTO.setContents(snippetUtils.populateFlowSnippet(snippet, false));
|
||||
snippetDTO.setContents(snippetUtils.populateFlowSnippet(snippet, false, false));
|
||||
return snippetDTO;
|
||||
}
|
||||
|
||||
|
|
|
@ -362,7 +362,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
contentType = MediaType.APPLICATION_OCTET_STREAM;
|
||||
}
|
||||
|
||||
return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=%s", content.getFilename())).build();
|
||||
return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=\"%s\"", content.getFilename())).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -461,7 +461,7 @@ public class ProvenanceResource extends ApplicationResource {
|
|||
contentType = MediaType.APPLICATION_OCTET_STREAM;
|
||||
}
|
||||
|
||||
return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=%s", content.getFilename())).build();
|
||||
return generateOkResponse(response).type(contentType).header("Content-Disposition", String.format("attachment; filename=\"%s\"", content.getFilename())).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -428,7 +428,7 @@ public class TemplateResource extends ApplicationResource {
|
|||
}
|
||||
|
||||
// generate the response
|
||||
return generateOkResponse(template).header("Content-Disposition", String.format("attachment; filename=%s.xml", attachmentName)).build();
|
||||
return generateOkResponse(template).header("Content-Disposition", String.format("attachment; filename=\"%s.xml\"", attachmentName)).build();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -797,6 +797,11 @@ public final class DtoFactory {
|
|||
copySnippet.getRemoteProcessGroups().add(copy(remoteGroup));
|
||||
}
|
||||
}
|
||||
if (originalSnippet.getControllerServices() != null) {
|
||||
for (final ControllerServiceDTO controllerService : originalSnippet.getControllerServices()) {
|
||||
copySnippet.getControllerServices().add(copy(controllerService));
|
||||
}
|
||||
}
|
||||
|
||||
return copySnippet;
|
||||
}
|
||||
|
|
|
@ -76,7 +76,7 @@ public class StandardSnippetDAO implements SnippetDAO {
|
|||
}
|
||||
|
||||
// generate the snippet contents
|
||||
FlowSnippetDTO snippetContents = snippetUtils.populateFlowSnippet(existingSnippet, true);
|
||||
FlowSnippetDTO snippetContents = snippetUtils.populateFlowSnippet(existingSnippet, true, false);
|
||||
|
||||
// resolve sensitive properties
|
||||
lookupSensitiveProperties(snippetContents);
|
||||
|
|
|
@ -72,9 +72,10 @@ public final class SnippetUtils {
|
|||
*
|
||||
* @param snippet snippet
|
||||
* @param recurse recurse
|
||||
* @param includeControllerServices whether or not to include controller services in the flow snippet dto
|
||||
* @return snippet
|
||||
*/
|
||||
public FlowSnippetDTO populateFlowSnippet(Snippet snippet, boolean recurse) {
|
||||
public FlowSnippetDTO populateFlowSnippet(Snippet snippet, boolean recurse, boolean includeControllerServices) {
|
||||
final FlowSnippetDTO snippetDto = new FlowSnippetDTO();
|
||||
final String groupId = snippet.getParentGroupId();
|
||||
final ProcessGroup processGroup = flowController.getGroup(groupId);
|
||||
|
@ -188,7 +189,9 @@ public final class SnippetUtils {
|
|||
snippetDto.setRemoteProcessGroups(remoteProcessGroups);
|
||||
}
|
||||
|
||||
addControllerServicesToSnippet(snippetDto);
|
||||
if (includeControllerServices) {
|
||||
addControllerServicesToSnippet(snippetDto);
|
||||
}
|
||||
|
||||
return snippetDto;
|
||||
}
|
||||
|
@ -570,8 +573,12 @@ public final class SnippetUtils {
|
|||
continue;
|
||||
}
|
||||
|
||||
final String newServiceId = serviceIdMap.get(currentServiceId);
|
||||
properties.put(descriptor.getName(), newServiceId);
|
||||
// if this is a copy/paste action, we can continue to reference the same service, in this case
|
||||
// the serviceIdMap will be empty
|
||||
if (serviceIdMap.containsKey(currentServiceId)) {
|
||||
final String newServiceId = serviceIdMap.get(currentServiceId);
|
||||
properties.put(descriptor.getName(), newServiceId);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -64,6 +64,9 @@ public class ContentViewerController extends HttpServlet {
|
|||
*/
|
||||
@Override
|
||||
protected void doGet(final HttpServletRequest request, final HttpServletResponse response) throws ServletException, IOException {
|
||||
// specify the charset in a response header
|
||||
response.addHeader("Content-Type", "text/html; charset=UTF-8");
|
||||
|
||||
// get the content
|
||||
final ServletContext servletContext = request.getServletContext();
|
||||
final ContentAccess contentAccess = (ContentAccess) servletContext.getAttribute("nifi-content-access");
|
||||
|
|
|
@ -70,7 +70,7 @@ public class DownloadSvg extends HttpServlet {
|
|||
}
|
||||
|
||||
response.setContentType("image/svg+xml");
|
||||
response.setHeader("Content-Disposition", "attachment; filename=" + filename);
|
||||
response.setHeader("Content-Disposition", "attachment; filename=\"" + filename + "\"");
|
||||
response.setStatus(HttpServletResponse.SC_OK);
|
||||
|
||||
response.getWriter().print(svg);
|
||||
|
|
|
@ -25,10 +25,6 @@
|
|||
<div class="setting-name">Name</div>
|
||||
<div class="controller-service-editable setting-field">
|
||||
<input type="text" id="controller-service-name" name="controller-service-name" class="setting-input"/>
|
||||
<div class="controller-service-enabled-container">
|
||||
<div id="controller-service-enabled" class="nf-checkbox checkbox-unchecked"></div>
|
||||
<span> Enabled</span>
|
||||
</div>
|
||||
</div>
|
||||
<div class="controller-service-read-only setting-field hidden">
|
||||
<span id="read-only-controller-service-name"></span>
|
||||
|
|
|
@ -79,19 +79,6 @@ div.controller-service-configuration-tab-container {
|
|||
float: left;
|
||||
}
|
||||
|
||||
#controller-service-enabled {
|
||||
width: 12px;
|
||||
height: 12px;
|
||||
float: left;
|
||||
margin-right: 4px;
|
||||
}
|
||||
|
||||
div.controller-service-enabled-container {
|
||||
float: left;
|
||||
margin-top: 5px;
|
||||
margin-left: 10px;
|
||||
}
|
||||
|
||||
div.availability-setting {
|
||||
float: left;
|
||||
width: 140px;
|
||||
|
|
Binary file not shown.
After Width: | Height: | Size: 215 B |
|
@ -914,6 +914,21 @@ nf.Actions = (function () {
|
|||
});
|
||||
},
|
||||
|
||||
/**
|
||||
* Moves the currently selected component into the current parent group.
|
||||
*/
|
||||
moveIntoParent: function () {
|
||||
var selection = nf.CanvasUtils.getSelection();
|
||||
|
||||
// ensure that components have been specified
|
||||
if (selection.empty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
// move the current selection into the parent group
|
||||
nf.CanvasUtils.moveComponentsToParent(selection);
|
||||
},
|
||||
|
||||
/**
|
||||
* Creates a new template based off the currently selected components. If no components
|
||||
* are selected, a template of the entire canvas is made.
|
||||
|
|
|
@ -50,6 +50,58 @@ nf.CanvasUtils = (function () {
|
|||
|
||||
return mid;
|
||||
};
|
||||
|
||||
var moveComponents = function (components, groupId) {
|
||||
return $.Deferred(function (deferred) {
|
||||
// ensure the current selection is eligible for move into the specified group
|
||||
nf.CanvasUtils.eligibleForMove(components, groupId).done(function () {
|
||||
// create a snippet for the specified components and link to the data flow
|
||||
var snippetDetails = nf.Snippet.marshal(components, true);
|
||||
nf.Snippet.create(snippetDetails).done(function (response) {
|
||||
var snippet = response.snippet;
|
||||
|
||||
// move the snippet into the target
|
||||
nf.Snippet.move(snippet.id, groupId).done(function () {
|
||||
var componentMap = d3.map();
|
||||
|
||||
// add the id to the type's array
|
||||
var addComponent = function (type, id) {
|
||||
if (!componentMap.has(type)) {
|
||||
componentMap.set(type, []);
|
||||
}
|
||||
componentMap.get(type).push(id);
|
||||
};
|
||||
|
||||
// go through each component being removed
|
||||
components.each(function (d) {
|
||||
addComponent(d.type, d.component.id);
|
||||
});
|
||||
|
||||
// refresh all component types as necessary (handle components that have been removed)
|
||||
componentMap.forEach(function (type, ids) {
|
||||
nf[type].remove(ids);
|
||||
});
|
||||
|
||||
// refresh the birdseye
|
||||
nf.Birdseye.refresh();
|
||||
deferred.resolve();
|
||||
}).fail(nf.Common.handleAjaxError).fail(function () {
|
||||
deferred.reject();
|
||||
}).always(function () {
|
||||
// unable to acutally move the components so attempt to
|
||||
// unlink and remove just the snippet
|
||||
nf.Snippet.unlink(snippet.id).done(function () {
|
||||
nf.Snippet.remove(snippet.id);
|
||||
});
|
||||
});
|
||||
}).fail(nf.Common.handleAjaxError).fail(function () {
|
||||
deferred.reject();
|
||||
});
|
||||
}).fail(function () {
|
||||
deferred.reject();
|
||||
});
|
||||
}).promise();
|
||||
};
|
||||
|
||||
return {
|
||||
config: {
|
||||
|
@ -1025,6 +1077,22 @@ nf.CanvasUtils = (function () {
|
|||
return origin;
|
||||
},
|
||||
|
||||
/**
|
||||
* Moves the specified components into the current parent group.
|
||||
*
|
||||
* @param {selection} components
|
||||
*/
|
||||
moveComponentsToParent: function (components) {
|
||||
var groupId = nf.Canvas.getParentGroupId();
|
||||
|
||||
// if the group id is null, we're already in the top most group
|
||||
if (groupId === null) {
|
||||
nf.Dialog.showOkDialog('Components are already in the topmost group.');
|
||||
} else {
|
||||
moveComponents(components, groupId);
|
||||
}
|
||||
},
|
||||
|
||||
/**
|
||||
* Moves the specified components into the specified group.
|
||||
*
|
||||
|
@ -1033,46 +1101,11 @@ nf.CanvasUtils = (function () {
|
|||
*/
|
||||
moveComponents: function (components, group) {
|
||||
var groupData = group.datum();
|
||||
|
||||
// ensure the current selection is eligible for move into the specified group
|
||||
nf.CanvasUtils.eligibleForMove(components, group).done(function () {
|
||||
// create a snippet for the specified components and link to the data flow
|
||||
var snippetDetails = nf.Snippet.marshal(components, true);
|
||||
nf.Snippet.create(snippetDetails).done(function (response) {
|
||||
var snippet = response.snippet;
|
||||
|
||||
// move the snippet into the target
|
||||
nf.Snippet.move(snippet.id, groupData.component.id).done(function () {
|
||||
var componentMap = d3.map();
|
||||
|
||||
// add the id to the type's array
|
||||
var addComponent = function (type, id) {
|
||||
if (!componentMap.has(type)) {
|
||||
componentMap.set(type, []);
|
||||
}
|
||||
componentMap.get(type).push(id);
|
||||
};
|
||||
|
||||
// go through each component being removed
|
||||
components.each(function (d) {
|
||||
addComponent(d.type, d.component.id);
|
||||
});
|
||||
|
||||
// refresh all component types as necessary (handle components that have been removed)
|
||||
componentMap.forEach(function (type, ids) {
|
||||
nf[type].remove(ids);
|
||||
});
|
||||
|
||||
// reload the target group
|
||||
nf.ProcessGroup.reload(groupData.component);
|
||||
}).fail(nf.Common.handleAjaxError).always(function () {
|
||||
// unable to acutally move the components so attempt to
|
||||
// unlink and remove just the snippet
|
||||
nf.Snippet.unlink(snippet.id).done(function () {
|
||||
nf.Snippet.remove(snippet.id);
|
||||
});
|
||||
});
|
||||
}).fail(nf.Common.handleAjaxError);
|
||||
|
||||
// move the components into the destination and...
|
||||
moveComponents(components, groupData.component.id).done(function () {
|
||||
// reload the target group
|
||||
nf.ProcessGroup.reload(groupData.component);
|
||||
});
|
||||
},
|
||||
|
||||
|
@ -1161,15 +1194,15 @@ nf.CanvasUtils = (function () {
|
|||
},
|
||||
|
||||
/**
|
||||
* Ensures components are eligible to be moved. The new target can be optionally specified.
|
||||
* Ensures components are eligible to be moved. The new group can be optionally specified.
|
||||
*
|
||||
* 1) Ensuring that the input and output ports are not connected outside of this group
|
||||
* 2) If the target is specified; ensuring there are no port name conflicts in the target group
|
||||
*
|
||||
* @argument {selection} selection The selection being moved
|
||||
* @argument {selection} group The selection containing the new group
|
||||
* @argument {string} groupId The id of the new group
|
||||
*/
|
||||
eligibleForMove: function (selection, group) {
|
||||
eligibleForMove: function (selection, groupId) {
|
||||
var inputPorts = [];
|
||||
var outputPorts = [];
|
||||
|
||||
|
@ -1191,7 +1224,7 @@ nf.CanvasUtils = (function () {
|
|||
// ports in the root group cannot be moved
|
||||
if (nf.Canvas.getParentGroupId() === null) {
|
||||
nf.Dialog.showOkDialog({
|
||||
dialogContent: 'Ports in the root group cannot be moved into another group.',
|
||||
dialogContent: 'Cannot move Ports out of the root group',
|
||||
overlayBackground: false
|
||||
});
|
||||
portConnectionDeferred.reject();
|
||||
|
@ -1245,12 +1278,11 @@ nf.CanvasUtils = (function () {
|
|||
// create a deferred for checking port names in the target
|
||||
var portNameCheck = function () {
|
||||
return $.Deferred(function (portNameDeferred) {
|
||||
var groupData = group.datum();
|
||||
|
||||
// add the get request
|
||||
$.ajax({
|
||||
type: 'GET',
|
||||
url: config.urls.controller + '/process-groups/' + encodeURIComponent(groupData.component.id),
|
||||
url: config.urls.controller + '/process-groups/' + encodeURIComponent(groupId),
|
||||
data: {
|
||||
verbose: true
|
||||
},
|
||||
|
@ -1294,7 +1326,7 @@ nf.CanvasUtils = (function () {
|
|||
|
||||
// execute the checks in order
|
||||
portConnectionCheck().done(function () {
|
||||
if (nf.Common.isDefinedAndNotNull(group)) {
|
||||
if (nf.Common.isDefinedAndNotNull(groupId)) {
|
||||
$.when(portNameCheck()).done(function () {
|
||||
deferred.resolve();
|
||||
}).fail(function () {
|
||||
|
|
|
@ -15,7 +15,7 @@
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
/* global nf */
|
||||
/* global nf, d3 */
|
||||
|
||||
nf.ContextMenu = (function () {
|
||||
|
||||
|
@ -276,6 +276,15 @@ nf.ContextMenu = (function () {
|
|||
var canStopTransmission = function (selection) {
|
||||
return nf.Common.isDFM() && nf.CanvasUtils.canAllStopTransmitting(selection);
|
||||
};
|
||||
|
||||
/**
|
||||
* Determines if the components in the specified selection can be moved into a parent group.
|
||||
*
|
||||
* @param {type} selection
|
||||
*/
|
||||
var canMoveToParent = function (selection) {
|
||||
return !selection.empty() && nf.CanvasUtils.isDisconnected(selection) && nf.Canvas.getParentGroupId() !== null;
|
||||
};
|
||||
|
||||
/**
|
||||
* Adds a menu item to the context menu.
|
||||
|
@ -363,6 +372,7 @@ nf.ContextMenu = (function () {
|
|||
{condition: isNotConnection, menuItem: {img: 'images/iconCenterView.png', text: 'Center in view', action: 'center'}},
|
||||
{condition: isCopyable, menuItem: {img: 'images/iconCopy.png', text: 'Copy', action: 'copy'}},
|
||||
{condition: isPastable, menuItem: {img: 'images/iconPaste.png', text: 'Paste', action: 'paste'}},
|
||||
{condition: canMoveToParent, menuItem: {img: 'images/iconMoveToParent.png', text: 'Move to parent group', action: 'moveIntoParent'}},
|
||||
{condition: isDeletable, menuItem: {img: 'images/iconDelete.png', text: 'Delete', action: 'delete'}}
|
||||
];
|
||||
|
||||
|
|
|
@ -70,12 +70,6 @@ nf.ControllerService = (function () {
|
|||
return true;
|
||||
}
|
||||
|
||||
if ($('#controller-service-enabled').hasClass('checkbox-checked') && details['state'] === 'DISABLED') {
|
||||
return true;
|
||||
} else if ($('#controller-service-enabled').hasClass('checkbox-unchecked') && details['state'] === 'ENABLED') {
|
||||
return true;
|
||||
}
|
||||
|
||||
// defer to the properties
|
||||
return $('#controller-service-properties').propertytable('isSaveRequired');
|
||||
};
|
||||
|
@ -98,13 +92,6 @@ nf.ControllerService = (function () {
|
|||
controllerServiceDto['properties'] = properties;
|
||||
}
|
||||
|
||||
// mark the controller service enabled if appropriate
|
||||
if ($('#controller-service-enabled').hasClass('checkbox-unchecked')) {
|
||||
controllerServiceDto['state'] = 'DISABLED';
|
||||
} else if ($('#controller-service-enabled').hasClass('checkbox-checked')) {
|
||||
controllerServiceDto['state'] = 'ENABLED';
|
||||
}
|
||||
|
||||
// create the controller service entity
|
||||
var controllerServiceEntity = {};
|
||||
controllerServiceEntity['revision'] = nf.Client.getRevision();
|
||||
|
@ -1438,17 +1425,10 @@ nf.ControllerService = (function () {
|
|||
// record the controller service details
|
||||
controllerServiceDialog.data('controllerServiceDetails', controllerService);
|
||||
|
||||
// determine if the enabled checkbox is checked or not
|
||||
var controllerServiceEnableStyle = 'checkbox-checked';
|
||||
if (controllerService['state'] === 'DISABLED') {
|
||||
controllerServiceEnableStyle = 'checkbox-unchecked';
|
||||
}
|
||||
|
||||
// populate the controller service settings
|
||||
nf.Common.populateField('controller-service-id', controllerService['id']);
|
||||
nf.Common.populateField('controller-service-type', nf.Common.substringAfterLast(controllerService['type'], '.'));
|
||||
$('#controller-service-name').val(controllerService['name']);
|
||||
$('#controller-service-enabled').removeClass('checkbox-checked checkbox-unchecked').addClass(controllerServiceEnableStyle);
|
||||
$('#controller-service-comments').val(controllerService['comments']);
|
||||
|
||||
// select the availability when appropriate
|
||||
|
|
|
@ -25,7 +25,6 @@ import java.util.Arrays;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import kafka.common.FailedToSendMessageException;
|
||||
import kafka.javaapi.producer.Producer;
|
||||
|
@ -33,21 +32,14 @@ import kafka.producer.KeyedMessage;
|
|||
import kafka.producer.ProducerConfig;
|
||||
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.provenance.ProvenanceReporter;
|
||||
import org.apache.nifi.provenance.ProvenanceEventRecord;
|
||||
import org.apache.nifi.provenance.ProvenanceEventType;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.MockFlowFileQueue;
|
||||
import org.apache.nifi.util.MockProcessSession;
|
||||
import org.apache.nifi.util.MockProvenanceReporter;
|
||||
import org.apache.nifi.util.MockSessionFactory;
|
||||
import org.apache.nifi.util.SharedSessionState;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.internal.util.reflection.Whitebox;
|
||||
|
||||
public class TestPutKafka {
|
||||
|
||||
|
@ -150,19 +142,7 @@ public class TestPutKafka {
|
|||
public void testProvenanceReporterMessagesCount() {
|
||||
final TestableProcessor processor = new TestableProcessor();
|
||||
|
||||
ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter());
|
||||
|
||||
AtomicLong idGenerator = new AtomicLong(0L);
|
||||
SharedSessionState sharedState = new SharedSessionState(processor, idGenerator);
|
||||
Whitebox.setInternalState(sharedState, "provenanceReporter", spyProvenanceReporter);
|
||||
MockFlowFileQueue flowFileQueue = sharedState.getFlowFileQueue();
|
||||
MockSessionFactory sessionFactory = Mockito.mock(MockSessionFactory.class);
|
||||
MockProcessSession mockProcessSession = new MockProcessSession(sharedState);
|
||||
Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession);
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(processor);
|
||||
Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue);
|
||||
Whitebox.setInternalState(runner, "sessionFactory", sessionFactory);
|
||||
|
||||
runner.setProperty(PutKafka.TOPIC, "topic1");
|
||||
runner.setProperty(PutKafka.KEY, "key1");
|
||||
|
@ -173,28 +153,19 @@ public class TestPutKafka {
|
|||
runner.enqueue(bytes);
|
||||
runner.run();
|
||||
|
||||
MockFlowFile mockFlowFile = mockProcessSession.getFlowFilesForRelationship(PutKafka.REL_SUCCESS).get(0);
|
||||
Mockito.verify(spyProvenanceReporter, Mockito.atLeastOnce()).send(mockFlowFile, "kafka://topic1", "Sent 4 messages");
|
||||
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
|
||||
assertEquals(1, events.size());
|
||||
final ProvenanceEventRecord event = events.get(0);
|
||||
assertEquals(ProvenanceEventType.SEND, event.getEventType());
|
||||
assertEquals("kafka://topic1", event.getTransitUri());
|
||||
assertEquals("Sent 4 messages", event.getDetails());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProvenanceReporterWithoutDelimiterMessagesCount() {
|
||||
final TestableProcessor processor = new TestableProcessor();
|
||||
|
||||
ProvenanceReporter spyProvenanceReporter = Mockito.spy(new MockProvenanceReporter());
|
||||
|
||||
AtomicLong idGenerator = new AtomicLong(0L);
|
||||
SharedSessionState sharedState = new SharedSessionState(processor, idGenerator);
|
||||
Whitebox.setInternalState(sharedState, "provenanceReporter", spyProvenanceReporter);
|
||||
MockFlowFileQueue flowFileQueue = sharedState.getFlowFileQueue();
|
||||
MockSessionFactory sessionFactory = Mockito.mock(MockSessionFactory.class);
|
||||
MockProcessSession mockProcessSession = new MockProcessSession(sharedState);
|
||||
Mockito.when(sessionFactory.createSession()).thenReturn(mockProcessSession);
|
||||
|
||||
final TestRunner runner = TestRunners.newTestRunner(processor);
|
||||
Whitebox.setInternalState(runner, "flowFileQueue", flowFileQueue);
|
||||
Whitebox.setInternalState(runner, "sessionFactory", sessionFactory);
|
||||
|
||||
runner.setProperty(PutKafka.TOPIC, "topic1");
|
||||
runner.setProperty(PutKafka.KEY, "key1");
|
||||
runner.setProperty(PutKafka.SEED_BROKERS, "localhost:1234");
|
||||
|
@ -203,8 +174,11 @@ public class TestPutKafka {
|
|||
runner.enqueue(bytes);
|
||||
runner.run();
|
||||
|
||||
MockFlowFile mockFlowFile = mockProcessSession.getFlowFilesForRelationship(PutKafka.REL_SUCCESS).get(0);
|
||||
Mockito.verify(spyProvenanceReporter, Mockito.atLeastOnce()).send(mockFlowFile, "kafka://topic1");
|
||||
final List<ProvenanceEventRecord> events = runner.getProvenanceEvents();
|
||||
assertEquals(1, events.size());
|
||||
final ProvenanceEventRecord event = events.get(0);
|
||||
assertEquals(ProvenanceEventType.SEND, event.getEventType());
|
||||
assertEquals("kafka://topic1", event.getTransitUri());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -44,9 +44,8 @@
|
|||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-mapper-asl</artifactId>
|
||||
<version>1.9.13</version>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.nifi.web;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
|
@ -33,7 +34,6 @@ import javax.xml.transform.TransformerFactoryConfigurationError;
|
|||
import javax.xml.transform.stream.StreamResult;
|
||||
import javax.xml.transform.stream.StreamSource;
|
||||
import org.apache.nifi.web.ViewableContent.DisplayMode;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
@WebServlet(name = "StandardContentViewer", urlPatterns = {"/view-content"})
|
||||
public class StandardContentViewerController extends HttpServlet {
|
||||
|
@ -60,7 +60,7 @@ public class StandardContentViewerController extends HttpServlet {
|
|||
if ("application/json".equals(content.getContentType())) {
|
||||
// format json
|
||||
final ObjectMapper mapper = new ObjectMapper();
|
||||
final Object objectJson = mapper.readValue(content.getContent(), Object.class);
|
||||
final Object objectJson = mapper.readValue(content.getContentStream(), Object.class);
|
||||
formatted = mapper.writerWithDefaultPrettyPrinter().writeValueAsString(objectJson);
|
||||
} else if ("application/xml".equals(content.getContentType())) {
|
||||
// format xml
|
||||
|
|
|
@ -27,25 +27,25 @@ import org.apache.nifi.util.MockFlowFile;
|
|||
import org.apache.nifi.util.MockProcessSession;
|
||||
import org.apache.nifi.util.SharedSessionState;
|
||||
import org.junit.Assert;
|
||||
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class NewestFirstPrioritizerTest {
|
||||
|
||||
@Test
|
||||
public void testPrioritizer() throws InstantiationException, IllegalAccessException {
|
||||
Processor processor = new SimpleProcessor();
|
||||
AtomicLong idGenerator = new AtomicLong(0L);
|
||||
MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator));
|
||||
final Processor processor = new SimpleProcessor();
|
||||
final AtomicLong idGenerator = new AtomicLong(0L);
|
||||
final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), Mockito.mock(Processor.class));
|
||||
|
||||
MockFlowFile flowFile1 = session.create();
|
||||
final MockFlowFile flowFile1 = session.create();
|
||||
try {
|
||||
Thread.sleep(2); // guarantee the FlowFile entryDate for flowFile2 is different than flowFile1
|
||||
} catch (InterruptedException e) {
|
||||
} catch (final InterruptedException e) {
|
||||
}
|
||||
MockFlowFile flowFile2 = session.create();
|
||||
final MockFlowFile flowFile2 = session.create();
|
||||
|
||||
NewestFlowFileFirstPrioritizer prioritizer = new NewestFlowFileFirstPrioritizer();
|
||||
final NewestFlowFileFirstPrioritizer prioritizer = new NewestFlowFileFirstPrioritizer();
|
||||
Assert.assertEquals(0, prioritizer.compare(null, null));
|
||||
Assert.assertEquals(-1, prioritizer.compare(flowFile1, null));
|
||||
Assert.assertEquals(1, prioritizer.compare(null, flowFile1));
|
||||
|
|
|
@ -27,25 +27,25 @@ import org.apache.nifi.util.MockFlowFile;
|
|||
import org.apache.nifi.util.MockProcessSession;
|
||||
import org.apache.nifi.util.SharedSessionState;
|
||||
import org.junit.Assert;
|
||||
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class OldestFirstPrioritizerTest {
|
||||
|
||||
@Test
|
||||
public void testPrioritizer() throws InstantiationException, IllegalAccessException {
|
||||
Processor processor = new SimpleProcessor();
|
||||
AtomicLong idGenerator = new AtomicLong(0L);
|
||||
MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator));
|
||||
final Processor processor = new SimpleProcessor();
|
||||
final AtomicLong idGenerator = new AtomicLong(0L);
|
||||
final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), Mockito.mock(Processor.class));
|
||||
|
||||
MockFlowFile flowFile1 = session.create();
|
||||
final MockFlowFile flowFile1 = session.create();
|
||||
try {
|
||||
Thread.sleep(2); // guarantee the FlowFile entryDate for flowFile2 is different than flowFile1
|
||||
} catch (InterruptedException e) {
|
||||
} catch (final InterruptedException e) {
|
||||
}
|
||||
MockFlowFile flowFile2 = session.create();
|
||||
final MockFlowFile flowFile2 = session.create();
|
||||
|
||||
OldestFlowFileFirstPrioritizer prioritizer = new OldestFlowFileFirstPrioritizer();
|
||||
final OldestFlowFileFirstPrioritizer prioritizer = new OldestFlowFileFirstPrioritizer();
|
||||
Assert.assertEquals(0, prioritizer.compare(null, null));
|
||||
Assert.assertEquals(-1, prioritizer.compare(flowFile1, null));
|
||||
Assert.assertEquals(1, prioritizer.compare(null, flowFile1));
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.nifi.prioritizer;
|
||||
|
||||
import org.apache.nifi.prioritizer.PriorityAttributePrioritizer;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
@ -31,9 +32,9 @@ import org.apache.nifi.processor.exception.ProcessException;
|
|||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.MockProcessSession;
|
||||
import org.apache.nifi.util.SharedSessionState;
|
||||
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
public class PriorityAttributePrioritizerTest {
|
||||
|
||||
|
@ -58,27 +59,27 @@ public class PriorityAttributePrioritizerTest {
|
|||
|
||||
@Test
|
||||
public void testPrioritizer() throws InstantiationException, IllegalAccessException {
|
||||
Processor processor = new SimpleProcessor();
|
||||
AtomicLong idGenerator = new AtomicLong(0L);
|
||||
MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator));
|
||||
final Processor processor = new SimpleProcessor();
|
||||
final AtomicLong idGenerator = new AtomicLong(0L);
|
||||
final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), Mockito.mock(Processor.class));
|
||||
|
||||
MockFlowFile ffNoPriority = session.create();
|
||||
MockFlowFile ffPri1 = session.create();
|
||||
final MockFlowFile ffNoPriority = session.create();
|
||||
final MockFlowFile ffPri1 = session.create();
|
||||
ffPri1.putAttributes(attrsPri1);
|
||||
MockFlowFile ffPri2 = session.create();
|
||||
final MockFlowFile ffPri2 = session.create();
|
||||
ffPri2.putAttributes(attrsPri2);
|
||||
MockFlowFile ffPrin1 = session.create();
|
||||
final MockFlowFile ffPrin1 = session.create();
|
||||
ffPrin1.putAttributes(attrsPrin1);
|
||||
MockFlowFile ffPriA = session.create();
|
||||
final MockFlowFile ffPriA = session.create();
|
||||
ffPriA.putAttributes(attrsPriA);
|
||||
MockFlowFile ffPriB = session.create();
|
||||
final MockFlowFile ffPriB = session.create();
|
||||
ffPriB.putAttributes(attrsPriB);
|
||||
MockFlowFile ffPriLP = session.create();
|
||||
final MockFlowFile ffPriLP = session.create();
|
||||
ffPriLP.putAttributes(attrsPriLP);
|
||||
MockFlowFile ffPriLN = session.create();
|
||||
final MockFlowFile ffPriLN = session.create();
|
||||
ffPriLN.putAttributes(attrsPriLN);
|
||||
|
||||
PriorityAttributePrioritizer prioritizer = new PriorityAttributePrioritizer();
|
||||
final PriorityAttributePrioritizer prioritizer = new PriorityAttributePrioritizer();
|
||||
assertEquals(0, prioritizer.compare(null, null));
|
||||
assertEquals(-1, prioritizer.compare(ffNoPriority, null));
|
||||
assertEquals(1, prioritizer.compare(null, ffNoPriority));
|
||||
|
|
|
@ -1,272 +1,284 @@
|
|||
<?xml version="1.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.
|
||||
-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-standard-bundle</artifactId>
|
||||
<version>0.2.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
<artifactId>nifi-standard-processors</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-processor-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-ssl-context-service-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-flowfile-packager</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-distributed-cache-client-service-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-http-context-map-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.sun.jersey</groupId>
|
||||
<artifactId>jersey-client</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.sun.jersey</groupId>
|
||||
<artifactId>jersey-server</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-net</groupId>
|
||||
<artifactId>commons-net</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-compress</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.bouncycastle</groupId>
|
||||
<artifactId>bcprov-jdk16</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.bouncycastle</groupId>
|
||||
<artifactId>bcpg-jdk16</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-codec</groupId>
|
||||
<artifactId>commons-codec</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-security-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.jcraft</groupId>
|
||||
<artifactId>jsch</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.jcraft</groupId>
|
||||
<artifactId>jzlib</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-server</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-servlet</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.httpcomponents</groupId>
|
||||
<artifactId>httpclient</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.mail</groupId>
|
||||
<artifactId>mail</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.jponge</groupId>
|
||||
<artifactId>lzma-java</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.tukaani</groupId>
|
||||
<artifactId>xz</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>net.sf.saxon</groupId>
|
||||
<artifactId>Saxon-HE</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-mock</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-socket-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-load-distribution-service-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-distributed-cache-client-service</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
<artifactId>joda-time</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.jms</groupId>
|
||||
<artifactId>javax.jms-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.activemq</groupId>
|
||||
<artifactId>activemq-client</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.jayway.jsonpath</groupId>
|
||||
<artifactId>json-path</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-ssl-context-service</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.tika</groupId>
|
||||
<artifactId>tika-core</artifactId>
|
||||
<version>1.7</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
<version>2.4.5</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes combine.children="append">
|
||||
<exclude>src/test/resources/localhost.cer</exclude>
|
||||
<exclude>src/test/resources/hello.txt</exclude>
|
||||
<exclude>src/test/resources/CharacterSetConversionSamples/Converted.txt</exclude>
|
||||
<exclude>src/test/resources/CharacterSetConversionSamples/Original.txt</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile.txt</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFileConcat.txt</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/1000bytes.txt</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/test.txt</exclude>
|
||||
<exclude>src/test/resources/ScanAttribute/dictionary-with-empty-new-lines</exclude>
|
||||
<exclude>src/test/resources/ScanAttribute/dictionary-with-extra-info</exclude>
|
||||
<exclude>src/test/resources/ScanAttribute/dictionary1</exclude>
|
||||
<exclude>src/test/resources/TestEncryptContent/text.txt</exclude>
|
||||
<exclude>src/test/resources/TestEncryptContent/text.txt.asc</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.txt</exclude>
|
||||
<exclude>src/test/resources/TestJson/json-sample.json</exclude>
|
||||
<exclude>src/test/resources/TestJson/control-characters.json</exclude>
|
||||
<exclude>src/test/resources/TestMergeContent/demarcate</exclude>
|
||||
<exclude>src/test/resources/TestMergeContent/foot</exclude>
|
||||
<exclude>src/test/resources/TestMergeContent/head</exclude>
|
||||
<exclude>src/test/resources/TestModifyBytes/noFooter.txt</exclude>
|
||||
<exclude>src/test/resources/TestModifyBytes/noFooter_noHeader.txt</exclude>
|
||||
<exclude>src/test/resources/TestModifyBytes/noHeader.txt</exclude>
|
||||
<exclude>src/test/resources/TestModifyBytes/testFile.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/$1$1.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/BRue_cRue_RiRey.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/Blu$2e_clu$2e.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/D$d_h$d.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/Good.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/Spider.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/[DODO].txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/cu[$1]_Po[$1].txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/cu_Po.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/food.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/testFile.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-backreference-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-blank-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-escaped-dollar-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-excessive-backreference-mapping-simple.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-excessive-backreference-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-invalid-backreference-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-no-match-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-space-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/colors-without-dashes.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/colors.txt</exclude>
|
||||
<exclude>src/test/resources/TestScanContent/helloWorld</exclude>
|
||||
<exclude>src/test/resources/TestScanContent/wellthengood-bye</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/1.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/2.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/3.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/4.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/5.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/6.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/original.txt</exclude>
|
||||
<exclude>src/test/resources/TestTransformXml/math.html</exclude>
|
||||
<exclude>src/test/resources/TestTransformXml/tokens.csv</exclude>
|
||||
<exclude>src/test/resources/TestTransformXml/tokens.xml</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/folder/cal.txt</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/folder/date.txt</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/data.flowfilev2</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/data.flowfilev3</exclude>
|
||||
<exclude>src/test/resources/TestXml/xml-bundle-1</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile.txt.bz2</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile.txt.gz</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile1.txt.bz2</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile1.txt.gz</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFileConcat.txt.bz2</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/TestIngestAndUpdate.jar</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/TestSuccess.jar</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/TestDynamicEnvironment.jar</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.jar</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.tar</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.tar.gz</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.txt.bz2</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.txt.gz</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.zip</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/flowfilev1.tar</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/data.tar</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/data.zip</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
<!-- 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. -->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-standard-bundle</artifactId>
|
||||
<version>0.2.0-incubating-SNAPSHOT</version>
|
||||
</parent>
|
||||
<artifactId>nifi-standard-processors</artifactId>
|
||||
<packaging>jar</packaging>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-processor-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-ssl-context-service-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-flowfile-packager</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-distributed-cache-client-service-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-http-context-map-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.sun.jersey</groupId>
|
||||
<artifactId>jersey-client</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.sun.jersey</groupId>
|
||||
<artifactId>jersey-server</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-net</groupId>
|
||||
<artifactId>commons-net</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-compress</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.bouncycastle</groupId>
|
||||
<artifactId>bcprov-jdk16</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.bouncycastle</groupId>
|
||||
<artifactId>bcpg-jdk16</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-codec</groupId>
|
||||
<artifactId>commons-codec</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-security-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.jcraft</groupId>
|
||||
<artifactId>jsch</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.jcraft</groupId>
|
||||
<artifactId>jzlib</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-server</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.eclipse.jetty</groupId>
|
||||
<artifactId>jetty-servlet</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.httpcomponents</groupId>
|
||||
<artifactId>httpclient</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.mail</groupId>
|
||||
<artifactId>mail</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.github.jponge</groupId>
|
||||
<artifactId>lzma-java</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.tukaani</groupId>
|
||||
<artifactId>xz</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>net.sf.saxon</groupId>
|
||||
<artifactId>Saxon-HE</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-mock</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-socket-utils</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-load-distribution-service-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-distributed-cache-client-service</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
<artifactId>joda-time</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.jms</groupId>
|
||||
<artifactId>javax.jms-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.activemq</groupId>
|
||||
<artifactId>activemq-client</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.jayway.jsonpath</groupId>
|
||||
<artifactId>json-path</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-ssl-context-service</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.tika</groupId>
|
||||
<artifactId>tika-core</artifactId>
|
||||
<version>1.7</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.avro</groupId>
|
||||
<artifactId>avro</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-dbcp-service-api</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.derby</groupId>
|
||||
<artifactId>derby</artifactId>
|
||||
<version>10.11.1.1</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes combine.children="append">
|
||||
<exclude>src/test/resources/localhost.cer</exclude>
|
||||
<exclude>src/test/resources/hello.txt</exclude>
|
||||
<exclude>src/test/resources/CharacterSetConversionSamples/Converted.txt</exclude>
|
||||
<exclude>src/test/resources/CharacterSetConversionSamples/Original.txt</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile.txt</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFileConcat.txt</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/1000bytes.txt</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/test.txt</exclude>
|
||||
<exclude>src/test/resources/ScanAttribute/dictionary-with-empty-new-lines</exclude>
|
||||
<exclude>src/test/resources/ScanAttribute/dictionary-with-extra-info</exclude>
|
||||
<exclude>src/test/resources/ScanAttribute/dictionary1</exclude>
|
||||
<exclude>src/test/resources/TestEncryptContent/text.txt</exclude>
|
||||
<exclude>src/test/resources/TestEncryptContent/text.txt.asc</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.txt</exclude>
|
||||
<exclude>src/test/resources/TestJson/json-sample.json</exclude>
|
||||
<exclude>src/test/resources/TestJson/control-characters.json</exclude>
|
||||
<exclude>src/test/resources/TestMergeContent/demarcate</exclude>
|
||||
<exclude>src/test/resources/TestMergeContent/foot</exclude>
|
||||
<exclude>src/test/resources/TestMergeContent/head</exclude>
|
||||
<exclude>src/test/resources/TestModifyBytes/noFooter.txt</exclude>
|
||||
<exclude>src/test/resources/TestModifyBytes/noFooter_noHeader.txt</exclude>
|
||||
<exclude>src/test/resources/TestModifyBytes/noHeader.txt</exclude>
|
||||
<exclude>src/test/resources/TestModifyBytes/testFile.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/$1$1.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/BRue_cRue_RiRey.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/Blu$2e_clu$2e.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/D$d_h$d.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/Good.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/Spider.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/[DODO].txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/cu[$1]_Po[$1].txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/cu_Po.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/food.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextLineByLine/testFile.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-backreference-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-blank-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-escaped-dollar-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-excessive-backreference-mapping-simple.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-excessive-backreference-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-invalid-backreference-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-no-match-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-space-mapping.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/colors-without-dashes.txt</exclude>
|
||||
<exclude>src/test/resources/TestReplaceTextWithMapping/colors.txt</exclude>
|
||||
<exclude>src/test/resources/TestScanContent/helloWorld</exclude>
|
||||
<exclude>src/test/resources/TestScanContent/wellthengood-bye</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/1.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/2.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/3.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/4.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/5.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/6.txt</exclude>
|
||||
<exclude>src/test/resources/TestSplitText/original.txt</exclude>
|
||||
<exclude>src/test/resources/TestTransformXml/math.html</exclude>
|
||||
<exclude>src/test/resources/TestTransformXml/tokens.csv</exclude>
|
||||
<exclude>src/test/resources/TestTransformXml/tokens.xml</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/folder/cal.txt</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/folder/date.txt</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/data.flowfilev2</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/data.flowfilev3</exclude>
|
||||
<exclude>src/test/resources/TestXml/xml-bundle-1</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile.txt.bz2</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile.txt.gz</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile1.txt.bz2</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFile1.txt.gz</exclude>
|
||||
<exclude>src/test/resources/CompressedData/SampleFileConcat.txt.bz2</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/TestIngestAndUpdate.jar</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/TestSuccess.jar</exclude>
|
||||
<exclude>src/test/resources/ExecuteCommand/TestDynamicEnvironment.jar</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.jar</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.tar</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.tar.gz</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.txt.bz2</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.txt.gz</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/1.zip</exclude>
|
||||
<exclude>src/test/resources/TestIdentifyMimeType/flowfilev1.tar</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/data.tar</exclude>
|
||||
<exclude>src/test/resources/TestUnpackContent/data.zip</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
||||
|
|
|
@ -89,7 +89,7 @@ import org.xml.sax.InputSource;
|
|||
+ "Name into which the result will be placed (if the Destination is flowfile-attribute; otherwise, the property name is ignored). "
|
||||
+ "The value of the property must be a valid XPath expression. If the XPath evaluates to more than one node and the Return Type is "
|
||||
+ "set to 'nodeset' (either directly, or via 'auto-detect' with a Destination of "
|
||||
+ "'flowfile-content', the FlowFile will be unmodified and will be routed to failure. If the XPath does not "
|
||||
+ "'flowfile-content'), the FlowFile will be unmodified and will be routed to failure. If the XPath does not "
|
||||
+ "evaluate to a Node, the FlowFile will be routed to 'unmatched' without having its contents modified. If Destination is "
|
||||
+ "flowfile-attribute and the expression matches nothing, attributes will be created with empty strings as the value, and the "
|
||||
+ "FlowFile will always be routed to 'matched'")
|
||||
|
|
|
@ -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.standard;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.sql.Connection;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.dbcp.DBCPService;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.util.JdbcCommon;
|
||||
import org.apache.nifi.util.LongHolder;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
@EventDriven
|
||||
@Tags({ "sql", "select", "jdbc", "query", "database" })
|
||||
@CapabilityDescription("Execute provided SQL select query. Query result will be converted to Avro format."
|
||||
+ " Streaming is used so arbitrarily large result sets are supported.")
|
||||
public class ExecuteSQL extends AbstractProcessor {
|
||||
|
||||
// Relationships
|
||||
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("Successfully created FlowFile from SQL query result set.")
|
||||
.build();
|
||||
public static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||
.name("failure")
|
||||
.description("SQL query execution failed. Incoming FlowFile will be penalized and routed to this relationship")
|
||||
.build();
|
||||
private final Set<Relationship> relationships;
|
||||
|
||||
public static final PropertyDescriptor DBCP_SERVICE = new PropertyDescriptor.Builder()
|
||||
.name("Database Connection Pooling Service")
|
||||
.description("The Controller Service that is used to obtain connection to database")
|
||||
.required(true)
|
||||
.identifiesControllerService(DBCPService.class)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor SQL_SELECT_QUERY = new PropertyDescriptor.Builder()
|
||||
.name("SQL select query")
|
||||
.description("SQL select query")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.expressionLanguageSupported(true)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor QUERY_TIMEOUT = new PropertyDescriptor.Builder()
|
||||
.name("Max Wait Time")
|
||||
.description("The maximum amount of time allowed for a running SQL select query "
|
||||
+ " , zero means there is no limit. Max time less than 1 second will be equal to zero.")
|
||||
.defaultValue("0 seconds")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||
.sensitive(false)
|
||||
.build();
|
||||
|
||||
private final List<PropertyDescriptor> propDescriptors;
|
||||
|
||||
public ExecuteSQL() {
|
||||
final Set<Relationship> r = new HashSet<>();
|
||||
r.add(REL_SUCCESS);
|
||||
relationships = Collections.unmodifiableSet(r);
|
||||
|
||||
final List<PropertyDescriptor> pds = new ArrayList<>();
|
||||
pds.add(DBCP_SERVICE);
|
||||
pds.add(SQL_SELECT_QUERY);
|
||||
pds.add(QUERY_TIMEOUT);
|
||||
propDescriptors = Collections.unmodifiableList(pds);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Relationship> getRelationships() {
|
||||
return relationships;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return propDescriptors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
final FlowFile incoming = session.get();
|
||||
if (incoming == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final ProcessorLog logger = getLogger();
|
||||
|
||||
final DBCPService dbcpService = context.getProperty(DBCP_SERVICE).asControllerService(DBCPService.class);
|
||||
final String selectQuery = context.getProperty(SQL_SELECT_QUERY).evaluateAttributeExpressions(incoming).getValue();
|
||||
final Integer queryTimeout = context.getProperty(QUERY_TIMEOUT).asTimePeriod(TimeUnit.SECONDS).intValue();
|
||||
|
||||
final StopWatch stopWatch = new StopWatch(true);
|
||||
|
||||
try (final Connection con = dbcpService.getConnection();
|
||||
final Statement st = con.createStatement()) {
|
||||
st.setQueryTimeout(queryTimeout); // timeout in seconds
|
||||
final LongHolder nrOfRows = new LongHolder(0L);
|
||||
final FlowFile outgoing = session.write(incoming, new OutputStreamCallback() {
|
||||
@Override
|
||||
public void process(final OutputStream out) throws IOException {
|
||||
try {
|
||||
logger.debug("Executing query {}", new Object[] { selectQuery });
|
||||
final ResultSet resultSet = st.executeQuery(selectQuery);
|
||||
nrOfRows.set(JdbcCommon.convertToAvroStream(resultSet, out));
|
||||
} catch (final SQLException e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
logger.info("{} contains {} Avro records", new Object[] { nrOfRows.get() });
|
||||
logger.info("Transferred {} to 'success'", new Object[] { outgoing });
|
||||
session.getProvenanceReporter().modifyContent(outgoing, "Retrieved " + nrOfRows.get() + " rows", stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||
session.transfer(outgoing, REL_SUCCESS);
|
||||
} catch (final ProcessException | SQLException e) {
|
||||
logger.error("Unable to execute SQL select query {} for {} due to {}; routing to failure", new Object[] { selectQuery, incoming, e });
|
||||
session.transfer(incoming, REL_FAILURE);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,165 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard.util;
|
||||
|
||||
import static java.sql.Types.BIGINT;
|
||||
import static java.sql.Types.BOOLEAN;
|
||||
import static java.sql.Types.CHAR;
|
||||
import static java.sql.Types.DATE;
|
||||
import static java.sql.Types.DECIMAL;
|
||||
import static java.sql.Types.DOUBLE;
|
||||
import static java.sql.Types.FLOAT;
|
||||
import static java.sql.Types.INTEGER;
|
||||
import static java.sql.Types.LONGNVARCHAR;
|
||||
import static java.sql.Types.LONGVARCHAR;
|
||||
import static java.sql.Types.NCHAR;
|
||||
import static java.sql.Types.NUMERIC;
|
||||
import static java.sql.Types.NVARCHAR;
|
||||
import static java.sql.Types.REAL;
|
||||
import static java.sql.Types.ROWID;
|
||||
import static java.sql.Types.SMALLINT;
|
||||
import static java.sql.Types.TIME;
|
||||
import static java.sql.Types.TIMESTAMP;
|
||||
import static java.sql.Types.TINYINT;
|
||||
import static java.sql.Types.VARCHAR;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.ResultSetMetaData;
|
||||
import java.sql.SQLException;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.SchemaBuilder;
|
||||
import org.apache.avro.SchemaBuilder.FieldAssembler;
|
||||
import org.apache.avro.file.DataFileWriter;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericDatumWriter;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.DatumWriter;
|
||||
|
||||
/**
|
||||
* JDBC / SQL common functions.
|
||||
*/
|
||||
public class JdbcCommon {
|
||||
|
||||
public static long convertToAvroStream(final ResultSet rs, final OutputStream outStream) throws SQLException, IOException {
|
||||
final Schema schema = createSchema(rs);
|
||||
final GenericRecord rec = new GenericData.Record(schema);
|
||||
|
||||
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<GenericRecord>(schema);
|
||||
try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter)) {
|
||||
dataFileWriter.create(schema, outStream);
|
||||
|
||||
final ResultSetMetaData meta = rs.getMetaData();
|
||||
final int nrOfColumns = meta.getColumnCount();
|
||||
long nrOfRows = 0;
|
||||
while (rs.next()) {
|
||||
for (int i = 1; i <= nrOfColumns; i++) {
|
||||
final Object value = rs.getObject(i);
|
||||
|
||||
// The different types that we support are numbers (int, long, double, float),
|
||||
// as well as boolean values and Strings. Since Avro doesn't provide
|
||||
// timestamp types, we want to convert those to Strings. So we will cast anything other
|
||||
// than numbers or booleans to strings by using to toString() method.
|
||||
if (value == null) {
|
||||
rec.put(i - 1, null);
|
||||
} else if (value instanceof Number || value instanceof Boolean) {
|
||||
rec.put(i - 1, value);
|
||||
} else {
|
||||
rec.put(i - 1, value.toString());
|
||||
}
|
||||
}
|
||||
dataFileWriter.append(rec);
|
||||
nrOfRows += 1;
|
||||
}
|
||||
|
||||
return nrOfRows;
|
||||
}
|
||||
}
|
||||
|
||||
public static Schema createSchema(final ResultSet rs) throws SQLException {
|
||||
final ResultSetMetaData meta = rs.getMetaData();
|
||||
final int nrOfColumns = meta.getColumnCount();
|
||||
final String tableName = meta.getTableName(1);
|
||||
|
||||
final FieldAssembler<Schema> builder = SchemaBuilder.record(tableName).namespace("any.data").fields();
|
||||
|
||||
/**
|
||||
* Some missing Avro types - Decimal, Date types. May need some additional work.
|
||||
*/
|
||||
for (int i = 1; i <= nrOfColumns; i++) {
|
||||
switch (meta.getColumnType(i)) {
|
||||
case CHAR:
|
||||
case LONGNVARCHAR:
|
||||
case LONGVARCHAR:
|
||||
case NCHAR:
|
||||
case NVARCHAR:
|
||||
case VARCHAR:
|
||||
builder.name(meta.getColumnName(i)).type().stringType().noDefault();
|
||||
break;
|
||||
|
||||
case BOOLEAN:
|
||||
builder.name(meta.getColumnName(i)).type().booleanType().noDefault();
|
||||
break;
|
||||
|
||||
case INTEGER:
|
||||
case SMALLINT:
|
||||
case TINYINT:
|
||||
builder.name(meta.getColumnName(i)).type().intType().noDefault();
|
||||
break;
|
||||
|
||||
case BIGINT:
|
||||
builder.name(meta.getColumnName(i)).type().longType().noDefault();
|
||||
break;
|
||||
|
||||
// java.sql.RowId is interface, is seems to be database
|
||||
// implementation specific, let's convert to String
|
||||
case ROWID:
|
||||
builder.name(meta.getColumnName(i)).type().stringType().noDefault();
|
||||
break;
|
||||
|
||||
case FLOAT:
|
||||
case REAL:
|
||||
builder.name(meta.getColumnName(i)).type().floatType().noDefault();
|
||||
break;
|
||||
|
||||
case DOUBLE:
|
||||
builder.name(meta.getColumnName(i)).type().doubleType().noDefault();
|
||||
break;
|
||||
|
||||
// Did not find direct suitable type, need to be clarified!!!!
|
||||
case DECIMAL:
|
||||
case NUMERIC:
|
||||
builder.name(meta.getColumnName(i)).type().stringType().noDefault();
|
||||
break;
|
||||
|
||||
// Did not find direct suitable type, need to be clarified!!!!
|
||||
case DATE:
|
||||
case TIME:
|
||||
case TIMESTAMP:
|
||||
builder.name(meta.getColumnName(i)).type().stringType().noDefault();
|
||||
break;
|
||||
|
||||
default:
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return builder.endRecord();
|
||||
}
|
||||
}
|
|
@ -66,3 +66,4 @@ org.apache.nifi.processors.standard.SplitXml
|
|||
org.apache.nifi.processors.standard.TransformXml
|
||||
org.apache.nifi.processors.standard.UnpackContent
|
||||
org.apache.nifi.processors.standard.ValidateXml
|
||||
org.apache.nifi.processors.standard.ExecuteSQL
|
||||
|
|
|
@ -0,0 +1,169 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.SQLException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.DatumReader;
|
||||
import org.apache.nifi.controller.AbstractControllerService;
|
||||
import org.apache.nifi.dbcp.DBCPService;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processors.standard.util.TestJdbcHugeStream;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.util.MockFlowFile;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.fusesource.hawtbuf.ByteArrayInputStream;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class TestExecuteSQL {
|
||||
|
||||
private static Logger LOGGER;
|
||||
|
||||
static {
|
||||
System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
|
||||
System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
|
||||
System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug");
|
||||
System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.ExecuteSQL", "debug");
|
||||
System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.TestExecuteSQL", "debug");
|
||||
LOGGER = LoggerFactory.getLogger(TestExecuteSQL.class);
|
||||
}
|
||||
|
||||
final static String DB_LOCATION = "target/db";
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
System.setProperty("derby.stream.error.file", "target/derby.log");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoTimeLimit() throws InitializationException, ClassNotFoundException, SQLException, IOException {
|
||||
invokeOnTrigger(null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testQueryTimeout() throws InitializationException, ClassNotFoundException, SQLException, IOException {
|
||||
// Does to seem to have any effect when using embedded Derby
|
||||
invokeOnTrigger(1); // 1 second max time
|
||||
}
|
||||
|
||||
public void invokeOnTrigger(final Integer queryTimeout) throws InitializationException, ClassNotFoundException, SQLException, IOException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(ExecuteSQL.class);
|
||||
|
||||
final DBCPService dbcp = new DBCPServiceSimpleImpl();
|
||||
final Map<String, String> dbcpProperties = new HashMap<>();
|
||||
|
||||
runner.addControllerService("dbcp", dbcp, dbcpProperties);
|
||||
|
||||
runner.enableControllerService(dbcp);
|
||||
runner.setProperty(ExecuteSQL.DBCP_SERVICE, "dbcp");
|
||||
|
||||
if (queryTimeout != null) {
|
||||
runner.setProperty(ExecuteSQL.QUERY_TIMEOUT, queryTimeout.toString() + " secs");
|
||||
}
|
||||
|
||||
// remove previous test database, if any
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
// load test data to database
|
||||
final Connection con = dbcp.getConnection();
|
||||
TestJdbcHugeStream.loadTestData2Database(con, 100, 2000, 1000);
|
||||
LOGGER.info("test data loaded");
|
||||
|
||||
// ResultSet size will be 1x2000x1000 = 2 000 000 rows
|
||||
// because of where PER.ID = ${person.id}
|
||||
final int nrOfRows = 2000000;
|
||||
final String query = "select "
|
||||
+ " PER.ID as PersonId, PER.NAME as PersonName, PER.CODE as PersonCode"
|
||||
+ ", PRD.ID as ProductId,PRD.NAME as ProductName,PRD.CODE as ProductCode"
|
||||
+ ", REL.ID as RelId, REL.NAME as RelName, REL.CODE as RelCode"
|
||||
+ ", ROW_NUMBER() OVER () as rownr "
|
||||
+ " from persons PER, products PRD, relationships REL"
|
||||
+ " where PER.ID = ${person.id}";
|
||||
|
||||
runner.setProperty(ExecuteSQL.SQL_SELECT_QUERY, query);
|
||||
|
||||
// incoming FlowFile content is not used, but attributes are used
|
||||
final Map<String, String> attributes = new HashMap<String, String>();
|
||||
attributes.put("person.id", "10");
|
||||
runner.enqueue("Hello".getBytes(), attributes);
|
||||
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(ExecuteSQL.REL_SUCCESS, 1);
|
||||
|
||||
// read all Avro records and verify created FlowFile contains 1000000
|
||||
// records
|
||||
final List<MockFlowFile> flowfiles = runner.getFlowFilesForRelationship(ExecuteSQL.REL_SUCCESS);
|
||||
final InputStream in = new ByteArrayInputStream(flowfiles.get(0).toByteArray());
|
||||
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
|
||||
final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<GenericRecord>(in, datumReader);
|
||||
GenericRecord record = null;
|
||||
long recordsFromStream = 0;
|
||||
while (dataFileReader.hasNext()) {
|
||||
// Reuse record object by passing it to next(). This saves us from
|
||||
// allocating and garbage collecting many objects for files with
|
||||
// many items.
|
||||
record = dataFileReader.next(record);
|
||||
recordsFromStream += 1;
|
||||
}
|
||||
|
||||
LOGGER.info("total nr of records from stream: " + recordsFromStream);
|
||||
assertEquals(nrOfRows, recordsFromStream);
|
||||
dataFileReader.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Simple implementation only for ExecuteSQL processor testing.
|
||||
*
|
||||
*/
|
||||
class DBCPServiceSimpleImpl extends AbstractControllerService implements DBCPService {
|
||||
|
||||
@Override
|
||||
public String getIdentifier() {
|
||||
return "dbcp";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Connection getConnection() throws ProcessException {
|
||||
try {
|
||||
Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
|
||||
final Connection con = DriverManager.getConnection("jdbc:derby:" + DB_LOCATION + ";create=true");
|
||||
return con;
|
||||
} catch (final Exception e) {
|
||||
throw new ProcessException("getConnection failed: " + e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,155 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard.util;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.DatumReader;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestJdbcCommon {
|
||||
|
||||
final static String DB_LOCATION = "target/db";
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
System.setProperty("derby.stream.error.file", "target/derby.log");
|
||||
}
|
||||
|
||||
String createTable = "create table restaurants(id integer, name varchar(20), city varchar(50))";
|
||||
String dropTable = "drop table restaurants";
|
||||
|
||||
@Test
|
||||
public void testCreateSchema() throws ClassNotFoundException, SQLException {
|
||||
|
||||
// remove previous test database, if any
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
final Connection con = createConnection();
|
||||
final Statement st = con.createStatement();
|
||||
|
||||
try {
|
||||
st.executeUpdate(dropTable);
|
||||
} catch (final Exception e) {
|
||||
// table may not exist, this is not serious problem.
|
||||
}
|
||||
|
||||
st.executeUpdate(createTable);
|
||||
st.executeUpdate("insert into restaurants values (1, 'Irifunes', 'San Mateo')");
|
||||
st.executeUpdate("insert into restaurants values (2, 'Estradas', 'Daly City')");
|
||||
st.executeUpdate("insert into restaurants values (3, 'Prime Rib House', 'San Francisco')");
|
||||
|
||||
final ResultSet resultSet = st.executeQuery("select * from restaurants");
|
||||
|
||||
final Schema schema = JdbcCommon.createSchema(resultSet);
|
||||
assertNotNull(schema);
|
||||
|
||||
// records name, should be result set first column table name
|
||||
// Notice! sql select may join data from different tables, other columns
|
||||
// may have different table names
|
||||
assertEquals("RESTAURANTS", schema.getName());
|
||||
assertNotNull(schema.getField("ID"));
|
||||
assertNotNull(schema.getField("NAME"));
|
||||
assertNotNull(schema.getField("CITY"));
|
||||
|
||||
st.close();
|
||||
con.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertToBytes() throws ClassNotFoundException, SQLException, IOException {
|
||||
// remove previous test database, if any
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
final Connection con = createConnection();
|
||||
final Statement st = con.createStatement();
|
||||
|
||||
try {
|
||||
st.executeUpdate(dropTable);
|
||||
} catch (final Exception e) {
|
||||
// table may not exist, this is not serious problem.
|
||||
}
|
||||
|
||||
st.executeUpdate(createTable);
|
||||
|
||||
st.executeUpdate("insert into restaurants values (1, 'Irifunes', 'San Mateo')");
|
||||
st.executeUpdate("insert into restaurants values (2, 'Estradas', 'Daly City')");
|
||||
st.executeUpdate("insert into restaurants values (3, 'Prime Rib House', 'San Francisco')");
|
||||
|
||||
final ResultSet resultSet = st.executeQuery("select R.*, ROW_NUMBER() OVER () as rownr from restaurants R");
|
||||
|
||||
final ByteArrayOutputStream outStream = new ByteArrayOutputStream();
|
||||
JdbcCommon.convertToAvroStream(resultSet, outStream);
|
||||
|
||||
final byte[] serializedBytes = outStream.toByteArray();
|
||||
assertNotNull(serializedBytes);
|
||||
System.out.println("Avro serialized result size in bytes: " + serializedBytes.length);
|
||||
|
||||
st.close();
|
||||
con.close();
|
||||
|
||||
// Deserialize bytes to records
|
||||
|
||||
final InputStream instream = new ByteArrayInputStream(serializedBytes);
|
||||
|
||||
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
|
||||
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<GenericRecord>(instream, datumReader)) {
|
||||
GenericRecord record = null;
|
||||
while (dataFileReader.hasNext()) {
|
||||
// Reuse record object by passing it to next(). This saves us from
|
||||
// allocating and garbage collecting many objects for files with
|
||||
// many items.
|
||||
record = dataFileReader.next(record);
|
||||
System.out.println(record);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// many test use Derby as database, so ensure driver is available
|
||||
@Test
|
||||
public void testDriverLoad() throws ClassNotFoundException {
|
||||
final Class<?> clazz = Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
|
||||
assertNotNull(clazz);
|
||||
}
|
||||
|
||||
private Connection createConnection() throws ClassNotFoundException, SQLException {
|
||||
|
||||
Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
|
||||
final Connection con = DriverManager.getConnection("jdbc:derby:" + DB_LOCATION + ";create=true");
|
||||
return con;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,267 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.nifi.processors.standard.util;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.DatumReader;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* Test streaming using large number of result set rows. 1. Read data from
|
||||
* database. 2. Create Avro schema from ResultSet meta data. 3. Read rows from
|
||||
* ResultSet and write rows to Avro writer stream (Avro will create record for
|
||||
* each row). 4. And finally read records from Avro stream to verify all data is
|
||||
* present in Avro stream.
|
||||
*
|
||||
*
|
||||
* Sql query will return all combinations from 3 table. For example when each
|
||||
* table contain 1000 rows, result set will be 1 000 000 000 rows.
|
||||
*
|
||||
*/
|
||||
public class TestJdbcHugeStream {
|
||||
|
||||
final static String DB_LOCATION = "target/db";
|
||||
|
||||
@BeforeClass
|
||||
public static void setup() {
|
||||
System.setProperty("derby.stream.error.file", "target/derby.log");
|
||||
}
|
||||
|
||||
/**
|
||||
* In case of large record set this will fail with
|
||||
* java.lang.OutOfMemoryError: Java heap space at
|
||||
* java.util.Arrays.copyOf(Arrays.java:2271) at
|
||||
* java.io.ByteArrayOutputStream.grow(ByteArrayOutputStream.java:113) at
|
||||
* java
|
||||
* .io.ByteArrayOutputStream.ensureCapacity(ByteArrayOutputStream.java:93)
|
||||
* at java.io.ByteArrayOutputStream.write(ByteArrayOutputStream.java:140) at
|
||||
* org .apache.avro.file.
|
||||
* DataFileWriter$BufferedFileOutputStream$PositionFilter
|
||||
* .write(DataFileWriter.java:446)
|
||||
*
|
||||
*/
|
||||
@Test
|
||||
@Ignore
|
||||
public void readSend2StreamHuge_InMemory() throws ClassNotFoundException, SQLException, IOException {
|
||||
|
||||
// remove previous test database, if any
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
try (final Connection con = createConnection()) {
|
||||
loadTestData2Database(con, 150, 150, 150);
|
||||
System.out.println("test data loaded");
|
||||
|
||||
try (final Statement st = con.createStatement()) {
|
||||
// Notice!
|
||||
// Following select is deliberately invalid!
|
||||
// For testing we need huge amount of rows, so where part is not
|
||||
// used.
|
||||
final ResultSet resultSet = st.executeQuery("select "
|
||||
+ " PER.ID as PersonId, PER.NAME as PersonName, PER.CODE as PersonCode"
|
||||
+ ", PRD.ID as ProductId,PRD.NAME as ProductName,PRD.CODE as ProductCode"
|
||||
+ ", REL.ID as RelId, REL.NAME as RelName, REL.CODE as RelCode"
|
||||
+ ", ROW_NUMBER() OVER () as rownr "
|
||||
+ " from persons PER, products PRD, relationships REL");
|
||||
|
||||
final ByteArrayOutputStream outStream = new ByteArrayOutputStream();
|
||||
final long nrOfRows = JdbcCommon.convertToAvroStream(resultSet, outStream);
|
||||
System.out.println("total nr of rows in resultset: " + nrOfRows);
|
||||
|
||||
final byte[] serializedBytes = outStream.toByteArray();
|
||||
assertNotNull(serializedBytes);
|
||||
System.out.println("Avro serialized result size in bytes: " + serializedBytes.length);
|
||||
|
||||
// Deserialize bytes to records
|
||||
|
||||
final InputStream instream = new ByteArrayInputStream(serializedBytes);
|
||||
|
||||
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
|
||||
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<GenericRecord>(instream, datumReader)) {
|
||||
GenericRecord record = null;
|
||||
long recordsFromStream = 0;
|
||||
while (dataFileReader.hasNext()) {
|
||||
// Reuse record object by passing it to next(). This
|
||||
// saves us from
|
||||
// allocating and garbage collecting many objects for
|
||||
// files with many items.
|
||||
record = dataFileReader.next(record);
|
||||
recordsFromStream += 1;
|
||||
}
|
||||
System.out.println("total nr of records from stream: " + recordsFromStream);
|
||||
assertEquals(nrOfRows, recordsFromStream);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void readSend2StreamHuge_FileBased() throws ClassNotFoundException, SQLException, IOException {
|
||||
|
||||
// remove previous test database, if any
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
try (final Connection con = createConnection()) {
|
||||
loadTestData2Database(con, 300, 300, 300);
|
||||
|
||||
try (final Statement st = con.createStatement()) {
|
||||
// Notice!
|
||||
// Following select is deliberately invalid!
|
||||
// For testing we need huge amount of rows, so where part is not
|
||||
// used.
|
||||
final ResultSet resultSet = st.executeQuery("select "
|
||||
+ " PER.ID as PersonId, PER.NAME as PersonName, PER.CODE as PersonCode"
|
||||
+ ", PRD.ID as ProductId,PRD.NAME as ProductName,PRD.CODE as ProductCode"
|
||||
+ ", REL.ID as RelId, REL.NAME as RelName, REL.CODE as RelCode"
|
||||
+ ", ROW_NUMBER() OVER () as rownr "
|
||||
+ " from persons PER, products PRD, relationships REL");
|
||||
|
||||
final OutputStream outStream = new FileOutputStream("target/data.avro");
|
||||
final long nrOfRows = JdbcCommon.convertToAvroStream(resultSet, outStream);
|
||||
|
||||
// Deserialize bytes to records
|
||||
final InputStream instream = new FileInputStream("target/data.avro");
|
||||
|
||||
final DatumReader<GenericRecord> datumReader = new GenericDatumReader<GenericRecord>();
|
||||
try (final DataFileStream<GenericRecord> dataFileReader = new DataFileStream<GenericRecord>(instream, datumReader)) {
|
||||
GenericRecord record = null;
|
||||
long recordsFromStream = 0;
|
||||
while (dataFileReader.hasNext()) {
|
||||
// Reuse record object by passing it to next(). This
|
||||
// saves us from
|
||||
// allocating and garbage collecting many objects for
|
||||
// files with many items.
|
||||
record = dataFileReader.next(record);
|
||||
recordsFromStream += 1;
|
||||
}
|
||||
System.out.println("total nr of records from stream: " + recordsFromStream);
|
||||
assertEquals(nrOfRows, recordsFromStream);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// ================================================ helpers
|
||||
// ===============================================
|
||||
|
||||
static String dropPersons = "drop table persons";
|
||||
static String dropProducts = "drop table products";
|
||||
static String dropRelationships = "drop table relationships";
|
||||
static String createPersons = "create table persons (id integer, name varchar(100), code integer)";
|
||||
static String createProducts = "create table products (id integer, name varchar(100), code integer)";
|
||||
static String createRelationships = "create table relationships (id integer,name varchar(100), code integer)";
|
||||
|
||||
static public void loadTestData2Database(Connection con, int nrOfPersons, int nrOfProducts, int nrOfRels) throws ClassNotFoundException, SQLException {
|
||||
|
||||
System.out.println(createRandomName());
|
||||
System.out.println(createRandomName());
|
||||
System.out.println(createRandomName());
|
||||
|
||||
final Statement st = con.createStatement();
|
||||
|
||||
// tables may not exist, this is not serious problem.
|
||||
try {
|
||||
st.executeUpdate(dropPersons);
|
||||
} catch (final Exception e) {
|
||||
}
|
||||
|
||||
try {
|
||||
st.executeUpdate(dropProducts);
|
||||
} catch (final Exception e) {
|
||||
}
|
||||
|
||||
try {
|
||||
st.executeUpdate(dropRelationships);
|
||||
} catch (final Exception e) {
|
||||
}
|
||||
|
||||
st.executeUpdate(createPersons);
|
||||
st.executeUpdate(createProducts);
|
||||
st.executeUpdate(createRelationships);
|
||||
|
||||
for (int i = 0; i < nrOfPersons; i++)
|
||||
loadPersons(st, i);
|
||||
|
||||
for (int i = 0; i < nrOfProducts; i++)
|
||||
loadProducts(st, i);
|
||||
|
||||
for (int i = 0; i < nrOfRels; i++)
|
||||
loadRelationships(st, i);
|
||||
|
||||
st.close();
|
||||
}
|
||||
|
||||
static Random rng = new Random(53495);
|
||||
|
||||
static private void loadPersons(Statement st, int nr) throws SQLException {
|
||||
st.executeUpdate("insert into persons values (" + nr + ", '" + createRandomName() + "', " + rng.nextInt(469946) + ")");
|
||||
}
|
||||
|
||||
static private void loadProducts(Statement st, int nr) throws SQLException {
|
||||
st.executeUpdate("insert into products values (" + nr + ", '" + createRandomName() + "', " + rng.nextInt(469946) + ")");
|
||||
}
|
||||
|
||||
static private void loadRelationships(Statement st, int nr) throws SQLException {
|
||||
st.executeUpdate("insert into relationships values (" + nr + ", '" + createRandomName() + "', " + rng.nextInt(469946) + ")");
|
||||
}
|
||||
|
||||
static private String createRandomName() {
|
||||
return createRandomString() + " " + createRandomString();
|
||||
}
|
||||
|
||||
static private String createRandomString() {
|
||||
|
||||
final int length = rng.nextInt(19);
|
||||
final String characters = "ABCDEFGHIJKLMNOPQRSTUVWXYZ";
|
||||
|
||||
final char[] text = new char[length];
|
||||
for (int i = 0; i < length; i++) {
|
||||
text[i] = characters.charAt(rng.nextInt(characters.length()));
|
||||
}
|
||||
return new String(text);
|
||||
}
|
||||
|
||||
private Connection createConnection() throws ClassNotFoundException, SQLException {
|
||||
Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
|
||||
final Connection con = DriverManager.getConnection("jdbc:derby:" + DB_LOCATION + ";create=true");
|
||||
return con;
|
||||
}
|
||||
|
||||
}
|
|
@ -29,7 +29,7 @@
|
|||
<module>nifi-standard-reporting-tasks</module>
|
||||
<module>nifi-standard-content-viewer</module>
|
||||
<module>nifi-standard-nar</module>
|
||||
</modules>
|
||||
</modules>
|
||||
<dependencyManagement>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
|
@ -53,6 +53,11 @@
|
|||
<type>war</type>
|
||||
<version>0.2.0-incubating-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
<version>2.4.5</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</dependencyManagement>
|
||||
|
||||
|
|
|
@ -41,84 +41,59 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
import org.apache.nifi.reporting.InitializationException;
|
||||
|
||||
/**
|
||||
* Implementation of for Database Connection Pooling Service.
|
||||
* Apache DBCP is used for connection pooling functionality.
|
||||
* Implementation of for Database Connection Pooling Service. Apache DBCP is used for connection pooling functionality.
|
||||
*
|
||||
*/
|
||||
@Tags({"dbcp", "jdbc", "database", "connection", "pooling", "store"})
|
||||
@Tags({ "dbcp", "jdbc", "database", "connection", "pooling", "store" })
|
||||
@CapabilityDescription("Provides Database Connection Pooling Service. Connections can be asked from pool and returned after usage.")
|
||||
public class DBCPConnectionPool extends AbstractControllerService implements DBCPService {
|
||||
|
||||
public static final DatabaseSystemDescriptor DEFAULT_DATABASE_SYSTEM = DatabaseSystems.getDescriptor("JavaDB");
|
||||
|
||||
public static final PropertyDescriptor DATABASE_SYSTEM = new PropertyDescriptor.Builder()
|
||||
.name("Database Type")
|
||||
.description("Database management system")
|
||||
.allowableValues(DatabaseSystems.knownDatabaseSystems)
|
||||
.defaultValue(DEFAULT_DATABASE_SYSTEM.getValue())
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor DB_HOST = new PropertyDescriptor.Builder()
|
||||
.name("Database Host")
|
||||
.description("Database Host")
|
||||
.defaultValue(null)
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor DB_PORT = new PropertyDescriptor.Builder()
|
||||
.name("Database Port")
|
||||
.description("Database server port")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.PORT_VALIDATOR)
|
||||
.build();
|
||||
public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
|
||||
.name("Database Connection URL")
|
||||
.description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
|
||||
+ " The exact syntax of a database connection URL is specified by your DBMS.")
|
||||
.defaultValue(null)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.required(true)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor DB_DRIVERNAME = new PropertyDescriptor.Builder()
|
||||
.name("Database Driver Class Name")
|
||||
.description("Database driver class name")
|
||||
.defaultValue(DEFAULT_DATABASE_SYSTEM.driverClassName)
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
.name("Database Driver Class Name")
|
||||
.description("Database driver class name")
|
||||
.defaultValue(null)
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor DB_DRIVER_JAR_URL = new PropertyDescriptor.Builder()
|
||||
.name("Database Driver Jar Url")
|
||||
.description("Optional database driver jar file path url. For example 'file:///var/tmp/mariadb-java-client-1.1.7.jar'")
|
||||
.defaultValue(null)
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.URL_VALIDATOR)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor DB_NAME = new PropertyDescriptor.Builder()
|
||||
.name("Database Name")
|
||||
.description("Database name")
|
||||
.defaultValue(null)
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
.name("Database Driver Jar Url")
|
||||
.description("Optional database driver jar file path url. For example 'file:///var/tmp/mariadb-java-client-1.1.7.jar'")
|
||||
.defaultValue(null)
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.URL_VALIDATOR)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
|
||||
.name("Database User")
|
||||
.description("Database user name")
|
||||
.defaultValue(null)
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
.name("Database User")
|
||||
.description("Database user name")
|
||||
.defaultValue(null)
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
|
||||
.name("Password")
|
||||
.description("The password for the database user")
|
||||
.defaultValue(null)
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.sensitive(true)
|
||||
.build();
|
||||
.name("Password")
|
||||
.description("The password for the database user")
|
||||
.defaultValue(null)
|
||||
.required(false)
|
||||
.sensitive(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
|
||||
.name("Max Wait Time")
|
||||
.description("The maximum amount of time that the pool will wait (when there are no available connections) "
|
||||
+ " for a connection to be returned before failing, or -1 to wait indefinitely. ")
|
||||
.name("Max Wait Time")
|
||||
.description("The maximum amount of time that the pool will wait (when there are no available connections) "
|
||||
+ " for a connection to be returned before failing, or -1 to wait indefinitely. ")
|
||||
.defaultValue("500 millis")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||
|
@ -126,9 +101,9 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
|
|||
.build();
|
||||
|
||||
public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
|
||||
.name("Max Total Connections")
|
||||
.description("The maximum number of active connections that can be allocated from this pool at the same time, "
|
||||
+ " or negative for no limit.")
|
||||
.name("Max Total Connections")
|
||||
.description("The maximum number of active connections that can be allocated from this pool at the same time, "
|
||||
+ " or negative for no limit.")
|
||||
.defaultValue("8")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.INTEGER_VALIDATOR)
|
||||
|
@ -138,13 +113,10 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
|
|||
private static final List<PropertyDescriptor> properties;
|
||||
|
||||
static {
|
||||
List<PropertyDescriptor> props = new ArrayList<>();
|
||||
props.add(DATABASE_SYSTEM);
|
||||
props.add(DB_HOST);
|
||||
props.add(DB_PORT);
|
||||
final List<PropertyDescriptor> props = new ArrayList<>();
|
||||
props.add(DATABASE_URL);
|
||||
props.add(DB_DRIVERNAME);
|
||||
props.add(DB_DRIVER_JAR_URL);
|
||||
props.add(DB_NAME);
|
||||
props.add(DB_USER);
|
||||
props.add(DB_PASSWORD);
|
||||
props.add(MAX_WAIT_TIME);
|
||||
|
@ -162,30 +134,29 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
|
|||
|
||||
/**
|
||||
* Create new pool, open some connections ready to be used
|
||||
* @param context the configuration context
|
||||
* @throws InitializationException if unable to create a database connection
|
||||
*
|
||||
* @param context
|
||||
* the configuration context
|
||||
* @throws InitializationException
|
||||
* if unable to create a database connection
|
||||
*/
|
||||
@OnEnabled
|
||||
public void onConfigured(final ConfigurationContext context) throws InitializationException {
|
||||
DatabaseSystemDescriptor dbsystem = DatabaseSystems.getDescriptor( context.getProperty(DATABASE_SYSTEM).getValue() );
|
||||
|
||||
String host = context.getProperty(DB_HOST).getValue();
|
||||
Integer port = context.getProperty(DB_PORT).asInteger();
|
||||
String drv = context.getProperty(DB_DRIVERNAME).getValue();
|
||||
String dbname = context.getProperty(DB_NAME).getValue();
|
||||
String user = context.getProperty(DB_USER).getValue();
|
||||
String passw = context.getProperty(DB_PASSWORD).getValue();
|
||||
Long maxWaitMillis = context.getProperty(MAX_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
|
||||
Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).asInteger();
|
||||
final String drv = context.getProperty(DB_DRIVERNAME).getValue();
|
||||
final String user = context.getProperty(DB_USER).getValue();
|
||||
final String passw = context.getProperty(DB_PASSWORD).getValue();
|
||||
final Long maxWaitMillis = context.getProperty(MAX_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
|
||||
final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).asInteger();
|
||||
|
||||
dataSource = new BasicDataSource();
|
||||
dataSource.setDriverClassName(drv);
|
||||
|
||||
// Optional driver URL, when exist, this URL will be used to locate driver jar file location
|
||||
String urlString = context.getProperty(DB_DRIVER_JAR_URL).getValue();
|
||||
dataSource.setDriverClassLoader( getDriverClassLoader(urlString, drv) );
|
||||
final String urlString = context.getProperty(DB_DRIVER_JAR_URL).getValue();
|
||||
dataSource.setDriverClassLoader(getDriverClassLoader(urlString, drv));
|
||||
|
||||
String dburl = dbsystem.buildUrl(host, port, dbname);
|
||||
final String dburl = context.getProperty(DATABASE_URL).getValue();
|
||||
|
||||
dataSource.setMaxWait(maxWaitMillis);
|
||||
dataSource.setMaxActive(maxTotal);
|
||||
|
@ -196,40 +167,41 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
|
|||
|
||||
// verify connection can be established.
|
||||
try {
|
||||
Connection con = dataSource.getConnection();
|
||||
if (con==null) {
|
||||
final Connection con = dataSource.getConnection();
|
||||
if (con == null) {
|
||||
throw new InitializationException("Connection to database cannot be established.");
|
||||
}
|
||||
con.close();
|
||||
} catch (SQLException e) {
|
||||
} catch (final SQLException e) {
|
||||
throw new InitializationException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* using Thread.currentThread().getContextClassLoader();
|
||||
* will ensure that you are using the ClassLoader for you NAR.
|
||||
* @throws InitializationException if there is a problem obtaining the ClassLoader
|
||||
* using Thread.currentThread().getContextClassLoader(); will ensure that you are using the ClassLoader for you NAR.
|
||||
*
|
||||
* @throws InitializationException
|
||||
* if there is a problem obtaining the ClassLoader
|
||||
*/
|
||||
protected ClassLoader getDriverClassLoader(String urlString, String drvName) throws InitializationException {
|
||||
if (urlString!=null && urlString.length()>0) {
|
||||
if (urlString != null && urlString.length() > 0) {
|
||||
try {
|
||||
URL[] urls = new URL[] { new URL(urlString) };
|
||||
URLClassLoader ucl = new URLClassLoader(urls);
|
||||
final URL[] urls = new URL[] { new URL(urlString) };
|
||||
final URLClassLoader ucl = new URLClassLoader(urls);
|
||||
|
||||
// Workaround which allows to use URLClassLoader for JDBC driver loading.
|
||||
// (Because the DriverManager will refuse to use a driver not loaded by the system ClassLoader.)
|
||||
Class<?> clazz = Class.forName(drvName, true, ucl);
|
||||
if (clazz==null) {
|
||||
final Class<?> clazz = Class.forName(drvName, true, ucl);
|
||||
if (clazz == null) {
|
||||
throw new InitializationException("Can't load Database Driver " + drvName);
|
||||
}
|
||||
Driver driver = (Driver) clazz.newInstance();
|
||||
DriverManager.registerDriver( new DriverShim(driver) );
|
||||
final Driver driver = (Driver) clazz.newInstance();
|
||||
DriverManager.registerDriver(new DriverShim(driver));
|
||||
|
||||
return ucl;
|
||||
} catch (MalformedURLException e) {
|
||||
} catch (final MalformedURLException e) {
|
||||
throw new InitializationException("Invalid Database Driver Jar Url", e);
|
||||
} catch (Exception e) {
|
||||
} catch (final Exception e) {
|
||||
throw new InitializationException("Can't load Database Driver", e);
|
||||
}
|
||||
} else {
|
||||
|
@ -239,24 +211,23 @@ public class DBCPConnectionPool extends AbstractControllerService implements DBC
|
|||
}
|
||||
|
||||
/**
|
||||
* Shutdown pool, close all open connections.
|
||||
* Shutdown pool, close all open connections.
|
||||
*/
|
||||
@OnDisabled
|
||||
public void shutdown() {
|
||||
try {
|
||||
dataSource.close();
|
||||
} catch (SQLException e) {
|
||||
} catch (final SQLException e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Connection getConnection() throws ProcessException {
|
||||
try {
|
||||
Connection con = dataSource.getConnection();
|
||||
final Connection con = dataSource.getConnection();
|
||||
return con;
|
||||
} catch (SQLException e) {
|
||||
} catch (final SQLException e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.nifi.components.AllowableValue;
|
|||
* An immutable object for holding information about a database system.
|
||||
*
|
||||
*/
|
||||
@Deprecated
|
||||
public class DatabaseSystemDescriptor extends AllowableValue {
|
||||
|
||||
public final String driverClassName;
|
||||
|
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.nifi.dbcp;
|
||||
|
||||
@Deprecated
|
||||
public class DatabaseSystems {
|
||||
|
||||
/**
|
||||
|
|
|
@ -16,7 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.dbcp;
|
||||
|
||||
import static org.apache.nifi.dbcp.DatabaseSystems.getDescriptor;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
||||
|
@ -53,23 +52,8 @@ public class DBCPServiceTest {
|
|||
System.setProperty("derby.stream.error.file", "target/derby.log");
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Unknown database system.
|
||||
*
|
||||
*/
|
||||
@Test
|
||||
public void testUnknownDatabaseSystem() throws InitializationException {
|
||||
final TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
|
||||
final DBCPConnectionPool service = new DBCPConnectionPool();
|
||||
final Map<String, String> properties = new HashMap<String, String>();
|
||||
properties.put(DBCPConnectionPool.DATABASE_SYSTEM.getName(), "garbage");
|
||||
runner.addControllerService("test-bad2", service, properties);
|
||||
runner.assertNotValid(service);
|
||||
}
|
||||
|
||||
/**
|
||||
* Missing property values.
|
||||
* Missing property values.
|
||||
*/
|
||||
@Test
|
||||
public void testMissingPropertyValues() throws InitializationException {
|
||||
|
@ -81,8 +65,7 @@ public class DBCPServiceTest {
|
|||
}
|
||||
|
||||
/**
|
||||
* Test database connection using Derby.
|
||||
* Connect, create table, insert, select, drop table.
|
||||
* Test database connection using Derby. Connect, create table, insert, select, drop table.
|
||||
*
|
||||
*/
|
||||
@Test
|
||||
|
@ -92,25 +75,21 @@ public class DBCPServiceTest {
|
|||
runner.addControllerService("test-good1", service);
|
||||
|
||||
// remove previous test database, if any
|
||||
File dbLocation = new File(DB_LOCATION);
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
// Should setProperty call also generate DBCPConnectionPool.onPropertyModified() method call?
|
||||
// It does not currently.
|
||||
|
||||
// Some properties already should have JavaDB/Derby default values, let's set only missing values.
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_HOST, "NA"); // Embedded Derby don't use host
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_PORT, "1"); // Embedded Derby don't use port, but must have value anyway
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_NAME, DB_LOCATION);
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
|
||||
// set embedded Derby database connection url
|
||||
runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
|
||||
|
||||
runner.enableControllerService(service);
|
||||
|
||||
runner.assertValid(service);
|
||||
DBCPService dbcpService = (DBCPService) runner.getProcessContext().getControllerServiceLookup().getControllerService("test-good1");
|
||||
final DBCPService dbcpService = (DBCPService) runner.getProcessContext().getControllerServiceLookup().getControllerService("test-good1");
|
||||
Assert.assertNotNull(dbcpService);
|
||||
Connection connection = dbcpService.getConnection();
|
||||
final Connection connection = dbcpService.getConnection();
|
||||
Assert.assertNotNull(connection);
|
||||
|
||||
createInsertSelectDrop(connection);
|
||||
|
@ -119,12 +98,9 @@ public class DBCPServiceTest {
|
|||
}
|
||||
|
||||
/**
|
||||
* NB!!!!
|
||||
* Prerequisite: file should be present in /var/tmp/mariadb-java-client-1.1.7.jar
|
||||
* Prerequisite: access to running MariaDb database server
|
||||
* NB!!!! Prerequisite: file should be present in /var/tmp/mariadb-java-client-1.1.7.jar Prerequisite: access to running MariaDb database server
|
||||
*
|
||||
* Test database connection using external JDBC jar located by URL.
|
||||
* Connect, create table, insert, select, drop table.
|
||||
* Test database connection using external JDBC jar located by URL. Connect, create table, insert, select, drop table.
|
||||
*
|
||||
*/
|
||||
@Ignore
|
||||
|
@ -134,42 +110,32 @@ public class DBCPServiceTest {
|
|||
final DBCPConnectionPool service = new DBCPConnectionPool();
|
||||
runner.addControllerService("test-external-jar", service);
|
||||
|
||||
DatabaseSystemDescriptor mariaDb = getDescriptor("MariaDB");
|
||||
assertNotNull(mariaDb);
|
||||
|
||||
// Set MariaDB properties values.
|
||||
runner.setProperty(service, DBCPConnectionPool.DATABASE_SYSTEM, mariaDb.getValue());
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_PORT, mariaDb.defaultPort.toString());
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, mariaDb.driverClassName);
|
||||
// set MariaDB database connection url
|
||||
runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:mariadb://localhost:3306/" + "testdb");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.mariadb.jdbc.Driver");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_DRIVER_JAR_URL, "file:///var/tmp/mariadb-java-client-1.1.7.jar");
|
||||
|
||||
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_HOST, "localhost"); // localhost
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_NAME, "testdb");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
|
||||
|
||||
runner.enableControllerService(service);
|
||||
|
||||
runner.assertValid(service);
|
||||
DBCPService dbcpService = (DBCPService) runner.getProcessContext().getControllerServiceLookup().getControllerService("test-external-jar");
|
||||
final DBCPService dbcpService = (DBCPService) runner.getProcessContext().getControllerServiceLookup().getControllerService("test-external-jar");
|
||||
Assert.assertNotNull(dbcpService);
|
||||
Connection connection = dbcpService.getConnection();
|
||||
final Connection connection = dbcpService.getConnection();
|
||||
Assert.assertNotNull(connection);
|
||||
|
||||
createInsertSelectDrop(connection);
|
||||
|
||||
connection.close(); // return to pool
|
||||
connection.close(); // return to pool
|
||||
}
|
||||
|
||||
|
||||
@Rule
|
||||
public ExpectedException exception = ExpectedException.none();
|
||||
|
||||
/**
|
||||
* Test get database connection using Derby.
|
||||
* Get many times, after a while pool should not contain any available connection
|
||||
* and getConnection should fail.
|
||||
* Test get database connection using Derby. Get many times, after a while pool should not contain any available connection and getConnection should fail.
|
||||
*/
|
||||
@Test
|
||||
public void testExhaustPool() throws InitializationException, SQLException {
|
||||
|
@ -178,33 +144,30 @@ public class DBCPServiceTest {
|
|||
runner.addControllerService("test-exhaust", service);
|
||||
|
||||
// remove previous test database, if any
|
||||
File dbLocation = new File(DB_LOCATION);
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_HOST, "NA"); // Embedded Derby don't use host
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_PORT, "1"); // Embedded Derby don't use port, but must have value anyway
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_NAME, DB_LOCATION);
|
||||
// set embedded Derby database connection url
|
||||
runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
|
||||
|
||||
runner.enableControllerService(service);
|
||||
|
||||
runner.assertValid(service);
|
||||
DBCPService dbcpService = (DBCPService) runner.getProcessContext().getControllerServiceLookup().getControllerService("test-exhaust");
|
||||
final DBCPService dbcpService = (DBCPService) runner.getProcessContext().getControllerServiceLookup().getControllerService("test-exhaust");
|
||||
Assert.assertNotNull(dbcpService);
|
||||
|
||||
exception.expect(ProcessException.class);
|
||||
exception.expectMessage("Cannot get a connection, pool error Timeout waiting for idle object");
|
||||
for (int i = 0; i < 100; i++) {
|
||||
Connection connection = dbcpService.getConnection();
|
||||
final Connection connection = dbcpService.getConnection();
|
||||
Assert.assertNotNull(connection);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test get database connection using Derby.
|
||||
* Get many times, release immediately
|
||||
* and getConnection should not fail.
|
||||
* Test get database connection using Derby. Get many times, release immediately and getConnection should not fail.
|
||||
*/
|
||||
@Test
|
||||
public void testGetManyNormal() throws InitializationException, SQLException {
|
||||
|
@ -213,103 +176,98 @@ public class DBCPServiceTest {
|
|||
runner.addControllerService("test-exhaust", service);
|
||||
|
||||
// remove previous test database, if any
|
||||
File dbLocation = new File(DB_LOCATION);
|
||||
final File dbLocation = new File(DB_LOCATION);
|
||||
dbLocation.delete();
|
||||
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_HOST, "NA"); // Embedded Derby don't use host
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_PORT, "1"); // Embedded Derby don't use port, but must have value anyway
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_NAME, DB_LOCATION);
|
||||
// set embedded Derby database connection url
|
||||
runner.setProperty(service, DBCPConnectionPool.DATABASE_URL, "jdbc:derby:" + DB_LOCATION + ";create=true");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_USER, "tester");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_PASSWORD, "testerp");
|
||||
runner.setProperty(service, DBCPConnectionPool.DB_DRIVERNAME, "org.apache.derby.jdbc.EmbeddedDriver");
|
||||
|
||||
runner.enableControllerService(service);
|
||||
|
||||
runner.assertValid(service);
|
||||
DBCPService dbcpService = (DBCPService) runner.getProcessContext().getControllerServiceLookup().getControllerService("test-exhaust");
|
||||
final DBCPService dbcpService = (DBCPService) runner.getProcessContext().getControllerServiceLookup().getControllerService("test-exhaust");
|
||||
Assert.assertNotNull(dbcpService);
|
||||
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
Connection connection = dbcpService.getConnection();
|
||||
final Connection connection = dbcpService.getConnection();
|
||||
Assert.assertNotNull(connection);
|
||||
connection.close(); // will return connection to pool
|
||||
connection.close(); // will return connection to pool
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testDriverLoad() throws ClassNotFoundException {
|
||||
Class<?> clazz = Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
|
||||
final Class<?> clazz = Class.forName("org.apache.derby.jdbc.EmbeddedDriver");
|
||||
assertNotNull(clazz);
|
||||
}
|
||||
|
||||
/**
|
||||
* NB!!!!
|
||||
* Prerequisite: file should be present in /var/tmp/mariadb-java-client-1.1.7.jar
|
||||
* NB!!!! Prerequisite: file should be present in /var/tmp/mariadb-java-client-1.1.7.jar
|
||||
*/
|
||||
@Test
|
||||
@Ignore("Intended only for local testing, not automated testing")
|
||||
public void testURLClassLoader() throws ClassNotFoundException, MalformedURLException, SQLException, InstantiationException, IllegalAccessException {
|
||||
|
||||
URL url = new URL("file:///var/tmp/mariadb-java-client-1.1.7.jar");
|
||||
URL[] urls = new URL[] { url };
|
||||
final URL url = new URL("file:///var/tmp/mariadb-java-client-1.1.7.jar");
|
||||
final URL[] urls = new URL[] { url };
|
||||
|
||||
ClassLoader parent = Thread.currentThread().getContextClassLoader();
|
||||
URLClassLoader ucl = new URLClassLoader(urls,parent);
|
||||
final ClassLoader parent = Thread.currentThread().getContextClassLoader();
|
||||
final URLClassLoader ucl = new URLClassLoader(urls, parent);
|
||||
|
||||
Class<?> clazz = Class.forName("org.mariadb.jdbc.Driver", true, ucl);
|
||||
final Class<?> clazz = Class.forName("org.mariadb.jdbc.Driver", true, ucl);
|
||||
assertNotNull(clazz);
|
||||
|
||||
Driver driver = (Driver) clazz.newInstance();
|
||||
Driver shim = new DriverShim(driver);
|
||||
DriverManager.registerDriver( shim );
|
||||
final Driver driver = (Driver) clazz.newInstance();
|
||||
final Driver shim = new DriverShim(driver);
|
||||
DriverManager.registerDriver(shim);
|
||||
|
||||
Driver driver2 = DriverManager.getDriver("jdbc:mariadb://localhost:3306/testdb");
|
||||
final Driver driver2 = DriverManager.getDriver("jdbc:mariadb://localhost:3306/testdb");
|
||||
assertNotNull(driver2);
|
||||
}
|
||||
|
||||
/**
|
||||
* NB!!!!
|
||||
* Prerequisite: file should be present in /var/tmp/mariadb-java-client-1.1.7.jar
|
||||
* Prerequisite: access to running MariaDb database server
|
||||
* NB!!!! Prerequisite: file should be present in /var/tmp/mariadb-java-client-1.1.7.jar Prerequisite: access to running MariaDb database server
|
||||
*/
|
||||
@Test
|
||||
@Ignore("Intended only for local testing, not automated testing")
|
||||
public void testURLClassLoaderGetConnection() throws ClassNotFoundException, MalformedURLException, SQLException, InstantiationException, IllegalAccessException {
|
||||
|
||||
URL url = new URL("file:///var/tmp/mariadb-java-client-1.1.7.jar");
|
||||
URL[] urls = new URL[] { url };
|
||||
final URL url = new URL("file:///var/tmp/mariadb-java-client-1.1.7.jar");
|
||||
final URL[] urls = new URL[] { url };
|
||||
|
||||
ClassLoader parent = Thread.currentThread().getContextClassLoader();
|
||||
URLClassLoader ucl = new URLClassLoader(urls,parent);
|
||||
final ClassLoader parent = Thread.currentThread().getContextClassLoader();
|
||||
final URLClassLoader ucl = new URLClassLoader(urls, parent);
|
||||
|
||||
Class<?> clazz = Class.forName("org.mariadb.jdbc.Driver", true, ucl);
|
||||
final Class<?> clazz = Class.forName("org.mariadb.jdbc.Driver", true, ucl);
|
||||
assertNotNull(clazz);
|
||||
|
||||
Driver driver = (Driver) clazz.newInstance();
|
||||
Driver shim = new DriverShim(driver);
|
||||
DriverManager.registerDriver( shim );
|
||||
final Driver driver = (Driver) clazz.newInstance();
|
||||
final Driver shim = new DriverShim(driver);
|
||||
DriverManager.registerDriver(shim);
|
||||
|
||||
Driver driver2 = DriverManager.getDriver("jdbc:mariadb://localhost:3306/testdb");
|
||||
final Driver driver2 = DriverManager.getDriver("jdbc:mariadb://localhost:3306/testdb");
|
||||
assertNotNull(driver2);
|
||||
|
||||
Connection connection = DriverManager.getConnection("jdbc:mariadb://localhost:3306/testdb","tester","testerp");
|
||||
final Connection connection = DriverManager.getConnection("jdbc:mariadb://localhost:3306/testdb", "tester", "testerp");
|
||||
assertNotNull(connection);
|
||||
connection.close();
|
||||
|
||||
DriverManager.deregisterDriver(shim);
|
||||
}
|
||||
|
||||
|
||||
String createTable = "create table restaurants(id integer, name varchar(20), city varchar(50))";
|
||||
String dropTable = "drop table restaurants";
|
||||
|
||||
protected void createInsertSelectDrop( Connection con) throws SQLException {
|
||||
protected void createInsertSelectDrop(Connection con) throws SQLException {
|
||||
|
||||
Statement st = con.createStatement();
|
||||
final Statement st = con.createStatement();
|
||||
|
||||
try {
|
||||
st.executeUpdate(dropTable);
|
||||
} catch (Exception e) {
|
||||
} catch (final Exception e) {
|
||||
// table may not exist, this is not serious problem.
|
||||
}
|
||||
|
||||
|
@ -320,7 +278,7 @@ public class DBCPServiceTest {
|
|||
st.executeUpdate("insert into restaurants values (3, 'Prime Rib House', 'San Francisco')");
|
||||
|
||||
int nrOfRows = 0;
|
||||
ResultSet resultSet = st.executeQuery("select * from restaurants");
|
||||
final ResultSet resultSet = st.executeQuery("select * from restaurants");
|
||||
while (resultSet.next())
|
||||
nrOfRows++;
|
||||
assertEquals(3, nrOfRows);
|
||||
|
|
|
@ -52,7 +52,6 @@ public abstract class AbstractCacheServer implements CacheServer {
|
|||
private final SSLContext sslContext;
|
||||
protected volatile boolean stopped = false;
|
||||
private final Set<Thread> processInputThreads = new CopyOnWriteArraySet<>();
|
||||
;
|
||||
|
||||
private volatile ServerSocketChannel serverSocketChannel;
|
||||
|
||||
|
@ -62,6 +61,11 @@ public abstract class AbstractCacheServer implements CacheServer {
|
|||
this.sslContext = sslContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPort() {
|
||||
return serverSocketChannel == null ? this.port : serverSocketChannel.socket().getLocalPort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() throws IOException {
|
||||
serverSocketChannel = ServerSocketChannel.open();
|
||||
|
@ -117,7 +121,7 @@ public abstract class AbstractCacheServer implements CacheServer {
|
|||
return;
|
||||
}
|
||||
try (final InputStream in = new BufferedInputStream(rawInputStream);
|
||||
final OutputStream out = new BufferedOutputStream(rawOutputStream)) {
|
||||
final OutputStream out = new BufferedOutputStream(rawOutputStream)) {
|
||||
|
||||
final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(1);
|
||||
|
||||
|
|
|
@ -24,4 +24,6 @@ public interface CacheServer {
|
|||
|
||||
void stop() throws IOException;
|
||||
|
||||
int getPort();
|
||||
|
||||
}
|
||||
|
|
|
@ -35,39 +35,39 @@ public abstract class DistributedCacheServer extends AbstractControllerService {
|
|||
public static final String EVICTION_STRATEGY_FIFO = "First In, First Out";
|
||||
|
||||
public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
|
||||
.name("Port")
|
||||
.description("The port to listen on for incoming connections")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.PORT_VALIDATOR)
|
||||
.defaultValue("4557")
|
||||
.build();
|
||||
.name("Port")
|
||||
.description("The port to listen on for incoming connections")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.PORT_VALIDATOR)
|
||||
.defaultValue("4557")
|
||||
.build();
|
||||
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
||||
.name("SSL Context Service")
|
||||
.description("If specified, this service will be used to create an SSL Context that will be used "
|
||||
+ "to secure communications; if not specified, communications will not be secure")
|
||||
.required(false)
|
||||
.identifiesControllerService(SSLContextService.class)
|
||||
.build();
|
||||
.name("SSL Context Service")
|
||||
.description("If specified, this service will be used to create an SSL Context that will be used "
|
||||
+ "to secure communications; if not specified, communications will not be secure")
|
||||
.required(false)
|
||||
.identifiesControllerService(SSLContextService.class)
|
||||
.build();
|
||||
public static final PropertyDescriptor MAX_CACHE_ENTRIES = new PropertyDescriptor.Builder()
|
||||
.name("Maximum Cache Entries")
|
||||
.description("The maximum number of cache entries that the cache can hold")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.defaultValue("10000")
|
||||
.build();
|
||||
.name("Maximum Cache Entries")
|
||||
.description("The maximum number of cache entries that the cache can hold")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||
.defaultValue("10000")
|
||||
.build();
|
||||
public static final PropertyDescriptor EVICTION_POLICY = new PropertyDescriptor.Builder()
|
||||
.name("Eviction Strategy")
|
||||
.description("Determines which strategy should be used to evict values from the cache to make room for new entries")
|
||||
.required(true)
|
||||
.allowableValues(EVICTION_STRATEGY_LFU, EVICTION_STRATEGY_LRU, EVICTION_STRATEGY_FIFO)
|
||||
.defaultValue(EVICTION_STRATEGY_LFU)
|
||||
.build();
|
||||
.name("Eviction Strategy")
|
||||
.description("Determines which strategy should be used to evict values from the cache to make room for new entries")
|
||||
.required(true)
|
||||
.allowableValues(EVICTION_STRATEGY_LFU, EVICTION_STRATEGY_LRU, EVICTION_STRATEGY_FIFO)
|
||||
.defaultValue(EVICTION_STRATEGY_LFU)
|
||||
.build();
|
||||
public static final PropertyDescriptor PERSISTENCE_PATH = new PropertyDescriptor.Builder()
|
||||
.name("Persistence Directory")
|
||||
.description("If specified, the cache will be persisted in the given directory; if not specified, the cache will be in-memory only")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.createDirectoryExistsValidator(true, true))
|
||||
.build();
|
||||
.name("Persistence Directory")
|
||||
.description("If specified, the cache will be persisted in the given directory; if not specified, the cache will be in-memory only")
|
||||
.required(false)
|
||||
.addValidator(StandardValidators.createDirectoryExistsValidator(true, true))
|
||||
.build();
|
||||
|
||||
private volatile CacheServer cacheServer;
|
||||
|
||||
|
@ -103,5 +103,12 @@ public abstract class DistributedCacheServer extends AbstractControllerService {
|
|||
shutdownServer();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the port that the server is listening on, or -1 if the server has not been started
|
||||
*/
|
||||
public int getPort() {
|
||||
return cacheServer == null ? -1 : cacheServer.getPort();
|
||||
}
|
||||
|
||||
protected abstract CacheServer createCacheServer(ConfigurationContext context);
|
||||
}
|
||||
|
|
|
@ -16,8 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.distributed.cache.server;
|
||||
|
||||
import org.apache.commons.lang3.SystemUtils;
|
||||
import org.apache.nifi.distributed.cache.server.DistributedSetCacheServer;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
@ -29,8 +27,11 @@ import java.io.OutputStream;
|
|||
import java.net.ConnectException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.commons.lang3.SerializationException;
|
||||
import org.apache.commons.lang3.SystemUtils;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.distributed.cache.client.Deserializer;
|
||||
import org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService;
|
||||
|
@ -38,16 +39,16 @@ import org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService
|
|||
import org.apache.nifi.distributed.cache.client.Serializer;
|
||||
import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
|
||||
import org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer;
|
||||
import org.apache.nifi.processor.Processor;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.reporting.InitializationException;
|
||||
import org.apache.nifi.ssl.SSLContextService.ClientAuth;
|
||||
import org.apache.nifi.ssl.StandardSSLContextService;
|
||||
import org.apache.nifi.util.MockConfigurationContext;
|
||||
import org.apache.nifi.util.MockControllerServiceInitializationContext;
|
||||
|
||||
import org.apache.commons.lang3.SerializationException;
|
||||
import org.apache.nifi.util.TestRunner;
|
||||
import org.apache.nifi.util.TestRunners;
|
||||
import org.junit.Assume;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -65,20 +66,24 @@ public class TestServerAndClient {
|
|||
LOGGER = LoggerFactory.getLogger(TestServerAndClient.class);
|
||||
}
|
||||
|
||||
@Ignore("Test fails when in a maven parallel build due to address/port already taken - need to vary these so tests can run in parallel")
|
||||
@Test
|
||||
public void testNonPersistentSetServerAndClient() throws InitializationException, IOException {
|
||||
|
||||
/**
|
||||
* This bypasses the test for build environments in OS X running Java 1.8 due to a JVM bug
|
||||
* See: https://issues.apache.org/jira/browse/NIFI-437
|
||||
*/
|
||||
Assume.assumeFalse("test is skipped due to build environment being OS X with JDK 1.8. See https://issues.apache.org/jira/browse/NIFI-437",
|
||||
SystemUtils.IS_OS_MAC && SystemUtils.IS_JAVA_1_8);
|
||||
|
||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||
// Create server
|
||||
final DistributedSetCacheServer server = new DistributedSetCacheServer();
|
||||
MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
|
||||
server.initialize(serverInitContext);
|
||||
final TestRunner runner = TestRunners.newTestRunner(Mockito.mock(Processor.class));
|
||||
final DistributedSetCacheServer server = new SetServer();
|
||||
runner.addControllerService("server", server);
|
||||
runner.enableControllerService(server);
|
||||
|
||||
final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
|
||||
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
|
||||
server.startServer(serverContext);
|
||||
|
||||
final DistributedSetCacheClientService client = createClient();
|
||||
final DistributedSetCacheClientService client = createClient(server.getPort());
|
||||
final Serializer<String> serializer = new StringSerializer();
|
||||
final boolean added = client.addIfAbsent("test", serializer);
|
||||
assertTrue(added);
|
||||
|
@ -98,24 +103,28 @@ public class TestServerAndClient {
|
|||
server.shutdownServer();
|
||||
}
|
||||
|
||||
@Ignore("Test fails when in a maven parallel build due to address/port already taken - need to vary these so tests can run in parallel")
|
||||
@Test
|
||||
public void testPersistentSetServerAndClient() throws InitializationException, IOException {
|
||||
/**
|
||||
* This bypasses the test for build environments in OS X running Java 1.8 due to a JVM bug
|
||||
* See: https://issues.apache.org/jira/browse/NIFI-437
|
||||
*/
|
||||
Assume.assumeFalse("test is skipped due to build environment being OS X with JDK 1.8. See https://issues.apache.org/jira/browse/NIFI-437",
|
||||
SystemUtils.IS_OS_MAC && SystemUtils.IS_JAVA_1_8);
|
||||
|
||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||
// Create server
|
||||
final DistributedSetCacheServer server = new DistributedSetCacheServer();
|
||||
MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
|
||||
server.initialize(serverInitContext);
|
||||
|
||||
final File dataFile = new File("target/cache-data");
|
||||
deleteRecursively(dataFile);
|
||||
|
||||
final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
|
||||
serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
|
||||
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
|
||||
server.startServer(serverContext);
|
||||
// Create server
|
||||
final TestRunner runner = TestRunners.newTestRunner(Mockito.mock(Processor.class));
|
||||
final DistributedSetCacheServer server = new SetServer();
|
||||
runner.addControllerService("server", server);
|
||||
runner.setProperty(server, DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
|
||||
runner.enableControllerService(server);
|
||||
|
||||
final DistributedSetCacheClientService client = createClient();
|
||||
DistributedSetCacheClientService client = createClient(server.getPort());
|
||||
final Serializer<String> serializer = new StringSerializer();
|
||||
final boolean added = client.addIfAbsent("test", serializer);
|
||||
final boolean added2 = client.addIfAbsent("test2", serializer);
|
||||
|
@ -137,41 +146,45 @@ public class TestServerAndClient {
|
|||
assertFalse(containedAfterRemove);
|
||||
|
||||
server.shutdownServer();
|
||||
client.close();
|
||||
|
||||
final DistributedSetCacheServer newServer = new DistributedSetCacheServer();
|
||||
MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2");
|
||||
newServer.initialize(newServerInitContext);
|
||||
|
||||
final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties,
|
||||
newServerInitContext.getControllerServiceLookup());
|
||||
newServer.startServer(newServerContext);
|
||||
final DistributedSetCacheServer newServer = new SetServer();
|
||||
runner.addControllerService("server2", newServer);
|
||||
runner.setProperty(newServer, DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
|
||||
runner.enableControllerService(newServer);
|
||||
client = createClient(newServer.getPort());
|
||||
|
||||
assertFalse(client.contains("test", serializer));
|
||||
assertTrue(client.contains("test2", serializer));
|
||||
|
||||
newServer.shutdownServer();
|
||||
client.close();
|
||||
}
|
||||
|
||||
@Ignore("Test fails when in a maven parallel build due to address/port already taken - need to vary these so tests can run in parallel")
|
||||
@Test
|
||||
public void testPersistentSetServerAndClientWithLFUEvictions() throws InitializationException, IOException {
|
||||
/**
|
||||
* This bypasses the test for build environments in OS X running Java 1.8 due to a JVM bug
|
||||
* See: https://issues.apache.org/jira/browse/NIFI-437
|
||||
*/
|
||||
Assume.assumeFalse("test is skipped due to build environment being OS X with JDK 1.8. See https://issues.apache.org/jira/browse/NIFI-437",
|
||||
SystemUtils.IS_OS_MAC && SystemUtils.IS_JAVA_1_8);
|
||||
|
||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||
// Create server
|
||||
final DistributedSetCacheServer server = new DistributedSetCacheServer();
|
||||
MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
|
||||
server.initialize(serverInitContext);
|
||||
|
||||
final File dataFile = new File("target/cache-data");
|
||||
deleteRecursively(dataFile);
|
||||
|
||||
final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
|
||||
serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
|
||||
serverProperties.put(DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3");
|
||||
// Create server
|
||||
final TestRunner runner = TestRunners.newTestRunner(Mockito.mock(Processor.class));
|
||||
final DistributedSetCacheServer server = new SetServer();
|
||||
runner.addControllerService("server", server);
|
||||
runner.setProperty(server, DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
|
||||
runner.setProperty(server, DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3");
|
||||
runner.setProperty(server, DistributedSetCacheServer.EVICTION_POLICY, DistributedSetCacheServer.EVICTION_STRATEGY_LFU);
|
||||
runner.enableControllerService(server);
|
||||
|
||||
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
|
||||
server.startServer(serverContext);
|
||||
|
||||
final DistributedSetCacheClientService client = createClient();
|
||||
DistributedSetCacheClientService client = createClient(server.getPort());
|
||||
final Serializer<String> serializer = new StringSerializer();
|
||||
final boolean added = client.addIfAbsent("test", serializer);
|
||||
waitABit();
|
||||
|
@ -199,13 +212,13 @@ public class TestServerAndClient {
|
|||
|
||||
server.shutdownServer();
|
||||
|
||||
final DistributedSetCacheServer newServer = new DistributedSetCacheServer();
|
||||
MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2");
|
||||
newServer.initialize(newServerInitContext);
|
||||
|
||||
final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties,
|
||||
newServerInitContext.getControllerServiceLookup());
|
||||
newServer.startServer(newServerContext);
|
||||
final DistributedSetCacheServer newServer = new SetServer();
|
||||
runner.addControllerService("server2", newServer);
|
||||
runner.setProperty(newServer, DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
|
||||
runner.enableControllerService(newServer);
|
||||
client.close();
|
||||
client = createClient(newServer.getPort());
|
||||
|
||||
assertTrue(client.contains("test", serializer));
|
||||
assertTrue(client.contains("test2", serializer));
|
||||
|
@ -213,29 +226,33 @@ public class TestServerAndClient {
|
|||
assertTrue(client.contains("test4", serializer));
|
||||
|
||||
newServer.shutdownServer();
|
||||
client.close();
|
||||
}
|
||||
|
||||
@Ignore("Test fails when in a maven parallel build due to address/port already taken - need to vary these so tests can run in parallel")
|
||||
@Test
|
||||
public void testPersistentSetServerAndClientWithFIFOEvictions() throws InitializationException, IOException {
|
||||
/**
|
||||
* This bypasses the test for build environments in OS X running Java 1.8 due to a JVM bug
|
||||
* See: https://issues.apache.org/jira/browse/NIFI-437
|
||||
*/
|
||||
Assume.assumeFalse("test is skipped due to build environment being OS X with JDK 1.8. See https://issues.apache.org/jira/browse/NIFI-437",
|
||||
SystemUtils.IS_OS_MAC && SystemUtils.IS_JAVA_1_8);
|
||||
|
||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||
// Create server
|
||||
final DistributedSetCacheServer server = new DistributedSetCacheServer();
|
||||
MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
|
||||
server.initialize(serverInitContext);
|
||||
|
||||
final File dataFile = new File("target/cache-data");
|
||||
deleteRecursively(dataFile);
|
||||
|
||||
final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
|
||||
serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
|
||||
serverProperties.put(DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3");
|
||||
serverProperties.put(DistributedSetCacheServer.EVICTION_POLICY, DistributedSetCacheServer.EVICTION_STRATEGY_FIFO);
|
||||
// Create server
|
||||
final TestRunner runner = TestRunners.newTestRunner(Mockito.mock(Processor.class));
|
||||
final DistributedSetCacheServer server = new SetServer();
|
||||
runner.addControllerService("server", server);
|
||||
runner.setProperty(server, DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
|
||||
runner.setProperty(server, DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3");
|
||||
runner.setProperty(server, DistributedSetCacheServer.EVICTION_POLICY, DistributedSetCacheServer.EVICTION_STRATEGY_FIFO);
|
||||
runner.enableControllerService(server);
|
||||
|
||||
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
|
||||
server.startServer(serverContext);
|
||||
|
||||
final DistributedSetCacheClientService client = createClient();
|
||||
DistributedSetCacheClientService client = createClient(server.getPort());
|
||||
final Serializer<String> serializer = new StringSerializer();
|
||||
|
||||
// add 3 entries to the cache. But, if we add too fast, we'll have the same millisecond
|
||||
|
@ -267,35 +284,42 @@ public class TestServerAndClient {
|
|||
assertTrue(client.contains("test3", serializer));
|
||||
|
||||
server.shutdownServer();
|
||||
client.close();
|
||||
|
||||
final DistributedSetCacheServer newServer = new DistributedSetCacheServer();
|
||||
MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2");
|
||||
newServer.initialize(newServerInitContext);
|
||||
|
||||
final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties,
|
||||
newServerInitContext.getControllerServiceLookup());
|
||||
newServer.startServer(newServerContext);
|
||||
final DistributedSetCacheServer newServer = new SetServer();
|
||||
runner.addControllerService("server2", newServer);
|
||||
runner.setProperty(newServer, DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
|
||||
runner.setProperty(newServer, DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3");
|
||||
runner.setProperty(newServer, DistributedSetCacheServer.EVICTION_POLICY, DistributedSetCacheServer.EVICTION_STRATEGY_FIFO);
|
||||
runner.enableControllerService(newServer);
|
||||
|
||||
client = createClient(newServer.getPort());
|
||||
assertFalse(client.contains("test", serializer));
|
||||
assertTrue(client.contains("test2", serializer));
|
||||
assertTrue(client.contains("test3", serializer));
|
||||
assertTrue(client.contains("test4", serializer));
|
||||
|
||||
newServer.shutdownServer();
|
||||
client.close();
|
||||
}
|
||||
|
||||
@Ignore("Test fails when in a maven parallel build due to address/port already taken - need to vary these so tests can run in parallel")
|
||||
@Test
|
||||
public void testNonPersistentMapServerAndClient() throws InitializationException, IOException, InterruptedException {
|
||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||
// Create server
|
||||
final DistributedMapCacheServer server = new DistributedMapCacheServer();
|
||||
MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
|
||||
server.initialize(serverInitContext);
|
||||
/**
|
||||
* This bypasses the test for build environments in OS X running Java 1.8 due to a JVM bug
|
||||
* See: https://issues.apache.org/jira/browse/NIFI-437
|
||||
*/
|
||||
Assume.assumeFalse("test is skipped due to build environment being OS X with JDK 1.8. See https://issues.apache.org/jira/browse/NIFI-437",
|
||||
SystemUtils.IS_OS_MAC && SystemUtils.IS_JAVA_1_8);
|
||||
|
||||
final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
|
||||
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
|
||||
server.startServer(serverContext);
|
||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||
|
||||
// Create server
|
||||
final DistributedMapCacheServer server = new MapServer();
|
||||
final TestRunner runner = TestRunners.newTestRunner(Mockito.mock(Processor.class));
|
||||
runner.addControllerService("server", server);
|
||||
runner.enableControllerService(server);
|
||||
|
||||
DistributedMapCacheClientService client = new DistributedMapCacheClientService();
|
||||
MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
|
||||
|
@ -303,6 +327,7 @@ public class TestServerAndClient {
|
|||
|
||||
final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
|
||||
clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
|
||||
clientProperties.put(DistributedMapCacheClientService.PORT, String.valueOf(server.getPort()));
|
||||
clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
|
||||
MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
|
||||
client.cacheConfig(clientContext);
|
||||
|
@ -338,7 +363,7 @@ public class TestServerAndClient {
|
|||
try {
|
||||
client.containsKey("testKey", keySerializer);
|
||||
fail("Should be closed and not accessible");
|
||||
} catch (Exception e) {
|
||||
} catch (final Exception e) {
|
||||
|
||||
}
|
||||
client = null;
|
||||
|
@ -346,12 +371,11 @@ public class TestServerAndClient {
|
|||
clientContext = null;
|
||||
|
||||
DistributedMapCacheClientService client2 = new DistributedMapCacheClientService();
|
||||
|
||||
MockControllerServiceInitializationContext clientInitContext2 = new MockControllerServiceInitializationContext(client2, "client2");
|
||||
client2.initialize(clientInitContext2);
|
||||
|
||||
MockConfigurationContext clientContext2 = new MockConfigurationContext(clientProperties,
|
||||
clientInitContext2.getControllerServiceLookup());
|
||||
clientInitContext2.getControllerServiceLookup());
|
||||
client2.cacheConfig(clientContext2);
|
||||
assertFalse(client2.putIfAbsent("testKey", "test", keySerializer, valueSerializer));
|
||||
assertTrue(client2.containsKey("testKey", keySerializer));
|
||||
|
@ -360,13 +384,11 @@ public class TestServerAndClient {
|
|||
try {
|
||||
client2.containsKey("testKey", keySerializer);
|
||||
fail("Should have blown exception!");
|
||||
} catch (ConnectException e) {
|
||||
} catch (final ConnectException e) {
|
||||
client2 = null;
|
||||
clientContext2 = null;
|
||||
clientInitContext2 = null;
|
||||
}
|
||||
Thread.sleep(2000);
|
||||
System.gc();
|
||||
LOGGER.debug("end testNonPersistentMapServerAndClient");
|
||||
}
|
||||
|
||||
|
@ -377,12 +399,12 @@ public class TestServerAndClient {
|
|||
* This bypasses the test for build environments in OS X running Java 1.8 due to a JVM bug See: https://issues.apache.org/jira/browse/NIFI-437
|
||||
*/
|
||||
Assume.assumeFalse("testClientTermination is skipped due to build environment being OS X with JDK 1.8. See https://issues.apache.org/jira/browse/NIFI-437",
|
||||
SystemUtils.IS_OS_MAC && SystemUtils.IS_JAVA_1_8);
|
||||
SystemUtils.IS_OS_MAC && SystemUtils.IS_JAVA_1_8);
|
||||
|
||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||
// Create server
|
||||
final DistributedMapCacheServer server = new DistributedMapCacheServer();
|
||||
MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
|
||||
final DistributedMapCacheServer server = new MapServer();
|
||||
final MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
|
||||
server.initialize(serverInitContext);
|
||||
|
||||
final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
|
||||
|
@ -428,65 +450,6 @@ public class TestServerAndClient {
|
|||
server.shutdownServer();
|
||||
}
|
||||
|
||||
@Ignore
|
||||
@Test
|
||||
public void testSSLWith2RequestsWithServerTimeout() throws InitializationException, IOException, InterruptedException {
|
||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||
// Create SSLContext Service
|
||||
final StandardSSLContextService sslService = new StandardSSLContextService();
|
||||
final MockControllerServiceInitializationContext sslServerInitContext = new MockControllerServiceInitializationContext(sslService,
|
||||
"ssl-context");
|
||||
sslService.initialize(sslServerInitContext);
|
||||
|
||||
final Map<PropertyDescriptor, String> sslServerProps = new HashMap<>();
|
||||
sslServerProps.put(StandardSSLContextService.KEYSTORE, "src/test/resources/localhost-ks.jks");
|
||||
sslServerProps.put(StandardSSLContextService.KEYSTORE_PASSWORD, "localtest");
|
||||
sslServerProps.put(StandardSSLContextService.KEYSTORE_TYPE, "JKS");
|
||||
sslServerProps.put(StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks");
|
||||
sslServerProps.put(StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest");
|
||||
sslServerProps.put(StandardSSLContextService.TRUSTSTORE_TYPE, "JKS");
|
||||
MockConfigurationContext sslServerContext = new MockConfigurationContext(sslServerProps, sslServerInitContext);
|
||||
sslService.onConfigured(sslServerContext);
|
||||
sslService.createSSLContext(ClientAuth.REQUIRED);
|
||||
// Create server
|
||||
final DistributedMapCacheServer server = new DistributedMapCacheServer();
|
||||
final MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
|
||||
server.initialize(serverInitContext);
|
||||
|
||||
final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
|
||||
serverProperties.put(DistributedMapCacheServer.SSL_CONTEXT_SERVICE, "ssl-context");
|
||||
final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
|
||||
server.startServer(serverContext);
|
||||
|
||||
DistributedMapCacheClientService client = new DistributedMapCacheClientService();
|
||||
MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
|
||||
client.initialize(clientInitContext);
|
||||
|
||||
final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
|
||||
clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
|
||||
clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
|
||||
clientProperties.put(DistributedMapCacheClientService.SSL_CONTEXT_SERVICE, "ssl-context");
|
||||
MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
|
||||
client.cacheConfig(clientContext);
|
||||
final Serializer<String> valueSerializer = new StringSerializer();
|
||||
final Serializer<String> keySerializer = new StringSerializer();
|
||||
final Deserializer<String> deserializer = new StringDeserializer();
|
||||
|
||||
final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
|
||||
assertEquals(null, original);
|
||||
|
||||
Thread.sleep(30000);
|
||||
try {
|
||||
final boolean contains = client.containsKey("testKey", keySerializer);
|
||||
assertTrue(contains);
|
||||
} catch (IOException e) {
|
||||
// this is due to the server timing out in the middle of this request
|
||||
assertTrue(e.getMessage().contains("Channel is closed"));
|
||||
}
|
||||
|
||||
server.shutdownServer();
|
||||
}
|
||||
|
||||
private void waitABit() {
|
||||
try {
|
||||
Thread.sleep(10L);
|
||||
|
@ -494,13 +457,14 @@ public class TestServerAndClient {
|
|||
}
|
||||
}
|
||||
|
||||
private DistributedSetCacheClientService createClient() throws InitializationException {
|
||||
private DistributedSetCacheClientService createClient(final int port) throws InitializationException {
|
||||
final DistributedSetCacheClientService client = new DistributedSetCacheClientService();
|
||||
MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
|
||||
final MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
|
||||
client.initialize(clientInitContext);
|
||||
|
||||
final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
|
||||
clientProperties.put(DistributedSetCacheClientService.HOSTNAME, "localhost");
|
||||
clientProperties.put(DistributedSetCacheClientService.PORT, String.valueOf(port));
|
||||
final MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
|
||||
client.onConfigured(clientContext);
|
||||
|
||||
|
@ -519,7 +483,7 @@ public class TestServerAndClient {
|
|||
|
||||
@Override
|
||||
public String deserialize(final byte[] input) throws DeserializationException, IOException {
|
||||
return (input.length == 0) ? null : new String(input, StandardCharsets.UTF_8);
|
||||
return input.length == 0 ? null : new String(input, StandardCharsets.UTF_8);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -543,4 +507,30 @@ public class TestServerAndClient {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static List<PropertyDescriptor> replacePortDescriptor(final List<PropertyDescriptor> descriptors) {
|
||||
descriptors.remove(DistributedCacheServer.PORT);
|
||||
descriptors.add(new PropertyDescriptor.Builder()
|
||||
.name("Port")
|
||||
.description("The port to listen on for incoming connections")
|
||||
.required(true)
|
||||
.addValidator(StandardValidators.createLongValidator(0L, 65535L, true))
|
||||
.defaultValue("0")
|
||||
.build());
|
||||
return descriptors;
|
||||
}
|
||||
|
||||
private static class SetServer extends DistributedSetCacheServer {
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return replacePortDescriptor(super.getSupportedPropertyDescriptors());
|
||||
}
|
||||
}
|
||||
|
||||
private static class MapServer extends DistributedMapCacheServer {
|
||||
@Override
|
||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||
return replacePortDescriptor(super.getSupportedPropertyDescriptors());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,19 +16,10 @@
|
|||
*/
|
||||
package org.apache.nifi.ssl;
|
||||
|
||||
import java.io.File;
|
||||
import java.net.MalformedURLException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnEnabled;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
|
@ -42,6 +33,19 @@ import org.apache.nifi.security.util.CertificateUtils;
|
|||
import org.apache.nifi.security.util.KeystoreType;
|
||||
import org.apache.nifi.security.util.SslContextFactory;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import java.io.File;
|
||||
import java.net.MalformedURLException;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
@Tags({"ssl", "secure", "certificate", "keystore", "truststore", "jks", "p12", "pkcs12", "pkcs"})
|
||||
@CapabilityDescription("Standard implementation of the SSLContextService. Provides the ability to configure "
|
||||
+ "keystore and/or truststore properties once and reuse that configuration throughout the application")
|
||||
|
@ -92,6 +96,15 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
|||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.sensitive(true)
|
||||
.build();
|
||||
public static final PropertyDescriptor SSL_ALGORITHM = new PropertyDescriptor.Builder()
|
||||
.name("SSL Protocol")
|
||||
.defaultValue("TLS")
|
||||
.required(false)
|
||||
.allowableValues(buildAlgorithmAllowableValues())
|
||||
.description("The algorithm to use for this SSL context")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.sensitive(false)
|
||||
.build();
|
||||
|
||||
private static final List<PropertyDescriptor> properties;
|
||||
private ConfigurationContext configContext;
|
||||
|
@ -104,6 +117,7 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
|||
props.add(TRUSTSTORE);
|
||||
props.add(TRUSTSTORE_PASSWORD);
|
||||
props.add(TRUSTSTORE_TYPE);
|
||||
props.add(SSL_ALGORITHM);
|
||||
properties = Collections.unmodifiableList(props);
|
||||
}
|
||||
|
||||
|
@ -207,13 +221,15 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
|||
}
|
||||
|
||||
private void verifySslConfig(final ValidationContext validationContext) throws ProcessException {
|
||||
final String protocol = validationContext.getProperty(SSL_ALGORITHM).getValue();
|
||||
try {
|
||||
final String keystoreFile = validationContext.getProperty(KEYSTORE).getValue();
|
||||
if (keystoreFile == null) {
|
||||
SslContextFactory.createTrustSslContext(
|
||||
validationContext.getProperty(TRUSTSTORE).getValue(),
|
||||
validationContext.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
|
||||
validationContext.getProperty(TRUSTSTORE_TYPE).getValue());
|
||||
validationContext.getProperty(TRUSTSTORE_TYPE).getValue(),
|
||||
protocol);
|
||||
return;
|
||||
}
|
||||
final String truststoreFile = validationContext.getProperty(TRUSTSTORE).getValue();
|
||||
|
@ -221,7 +237,8 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
|||
SslContextFactory.createSslContext(
|
||||
validationContext.getProperty(KEYSTORE).getValue(),
|
||||
validationContext.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(),
|
||||
validationContext.getProperty(KEYSTORE_TYPE).getValue());
|
||||
validationContext.getProperty(KEYSTORE_TYPE).getValue(),
|
||||
protocol);
|
||||
return;
|
||||
}
|
||||
|
||||
|
@ -232,7 +249,8 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
|||
validationContext.getProperty(TRUSTSTORE).getValue(),
|
||||
validationContext.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
|
||||
validationContext.getProperty(TRUSTSTORE_TYPE).getValue(),
|
||||
org.apache.nifi.security.util.SslContextFactory.ClientAuth.REQUIRED);
|
||||
org.apache.nifi.security.util.SslContextFactory.ClientAuth.REQUIRED,
|
||||
protocol);
|
||||
} catch (final Exception e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
|
@ -240,20 +258,23 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
|||
|
||||
@Override
|
||||
public SSLContext createSSLContext(final ClientAuth clientAuth) throws ProcessException {
|
||||
final String protocol = configContext.getProperty(SSL_ALGORITHM).getValue();
|
||||
try {
|
||||
final String keystoreFile = configContext.getProperty(KEYSTORE).getValue();
|
||||
if (keystoreFile == null) {
|
||||
return SslContextFactory.createTrustSslContext(
|
||||
configContext.getProperty(TRUSTSTORE).getValue(),
|
||||
configContext.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
|
||||
configContext.getProperty(TRUSTSTORE_TYPE).getValue());
|
||||
configContext.getProperty(TRUSTSTORE_TYPE).getValue(),
|
||||
protocol);
|
||||
}
|
||||
final String truststoreFile = configContext.getProperty(TRUSTSTORE).getValue();
|
||||
if (truststoreFile == null) {
|
||||
return SslContextFactory.createSslContext(
|
||||
configContext.getProperty(KEYSTORE).getValue(),
|
||||
configContext.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(),
|
||||
configContext.getProperty(KEYSTORE_TYPE).getValue());
|
||||
configContext.getProperty(KEYSTORE_TYPE).getValue(),
|
||||
protocol);
|
||||
}
|
||||
|
||||
return SslContextFactory.createSslContext(
|
||||
|
@ -263,7 +284,8 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
|||
configContext.getProperty(TRUSTSTORE).getValue(),
|
||||
configContext.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
|
||||
configContext.getProperty(TRUSTSTORE_TYPE).getValue(),
|
||||
org.apache.nifi.security.util.SslContextFactory.ClientAuth.valueOf(clientAuth.name()));
|
||||
org.apache.nifi.security.util.SslContextFactory.ClientAuth.valueOf(clientAuth.name()),
|
||||
protocol);
|
||||
} catch (final Exception e) {
|
||||
throw new ProcessException(e);
|
||||
}
|
||||
|
@ -309,8 +331,13 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
|||
return getKeyStoreFile() != null && getKeyStorePassword() != null && getKeyStoreType() != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getSslAlgorithm() {
|
||||
return configContext.getProperty(SSL_ALGORITHM).getValue();
|
||||
}
|
||||
|
||||
private static Collection<ValidationResult> validateStore(final Map<PropertyDescriptor, String> properties,
|
||||
final KeystoreValidationGroup keyStoreOrTrustStore) {
|
||||
final KeystoreValidationGroup keyStoreOrTrustStore) {
|
||||
final Collection<ValidationResult> results = new ArrayList<>();
|
||||
|
||||
final String filename;
|
||||
|
@ -382,6 +409,36 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
|||
KEYSTORE, TRUSTSTORE
|
||||
}
|
||||
|
||||
private static AllowableValue[] buildAlgorithmAllowableValues() {
|
||||
final Set<String> supportedProtocols = new HashSet<>();
|
||||
|
||||
/*
|
||||
* Prepopulate protocols with generic instance types commonly used
|
||||
* see: http://docs.oracle.com/javase/7/docs/technotes/guides/security/StandardNames.html#SSLContext
|
||||
*/
|
||||
supportedProtocols.add("SSL");
|
||||
supportedProtocols.add("TLS");
|
||||
|
||||
// Determine those provided by the JVM on the system
|
||||
try {
|
||||
supportedProtocols.addAll(Arrays.asList(SSLContext.getDefault().createSSLEngine().getSupportedProtocols()));
|
||||
} catch (NoSuchAlgorithmException e) {
|
||||
// ignored as default is used
|
||||
}
|
||||
|
||||
final int numProtocols = supportedProtocols.size();
|
||||
|
||||
// Sort for consistent presentation in configuraiton views
|
||||
final List<String> supportedProtocolList = new ArrayList<>(supportedProtocols);
|
||||
Collections.sort(supportedProtocolList);
|
||||
|
||||
final List<AllowableValue> protocolAllowableValues = new ArrayList<>();
|
||||
for (final String protocol : supportedProtocolList) {
|
||||
protocolAllowableValues.add(new AllowableValue(protocol));
|
||||
}
|
||||
return protocolAllowableValues.toArray(new AllowableValue[numProtocols]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SSLContextService[id=" + getIdentifier() + "]";
|
||||
|
|
|
@ -57,4 +57,5 @@ public interface SSLContextService extends ControllerService {
|
|||
|
||||
public boolean isKeyStoreConfigured();
|
||||
|
||||
String getSslAlgorithm();
|
||||
}
|
||||
|
|
10
nifi/pom.xml
10
nifi/pom.xml
|
@ -847,11 +847,21 @@
|
|||
<artifactId>nifi-dbcp-service</artifactId>
|
||||
<version>0.2.0-incubating-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.nifi</groupId>
|
||||
<artifactId>nifi-dbcp-service-api</artifactId>
|
||||
<version>0.2.0-incubating-SNAPSHOT</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.jayway.jsonpath</groupId>
|
||||
<artifactId>json-path</artifactId>
|
||||
<version>2.0.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.derby</groupId>
|
||||
<artifactId>derby</artifactId>
|
||||
<version>10.11.1.1</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</dependencyManagement>
|
||||
<dependencies>
|
||||
|
|
Loading…
Reference in New Issue