You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by al...@apache.org on 2015/09/21 15:37:59 UTC

[1/2] incubator-brooklyn git commit: Capture exit code in AbstractSoftwareProcessWinrmDriver - Refactor AbstractSoftwareProcessWinrmDriver implementation like AbstractSoftwareProcessSshDriver - ScriptHelper exit code test - Test no output gathering for S

Repository: incubator-brooklyn
Updated Branches:
  refs/heads/master 10c657648 -> 032c53309


Capture exit code in AbstractSoftwareProcessWinrmDriver
- Refactor AbstractSoftwareProcessWinrmDriver implementation like AbstractSoftwareProcessSshDriver
- ScriptHelper exit code test
- Test no output gathering for ScriptHelper


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

Branch: refs/heads/master
Commit: b6bc247f955578798e23d1829d9abd099a41ac6e
Parents: 08995e0
Author: Valentin Aitken <va...@cloudsoftcorp.com>
Authored: Wed Sep 9 22:47:24 2015 +0300
Committer: Valentin Aitken <va...@cloudsoftcorp.com>
Committed: Mon Sep 21 15:43:40 2015 +0300

----------------------------------------------------------------------
 .../AbstractSoftwareProcessWinRmDriver.java     |  98 +++++++--
 .../base/VanillaWindowsProcessWinRmDriver.java  |  33 ++-
 .../lifecycle/NativeWindowsScriptRunner.java    |  29 +++
 .../base/lifecycle/WinRmExecuteHelper.java      | 217 +++++++++++++++++++
 .../AbstractSoftwareProcessStreamsTest.java     |   8 +-
 .../DoNothingWinRmSoftwareProcessDriver.java    |  68 ++++++
 ...laSoftwareProcessStreamsIntegrationTest.java | 107 ++-------
 ...nillaWindowsProcessWinrmStreamsLiveTest.java |  42 ++--
 .../base/lifecycle/ScriptHelperUnitTest.java    | 146 +++++++++++++
 .../lifecycle/WinRmExecuteHelperUnitTest.java   |  62 ++++++
 10 files changed, 669 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/main/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessWinRmDriver.java
----------------------------------------------------------------------
diff --git a/software/base/src/main/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessWinRmDriver.java b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessWinRmDriver.java
index 8831ba2..7091147 100644
--- a/software/base/src/main/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessWinRmDriver.java
+++ b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessWinRmDriver.java
@@ -19,14 +19,17 @@
 package org.apache.brooklyn.entity.software.base;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
 import io.cloudsoft.winrm4j.winrm.WinRmToolResponse;
 import org.apache.brooklyn.api.entity.EntityLocal;
 import org.apache.brooklyn.api.mgmt.Task;
 import org.apache.brooklyn.api.sensor.AttributeSensor;
-import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.core.entity.BrooklynConfigKeys;
+import org.apache.brooklyn.core.entity.Entities;
 import org.apache.brooklyn.core.mgmt.BrooklynTaskTags;
 import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.entity.software.base.lifecycle.NativeWindowsScriptRunner;
+import org.apache.brooklyn.entity.software.base.lifecycle.WinRmExecuteHelper;
 import org.apache.brooklyn.location.winrm.WinRmMachineLocation;
 import org.apache.brooklyn.util.core.task.Tasks;
 import org.apache.brooklyn.util.exceptions.ReferenceWithError;
