You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2014/12/16 21:10:51 UTC

[1/8] incubator-nifi git commit: NIFI-145: Added PID detection for Linux

Repository: incubator-nifi
Updated Branches:
  refs/heads/develop a94cf4be4 -> 575cc42c7


NIFI-145: Added PID detection for Linux


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/d517b3fd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/d517b3fd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/d517b3fd

Branch: refs/heads/develop
Commit: d517b3fd0932676edddfa2cfbacb320b6142e882
Parents: 2bcd1e6
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Dec 11 13:40:56 2014 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Thu Dec 11 13:40:56 2014 -0500

----------------------------------------------------------------------
 .../resources/src/main/resources/bin/nifi.sh    |   2 +-
 .../src/main/resources/conf/bootstrap.conf      |   8 +-
 .../apache/nifi/bootstrap/BootstrapCodec.java   |   1 +
 .../java/org/apache/nifi/bootstrap/RunNiFi.java | 439 ++++++++++++++++---
 4 files changed, 387 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d517b3fd/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh b/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
index 60afa48..9dc7ef6 100644
--- a/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
+++ b/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
@@ -172,7 +172,7 @@ 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 $1
+    exec "$JAVA" -cp "$NIFI_HOME"/lib/bootstrap/* -Xms12m -Xmx24m -Dorg.apache.nifi.bootstrap.config.file="$BOOTSTRAP_CONF" org.apache.nifi.bootstrap.RunNiFi $1 $2
 }
 
 main() {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d517b3fd/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf b/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf
index c45d8f8..6b32b2b 100644
--- a/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf
+++ b/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf
@@ -13,4 +13,10 @@ java.arg.2=-Xms256m
 java.arg.3=-Xmx512m
 
 # Enable Remote Debugging
-#java.arg.2=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
\ No newline at end of file
+#java.arg.2=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
+
+# Java command to use when running NiFi
+java=java
+
+# Username to use when running NiFi. This value will be ignored on Windows.
+run.as=

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d517b3fd/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java
index 8138c02..fb10930 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java
@@ -27,6 +27,7 @@ import java.util.Arrays;
 
 import org.apache.nifi.bootstrap.exception.InvalidCommandException;
 
+
 public class BootstrapCodec {
 	private final RunNiFi runner;
 	private final BufferedReader reader;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/d517b3fd/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
index 1b82a3c..a5987bc 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
@@ -26,10 +26,11 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
+import java.io.Reader;
+import java.lang.reflect.Field;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -39,6 +40,9 @@ 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;
 
 
 /**
@@ -60,6 +64,8 @@ public class RunNiFi {
 	public static final String GRACEFUL_SHUTDOWN_PROP = "graceful.shutdown.seconds";
 	public static final String DEFAULT_GRACEFUL_SHUTDOWN_VALUE = "20";
 	
+	public static final String RUN_AS_PROP = "run.as";
+	
 	public static final int MAX_RESTART_ATTEMPTS = 5;
 	public static final int STARTUP_WAIT_SECONDS = 60;
 	
@@ -68,20 +74,32 @@ public class RunNiFi {
 	
 	private volatile boolean autoRestartNiFi = true;
 	private volatile int ccPort = -1;
+	private volatile long nifiPid = -1L;
 	
 	private final Lock lock = new ReentrantLock();
 	private final Condition startupCondition = lock.newCondition();
 	
 	private final File bootstrapConfigFile;
+
+	private final java.util.logging.Logger logger;
 	
-	public RunNiFi(final File bootstrapConfigFile) {
+	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);
+		}
 	}
 	
 	private static void printUsage() {
 		System.out.println("Usage:");
 		System.out.println();
-		System.out.println("java org.apache.nifi.bootstrap.RunNiFi <command>");
+		System.out.println("java org.apache.nifi.bootstrap.RunNiFi [<-verbose>] <command>");
 		System.out.println();
 		System.out.println("Valid commands include:");
 		System.out.println("");
@@ -91,22 +109,33 @@ public class RunNiFi {
 		System.out.println("Run : Start a new instance of Apache NiFi and monitor the Process, restarting if the instance dies");
 		System.out.println();
 	}
+
 	
 	public static void main(final String[] args) throws IOException, InterruptedException {
-		if ( args.length != 1 ) {
+		if ( args.length < 1 || args.length > 2 ) {
 			printUsage();
 			return;
 		}
 		
-		switch (args[0].toLowerCase()) {
+		boolean verbose = false;
+		if ( args.length == 2 ) {
+		    if ( args[0].equals("-verbose") ) {
+		        verbose = true;
+		    } else {
+		        printUsage();
+		        return;
+		    }
+		}
+		
+		final String cmd = args.length == 1 ? args[0] : args[1];
+		
+		switch (cmd.toLowerCase()) {
 			case "start":
 			case "run":
 			case "stop":
 			case "status":
 				break;
 			default:
-				System.out.println("Invalid argument: " + args[0]);
-				System.out.println();
 				printUsage();
 				return;
 		}
@@ -128,9 +157,9 @@ public class RunNiFi {
 		
 		final File configFile = new File(configFilename);
 		
-		final RunNiFi runNiFi = new RunNiFi(configFile);
+		final RunNiFi runNiFi = new RunNiFi(configFile, verbose);
 		
-		switch (args[0].toLowerCase()) {
+		switch (cmd.toLowerCase()) {
 			case "start":
 				runNiFi.start(false);
 				break;
@@ -151,49 +180,189 @@ public class RunNiFi {
 		final File confDir = bootstrapConfigFile.getParentFile();
 		final File nifiHome = confDir.getParentFile();
 		final File bin = new File(nifiHome, "bin");
-		final File statusFile = new File(bin, "nifi.port");
+		final File statusFile = new File(bin, "nifi.pid");
+		
+		logger.fine("Status File: " + statusFile);
+		
 		return statusFile;
 	}
+	
+	private Properties loadProperties() throws IOException {
+	    final Properties props = new Properties();
+	    final File statusFile = getStatusFile();
+	    if ( statusFile == null || !statusFile.exists() ) {
+	        logger.fine("No status file to load properties from");
+	        return props;
+	    }
+	    
+	    try (final FileInputStream fis = new FileInputStream(getStatusFile())) {
+	        props.load(fis);
+	    }
+	    
+	    logger.fine("Properties: " + props);
+	    return props;
+	}
+	
+	private synchronized void saveProperties(final Properties nifiProps) throws IOException {
+	    final File statusFile = getStatusFile();
+        try (final FileOutputStream fos = new FileOutputStream(statusFile)) {
+            nifiProps.store(fos, null);
+            fos.getFD().sync();
+        }
+        
+        logger.fine("Saved Properties " + nifiProps + " to " + statusFile);
+	}
+
+	private boolean isPingSuccessful(final int port) {
+	    logger.fine("Pinging " + port);
+	    
+	    try (final Socket socket = new Socket("localhost", port)) {
+            final OutputStream out = socket.getOutputStream();
+            out.write((PING_CMD + "\n").getBytes(StandardCharsets.UTF_8));
+            out.flush();
 
+            logger.fine("Sent PING command");
+            
+            final InputStream in = socket.getInputStream();
+            final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+            final String response = reader.readLine();
+            logger.fine("PING response: " + response);
+            
+            return PING_CMD.equals(response);
+	    } catch (final IOException ioe) {
+	        return false;
+	    }
+	}
+	
 	private Integer getCurrentPort() throws IOException {
-		try {
-			final File statusFile = getStatusFile();
-			final byte[] info = Files.readAllBytes(statusFile.toPath());
-			final String text = new String(info);
-			
-			final int port = Integer.parseInt(text);
-			
-			try (final Socket socket = new Socket("localhost", port)) {
-				final OutputStream out = socket.getOutputStream();
-				out.write((PING_CMD + "\n").getBytes(StandardCharsets.UTF_8));
-				out.flush();
-				
-				final InputStream in = socket.getInputStream();
-				final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
-				final String response = reader.readLine();
-				if ( response.equals(PING_CMD) ) {
-					return port;
-				}
-			} catch (final IOException ioe) {
-				System.out.println("Found NiFi instance info at " + statusFile + " indicating that NiFi is running and listening to port " + port + " but unable to communicate with NiFi on that port. The process may have died or may be hung.");
-				throw ioe;
-			}
-		} catch (final Exception e) {
-			return null;
+		final Properties props = loadProperties();
+		final String portVal = props.getProperty("port");
+		if ( portVal == null ) {
+		    logger.fine("No Port found in status file");
+		    return null;
+		} else { 
+		    logger.fine("Port defined in status file: " + portVal);
 		}
 		
-		return null;
+		final int port = Integer.parseInt(portVal);
+	    final boolean success = isPingSuccessful(port);
+	    if ( success ) {
+	        logger.fine("Successful PING on port " + port);
+	        return port;
+	    }
+
+	    final String pid = props.getProperty("pid");
+	    logger.fine("PID in status file is " + pid);
+	    if ( pid != null ) {
+	        final boolean procRunning = isProcessRunning(pid);
+	        if ( procRunning ) {
+	            return port;
+	        } else {
+	            return null;
+	        }
+	    }
+	    
+	    return null;
 	}
 	
 	
+	private boolean isProcessRunning(final String pid) {
+	    try {
+	        // We use the "ps" command to check if the process is still running.
+	        final ProcessBuilder builder = new ProcessBuilder();
+	        
+	        // ps -p <pid> -o comm=
+	        // -> -p <pid> to filter just the pid we care about
+	        // -> -o comm= to remove headers from the output
+	        builder.command("ps", "-p", pid, "-o", "comm=");
+	        final Process proc = builder.start();
+	        
+	        // Read how many lines are output by the 'ps' command
+	        int lineCount = 0;
+	        String line;
+	        try (final InputStream in = proc.getInputStream();
+	             final Reader streamReader = new InputStreamReader(in);
+	             final BufferedReader reader = new BufferedReader(streamReader)) {
+	            
+	            while ((line = reader.readLine()) != null) {
+	                if ( !line.trim().isEmpty() ) {
+	                    lineCount++;
+	                }
+	            }
+	        }
+	        
+	        // If anything was output, the process is running.
+	        final boolean running = lineCount > 0;
+	        if ( running ) {
+	            logger.fine("Process with PID " + pid + " is running");
+	        } else {
+	            logger.fine("Process with PID " + pid + " is not running");
+	        }
+	        
+	        return running;
+	    } catch (final IOException ioe) {
+	        System.err.println("Failed to determine if Process " + pid + " is running; assuming that it is not");
+	        return false;
+	    }
+	}
+	
+	
+	private Status getStatus() {
+	    final Properties props;
+	    try {
+	        props = loadProperties();
+	    } catch (final IOException ioe) {
+	        return new Status(null, null, false, false);
+	    }
+	    
+	    if ( props == null ) {
+	        return new Status(null, null, false, false);
+	    }
+	    
+        final String portValue = props.getProperty("port");
+        final String pid = props.getProperty("pid");
+        
+        if ( portValue == null && pid == null ) {
+            return new Status(null, null, false, false);
+        }
+        
+        Integer port = null;
+        boolean pingSuccess = false;
+        if ( portValue != null ) {
+            try {
+                port = Integer.parseInt(portValue);
+                pingSuccess = isPingSuccessful(port);
+            } catch (final NumberFormatException nfe) {
+                return new Status(null, null, false, false);
+            }
+        }
+        
+        if ( pingSuccess ) {
+            return new Status(port, pid, true, true);
+        }
+        
+        final boolean alive = (pid == null) ? false : isProcessRunning(pid);
+        return new Status(port, pid, pingSuccess, alive);
+	}
+	
 	public void status() throws IOException {
-		final Integer port = getCurrentPort();
-		if ( port == null ) {
-			System.out.println("Apache NiFi does not appear to be running");
-		} else {
-			System.out.println("Apache NiFi is currently running, listening on port " + port);
-		}
-		return;
+	    final Status status = getStatus();
+	    if ( status.isRespondingToPing() ) {
+	        logger.info("Apache NiFi is currently running, listening to Bootstrap on port " + status.getPort() + 
+	                ", PID=" + (status.getPid() == null ? "unknkown" : status.getPid()));
+	        return;
+	    }
+
+	    if ( status.isProcessRunning() ) {
+	        logger.info("Apache NiFi is running at PID " + status.getPid() + " but is not responding to ping requests");
+	        return;
+	    }
+	    
+	    if ( status.getPid() == null ) {
+	        logger.info("Apache NiFi is not responding to Ping requests. The process may have died or may be hung");
+	    } else {
+	        logger.info("Apache NiFi is not running");
+	    }
 	}
 	
 	
@@ -209,6 +378,7 @@ public class RunNiFi {
 			socket.connect(new InetSocketAddress("localhost", port));
 			socket.setSoTimeout(60000);
 			
+			logger.fine("Sending SHUTDOWN Command to port " + port);
 			final OutputStream out = socket.getOutputStream();
 			out.write((SHUTDOWN_CMD + "\n").getBytes(StandardCharsets.UTF_8));
 			out.flush();
@@ -216,23 +386,72 @@ public class RunNiFi {
 			final InputStream in = socket.getInputStream();
 			final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
 			final String response = reader.readLine();
+			
+			logger.fine("Received response to SHUTDOWN command: " + response);
+			
 			if ( SHUTDOWN_CMD.equals(response) ) {
-				System.out.println("Apache NiFi has accepted the Shutdown Command and is shutting down now");
+				logger.info("Apache NiFi has accepted the Shutdown Command and is shutting down now");
+				
+				final Properties nifiProps = loadProperties();
+				final String pid = nifiProps.getProperty("pid");
+				if ( pid != null ) {
+
+			        final Properties bootstrapProperties = new Properties();
+			        try (final FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
+			            bootstrapProperties.load(fis);
+			        }
+
+				    String gracefulShutdown = bootstrapProperties.getProperty(GRACEFUL_SHUTDOWN_PROP, DEFAULT_GRACEFUL_SHUTDOWN_VALUE);
+				    int gracefulShutdownSeconds;
+				    try {
+				        gracefulShutdownSeconds = Integer.parseInt(gracefulShutdown);
+				    } catch (final NumberFormatException nfe) {
+				        gracefulShutdownSeconds = Integer.parseInt(DEFAULT_GRACEFUL_SHUTDOWN_VALUE);
+				    }
+			        
+			        final long startWait = System.nanoTime();
+			        while ( isProcessRunning(pid) ) {
+			            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.warning("NiFi has not finished shutting down after " + gracefulShutdownSeconds + " seconds. Killing process.");
+			                    try {
+			                        killProcess(pid);
+			                    } catch (final IOException ioe) {
+			                        logger.severe("Failed to kill Process with PID " + pid);
+			                    }
+			                }
+			                break;
+			            } else {
+			                try {
+			                    Thread.sleep(2000L);
+			                } catch (final InterruptedException ie) {}
+			            }
+			        }
+			        
+			        logger.info("NiFi has finished shutting down.");
+				}
 				
 				final File statusFile = getStatusFile();
 				if ( !statusFile.delete() ) {
-					System.err.println("Failed to delete status file " + statusFile + "; this file should be cleaned up manually");
+					logger.severe("Failed to delete status file " + statusFile + "; this file should be cleaned up manually");
 				}
 			} else {
-				System.err.println("When sending SHUTDOWN command to NiFi, got unexpected response " + response);
+				logger.severe("When sending SHUTDOWN command to NiFi, got unexpected response " + response);
 			}
 		} catch (final IOException ioe) {
-			System.err.println("Failed to communicate with Apache NiFi");
+		    logger.severe("Failed to send shutdown command to port " + port + " due to " + ioe);
 			return;
 		}
 	}
 	
 	
+	private static void killProcess(final String pid) throws IOException {
+	    Runtime.getRuntime().exec(new String[] {"kill", "-9", pid});
+	}
+	
 	public static boolean isAlive(final Process process) {
 		try {
 			process.exitValue();
@@ -246,7 +465,7 @@ public class RunNiFi {
 	public void start(final boolean monitor) throws IOException, InterruptedException {
 		final Integer port = getCurrentPort();
 		if ( port != null ) {
-			System.out.println("Apache NiFi is already running, listening on port " + port);
+			System.out.println("Apache NiFi is already running, listening to Bootstrap on port " + port);
 			return;
 		}
 		
@@ -344,7 +563,20 @@ public class RunNiFi {
 		final NiFiListener listener = new NiFiListener();
 		final int listenPort = listener.start(this);
 		
+		String runAs = isWindows() ? null : props.get(RUN_AS_PROP);
+		if ( runAs != null ) {
+		    runAs = runAs.trim();
+		    if ( runAs.isEmpty() ) {
+		        runAs = null;
+		    }
+		}
+		
 		final List<String> cmd = new ArrayList<>();
+		if ( runAs != null ) {
+		    cmd.add("sudo");
+		    cmd.add("-u");
+		    cmd.add(runAs);
+		}
 		cmd.add(javaCmd);
 		cmd.add("-classpath");
 		cmd.add(classPath);
@@ -361,9 +593,9 @@ public class RunNiFi {
 			cmdBuilder.append(s).append(" ");
 		}
 
-		System.out.println("Starting Apache NiFi...");
-		System.out.println("Working Directory: " + workingDir.getAbsolutePath());
-		System.out.println("Command: " + cmdBuilder.toString());
+		logger.info("Starting Apache NiFi...");
+		logger.info("Working Directory: " + workingDir.getAbsolutePath());
+		logger.info("Command: " + cmdBuilder.toString());
 		
 		if ( monitor ) {
 			String gracefulShutdown = props.get(GRACEFUL_SHUTDOWN_PROP);
@@ -383,6 +615,13 @@ public class RunNiFi {
 			}
 			
 			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(properties);
+            }
 			
 			ShutdownHook shutdownHook = new ShutdownHook(process, this, gracefulShutdownSeconds);
 			final Runtime runtime = Runtime.getRuntime();
@@ -404,18 +643,26 @@ public class RunNiFi {
 				    }
 					
 					if (autoRestartNiFi) {
-						System.out.println("Apache NiFi appears to have died. Restarting...");
+						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(properties);
+			            }
+						
 						shutdownHook = new ShutdownHook(process, this, gracefulShutdownSeconds);
 						runtime.addShutdownHook(shutdownHook);
 						
 						final boolean started = waitForStart();
 						
 						if ( started ) {
-							System.out.println("Successfully started Apache NiFi");
+							logger.info("Successfully started Apache NiFi" + (pid == null ? "" : " with PID " + pid));
 						} else {
-							System.err.println("Apache NiFi does not appear to have started");
+							logger.severe("Apache NiFi does not appear to have started");
 						}
 					} else {
 						return;
@@ -423,13 +670,22 @@ public class RunNiFi {
 				}
 			}
 		} else {
-			builder.start();
+			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(properties);
+			}
+			
 			boolean started = waitForStart();
 			
 			if ( started ) {
-				System.out.println("Successfully started Apache NiFi");
+				logger.info("Successfully started Apache NiFi" + (pid == null ? "" : " with PID " + pid));
 			} else {
-				System.err.println("Apache NiFi does not appear to have started");
+				logger.severe("Apache NiFi does not appear to have started");
 			}
 			
 			listener.stop();
@@ -437,6 +693,30 @@ public class RunNiFi {
 	}
 	
 	
+	private Long getPid(final Process process) {
+	    try {
+            final Class<?> procClass = process.getClass();
+            final Field pidField = procClass.getDeclaredField("pid");
+            pidField.setAccessible(true);
+            final Object pidObject = pidField.get(process);
+            
+            logger.fine("PID Object = " + pidObject);
+            
+            if ( pidObject instanceof Number ) {
+                return ((Number) pidObject).longValue();
+            }
+            return null;
+        } catch (final IllegalAccessException | NoSuchFieldException nsfe) {
+            logger.fine("Could not find PID for child process due to " + nsfe);
+            return null;
+        }
+	}
+	
+	private boolean isWindows() {
+	    final String osName = System.getProperty("os.name");
+	    return osName != null && osName.toLowerCase().contains("win");
+	}
+	
 	private boolean waitForStart() {
 		lock.lock();
 		try {
@@ -482,17 +762,54 @@ public class RunNiFi {
 		this.ccPort = port;
 
 		final File statusFile = getStatusFile();
-		try (final FileOutputStream fos = new FileOutputStream(statusFile)) {
-			fos.write(String.valueOf(port).getBytes(StandardCharsets.UTF_8));
-			fos.getFD().sync();
+		
+		final Properties nifiProps = new Properties();
+		if ( nifiPid != -1 ) {
+		    nifiProps.setProperty("pid", String.valueOf(nifiPid));
+		}
+		nifiProps.setProperty("port", String.valueOf(ccPort));
+		try {
+		    saveProperties(nifiProps);
 		} catch (final IOException ioe) {
-			System.err.println("Apache NiFi has started but failed to persist NiFi Port information to " + statusFile.getAbsolutePath() + " due to " + ioe);
+		    logger.warning("Apache NiFi has started but failed to persist NiFi Port information to " + statusFile.getAbsolutePath() + " due to " + ioe);
 		}
 		
-		System.out.println("Apache NiFi now running and listening for requests on port " + port);
+		logger.info("Apache NiFi now running and listening for Bootstrap requests on port " + port);
 	}
 	
 	int getNiFiCommandControlPort() {
 		return this.ccPort;
 	}
+	
+	
+	private static class Status {
+	    private final Integer port;
+	    private final String pid;
+	    
+	    private final Boolean respondingToPing;
+	    private final Boolean processRunning;
+	    
+	    public Status(final Integer port, final String pid, final Boolean respondingToPing, final Boolean processRunning) {
+	        this.port = port;
+	        this.pid = pid;
+	        this.respondingToPing = respondingToPing;
+	        this.processRunning = processRunning;
+	    }
+	    
+	    public String getPid() {
+	        return pid;
+	    }
+	    
+	    public Integer getPort() {
+	        return port;
+	    }
+	    
+	    public boolean isRespondingToPing() {
+	        return Boolean.TRUE.equals(respondingToPing);
+	    }
+	    
+        public boolean isProcessRunning() {
+            return Boolean.TRUE.equals(processRunning);
+        }
+	}
 }


[4/8] incubator-nifi git commit: NIFI-145: Fixed issue that indicated that NiFi was hung or died when the .pid file wasn't found

Posted by ma...@apache.org.
NIFI-145: Fixed issue that indicated that NiFi was hung or died when the .pid file wasn't found


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/2a2f0f03
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/2a2f0f03
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/2a2f0f03

Branch: refs/heads/develop
Commit: 2a2f0f036434c0fa2dd0f6af67e6cf13b0b3183e
Parents: 8cf0c78
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Dec 12 10:18:14 2014 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Dec 12 10:18:14 2014 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/nifi/bootstrap/RunNiFi.java        | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2a2f0f03/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
index a230711..270ba48 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
@@ -373,6 +373,11 @@ public class RunNiFi {
 	        return;
 	    }
 	    
+	    if ( status.getPort() == null ) {
+	        logger.info("Apache NiFi is not running");
+	        return;
+	    }
+	    
 	    if ( status.getPid() == null ) {
 	        logger.info("Apache NiFi is not responding to Ping requests. The process may have died or may be hung");
 	    } else {


[7/8] incubator-nifi git commit: NIFI-145: Make nifi.pid writable by owner

Posted by ma...@apache.org.
NIFI-145: Make nifi.pid writable by owner


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/c1681d95
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/c1681d95
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/c1681d95

Branch: refs/heads/develop
Commit: c1681d95f690a971e9a54e424e77661124077c9d
Parents: 1a231d8
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Dec 16 14:36:38 2014 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Dec 16 14:36:38 2014 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/nifi/bootstrap/RunNiFi.java      | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c1681d95/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
index 270ba48..437493e 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
@@ -36,9 +36,11 @@ import java.nio.file.attribute.PosixFilePermission;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
@@ -217,7 +219,10 @@ public class RunNiFi {
 	    }
 
 	    try {
-	        Files.setPosixFilePermissions(statusFile.toPath(), Collections.singleton(PosixFilePermission.OWNER_READ));
+	        final Set<PosixFilePermission> perms = new HashSet<>();
+	        perms.add(PosixFilePermission.OWNER_READ);
+	        perms.add(PosixFilePermission.OWNER_WRITE);
+	        Files.setPosixFilePermissions(statusFile.toPath(), perms);
 	    } catch (final Exception e) {
 	        logger.warning("Failed to set permissions so that only the owner can read status file " + statusFile + "; 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");
 	    }


[6/8] incubator-nifi git commit: NIFI-145: Update nifi.sh to avoid readlink that's not always compatible; added 'install command'; made nifi.sh chkconfig compatible

Posted by ma...@apache.org.
NIFI-145: Update nifi.sh to avoid readlink that's not always compatible; added 'install command'; made nifi.sh chkconfig compatible


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/1a231d8b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/1a231d8b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/1a231d8b

Branch: refs/heads/develop
Commit: 1a231d8b9599d6ff3aeddb97392acec1b885be70
Parents: d1271a2
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Dec 15 13:04:54 2014 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Dec 15 13:04:54 2014 -0500

----------------------------------------------------------------------
 .../resources/src/main/resources/bin/nifi.sh    | 80 ++++++++++----------
 1 file changed, 39 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/1a231d8b/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh b/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
index 9dc7ef6..163f8e2 100644
--- a/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
+++ b/nar-bundles/framework-bundle/framework/resources/src/main/resources/bin/nifi.sh
@@ -15,19 +15,16 @@
 #    See the License for the specific language governing permissions and
 #    limitations under the License.
 #
+# chkconfig: 2345 20 80
+# description: Apache NiFi is a dataflow system based on the principles of Flow-Based Programming.
+#
 
 # Script structure inspired from Apache Karaf and other Apache projects with similar startup approaches
 
-DIRNAME=`dirname "$0"`
+NIFI_HOME=`cd $(dirname "$0") && cd .. && pwd`
 PROGNAME=`basename "$0"`
 
 
-#
-#Readlink is not available on all systems. Change variable to appropriate alternative as part of OS detection
-#
-
-READLINK="readlink"
-
 warn() {
     echo "${PROGNAME}: $*"
 }
@@ -54,17 +51,14 @@ detectOS() {
             os400=true
             ;;
         Darwin)
-        	darwin=true
-        	;;
+                darwin=true
+                ;;
     esac
     # For AIX, set an environment variable
     if $aix; then
          export LDR_CNTRL=MAXDATA=0xB0000000@DSA
          echo $LDR_CNTRL
     fi
-    if $darwin; then
-    	READLINK="greadlink"
-    fi
 }
 
 unlimitFD() {
@@ -95,22 +89,6 @@ unlimitFD() {
     fi
 }
 
-locateHome() {
-    if [ "x$NIFI_HOME" != "x" ]; then
-        warn "Ignoring predefined value for NIFI_HOME"
-    fi
-
-    # In POSIX shells, CDPATH may cause cd to write to stdout
-    (unset CDPATH) >/dev/null 2>&1 && unset CDPATH
-    NIFI_HOME=$(dirname $($READLINK -f $0))/../
-    NIFI_HOME=$($READLINK -f $NIFI_HOME)
-    cd $NIFI_HOME
-    echo "Directory changed to NIFI_HOME of '$NIFI_HOME'"
-    if [ ! -d "$NIFI_HOME" ]; then
-        die "NIFI_HOME is not valid: $NIFI_HOME"
-    fi
-
-}
 
 
 locateJava() {
@@ -138,9 +116,6 @@ locateJava() {
             fi
         fi
     fi
-    if [ "x$JAVA_HOME" = "x" ]; then
-        JAVA_HOME="$(dirname $(dirname $(pathCanonical "$JAVA")))"
-    fi
 }
 
 init() {
@@ -150,29 +125,41 @@ init() {
     # Unlimit the number of file descriptors if possible
     unlimitFD
 
-    # Locate the NiFi home directory
-    locateHome
-
     # Locate the Java VM to execute
     locateJava
 }
 
+
+install() {
+        SVC_NAME=nifi
+        if [ "x$2" != "x" ] ; then
+                SVC_NAME=$2
+        fi
+
+        SVC_FILE=/etc/init.d/$SVC_NAME
+        cp $0 $SVC_FILE
+        sed -i s:NIFI_HOME=.*:NIFI_HOME="$NIFI_HOME": $SVC_FILE
+        sed -i s:PROGNAME=.*:PROGNAME=$(basename "$0"): $SVC_FILE
+        echo Service $SVC_NAME installed
+}
+
+
 run() {
-	BOOTSTRAP_CONF="$NIFI_HOME/conf/bootstrap.conf";
-	
+    BOOTSTRAP_CONF="$NIFI_HOME/conf/bootstrap.conf";
+
     if $cygwin; then
         NIFI_HOME=`cygpath --path --windows "$NIFI_HOME"`
         BOOTSTRAP_CONF=`cygpath --path --windows "$BOOTSTRAP_CONF"`
     fi
-    
-    echo 
+
+    echo
     echo "Java home: $JAVA_HOME"
     echo "NiFi home: $NIFI_HOME"
     echo
     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 $1 $2
+
+    exec "$JAVA" -cp "$NIFI_HOME"/lib/bootstrap/* -Xms12m -Xmx24m -Dorg.apache.nifi.bootstrap.config.file="$BOOTSTRAP_CONF" org.apache.nifi.bootstrap.RunNiFi $@
 }
 
 main() {
@@ -180,4 +167,15 @@ main() {
     run "$@"
 }
 
-main "$@"
+
+case "$1" in
+    install)
+        install "$@"
+        ;;
+    start|stop|run|status)
+        main "$@"
+        ;;
+    *)
+        echo "Usage nifi {start|stop|run|status|install}"
+        ;;
+esac


[3/8] incubator-nifi git commit: Merge branch 'NIFI164' into bootstrap

Posted by ma...@apache.org.
Merge branch 'NIFI164' into bootstrap


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/8cf0c781
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/8cf0c781
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/8cf0c781

Branch: refs/heads/develop
Commit: 8cf0c7814e7c3b451ba58eb97b8e93a3ce518ed6
Parents: 76f54f8 55d4b1c
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Dec 12 10:01:17 2014 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Dec 12 10:01:17 2014 -0500

----------------------------------------------------------------------
 .../main/java/org/apache/nifi/controller/FlowController.java   | 4 ++++
 .../nifi/controller/repository/FileSystemRepository.java       | 6 ++++++
 .../nifi/controller/repository/VolatileContentRepository.java  | 5 +++++
 .../apache/nifi/controller/repository/ContentRepository.java   | 6 ++++++
 4 files changed, 21 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/8cf0c781/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java
----------------------------------------------------------------------


[5/8] incubator-nifi git commit: Merge branch 'develop' into bootstrap

Posted by ma...@apache.org.
Merge branch 'develop' into bootstrap


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/d1271a28
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/d1271a28
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/d1271a28

Branch: refs/heads/develop
Commit: d1271a281e79a714a1c098465e29ef2925675e2b
Parents: 2a2f0f0 67354bd
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Dec 12 10:31:50 2014 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Dec 12 10:31:50 2014 -0500

----------------------------------------------------------------------
 .gitignore                                        |  1 +
 .../repository/TestStandardProcessSession.java    |  4 ++++
 .../web/nifi-web-api/nb-configuration.xml         | 18 ------------------
 .../provenance/VolatileProvenanceRepository.java  |  2 ++
 4 files changed, 7 insertions(+), 18 deletions(-)
----------------------------------------------------------------------



[2/8] incubator-nifi git commit: NIFI-145: Allow a run.as user to be set in bootstrap.conf file; addressed concerns where an un-priveleged user could issue commands to running NiFi to shutdown; addressed concerns where an un-priveleged user could push la

Posted by ma...@apache.org.
NIFI-145: Allow a run.as user to be set in bootstrap.conf file; addressed concerns where an un-priveleged user could issue commands to running NiFi to shutdown; addressed concerns where an un-priveleged user could push large amounts of data to the Bootstrap or NiFi causing OOME


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/76f54f86
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/76f54f86
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/76f54f86

Branch: refs/heads/develop
Commit: 76f54f86115e42d16f700aa0d1c5bce22f830734
Parents: d517b3f
Author: Mark Payne <ma...@hotmail.com>
Authored: Fri Dec 12 09:47:19 2014 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Fri Dec 12 09:47:19 2014 -0500

----------------------------------------------------------------------
 .../repository/VolatileContentRepository.java   |   2 +-
 .../src/main/resources/conf/bootstrap.conf      |   2 +-
 .../java/org/apache/nifi/BootstrapListener.java |  46 ++++++--
 .../src/main/java/org/apache/nifi/NiFi.java     |   2 -
 .../apache/nifi/util/LimitingInputStream.java   | 107 +++++++++++++++++++
 .../apache/nifi/bootstrap/BootstrapCodec.java   |   6 +-
 .../org/apache/nifi/bootstrap/NiFiListener.java |  18 +++-
 .../java/org/apache/nifi/bootstrap/RunNiFi.java |  81 ++++++++++----
 .../bootstrap/util/LimitingInputStream.java     | 107 +++++++++++++++++++
 9 files changed, 332 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/76f54f86/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java
index e14ec5d..1a44725 100644
--- a/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java
+++ b/nar-bundles/framework-bundle/framework/core/src/main/java/org/apache/nifi/controller/repository/VolatileContentRepository.java
@@ -92,7 +92,7 @@ public class VolatileContentRepository implements ContentRepository {
     public static final String MAX_SIZE_PROPERTY = "nifi.volatile.content.repository.max.size";
     public static final String BLOCK_SIZE_PROPERTY = "nifi.volatile.content.repository.block.size";
 
-    private final ScheduledExecutorService executor = new FlowEngine(3, "VolatileContentRepository Workers");
+    private final ScheduledExecutorService executor = new FlowEngine(3, "VolatileContentRepository Workers", true);
     private final ConcurrentMap<ContentClaim, ContentBlock> claimMap = new ConcurrentHashMap<>(256);
     private final AtomicLong repoSize = new AtomicLong(0L);
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/76f54f86/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf b/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf
index 6b32b2b..37ec474 100644
--- a/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf
+++ b/nar-bundles/framework-bundle/framework/resources/src/main/resources/conf/bootstrap.conf
@@ -13,7 +13,7 @@ java.arg.2=-Xms256m
 java.arg.3=-Xmx512m
 
 # Enable Remote Debugging
-#java.arg.2=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
+#java.arg.debug=-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=8000
 
 # Java command to use when running NiFi
 java=java

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/76f54f86/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java b/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java
index 31f336c..3393952 100644
--- a/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java
+++ b/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/BootstrapListener.java
@@ -27,9 +27,11 @@ import java.net.Socket;
 import java.net.SocketTimeoutException;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
+import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
+import org.apache.nifi.util.LimitingInputStream;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,14 +40,16 @@ public class BootstrapListener {
 	
 	private final NiFi nifi;
 	private final int bootstrapPort;
-
+	private final String secretKey;
+	
 	private volatile Listener listener;
 	private volatile ServerSocket serverSocket;
 	
 	
-	public BootstrapListener(final NiFi nifi, final int port) {
+	public BootstrapListener(final NiFi nifi, final int bootstrapPort) {
 		this.nifi = nifi;
-		this.bootstrapPort = port;
+		this.bootstrapPort = bootstrapPort;
+		secretKey = UUID.randomUUID().toString();
 	}
 	
 	public void start() throws IOException {
@@ -71,7 +75,7 @@ public class BootstrapListener {
 			socket.setSoTimeout(60000);
 			
 			final OutputStream out = socket.getOutputStream();
-			out.write(("PORT " + localPort + "\n").getBytes(StandardCharsets.UTF_8));
+			out.write(("PORT " + localPort + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
 			out.flush();
 			
 			logger.debug("Awaiting response from Bootstrap...");
@@ -121,6 +125,7 @@ public class BootstrapListener {
 				try {
 					final Socket socket;
 					try {
+					    logger.debug("Listening for Bootstrap Requests");
 						socket = serverSocket.accept();
 					} catch (final SocketTimeoutException ste) {
 						if ( stopped ) {
@@ -136,6 +141,9 @@ public class BootstrapListener {
 						throw ioe;
 					}
 					
+					logger.debug("Received connection from Bootstrap");
+					socket.setSoTimeout(5000);
+					
 					executor.submit(new Runnable() {
 						@Override
 						public void run() {
@@ -184,27 +192,42 @@ public class BootstrapListener {
 		out.flush();
 	}
 	
-	private BootstrapRequest readRequest(final InputStream in) throws IOException {
-		final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+	
+	@SuppressWarnings("resource")  // we don't want to close the stream, as the caller will do that
+    private BootstrapRequest readRequest(final InputStream in) throws IOException {
+	    // We want to ensure that we don't try to read data from an InputStream directly
+	    // by a BufferedReader because any user on the system could open a socket and send
+	    // a multi-gigabyte file without any new lines in order to crash the NiFi instance
+	    // (or at least cause OutOfMemoryErrors, which can wreak havoc on the running instance).
+	    // So we will limit the Input Stream to only 4 KB, which should be plenty for any request.
+	    final LimitingInputStream limitingIn = new LimitingInputStream(in, 4096);
+		final BufferedReader reader = new BufferedReader(new InputStreamReader(limitingIn));
 		
 		final String line = reader.readLine();
 		final String[] splits = line.split(" ");
 		if ( splits.length < 0 ) {
-			throw new IOException("Received invalid command from NiFi: " + line);
+			throw new IOException("Received invalid request from Bootstrap: " + line);
 		}
 		
 		final String requestType = splits[0];
 		final String[] args;
 		if ( splits.length == 1 ) {
-			args = new String[0];
+			throw new IOException("Received invalid request from Bootstrap; request did not have a secret key; request type = " + requestType);
+		} else if ( splits.length == 2 ) {
+		    args = new String[0];
 		} else {
-			args = Arrays.copyOfRange(splits, 1, splits.length);
+			args = Arrays.copyOfRange(splits, 2, splits.length);
+		}
+		
+		final String requestKey = splits[1];
+		if ( !secretKey.equals(requestKey) ) {
+		    throw new IOException("Received invalid Secret Key for request type " + requestType);
 		}
 		
 		try {
 			return new BootstrapRequest(requestType, args);
 		} catch (final Exception e) {
-			throw new IOException("Received invalid request from bootstrap; request type = " + requestType);
+			throw new IOException("Received invalid request from Bootstrap; request type = " + requestType);
 		}
 	}
 	
@@ -227,7 +250,8 @@ public class BootstrapListener {
 			return requestType;
 		}
 		
-		public String[] getArgs() {
+		@SuppressWarnings("unused")
+        public String[] getArgs() {
 			return args;
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/76f54f86/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/NiFi.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/NiFi.java b/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/NiFi.java
index 13cd4d6..98489af 100644
--- a/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/NiFi.java
+++ b/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/NiFi.java
@@ -36,7 +36,6 @@ import org.apache.nifi.nar.NarClassLoaders;
 import org.apache.nifi.nar.NarUnpacker;
 import org.apache.nifi.util.FileUtils;
 import org.apache.nifi.util.NiFiProperties;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.bridge.SLF4JBridgeHandler;
@@ -61,7 +60,6 @@ public class NiFi {
 
         // register the shutdown hook
         Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() {
-
             @Override
             public void run() {
                 // shutdown the jetty server

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/76f54f86/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/util/LimitingInputStream.java
----------------------------------------------------------------------
diff --git a/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/util/LimitingInputStream.java b/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/util/LimitingInputStream.java
new file mode 100644
index 0000000..ce3a6db
--- /dev/null
+++ b/nar-bundles/framework-bundle/framework/runtime/src/main/java/org/apache/nifi/util/LimitingInputStream.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class LimitingInputStream extends InputStream {
+
+    private final InputStream in;
+    private final long limit;
+    private long bytesRead = 0;
+
+    public LimitingInputStream(final InputStream in, final long limit) {
+        this.in = in;
+        this.limit = limit;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int val = in.read();
+        if (val > -1) {
+            bytesRead++;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(final byte[] b) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(b.length, limit - bytesRead);
+
+        final int val = in.read(b, 0, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(len, limit - bytesRead);
+
+        final int val = in.read(b, off, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public long skip(final long n) throws IOException {
+        final long skipped = in.skip(Math.min(n, limit - bytesRead));
+        bytesRead += skipped;
+        return skipped;
+    }
+
+    @Override
+    public int available() throws IOException {
+        return in.available();
+    }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
+
+    @Override
+    public void mark(int readlimit) {
+        in.mark(readlimit);
+    }
+
+    @Override
+    public boolean markSupported() {
+        return in.markSupported();
+    }
+
+    @Override
+    public void reset() throws IOException {
+        in.reset();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/76f54f86/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java
index fb10930..f03bf1e 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/BootstrapCodec.java
@@ -64,7 +64,7 @@ public class BootstrapCodec {
 	private void processRequest(final String cmd, final String[] args) throws InvalidCommandException, IOException {
 		switch (cmd) {
 			case "PORT": {
-				if ( args.length != 1 ) {
+				if ( args.length != 2 ) {
 					throw new InvalidCommandException();
 				}
 				
@@ -78,8 +78,10 @@ public class BootstrapCodec {
 				if ( port < 1 || port > 65535 ) {
 					throw new InvalidCommandException("Invalid Port number; should be integer between 1 and 65535");
 				}
+
+				final String secretKey = args[1];
 				
-				runner.setNiFiCommandControlPort(port);
+				runner.setNiFiCommandControlPort(port, secretKey);
 				writer.write("OK");
 				writer.newLine();
 				writer.flush();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/76f54f86/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java
index c831351..f05d45a 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/NiFiListener.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.bootstrap;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
@@ -24,6 +25,8 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.nifi.bootstrap.util.LimitingInputStream;
+
 public class NiFiListener {
 	private ServerSocket serverSocket;
 	private volatile Listener listener;
@@ -92,17 +95,26 @@ public class NiFiListener {
 						throw ioe;
 					}
 					
-					
 					executor.submit(new Runnable() {
 						@Override
 						public void run() {
 							try {
-								final BootstrapCodec codec = new BootstrapCodec(runner, socket.getInputStream(), socket.getOutputStream());
+							    // we want to ensure that we don't try to read data from an InputStream directly
+						        // by a BufferedReader because any user on the system could open a socket and send
+						        // a multi-gigabyte file without any new lines in order to crash the Bootstrap,
+							    // which in turn may cause the Shutdown Hook to shutdown NiFi.
+						        // So we will limit the amount of data to read to 4 KB
+						        final InputStream limitingIn = new LimitingInputStream(socket.getInputStream(), 4096);
+								final BootstrapCodec codec = new BootstrapCodec(runner, limitingIn, socket.getOutputStream());
 								codec.communicate();
-								socket.close();
 							} catch (final Throwable t) {
 								System.out.println("Failed to communicate with NiFi due to " + t);
 								t.printStackTrace();
+							} finally {
+							    try {
+							        socket.close();
+							    } catch (final IOException ioe) {
+							    }
 							}
 						}
 					});

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/76f54f86/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
index a5987bc..a230711 100644
--- a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/RunNiFi.java
@@ -31,7 +31,10 @@ import java.lang.reflect.Field;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFilePermission;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -205,6 +208,20 @@ public class RunNiFi {
 	
 	private synchronized void saveProperties(final Properties nifiProps) throws IOException {
 	    final File statusFile = getStatusFile();
+	    if ( statusFile.exists() && !statusFile.delete() ) {
+	        logger.warning("Failed to delete " + statusFile);
+	    }
+
+	    if ( !statusFile.createNewFile() ) {
+	        throw new IOException("Failed to create file " + statusFile);
+	    }
+
+	    try {
+	        Files.setPosixFilePermissions(statusFile.toPath(), Collections.singleton(PosixFilePermission.OWNER_READ));
+	    } catch (final Exception e) {
+	        logger.warning("Failed to set permissions so that only the owner can read status file " + statusFile + "; 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");
+	    }
+	    
         try (final FileOutputStream fos = new FileOutputStream(statusFile)) {
             nifiProps.store(fos, null);
             fos.getFD().sync();
@@ -213,16 +230,16 @@ public class RunNiFi {
         logger.fine("Saved Properties " + nifiProps + " to " + statusFile);
 	}
 
-	private boolean isPingSuccessful(final int port) {
+	private boolean isPingSuccessful(final int port, final String secretKey) {
 	    logger.fine("Pinging " + port);
 	    
 	    try (final Socket socket = new Socket("localhost", port)) {
             final OutputStream out = socket.getOutputStream();
-            out.write((PING_CMD + "\n").getBytes(StandardCharsets.UTF_8));
+            out.write((PING_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
             out.flush();
 
             logger.fine("Sent PING command");
-            
+            socket.setSoTimeout(5000);
             final InputStream in = socket.getInputStream();
             final BufferedReader reader = new BufferedReader(new InputStreamReader(in));
             final String response = reader.readLine();
@@ -245,7 +262,7 @@ public class RunNiFi {
 		}
 		
 		final int port = Integer.parseInt(portVal);
-	    final boolean success = isPingSuccessful(port);
+	    final boolean success = isPingSuccessful(port, props.getProperty("secret.key"));
 	    if ( success ) {
 	        logger.fine("Successful PING on port " + port);
 	        return port;
@@ -271,10 +288,7 @@ public class RunNiFi {
 	        // We use the "ps" command to check if the process is still running.
 	        final ProcessBuilder builder = new ProcessBuilder();
 	        
-	        // ps -p <pid> -o comm=
-	        // -> -p <pid> to filter just the pid we care about
-	        // -> -o comm= to remove headers from the output
-	        builder.command("ps", "-p", pid, "-o", "comm=");
+	        builder.command("ps", "-p", pid, "--no-headers");
 	        final Process proc = builder.start();
 	        
 	        // Read how many lines are output by the 'ps' command
@@ -321,6 +335,7 @@ public class RunNiFi {
 	    
         final String portValue = props.getProperty("port");
         final String pid = props.getProperty("pid");
+        final String secretKey = props.getProperty("secret.key");
         
         if ( portValue == null && pid == null ) {
             return new Status(null, null, false, false);
@@ -331,7 +346,7 @@ public class RunNiFi {
         if ( portValue != null ) {
             try {
                 port = Integer.parseInt(portValue);
-                pingSuccess = isPingSuccessful(port);
+                pingSuccess = isPingSuccessful(port, secretKey);
             } catch (final NumberFormatException nfe) {
                 return new Status(null, null, false, false);
             }
@@ -373,14 +388,19 @@ public class RunNiFi {
 			return;
 		}
 		
+		final Properties nifiProps = loadProperties();
+		final String secretKey = nifiProps.getProperty("secret.key");
+		
 		try (final Socket socket = new Socket()) {
+		    logger.fine("Connecting to NiFi instance");
 			socket.setSoTimeout(60000);
 			socket.connect(new InetSocketAddress("localhost", port));
+			logger.fine("Established connection to NiFi instance.");
 			socket.setSoTimeout(60000);
 			
 			logger.fine("Sending SHUTDOWN Command to port " + port);
 			final OutputStream out = socket.getOutputStream();
-			out.write((SHUTDOWN_CMD + "\n").getBytes(StandardCharsets.UTF_8));
+			out.write((SHUTDOWN_CMD + " " + secretKey + "\n").getBytes(StandardCharsets.UTF_8));
 			out.flush();
 			
 			final InputStream in = socket.getInputStream();
@@ -392,10 +412,8 @@ public class RunNiFi {
 			if ( SHUTDOWN_CMD.equals(response) ) {
 				logger.info("Apache NiFi has accepted the Shutdown Command and is shutting down now");
 				
-				final Properties nifiProps = loadProperties();
 				final String pid = nifiProps.getProperty("pid");
 				if ( pid != null ) {
-
 			        final Properties bootstrapProperties = new Properties();
 			        try (final FileInputStream fis = new FileInputStream(bootstrapConfigFile)) {
 			            bootstrapProperties.load(fis);
@@ -418,7 +436,7 @@ public class RunNiFi {
 			                if ( isProcessRunning(pid) ) {
 			                    logger.warning("NiFi has not finished shutting down after " + gracefulShutdownSeconds + " seconds. Killing process.");
 			                    try {
-			                        killProcess(pid);
+			                        killProcessTree(pid);
 			                    } catch (final IOException ioe) {
 			                        logger.severe("Failed to kill Process with PID " + pid);
 			                    }
@@ -448,7 +466,31 @@ public class RunNiFi {
 	}
 	
 	
-	private static void killProcess(final String pid) throws IOException {
+	private static List<String> getChildProcesses(final String ppid) throws IOException {
+	    final Process proc = Runtime.getRuntime().exec(new String[] {"ps", "-o", "pid", "--no-headers", "--ppid", ppid});
+	    final List<String> childPids = new ArrayList<>();
+	    try (final InputStream in = proc.getInputStream();
+	         final BufferedReader reader = new BufferedReader(new InputStreamReader(in))) {
+	        
+	        String line;
+	        while ((line = reader.readLine()) != null) {
+	            childPids.add(line.trim());
+	        }
+	    }
+	    
+	    return childPids;
+	}
+	
+	private void killProcessTree(final String pid) throws IOException {
+	    logger.fine("Killing Process Tree for PID " + pid);
+	    
+	    final List<String> children = getChildProcesses(pid);
+	    logger.fine("Children of PID " + pid + ": " + children);
+	    
+	    for ( final String childPid : children ) {
+	        killProcessTree(childPid);
+	    }
+	    
 	    Runtime.getRuntime().exec(new String[] {"kill", "-9", pid});
 	}
 	
@@ -620,7 +662,7 @@ public class RunNiFi {
                 nifiPid = pid;
                 final Properties nifiProps = new Properties();
                 nifiProps.setProperty("pid", String.valueOf(nifiPid));
-                saveProperties(properties);
+                saveProperties(nifiProps);
             }
 			
 			ShutdownHook shutdownHook = new ShutdownHook(process, this, gracefulShutdownSeconds);
@@ -651,7 +693,7 @@ public class RunNiFi {
 			                nifiPid = pid;
 			                final Properties nifiProps = new Properties();
 			                nifiProps.setProperty("pid", String.valueOf(nifiPid));
-			                saveProperties(properties);
+			                saveProperties(nifiProps);
 			            }
 						
 						shutdownHook = new ShutdownHook(process, this, gracefulShutdownSeconds);
@@ -677,7 +719,7 @@ public class RunNiFi {
 			    nifiPid = pid;
                 final Properties nifiProps = new Properties();
                 nifiProps.setProperty("pid", String.valueOf(nifiPid));
-                saveProperties(properties);
+                saveProperties(nifiProps);
 			}
 			
 			boolean started = waitForStart();
@@ -758,9 +800,8 @@ public class RunNiFi {
 		this.autoRestartNiFi = restart;
 	}
 	
-	void setNiFiCommandControlPort(final int port) {
+	void setNiFiCommandControlPort(final int port, final String secretKey) {
 		this.ccPort = port;
-
 		final File statusFile = getStatusFile();
 		
 		final Properties nifiProps = new Properties();
@@ -768,6 +809,8 @@ public class RunNiFi {
 		    nifiProps.setProperty("pid", String.valueOf(nifiPid));
 		}
 		nifiProps.setProperty("port", String.valueOf(ccPort));
+		nifiProps.setProperty("secret.key", secretKey);
+		
 		try {
 		    saveProperties(nifiProps);
 		} catch (final IOException ioe) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/76f54f86/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/util/LimitingInputStream.java
----------------------------------------------------------------------
diff --git a/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/util/LimitingInputStream.java b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/util/LimitingInputStream.java
new file mode 100644
index 0000000..2149342
--- /dev/null
+++ b/nifi-bootstrap/src/main/java/org/apache/nifi/bootstrap/util/LimitingInputStream.java
@@ -0,0 +1,107 @@
+/*
+ * 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.bootstrap.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public class LimitingInputStream extends InputStream {
+
+    private final InputStream in;
+    private final long limit;
+    private long bytesRead = 0;
+
+    public LimitingInputStream(final InputStream in, final long limit) {
+        this.in = in;
+        this.limit = limit;
+    }
+
+    @Override
+    public int read() throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int val = in.read();
+        if (val > -1) {
+            bytesRead++;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(final byte[] b) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(b.length, limit - bytesRead);
+
+        final int val = in.read(b, 0, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        if (bytesRead >= limit) {
+            return -1;
+        }
+
+        final int maxToRead = (int) Math.min(len, limit - bytesRead);
+
+        final int val = in.read(b, off, maxToRead);
+        if (val > 0) {
+            bytesRead += val;
+        }
+        return val;
+    }
+
+    @Override
+    public long skip(final long n) throws IOException {
+        final long skipped = in.skip(Math.min(n, limit - bytesRead));
+        bytesRead += skipped;
+        return skipped;
+    }
+
+    @Override
+    public int available() throws IOException {
+        return in.available();
+    }
+
+    @Override
+    public void close() throws IOException {
+        in.close();
+    }
+
+    @Override
+    public void mark(int readlimit) {
+        in.mark(readlimit);
+    }
+
+    @Override
+    public boolean markSupported() {
+        return in.markSupported();
+    }
+
+    @Override
+    public void reset() throws IOException {
+        in.reset();
+    }
+}


[8/8] incubator-nifi git commit: Merge branch 'bootstrap' into develop

Posted by ma...@apache.org.
Merge branch 'bootstrap' into develop


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/575cc42c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/575cc42c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/575cc42c

Branch: refs/heads/develop
Commit: 575cc42c72282b24d9a52c666a5934dbd666b116
Parents: a94cf4b c1681d9
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Dec 16 15:10:20 2014 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Dec 16 15:10:20 2014 -0500

----------------------------------------------------------------------
 .../repository/VolatileContentRepository.java   |   2 +-
 .../resources/src/main/resources/bin/nifi.sh    |  80 ++-
 .../src/main/resources/conf/bootstrap.conf      |   8 +-
 .../java/org/apache/nifi/BootstrapListener.java |  46 +-
 .../src/main/java/org/apache/nifi/NiFi.java     |   2 -
 .../apache/nifi/util/LimitingInputStream.java   | 107 ++++
 .../apache/nifi/bootstrap/BootstrapCodec.java   |   7 +-
 .../org/apache/nifi/bootstrap/NiFiListener.java |  18 +-
 .../java/org/apache/nifi/bootstrap/RunNiFi.java | 496 ++++++++++++++++---
 .../bootstrap/util/LimitingInputStream.java     | 107 ++++
 10 files changed, 749 insertions(+), 124 deletions(-)
----------------------------------------------------------------------