@@ -46,7 +49,9 @@ import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
-public abstract class AbstractSoftwareProcessWinRmDriver extends AbstractSoftwareProcessDriver {
+import static org.apache.brooklyn.util.JavaGroovyEquivalents.elvis;
+
+public abstract class AbstractSoftwareProcessWinRmDriver extends AbstractSoftwareProcessDriver implements NativeWindowsScriptRunner {
     private static final Logger LOG = LoggerFactory.getLogger(AbstractSoftwareProcessWinRmDriver.class);
 
     AttributeSensor<String> WINDOWS_USERNAME = Sensors.newStringSensor("windows.username",
@@ -60,10 +65,26 @@ public abstract class AbstractSoftwareProcessWinRmDriver extends AbstractSoftwar
         entity.setAttribute(WINDOWS_PASSWORD, location.config().get(WinRmMachineLocation.PASSWORD));
     }
 
+    /** @see #newScript(Map, String) */
+    protected WinRmExecuteHelper newScript(String phase) {
+        return newScript(Maps.<String, Object>newLinkedHashMap(), phase);
+    }
+
+    protected WinRmExecuteHelper newScript(Map<String, ?> flags, String phase) {
+        if (!Entities.isManaged(getEntity()))
+            throw new IllegalStateException(getEntity() + " is no longer managed; cannot create script to run here (" + phase + ")");
+
+        WinRmExecuteHelper s = new WinRmExecuteHelper(this, phase + " " + elvis(entity, this));
+        return s;
+    }
+
     @Override
     public void runPreInstallCommand() {
         if (Strings.isNonBlank(getEntity().getConfig(VanillaWindowsProcess.PRE_INSTALL_COMMAND)) || Strings.isNonBlank(getEntity().getConfig(VanillaWindowsProcess.PRE_INSTALL_POWERSHELL_COMMAND))) {
-            executeCommand(VanillaWindowsProcess.PRE_INSTALL_COMMAND, VanillaWindowsProcess.PRE_INSTALL_POWERSHELL_COMMAND, true);
+            executeCommandInTask(
+                    getEntity().getConfig(VanillaWindowsProcess.PRE_INSTALL_COMMAND),
+                    getEntity().getConfig(VanillaWindowsProcess.PRE_INSTALL_POWERSHELL_COMMAND),
+                    "pre-install-command");
         }
         if (entity.getConfig(VanillaWindowsProcess.PRE_INSTALL_REBOOT_REQUIRED)) {
             rebootAndWait();
@@ -78,21 +99,30 @@ public abstract class AbstractSoftwareProcessWinRmDriver extends AbstractSoftwar
     @Override
     public void runPostInstallCommand() {
         if (Strings.isNonBlank(entity.getConfig(BrooklynConfigKeys.POST_INSTALL_COMMAND)) || Strings.isNonBlank(getEntity().getConfig(VanillaWindowsProcess.POST_INSTALL_POWERSHELL_COMMAND))) {
-            executeCommand(BrooklynConfigKeys.POST_INSTALL_COMMAND, VanillaWindowsProcess.POST_INSTALL_POWERSHELL_COMMAND, true);
+            executeCommandInTask(
+                    getEntity().getConfig(VanillaWindowsProcess.POST_INSTALL_COMMAND),
+                    getEntity().getConfig(VanillaWindowsProcess.POST_INSTALL_POWERSHELL_COMMAND),
+                    "post-install-command");
         }
     }
 
     @Override
     public void runPreLaunchCommand() {
         if (Strings.isNonBlank(entity.getConfig(BrooklynConfigKeys.PRE_LAUNCH_COMMAND)) || Strings.isNonBlank(entity.getConfig(VanillaWindowsProcess.PRE_LAUNCH_POWERSHELL_COMMAND))) {
-            executeCommand(BrooklynConfigKeys.PRE_LAUNCH_COMMAND, VanillaWindowsProcess.PRE_LAUNCH_POWERSHELL_COMMAND, true);
+            executeCommandInTask(
+                    getEntity().getConfig(VanillaWindowsProcess.PRE_LAUNCH_COMMAND),
+                    getEntity().getConfig(VanillaWindowsProcess.PRE_LAUNCH_POWERSHELL_COMMAND),
+                    "pre-launch-command");
         }
     }
 
     @Override
     public void runPostLaunchCommand() {
         if (Strings.isNonBlank(entity.getConfig(BrooklynConfigKeys.POST_LAUNCH_COMMAND)) || Strings.isNonBlank(entity.getConfig(VanillaWindowsProcess.POST_LAUNCH_POWERSHELL_COMMAND))) {
-            executeCommand(BrooklynConfigKeys.POST_LAUNCH_COMMAND, VanillaWindowsProcess.POST_LAUNCH_POWERSHELL_COMMAND, true);
+            executeCommandInTask(
+                    getEntity().getConfig(VanillaWindowsProcess.POST_LAUNCH_COMMAND),
+                    getEntity().getConfig(VanillaWindowsProcess.POST_LAUNCH_POWERSHELL_COMMAND),
+                    "post-launch-command");
         }
     }
 
@@ -101,6 +131,29 @@ public abstract class AbstractSoftwareProcessWinRmDriver extends AbstractSoftwar
         return (WinRmMachineLocation)super.getLocation();
     }
 
+    public WinRmMachineLocation getMachine() {
+        return getLocation();
+    }
+
+    protected int executeCommandInTask(String command, String psCommand, String phase) {
+        return newScript(phase)
+                .setCommand(command)
+                .setPsCommand(psCommand)
+                .failOnNonZeroResultCode()
+                .gatherOutput()
+                .execute();
+    }
+
+    @Override
+    public int executeNativeCommand(Map flags, String command, String phase) {
+        return executeNativeOrPsCommand(flags, command, null, phase, true);
+    }
+
+    @Override
+    public int executePsCommand(Map flags, String command, String phase) {
+        return executeNativeOrPsCommand(flags, null, command, phase, true);
+    }
+
     @Override
     public String getRunDir() {
         // TODO: This needs to be tidied, and read from the appropriate flags (if set)
@@ -134,37 +187,40 @@ public abstract class AbstractSoftwareProcessWinRmDriver extends AbstractSoftwar
         getLocation().executePsScript("New-Item -path \"" + directoryName + "\" -type directory -ErrorAction SilentlyContinue");
     }
 
-    protected WinRmToolResponse executeCommand(ConfigKey<String> regularCommandKey, ConfigKey<String> powershellCommandKey, boolean allowNoOp) {
-        String regularCommand = getEntity().getConfig(regularCommandKey);
-        String powershellCommand = getEntity().getConfig(powershellCommandKey);
-        if (Strings.isBlank(regularCommand) && Strings.isBlank(powershellCommand)) {
+    @Override
+    public Integer executeNativeOrPsCommand(Map flags, String regularCommand, String powerShellCommand, String phase, Boolean allowNoOp) {
+        if (Strings.isBlank(regularCommand) && Strings.isBlank(powerShellCommand)) {
             if (allowNoOp) {
-                return new WinRmToolResponse("", "", 0);
+                return new WinRmToolResponse("", "", 0).getStatusCode();
             } else {
-                throw new IllegalStateException(String.format("Exactly one of %s or %s must be set", regularCommandKey.getName(), powershellCommandKey.getName()));
+                throw new IllegalStateException(String.format("Exactly one of %s or %s must be set", regularCommand, powerShellCommand));
             }
-        } else if (!Strings.isBlank(regularCommand) && !Strings.isBlank(powershellCommand)) {
-            throw new IllegalStateException(String.format("%s and %s cannot both be set", regularCommandKey.getName(), powershellCommandKey.getName()));
+        } else if (!Strings.isBlank(regularCommand) && !Strings.isBlank(powerShellCommand)) {
+            throw new IllegalStateException(String.format("%s and %s cannot both be set", regularCommand, powerShellCommand));
         }
 
         ByteArrayOutputStream stdIn = new ByteArrayOutputStream();
-        ByteArrayOutputStream stdOut = new ByteArrayOutputStream();
-        ByteArrayOutputStream stdErr = new ByteArrayOutputStream();
+        ByteArrayOutputStream stdOut = flags.get("out") != null ? (ByteArrayOutputStream)flags.get("out") : new ByteArrayOutputStream();
+        ByteArrayOutputStream stdErr = flags.get("err") != null ? (ByteArrayOutputStream)flags.get("err") : new ByteArrayOutputStream();
 
         Task<?> currentTask = Tasks.current();
         if (currentTask != null) {
-            writeToStream(stdIn, Strings.isBlank(regularCommand) ? powershellCommand : regularCommand);
-            Tasks.addTagDynamically(BrooklynTaskTags.tagForStreamSoft(BrooklynTaskTags.STREAM_STDIN, stdIn));
+            if (BrooklynTaskTags.stream(Tasks.current(), BrooklynTaskTags.STREAM_STDIN)==null) {
+                writeToStream(stdIn, Strings.isBlank(regularCommand) ? powerShellCommand : regularCommand);
+                Tasks.addTagDynamically(BrooklynTaskTags.tagForStreamSoft(BrooklynTaskTags.STREAM_STDIN, stdIn));
+            }
 
             if (BrooklynTaskTags.stream(currentTask, BrooklynTaskTags.STREAM_STDOUT)==null) {
                 Tasks.addTagDynamically(BrooklynTaskTags.tagForStreamSoft(BrooklynTaskTags.STREAM_STDOUT, stdOut));
+                flags.put("out", stdOut);
                 Tasks.addTagDynamically(BrooklynTaskTags.tagForStreamSoft(BrooklynTaskTags.STREAM_STDERR, stdErr));
+                flags.put("err", stdErr);
             }
         }
 
         WinRmToolResponse response;
         if (Strings.isBlank(regularCommand)) {
-            response = getLocation().executePsScript(ImmutableList.of(powershellCommand));
+            response = getLocation().executePsScript(ImmutableList.of(powerShellCommand));
         } else {
             response = getLocation().executeScript(ImmutableList.of(regularCommand));
         }
@@ -174,10 +230,10 @@ public abstract class AbstractSoftwareProcessWinRmDriver extends AbstractSoftwar
             writeToStream(stdErr, response.getStdErr());
         }
 
-        return response;
+        return response.getStatusCode();
     }
 
-    private void writeToStream(ByteArrayOutputStream stream, String string)  {
+    private void writeToStream(ByteArrayOutputStream stream, String string) {
         try {
             stream.write(string.getBytes());
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/main/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinRmDriver.java
----------------------------------------------------------------------
diff --git a/software/base/src/main/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinRmDriver.java b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinRmDriver.java
index c70a35e..dad0e04 100644
--- a/software/base/src/main/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinRmDriver.java
+++ b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinRmDriver.java
@@ -18,7 +18,6 @@
  */
 package org.apache.brooklyn.entity.software.base;
 
-import io.cloudsoft.winrm4j.winrm.WinRmToolResponse;
 import org.apache.brooklyn.api.entity.EntityLocal;
 import org.apache.brooklyn.core.entity.Attributes;
 import org.apache.brooklyn.location.winrm.WinRmMachineLocation;
@@ -47,7 +46,10 @@ public class VanillaWindowsProcessWinRmDriver extends AbstractSoftwareProcessWin
     public void install() {
         // TODO: Follow install path of VanillaSoftwareProcessSshDriver
         if(Strings.isNonBlank(getEntity().getConfig(VanillaWindowsProcess.INSTALL_COMMAND)) || Strings.isNonBlank(getEntity().getConfig(VanillaWindowsProcess.INSTALL_POWERSHELL_COMMAND))) {
-            executeCommand(VanillaWindowsProcess.INSTALL_COMMAND, VanillaWindowsProcess.INSTALL_POWERSHELL_COMMAND, true);
+            executeCommandInTask(
+                    getEntity().getConfig(VanillaWindowsProcess.INSTALL_COMMAND),
+                    getEntity().getConfig(VanillaWindowsProcess.INSTALL_POWERSHELL_COMMAND),
+                    "install-command");
         }
         if (entity.getConfig(VanillaWindowsProcess.INSTALL_REBOOT_REQUIRED)) {
             rebootAndWait();
@@ -56,9 +58,11 @@ public class VanillaWindowsProcessWinRmDriver extends AbstractSoftwareProcessWin
 
     @Override
     public void customize() {
-        // TODO: Follow customize path of VanillaSoftwareProcessSshDriver
         if(Strings.isNonBlank(getEntity().getConfig(VanillaWindowsProcess.CUSTOMIZE_COMMAND)) || Strings.isNonBlank(getEntity().getConfig(VanillaWindowsProcess.CUSTOMIZE_POWERSHELL_COMMAND))) {
-            executeCommand(VanillaWindowsProcess.CUSTOMIZE_COMMAND, VanillaWindowsProcess.CUSTOMIZE_POWERSHELL_COMMAND, true);
+            executeCommandInTask(
+                    getEntity().getConfig(VanillaWindowsProcess.CUSTOMIZE_COMMAND),
+                    getEntity().getConfig(VanillaWindowsProcess.CUSTOMIZE_POWERSHELL_COMMAND),
+                    "customize-command");
         }
         if (entity.getConfig(VanillaWindowsProcess.CUSTOMIZE_REBOOT_REQUIRED)) {
             rebootAndWait();
@@ -67,22 +71,29 @@ public class VanillaWindowsProcessWinRmDriver extends AbstractSoftwareProcessWin
 
     @Override
     public void launch() {
-        executeCommand(VanillaWindowsProcess.LAUNCH_COMMAND, VanillaWindowsProcess.LAUNCH_POWERSHELL_COMMAND, true);
+        executeCommandInTask(
+                getEntity().getConfig(VanillaWindowsProcess.LAUNCH_COMMAND),
+                getEntity().getConfig(VanillaWindowsProcess.LAUNCH_POWERSHELL_COMMAND),
+                "launch-command");
     }
 
     @Override
     public boolean isRunning() {
-        WinRmToolResponse runningCheck = executeCommand(VanillaWindowsProcess.CHECK_RUNNING_COMMAND,
-                VanillaWindowsProcess.CHECK_RUNNING_POWERSHELL_COMMAND, false);
-        if(runningCheck.getStatusCode() != 0) {
-            LOG.info(getEntity() + " isRunning check failed: exit code "  + runningCheck.getStatusCode() + "; " + runningCheck.getStdErr());
+        int exitCode = executeCommandInTask(
+                getEntity().getConfig(VanillaWindowsProcess.CHECK_RUNNING_COMMAND),
+                getEntity().getConfig(VanillaWindowsProcess.CHECK_RUNNING_POWERSHELL_COMMAND), "is-running-command");
+        if(exitCode != 0) {
+            LOG.info(getEntity() + " isRunning check failed: exit code "  + exitCode);
         }
-        return runningCheck.getStatusCode() == 0;
+        return exitCode == 0;
     }
 
     @Override
     public void stop() {
-        executeCommand(VanillaWindowsProcess.STOP_COMMAND, VanillaWindowsProcess.STOP_POWERSHELL_COMMAND, true);
+        executeCommandInTask(
+                getEntity().getConfig(VanillaWindowsProcess.STOP_COMMAND),
+                getEntity().getConfig(VanillaWindowsProcess.STOP_POWERSHELL_COMMAND),
+                "stop-command");
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/main/java/org/apache/brooklyn/entity/software/base/lifecycle/NativeWindowsScriptRunner.java
----------------------------------------------------------------------
diff --git a/software/base/src/main/java/org/apache/brooklyn/entity/software/base/lifecycle/NativeWindowsScriptRunner.java b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/lifecycle/NativeWindowsScriptRunner.java
new file mode 100644
index 0000000..26fde11
--- /dev/null
+++ b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/lifecycle/NativeWindowsScriptRunner.java
@@ -0,0 +1,29 @@
+/*
+ * 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.brooklyn.entity.software.base.lifecycle;
+
+import java.util.Map;
+
+public interface NativeWindowsScriptRunner {
+
+    /** Runs a command and returns the result code */
+    int executeNativeCommand(Map flags, String windowsCommand, String summaryForLogging);
+    int executePsCommand(Map flags, String powerShellCommand, String summaryForLogging);
+    Integer executeNativeOrPsCommand(Map flags, String regularCommand, String powershellCommand, String phase, Boolean allowNoOp);
+}

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/main/java/org/apache/brooklyn/entity/software/base/lifecycle/WinRmExecuteHelper.java
----------------------------------------------------------------------
diff --git a/software/base/src/main/java/org/apache/brooklyn/entity/software/base/lifecycle/WinRmExecuteHelper.java b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/lifecycle/WinRmExecuteHelper.java
new file mode 100644
index 0000000..60b58c4
--- /dev/null
+++ b/software/base/src/main/java/org/apache/brooklyn/entity/software/base/lifecycle/WinRmExecuteHelper.java
@@ -0,0 +1,217 @@
+/*
+ * 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.brooklyn.entity.software.base.lifecycle;
+
+import com.google.common.annotations.Beta;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.mgmt.ExecutionContext;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.api.mgmt.TaskQueueingContext;
+import org.apache.brooklyn.core.mgmt.BrooklynTaskTags;
+import org.apache.brooklyn.util.core.task.DynamicTasks;
+import org.apache.brooklyn.util.core.task.TaskBuilder;
+import org.apache.brooklyn.util.core.task.Tasks;
+import org.apache.brooklyn.util.stream.Streams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+
+import static java.lang.String.format;
+
+/**
+ * <code>org.apache.brooklyn.entity.software.base.lifecycle.ScriptHelper</code> analog for WinRM
+ */
+public class WinRmExecuteHelper {
+    public static final Logger LOG = LoggerFactory.getLogger(WinRmExecuteHelper.class);
+
+    private Entity entity;
+    private Task<Integer> task;
+
+    protected final NativeWindowsScriptRunner runner;
+    public final String summary;
+
+    private String command;
+    private String psCommand;
+
+    @SuppressWarnings("rawtypes")
+    protected final Map flags = new LinkedHashMap();
+    protected Predicate<? super Integer> resultCodeCheck = Predicates.alwaysTrue();
+    protected ByteArrayOutputStream stdout, stderr, stdin;
+
+    public WinRmExecuteHelper(NativeWindowsScriptRunner runner, String summary) {
+        this.runner = runner;
+        this.summary = summary;
+    }
+
+    public WinRmExecuteHelper setCommand(String command) {
+        this.command = command;
+        return this;
+    }
+
+    public WinRmExecuteHelper setPsCommand(String psCommand) {
+        this.psCommand = psCommand;
+        return this;
+    }
+
+    protected Entity getEntity() {
+        return entity;
+    }
+
+    /** queues the task for execution if we are in a {@link TaskQueueingContext} (e.g. EffectorTaskFactory);
+     * or if we aren't in a queueing context, it will submit the task (assuming there is an {@link ExecutionContext}
+     * _and_ block until completion, throwing on error */
+    @Beta
+    public Task<Integer> queue() {
+        return DynamicTasks.queueIfPossible(newTask()).orSubmitAndBlock().getTask();
+    }
+
+    /** creates a task which will execute this script; note this can only be run once per instance of this class */
+    public synchronized Task<Integer> newTask() {
+        if (task!=null) throw new IllegalStateException("task can only be generated once");
+        TaskBuilder<Integer> tb = Tasks.<Integer>builder().displayName("winrm: "+summary).body(
+                new Callable<Integer>() {
+                    public Integer call() throws Exception {
+                        return executeInternal();
+                    }
+                });
+
+        try {
+            ByteArrayOutputStream stdin = new ByteArrayOutputStream();
+            stdin.write((command != null ? command : psCommand).getBytes());
+            tb.tag(BrooklynTaskTags.tagForStreamSoft(BrooklynTaskTags.STREAM_STDIN, stdin));
+        } catch (IOException e) {
+            LOG.warn("Error registering stream "+BrooklynTaskTags.STREAM_STDIN+" on "+tb+": "+e, e);
+        }
+
+        Map flags = getFlags();
+
+        Map<?,?> env = (Map<?,?>) flags.get("env");
+        if (env!=null) {
+            // if not explicitly set, env will come from getShellEnv in AbstractSoftwareProcessSshDriver.execute,
+            // which will also update this tag appropriately
+            tb.tag(BrooklynTaskTags.tagForEnvStream(BrooklynTaskTags.STREAM_ENV, env));
+        }
+
+        if (gatherOutput) {
+            stdout = new ByteArrayOutputStream();
+            tb.tag(BrooklynTaskTags.tagForStreamSoft(BrooklynTaskTags.STREAM_STDOUT, stdout));
+            stderr = new ByteArrayOutputStream();
+            tb.tag(BrooklynTaskTags.tagForStreamSoft(BrooklynTaskTags.STREAM_STDERR, stderr));
+        }
+        task = tb.build();
+        return task;
+    }
+
+    public int execute() {
+        if (DynamicTasks.getTaskQueuingContext()!=null) {
+            return queue().getUnchecked();
+        } else {
+            return executeInternal();
+        }
+    }
+
+    public int executeInternal() {
+        int result;
+        if (gatherOutput) {
+            if (stdout==null) stdout = new ByteArrayOutputStream();
+            if (stderr==null) stderr = new ByteArrayOutputStream();
+            flags.put("out", stdout);
+            flags.put("err", stderr);
+        }
+        result = runner.executeNativeOrPsCommand(flags, command, psCommand, summary, false);
+        if (!resultCodeCheck.apply(result)) {
+            throw logWithDetailsAndThrow(format("Execution failed, invalid result %s for %s", result, summary), null);
+        }
+        return result;
+    }
+
+    public WinRmExecuteHelper failOnNonZeroResultCode() {
+        return updateTaskAndFailOnNonZeroResultCode();
+    }
+
+    public WinRmExecuteHelper updateTaskAndFailOnNonZeroResultCode() {
+        gatherOutput();
+        // a failure listener would be a cleaner way
+
+        resultCodeCheck = new Predicate<Integer>() {
+            @Override
+            public boolean apply(@Nullable Integer input) {
+                if (input==0) return true;
+
+                try {
+                    String notes = "";
+                    if (!getResultStderr().isEmpty())
+                        notes += "STDERR\n" + getResultStderr()+"\n";
+                    if (!getResultStdout().isEmpty())
+                        notes += "\n" + "STDOUT\n" + getResultStdout()+"\n";
+                    Tasks.setExtraStatusDetails(notes.trim());
+                } catch (Exception e) {
+                    LOG.warn("Unable to collect additional metadata on failure of "+summary+": "+e);
+                }
+
+                return false;
+            }
+        };
+
+        return this;
+    }
+
+    protected boolean gatherOutput = false;
+
+    public WinRmExecuteHelper gatherOutput() {
+        return gatherOutput(true);
+    }
+    public WinRmExecuteHelper gatherOutput(boolean gather) {
+        gatherOutput = gather;
+        return this;
+    }
+
+    protected RuntimeException logWithDetailsAndThrow(String message, Throwable optionalCause) {
+        LOG.warn(message + " (throwing)");
+        int maxLength = 1024;
+        LOG.warn(message + " (throwing)");
+        Streams.logStreamTail(LOG, "STDERR of problem in "+Tasks.current(), stderr, maxLength);
+        Streams.logStreamTail(LOG, "STDOUT of problem in "+Tasks.current(), stdout, maxLength);
+        Streams.logStreamTail(LOG, "STDIN of problem in "+Tasks.current(), Streams.byteArrayOfString(command != null ? command : psCommand), 4096);
+        if (optionalCause!=null) throw new IllegalStateException(message, optionalCause);
+        throw new IllegalStateException(message);
+    }
+
+    @SuppressWarnings("rawtypes")
+    public Map getFlags() {
+        return flags;
+    }
+
+    public String getResultStdout() {
+        if (stdout==null) throw new IllegalStateException("output not available on "+this+"; ensure gatherOutput(true) is set");
+        return stdout.toString();
+    }
+    public String getResultStderr() {
+        if (stderr==null) throw new IllegalStateException("output not available on "+this+"; ensure gatherOutput(true) is set");
+        return stderr.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/test/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessStreamsTest.java
----------------------------------------------------------------------
diff --git a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessStreamsTest.java b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessStreamsTest.java
index a1f006a..78f991d 100644
--- a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessStreamsTest.java
+++ b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/AbstractSoftwareProcessStreamsTest.java
@@ -63,9 +63,9 @@ public abstract class AbstractSoftwareProcessStreamsTest extends BrooklynAppLive
         }
     }
 
-    protected String getStreamOrFail(Task<?> task, String streamType) {
+    public static String getStreamOrFail(Task<?> task, String streamType) {
         String msg = "task="+task+"; stream="+streamType;
-        BrooklynTaskTags.WrappedStream stream = checkNotNull(BrooklynTaskTags.stream(task, BrooklynTaskTags.STREAM_STDIN), "Stream null: " + msg);
+        BrooklynTaskTags.WrappedStream stream = checkNotNull(BrooklynTaskTags.stream(task, streamType), "Stream null: " + msg);
         return checkNotNull(stream.streamContents.get(), "Contents null: "+msg);
     }
 
@@ -105,8 +105,8 @@ public abstract class AbstractSoftwareProcessStreamsTest extends BrooklynAppLive
             String stdin = getStreamOrFail(subTask, BrooklynTaskTags.STREAM_STDIN);
             String stdout = getStreamOrFail(subTask, BrooklynTaskTags.STREAM_STDOUT);
             String stderr = getStreamOrFail(subTask, BrooklynTaskTags.STREAM_STDERR);
-            String env = getStreamOrFail(subTask, BrooklynTaskTags.STREAM_ENV);
-            String msg = "stdin="+stdin+"; stdout="+stdout+"; stderr="+stderr+"; env="+env;
+//            String env = getStreamOrFail(subTask, BrooklynTaskTags.STREAM_ENV);
+            String msg = "stdin="+stdin+"; stdout="+stdout+"; stderr="+stderr; //+"; env="+env;
 
             assertTrue(stdin.contains("echo "+echoed), msg);
             assertTrue(stdout.contains(echoed), msg);

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/test/java/org/apache/brooklyn/entity/software/base/DoNothingWinRmSoftwareProcessDriver.java
----------------------------------------------------------------------
diff --git a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/DoNothingWinRmSoftwareProcessDriver.java b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/DoNothingWinRmSoftwareProcessDriver.java
new file mode 100644
index 0000000..cbf0b3d
--- /dev/null
+++ b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/DoNothingWinRmSoftwareProcessDriver.java
@@ -0,0 +1,68 @@
+/*
+ * 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.brooklyn.entity.software.base;
+
+import org.apache.brooklyn.api.entity.EntityLocal;
+import org.apache.brooklyn.location.winrm.WinRmMachineLocation;
+
+/**
+ * Implements methods in {@link AbstractSoftwareProcessWinRmDriver}
+ * such that no actions are performed.
+ * <p>
+ * {@link #isRunning()} returns true.
+ */
+public class DoNothingWinRmSoftwareProcessDriver extends AbstractSoftwareProcessWinRmDriver {
+
+    public DoNothingWinRmSoftwareProcessDriver(EntityLocal entity, WinRmMachineLocation machine) {
+        super(entity, machine);
+    }
+
+    @Override
+    public boolean isRunning() {
+        return true;
+    }
+
+    @Override
+    public void copyPreInstallResources() {
+    }
+
+    @Override
+    public void copyInstallResources() {
+    }
+
+    @Override
+    public void copyRuntimeResources() {
+    }
+
+    @Override
+    public void install() {
+    }
+
+    @Override
+    public void customize() {
+    }
+
+    @Override
+    public void launch() {
+    }
+
+    @Override
+    public void stop() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaSoftwareProcessStreamsIntegrationTest.java
----------------------------------------------------------------------
diff --git a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaSoftwareProcessStreamsIntegrationTest.java b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaSoftwareProcessStreamsIntegrationTest.java
index d322a3c..7e5f122 100644
--- a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaSoftwareProcessStreamsIntegrationTest.java
+++ b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaSoftwareProcessStreamsIntegrationTest.java
@@ -18,37 +18,16 @@
  */
 package org.apache.brooklyn.entity.software.base;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.testng.Assert.assertTrue;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import org.apache.brooklyn.api.entity.EntitySpec;
 import org.apache.brooklyn.api.location.Location;
-import org.apache.brooklyn.api.mgmt.HasTaskChildren;
-import org.apache.brooklyn.api.mgmt.Task;
-import org.apache.brooklyn.core.mgmt.BrooklynTaskTags;
-import org.apache.brooklyn.core.mgmt.BrooklynTaskTags.WrappedStream;
-import org.apache.brooklyn.core.test.BrooklynAppUnitTestSupport;
-import org.apache.brooklyn.util.core.task.TaskPredicates;
-import org.apache.brooklyn.util.text.StringPredicates;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-import com.google.common.base.Optional;
-import com.google.common.base.Predicate;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-
-public class VanillaSoftwareProcessStreamsIntegrationTest extends BrooklynAppUnitTestSupport {
-
-    private static final Logger log = LoggerFactory.getLogger(VanillaSoftwareProcessStreamsIntegrationTest.class);
+import java.util.Map;
 
+public class VanillaSoftwareProcessStreamsIntegrationTest extends AbstractSoftwareProcessStreamsTest {
     private Location localhost;
 
     @BeforeMethod(alwaysRun=true)
@@ -59,8 +38,26 @@ public class VanillaSoftwareProcessStreamsIntegrationTest extends BrooklynAppUni
     }
 
     @Test(groups = "Integration")
+    @Override
     public void testGetsStreams() {
-        Map<String, String> cmds = ImmutableMap.<String, String>builder()
+        Map<String, String> cmds = getCommands();
+        VanillaSoftwareProcess entity = app.createAndManageChild(EntitySpec.create(VanillaSoftwareProcess.class)
+                .configure(VanillaSoftwareProcess.PRE_INSTALL_COMMAND, "echo " + cmds.get("pre-install-command"))
+                .configure(VanillaSoftwareProcess.INSTALL_COMMAND, "echo " + cmds.get("ssh: installing.*"))
+                .configure(VanillaSoftwareProcess.POST_INSTALL_COMMAND, "echo " + cmds.get("post-install-command"))
+                .configure(VanillaSoftwareProcess.CUSTOMIZE_COMMAND, "echo " + cmds.get("ssh: customizing.*"))
+                .configure(VanillaSoftwareProcess.PRE_LAUNCH_COMMAND, "echo " + cmds.get("pre-launch-command"))
+                .configure(VanillaSoftwareProcess.LAUNCH_COMMAND, "echo " + cmds.get("ssh: launching.*"))
+                .configure(VanillaSoftwareProcess.POST_LAUNCH_COMMAND, "echo " + cmds.get("post-launch-command"))
+                .configure(VanillaSoftwareProcess.CHECK_RUNNING_COMMAND, "true"));
+        app.start(ImmutableList.of(localhost));
+
+        assertStreams(entity);
+    }
+
+    @Override
+    protected Map<String, String> getCommands() {
+        return ImmutableMap.<String, String>builder()
                 .put("pre-install-command", "myPreInstall")
                 .put("ssh: installing.*", "myInstall")
                 .put("post-install-command", "myPostInstall")
@@ -69,63 +66,5 @@ public class VanillaSoftwareProcessStreamsIntegrationTest extends BrooklynAppUni
                 .put("ssh: launching.*", "myLaunch")
                 .put("post-launch-command", "myPostLaunch")
                 .build();
-        VanillaSoftwareProcess entity = app.createAndManageChild(EntitySpec.create(VanillaSoftwareProcess.class)
-                .configure(VanillaSoftwareProcess.PRE_INSTALL_COMMAND, "echo "+cmds.get("pre-install-command"))
-                .configure(VanillaSoftwareProcess.INSTALL_COMMAND, "echo "+cmds.get("ssh: installing.*"))
-                .configure(VanillaSoftwareProcess.POST_INSTALL_COMMAND, "echo "+cmds.get("post-install-command"))
-                .configure(VanillaSoftwareProcess.CUSTOMIZE_COMMAND, "echo "+cmds.get("ssh: customizing.*"))
-                .configure(VanillaSoftwareProcess.PRE_LAUNCH_COMMAND, "echo "+cmds.get("pre-launch-command"))
-                .configure(VanillaSoftwareProcess.LAUNCH_COMMAND, "echo "+cmds.get("ssh: launching.*"))
-                .configure(VanillaSoftwareProcess.POST_LAUNCH_COMMAND, "echo "+cmds.get("post-launch-command"))
-                .configure(VanillaSoftwareProcess.CHECK_RUNNING_COMMAND, "true"));
-        app.start(ImmutableList.of(localhost));
-
-        Set<Task<?>> tasks = BrooklynTaskTags.getTasksInEntityContext(mgmt.getExecutionManager(), entity);
-
-        for (Map.Entry<String, String> entry : cmds.entrySet()) {
-            String taskNameRegex = entry.getKey();
-            String echoed = entry.getValue();
-
-            Task<?> subTask = findTaskOrSubTask(tasks, TaskPredicates.displayNameMatches(StringPredicates.matchesRegex(taskNameRegex))).get();
-
-            String stdin = getStreamOrFail(subTask, BrooklynTaskTags.STREAM_STDIN);
-            String stdout = getStreamOrFail(subTask, BrooklynTaskTags.STREAM_STDOUT);
-            String stderr = getStreamOrFail(subTask, BrooklynTaskTags.STREAM_STDERR);
-            String env = getStreamOrFail(subTask, BrooklynTaskTags.STREAM_ENV);
-            String msg = "stdin="+stdin+"; stdout="+stdout+"; stderr="+stderr+"; env="+env;
-
-            assertTrue(stdin.contains("echo "+echoed), msg);
-            assertTrue(stdout.contains(echoed), msg);
-        }
-    }
-
-    protected String getStreamOrFail(Task<?> task, String streamType) {
-        String msg = "task="+task+"; stream="+streamType;
-        WrappedStream stream = checkNotNull(BrooklynTaskTags.stream(task, BrooklynTaskTags.STREAM_STDIN), "Stream null: "+msg);
-        return checkNotNull(stream.streamContents.get(), "Contents null: "+msg);
-    }
-
-    protected Optional<Task<?>> findTaskOrSubTask(Iterable<? extends Task<?>> tasks, Predicate<? super Task<?>> matcher) {
-        List<String> taskNames = Lists.newArrayList();
-        Optional<Task<?>> result = findTaskOrSubTaskImpl(tasks, matcher, taskNames);
-        if (!result.isPresent() && log.isDebugEnabled()) {
-            log.debug("Task not found matching " + matcher + "; contender names were " + taskNames);
-        }
-        return result;
-    }
-
-    protected Optional<Task<?>> findTaskOrSubTaskImpl(Iterable<? extends Task<?>> tasks, Predicate<? super Task<?>> matcher, List<String> taskNames) {
-        for (Task<?> task : tasks) {
-            if (matcher.apply(task)) return Optional.<Task<?>>of(task);
-
-            if (!(task instanceof HasTaskChildren)) {
-                return Optional.absent();
-            } else {
-                Optional<Task<?>> subResult = findTaskOrSubTask(((HasTaskChildren) task).getChildren(), matcher);
-                if (subResult.isPresent()) return subResult;
-            }
-        }
-
-        return Optional.<Task<?>>absent();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinrmStreamsLiveTest.java
----------------------------------------------------------------------
diff --git a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinrmStreamsLiveTest.java b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinrmStreamsLiveTest.java
index 788df8a..2ee310b 100644
--- a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinrmStreamsLiveTest.java
+++ b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/VanillaWindowsProcessWinrmStreamsLiveTest.java
@@ -53,13 +53,13 @@ public class VanillaWindowsProcessWinrmStreamsLiveTest extends AbstractSoftwareP
     @Override
     public void testGetsStreams() {
         VanillaWindowsProcess entity = app.createAndManageChild(EntitySpec.create(VanillaWindowsProcess.class)
-                .configure(VanillaSoftwareProcess.PRE_INSTALL_COMMAND, "echo " + getCommands().get("pre-install-command"))
-                .configure(VanillaSoftwareProcess.INSTALL_COMMAND, "echo " + getCommands().get("^install$"))
-                .configure(VanillaSoftwareProcess.POST_INSTALL_COMMAND, "echo " + getCommands().get("post-install-command"))
-                .configure(VanillaSoftwareProcess.CUSTOMIZE_COMMAND, "echo " + getCommands().get("^customize$"))
-                .configure(VanillaSoftwareProcess.PRE_LAUNCH_COMMAND, "echo " + getCommands().get("pre-launch-command"))
-                .configure(VanillaSoftwareProcess.LAUNCH_COMMAND, "echo " + getCommands().get("^launch$"))
-                .configure(VanillaSoftwareProcess.POST_LAUNCH_COMMAND, "echo " + getCommands().get("post-launch-command"))
+                .configure(VanillaSoftwareProcess.PRE_INSTALL_COMMAND, "echo " + getCommands().get("winrm: pre-install-command.*"))
+                .configure(VanillaSoftwareProcess.INSTALL_COMMAND, "echo " + getCommands().get("winrm: install.*"))
+                .configure(VanillaSoftwareProcess.POST_INSTALL_COMMAND, "echo " + getCommands().get("winrm: post-install-command.*"))
+                .configure(VanillaSoftwareProcess.CUSTOMIZE_COMMAND, "echo " + getCommands().get("winrm: customize.*"))
+                .configure(VanillaSoftwareProcess.PRE_LAUNCH_COMMAND, "echo " + getCommands().get("winrm: pre-launch-command.*"))
+                .configure(VanillaSoftwareProcess.LAUNCH_COMMAND, "echo " + getCommands().get("winrm: launch.*"))
+                .configure(VanillaSoftwareProcess.POST_LAUNCH_COMMAND, "echo " + getCommands().get("winrm: post-launch-command.*"))
                 .configure(VanillaSoftwareProcess.CHECK_RUNNING_COMMAND, "echo true"));
         app.start(ImmutableList.of(location));
         assertStreams(entity);
@@ -68,13 +68,13 @@ public class VanillaWindowsProcessWinrmStreamsLiveTest extends AbstractSoftwareP
     @Test(groups = "Live")
     public void testGetsStreamsPowerShell() {
         VanillaWindowsProcess entity = app.createAndManageChild(EntitySpec.create(VanillaWindowsProcess.class)
-                .configure(VanillaWindowsProcess.PRE_INSTALL_POWERSHELL_COMMAND, "echo " + getCommands().get("pre-install-command"))
-                .configure(VanillaWindowsProcess.INSTALL_POWERSHELL_COMMAND, "echo " + getCommands().get("^install$"))
-                .configure(VanillaWindowsProcess.POST_INSTALL_POWERSHELL_COMMAND, "echo " + getCommands().get("post-install-command"))
-                .configure(VanillaWindowsProcess.CUSTOMIZE_POWERSHELL_COMMAND, "echo " + getCommands().get("^customize$"))
-                .configure(VanillaWindowsProcess.PRE_LAUNCH_POWERSHELL_COMMAND, "echo " + getCommands().get("pre-launch-command"))
-                .configure(VanillaWindowsProcess.LAUNCH_POWERSHELL_COMMAND, "echo " + getCommands().get("^launch$"))
-                .configure(VanillaWindowsProcess.POST_LAUNCH_POWERSHELL_COMMAND, "echo " + getCommands().get("post-launch-command"))
+                .configure(VanillaWindowsProcess.PRE_INSTALL_POWERSHELL_COMMAND, "echo " + getCommands().get("winrm: pre-install-command.*"))
+                .configure(VanillaWindowsProcess.INSTALL_POWERSHELL_COMMAND, "echo " + getCommands().get("winrm: install.*"))
+                .configure(VanillaWindowsProcess.POST_INSTALL_POWERSHELL_COMMAND, "echo " + getCommands().get("winrm: post-install-command.*"))
+                .configure(VanillaWindowsProcess.CUSTOMIZE_POWERSHELL_COMMAND, "echo " + getCommands().get("winrm: customize.*"))
+                .configure(VanillaWindowsProcess.PRE_LAUNCH_POWERSHELL_COMMAND, "echo " + getCommands().get("winrm: pre-launch-command.*"))
+                .configure(VanillaWindowsProcess.LAUNCH_POWERSHELL_COMMAND, "echo " + getCommands().get("winrm: launch.*"))
+                .configure(VanillaWindowsProcess.POST_LAUNCH_POWERSHELL_COMMAND, "echo " + getCommands().get("winrm: post-launch-command.*"))
                 .configure(VanillaWindowsProcess.CHECK_RUNNING_POWERSHELL_COMMAND, "echo true"));
         app.start(ImmutableList.of(location));
         assertStreams(entity);
@@ -83,13 +83,13 @@ public class VanillaWindowsProcessWinrmStreamsLiveTest extends AbstractSoftwareP
     @Override
     protected Map<String, String> getCommands() {
         return ImmutableMap.<String, String>builder()
-                .put("pre-install-command", "myPreInstall")
-                .put("^install$", "myInstall")
-                .put("post-install-command", "pre install command output")
-                .put("^customize$", "myCustomizing")
-                .put("pre-launch-command", "pre launch command output")
-                .put("^launch$", "myLaunch")
-                .put("post-launch-command", "post launch command output")
+                .put("winrm: pre-install-command.*", "myPreInstall")
+                .put("winrm: install.*", "myInstall")
+                .put("winrm: post-install-command.*", "pre_install_command_output")
+                .put("winrm: customize.*", "myCustomizing")
+                .put("winrm: pre-launch-command.*", "pre_launch_command_output")
+                .put("winrm: launch.*", "myLaunch")
+                .put("winrm: post-launch-command.*", "post_launch_command_output")
                 .build();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/test/java/org/apache/brooklyn/entity/software/base/lifecycle/ScriptHelperUnitTest.java
----------------------------------------------------------------------
diff --git a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/lifecycle/ScriptHelperUnitTest.java b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/lifecycle/ScriptHelperUnitTest.java
new file mode 100644
index 0000000..4cb3dd0
--- /dev/null
+++ b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/lifecycle/ScriptHelperUnitTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.brooklyn.entity.software.base.lifecycle;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.brooklyn.api.mgmt.Task;
+import org.apache.brooklyn.core.mgmt.BrooklynTaskTags;
+import org.apache.brooklyn.entity.software.base.AbstractSoftwareProcessSshDriver;
+import org.apache.brooklyn.entity.software.base.DoNothingSoftwareProcessDriver;
+import org.apache.brooklyn.location.ssh.SshMachineLocation;
+import org.apache.brooklyn.util.core.task.BasicExecutionContext;
+import org.apache.brooklyn.util.core.task.BasicExecutionManager;
+import org.apache.brooklyn.util.core.task.DynamicTasks;
+import org.apache.brooklyn.util.text.Strings;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import static org.apache.brooklyn.entity.software.base.VanillaSoftwareProcessStreamsIntegrationTest.getStreamOrFail;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class ScriptHelperUnitTest {
+    private static final String NON_ZERO_CODE_COMMAND = "false";
+
+    @Test
+    public void testZeroExitCode() {
+        DoNothingSoftwareProcessDriver sshRunner = mock(DoNothingSoftwareProcessDriver.class);
+
+        ScriptHelper scriptHelper = new ScriptHelper(sshRunner, "test-zero-code-task");
+        Assert.assertEquals(scriptHelper.executeInternal(), 0, "ScriptHelper doesn't return zero code");
+    }
+
+    @Test
+    public void testNonZeroExitCode() {
+        DoNothingSoftwareProcessDriver sshRunner = mock(DoNothingSoftwareProcessDriver.class);
+        when(sshRunner.execute(any(Map.class), any(List.class), any(String.class))).thenReturn(1);
+
+        ScriptHelper scriptHelper = new ScriptHelper(sshRunner, "test-zero-code-task")
+                .body.append(NON_ZERO_CODE_COMMAND);
+        Assert.assertNotEquals(scriptHelper.executeInternal(), 0, "ScriptHelper return zero code for non-zero code task");
+    }
+
+    @Test(expectedExceptions = IllegalStateException.class)
+    public void testNonZeroExitCodeException() {
+        DoNothingSoftwareProcessDriver sshRunner = mock(DoNothingSoftwareProcessDriver.class);
+        when(sshRunner.execute(any(Map.class), any(List.class), any(String.class))).thenReturn(1);
+
+        ScriptHelper scriptHelper = new ScriptHelper(sshRunner, "test-zero-code-task")
+                .failOnNonZeroResultCode()
+                .body.append(NON_ZERO_CODE_COMMAND);
+        scriptHelper.executeInternal();
+    }
+
+    private final String command = "echo Hello World!";
+    private final String output = "Hello World!";
+    private final String errorStd = "Error output";
+
+    @Test
+    public void testTaskGatherOutput() {
+        Task<Integer> task = executeSampleScript(new Function<ScriptHelper, Void>() {
+            @Override
+            public Void apply(ScriptHelper scriptHelper) {
+                return null;
+            }
+        });
+
+        String stdOut = getStreamOrFail(task, BrooklynTaskTags.STREAM_STDOUT);
+        String stdErr = getStreamOrFail(task, BrooklynTaskTags.STREAM_STDERR);
+        Assert.assertEquals(stdOut, output);
+        Assert.assertEquals(stdErr, errorStd);
+    }
+
+    @Test(groups="WIP")
+    public void testTaskNoGatherOutput() {
+        Task<Integer> task = executeSampleScript(new Function<ScriptHelper, Void>() {
+            @Override
+            public Void apply(ScriptHelper scriptHelper) {
+                scriptHelper.gatherOutput(false);
+                return null;
+            }
+        });
+
+        String stdOut = getStreamOrFail(task, BrooklynTaskTags.STREAM_STDOUT);
+        String stdErr = getStreamOrFail(task, BrooklynTaskTags.STREAM_STDERR);
+        Assert.assertTrue(Strings.isBlank(stdOut));
+        Assert.assertTrue(Strings.isBlank(stdErr));
+    }
+
+    private Task<Integer> executeSampleScript(Function<ScriptHelper, Void> visitor) {
+        SshMachineLocation sshMachineLocation = new SshMachineLocation(ImmutableMap.of("address", "localhost")) {
+            @Override
+            public int execScript(Map<String,?> props, String summaryForLogging, List<String> commands, Map<String,?> env) {
+                Map<String, Object> props2 = (Map<String, Object>)props;
+                ByteArrayOutputStream outputStream = (ByteArrayOutputStream)props2.get("out");
+                ByteArrayOutputStream errorStream = (ByteArrayOutputStream)props2.get("err");
+                try {
+                    outputStream.write(output.getBytes());
+                    errorStream.write(errorStd.getBytes());
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+                return 0;
+            }
+        };
+        AbstractSoftwareProcessSshDriver sshDriver = mock(AbstractSoftwareProcessSshDriver.class);
+        when(sshDriver.execute(any(Map.class), any(List.class), any(String.class))).thenCallRealMethod();
+        when(sshDriver.getMachine()).thenReturn(sshMachineLocation);
+
+        ScriptHelper scriptHelper = new ScriptHelper(sshDriver, "test");
+        scriptHelper.setFlag("logPrefix", "./");
+        scriptHelper.body.append(command);
+        visitor.apply(scriptHelper);
+
+        Task<Integer> task = scriptHelper.newTask();
+        DynamicTasks.TaskQueueingResult<Integer> taskQueueingResult = DynamicTasks.queueIfPossible(task);
+        BasicExecutionManager em = new BasicExecutionManager("tests");
+        BasicExecutionContext ec = new BasicExecutionContext(em);
+        taskQueueingResult.executionContext(ec);
+        taskQueueingResult.orSubmitAndBlock();
+
+        return task;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/b6bc247f/software/base/src/test/java/org/apache/brooklyn/entity/software/base/lifecycle/WinRmExecuteHelperUnitTest.java
----------------------------------------------------------------------
diff --git a/software/base/src/test/java/org/apache/brooklyn/entity/software/base/lifecycle/WinRmExecuteHelperUnitTest.java b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/lifecycle/WinRmExecuteHelperUnitTest.java
new file mode 100644
index 0000000..3005ca3
--- /dev/null
+++ b/software/base/src/test/java/org/apache/brooklyn/entity/software/base/lifecycle/WinRmExecuteHelperUnitTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.brooklyn.entity.software.base.lifecycle;
+
+import org.apache.brooklyn.entity.software.base.DoNothingWinRmSoftwareProcessDriver;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+import java.util.Map;
+
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class WinRmExecuteHelperUnitTest {
+    private static final String NON_ZERO_CODE_COMMAND = "false";
+
+    @Test
+    public void testZeroExitCode() {
+        DoNothingWinRmSoftwareProcessDriver nativeWindowsScriptRunner = mock(DoNothingWinRmSoftwareProcessDriver.class);
+
+        WinRmExecuteHelper scriptHelper = new WinRmExecuteHelper(nativeWindowsScriptRunner, "test-zero-code-task");
+        Assert.assertEquals(scriptHelper.executeInternal(), 0, "WinRmExecuteHelper doesn't return zero code");
+    }
+
+    @Test
+    public void testNonZeroExitCode() {
+        DoNothingWinRmSoftwareProcessDriver nativeWindowsScriptRunner = mock(DoNothingWinRmSoftwareProcessDriver.class);
+        when(nativeWindowsScriptRunner.executeNativeOrPsCommand(any(Map.class), any(String.class), any(String.class), any(String.class), any(Boolean.class))).thenReturn(1);
+
+        WinRmExecuteHelper scriptHelper = new WinRmExecuteHelper(nativeWindowsScriptRunner, "test-zero-code-task")
+                .setCommand(NON_ZERO_CODE_COMMAND);
+        Assert.assertNotEquals(scriptHelper.executeInternal(), 0, "WinRmExecuteHelper return zero code for non-zero code task");
+    }
+
+    @Test(expectedExceptions = IllegalStateException.class)
+    public void testNonZeroExitCodeException() {
+        DoNothingWinRmSoftwareProcessDriver nativeWindowsScriptRunner = mock(DoNothingWinRmSoftwareProcessDriver.class);
+        when(nativeWindowsScriptRunner.executeNativeOrPsCommand(any(Map.class), any(String.class), any(String.class), any(String.class), any(Boolean.class))).thenReturn(1);
+
+        WinRmExecuteHelper scriptHelper = new WinRmExecuteHelper(nativeWindowsScriptRunner, "test-zero-code-task")
+                .failOnNonZeroResultCode()
+                .setCommand(NON_ZERO_CODE_COMMAND);
+        scriptHelper.executeInternal();
+    }
+}


[2/2] incubator-brooklyn git commit: This closes #906

Posted by al...@apache.org.
This closes #906


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

Branch: refs/heads/master
Commit: 032c53309e5d084a7754512e37f6010cee5e24af
Parents: 10c6576 b6bc247
Author: Aled Sage <al...@gmail.com>
Authored: Mon Sep 21 14:37:42 2015 +0100
Committer: Aled Sage <al...@gmail.com>
Committed: Mon Sep 21 14:37:42 2015 +0100

----------------------------------------------------------------------
 .../AbstractSoftwareProcessWinRmDriver.java     |  98 +++++++--
 .../base/VanillaWindowsProcessWinRmDriver.java  |  33 ++-
 .../lifecycle/NativeWindowsScriptRunner.java    |  29 +++
 .../base/lifecycle/WinRmExecuteHelper.java      | 217 +++++++++++++++++++
 .../AbstractSoftwareProcessStreamsTest.java     |   8 +-
 .../DoNothingWinRmSoftwareProcessDriver.java    |  68 ++++++
 ...laSoftwareProcessStreamsIntegrationTest.java | 107 ++-------
 ...nillaWindowsProcessWinrmStreamsLiveTest.java |  42 ++--
 .../base/lifecycle/ScriptHelperUnitTest.java    | 146 +++++++++++++
 .../lifecycle/WinRmExecuteHelperUnitTest.java   |  62 ++++++
 10 files changed, 669 insertions(+), 141 deletions(-)
----------------------------------------------------------------------