You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@brooklyn.apache.org by geomacy <gi...@git.apache.org> on 2015/11/13 15:49:43 UTC

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

GitHub user geomacy opened a pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030

    SimpleCommand addition to brooklyn-test-framework

    ### Description
    
    An additional feature for the brooklyn-test-framework.
    
    Supports a test that runs a simple command (expected to return quickly) 
    on the host of the target entity and allows assertions to be made on the
    exit code, standard out and standard error of the command.  
    
    This is to support the use of arbitrary 'entities' (in the general sense)
    in tests using the brooklyn-test-framework.  Tests are regular blueprints
    so already allow the deployment of arbitrary Brooklyn entities. Additionally,
    to deploy arbitrary server type processes (expected to continue running),
    one can use the VanillaSoftwareProcess, which runs an arbitrary script.
    
    The SimpleCommandTest in this PR is to support a similar case to that of the 
    VanillaSoftwareProcess, except the command is expected to "do something", 
    finishing quickly, and return a result (process exit code), along with its 
    standard out and error streams, which can then be tested using assertions
    provided by the test-framework.
    
    See below for example YAML.
    
    ### Usage
    - The type is type: org.apache.brooklyn.test.framework.SimpleCommandTest.
    - The command can be specified as a script to be downloaded, by default
    to /tmp, using the "downloadUrl" flag,
    - Download directory can be changed using flag scriptDir.
    - Alternatively the command can be specified as a simple command
    to be executed, using the "defaultCommand" flag. This command can have
    bash style command syntax including pipes and redirects.
    - The command is run on the node of the target entity of the test via
    an SSH connection.
    - Assertions are made on the return code of the command using the 
    assertStatus flag, and on the standard output and error using assertOut
    and assertErr.
    
    ### TODO
    - The currently supported exceptions are equals, contains, matches and 
    isEmpty.  A TODO is to merge this with the refactored standalone assertion 
    class currently being developed for the test-framework.
    - Make the command execute against members of a cluster.  At present
    the command will only run against one of the cluster members.
    
    ### Example YAML
    ```
    name: simpleCommandTest 
    location: byon4
    services:
    - type: org.apache.brooklyn.test.framework.TestCase
      name: testcase1
      targetId: testprocess
      brooklyn.children:
        - type: org.apache.brooklyn.entity.software.base.EmptySoftwareProcess
          id: testprocess
    
        - type: org.apache.brooklyn.test.framework.SimpleCommandTest
          defaultCommand: hostname
          assertStatus:
            equals: 0
          assertOut: 
            equals: byon4
          assertErr: 
            isEmpty: true
    
        - type: org.apache.brooklyn.test.framework.SimpleCommandTest
          downloadUrl: http://localhost:8080/script1.sh
          assertStatus:
            equals: 0
          assertOut: 
            equals: hello world
          assertErr: 
            isEmpty: true
    ```

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/geomacy/incubator-brooklyn simplecommand

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-brooklyn/pull/1030.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1030
    
----
commit b4e3ac7b68ed26f3efa74be7285de013d1f8c02a
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-05T15:06:45Z

    Initial draft of SimpleCommand.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859808
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommand.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.ImplementedBy;
    +import org.apache.brooklyn.config.ConfigKey;
    +import org.apache.brooklyn.core.config.ConfigKeys;
    +import org.apache.brooklyn.core.entity.trait.Startable;
    +import org.apache.brooklyn.core.sensor.AttributeSensorAndConfigKey;
    +import org.apache.brooklyn.entity.software.base.SoftwareProcess;
    +import org.apache.brooklyn.util.core.flags.SetFromFlag;
    +import org.apache.brooklyn.util.os.Os;
    +
    +import static org.apache.brooklyn.core.config.ConfigKeys.newConfigKey;
    +
    +/**
    + * Entity to invoke on a node a simple command that will immediately succeed or fail.
    + *
    + * Invokes the command in the start operation, and declares itself RUNNING.
    + */
    +@ImplementedBy(SimpleCommandImpl.class)
    +public interface SimpleCommand extends Entity, Startable {
    +
    +    @SetFromFlag(nullable = false)
    +    ConfigKey<String> DEFAULT_COMMAND = ConfigKeys.newConfigKey(String.class, "defaultCommand",
    +            "Command to invoke if no script is provided via a downloadUrl");
    +
    +    @SetFromFlag("downloadUrl")
    +    AttributeSensorAndConfigKey<String, String> DOWNLOAD_URL = SoftwareProcess.DOWNLOAD_URL;
    +
    +    @SetFromFlag("scriptDir")
    +    ConfigKey<String> SCRIPT_DIR = newConfigKey("scriptDir", "directory where downloaded scripts should be put", Os.tmp());
    --- End diff --
    
    Is this the directory on the remote machine? Using `Os.tmp()` doesn't seem right - for example if Brooklyn were running on a windows machine then this would generate a windows directory, but the remote machine would most likely be Linux.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-158997862
  
    I'll resolve the merge conflicts.  I think I'd like to do the changes to the assertions as a separate PR, it seems to be a bit of 'mission creep' to do them alongside the SimpleShellCommand changes, if that's all right.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860211
  
    --- Diff: usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleCommandImplIntegrationTest.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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.test.framework;
    +
    +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.LocationSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
    +import org.apache.brooklyn.core.test.entity.TestApplication;
    +import org.apache.brooklyn.core.test.entity.TestEntity;
    +import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.AfterMethod;
    +import org.testng.annotations.BeforeMethod;
    +import org.testng.annotations.Test;
    +
    +import java.util.UUID;
    +
    +import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
    +import static org.apache.brooklyn.test.framework.SimpleCommand.DEFAULT_COMMAND;
    +import static org.apache.brooklyn.test.framework.SimpleCommandTest.*;
    +import static org.assertj.core.api.Assertions.assertThat;
    +
    +public class SimpleCommandImplIntegrationTest {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImplIntegrationTest.class);
    +
    +    private static final String UP = "up";
    +    private TestApplication app;
    +    private ManagementContext managementContext;
    +    private LocalhostMachineProvisioningLocation localhost;
    +    private String testId;
    +
    +
    +    @BeforeMethod
    +    public void setUp() {
    +
    +        testId = UUID.randomUUID().toString();
    +
    +        app = TestApplication.Factory.newManagedInstanceForTests();
    +        managementContext = app.getManagementContext();
    +
    +        localhost = managementContext.getLocationManager()
    +            .createLocation(LocationSpec.create(LocalhostMachineProvisioningLocation.class)
    +                .configure("name", testId));
    +    }
    +
    +    @AfterMethod(alwaysRun = true)
    +    public void tearDown() throws Exception {
    +        if (app != null) Entities.destroyAll(app.getManagementContext());
    +    }
    +
    +    @Test(groups = "Integration")
    +    public void shouldInvokeCommand() {
    --- End diff --
    
    Would be good to also see a test that confirms the `SimpleCommand` is using the machine from the testEntity (rather than potentially creating its own). For example, you could use a location that is a BYON with a single localhost VM. Then if the SimpleCommand were to try to get a new machine, it would fail.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-159020575
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-157748186
  
    Re-opening PR as noted above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45317643
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestCase.java ---
    @@ -19,6 +19,7 @@
     package org.apache.brooklyn.test.framework;
     
     import org.apache.brooklyn.api.entity.ImplementedBy;
    +import org.apache.brooklyn.core.entity.trait.Startable;
    --- End diff --
    
    not used?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45317630
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java ---
    @@ -58,7 +58,7 @@ public void start(Collection<? extends Location> locations) {
                 sensors().set(SERVICE_UP, true);
                 ServiceStateLogic.setExpectedState(this, Lifecycle.RUNNING);
             } catch (Throwable t) {
    -            LOG.info("Url [{}] test failed", url);
    +            LOG.debug("Url [{}] test failed", url);
    --- End diff --
    
    include `this` in the log message for when running at scale


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45320228
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.test.framework;
    +
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Splitter;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.TaskFactory;
    +import org.apache.brooklyn.core.effector.ssh.SshEffectorTasks;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.location.Machines;
    +import org.apache.brooklyn.location.ssh.SshMachineLocation;
    +import org.apache.brooklyn.test.Asserts;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.core.task.DynamicTasks;
    +import org.apache.brooklyn.util.core.task.ssh.SshTasks;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskWrapper;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Identifiers;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.*;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +import static org.apache.brooklyn.util.groovy.GroovyJavaMethods.truth;
    +import static org.apache.brooklyn.util.text.Strings.isBlank;
    +import static org.apache.brooklyn.util.text.Strings.isNonBlank;
    +import static org.apache.commons.collections.MapUtils.isEmpty;
    +
    +public class SimpleShellCommandTestImpl extends AbstractTest implements SimpleShellCommandTest {
    +
    +    public static final int SUCCESS = 0;
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleShellCommandTestImpl.class);
    +    private static final int A_LINE = 80;
    +    public static final String DEFAULT_NAME = "download.sh";
    +    private static final String CD = "cd";
    +
    +    @Override
    +    public void start(Collection<? extends Location> locations) {
    +        setExpectedState(this, STARTING);
    +        execute();
    +    }
    +
    +    @Override
    +    public void stop() {
    +        LOG.debug("{} Stopping simple command", this);
    +        setUpAndRunState(false, STOPPED);
    +    }
    +
    +    @Override
    +    public void restart() {
    +        LOG.debug("{} Restarting simple command", this);
    +        execute();
    +    }
    +
    +    private void setUpAndRunState(boolean up, Lifecycle status) {
    +        sensors().set(SERVICE_UP, up);
    +        setExpectedState(this, status);
    +    }
    +
    +    private static class Result {
    +        int exitCode;
    +        String stdout;
    +        String stderr;
    +        public Result(final ProcessTaskWrapper<Integer> job) {
    +            exitCode = job.get();
    +            stdout = job.getStdout().trim();
    +            stderr = job.getStderr().trim();
    +        }
    +        public int getExitCode() {
    +            return exitCode;
    +        }
    +        public String getStdout() {
    +            return stdout;
    +        }
    +        public String getStderr() {
    +            return stderr;
    +        }
    +    }
    +
    +    protected void handle(Result result) {
    +        LOG.debug("{}, Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +            this, result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +        AssertionSupport support = new AssertionSupport();
    +        checkAssertions(support, exitCodeAssertions(), "exit code", result.getExitCode());
    +        checkAssertions(support, getConfig(ASSERT_OUT), "stdout", result.getStdout());
    +        checkAssertions(support, getConfig(ASSERT_ERR), "stderr", result.getStderr());
    +        support.validate();
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    public void execute() {
    +        try {
    +            SshMachineLocation machineLocation =
    +                Machines.findUniqueMachineLocation(resolveTarget().getLocations(), SshMachineLocation.class).get();
    +            executeCommand(machineLocation);
    +            setUpAndRunState(true, RUNNING);
    +        } catch (Throwable t) {
    +            setUpAndRunState(false, ON_FIRE);
    +            throw Exceptions.propagate(t);
    +        }
    +    }
    +
    +    private void executeCommand(SshMachineLocation machineLocation) {
    +
    +        Result result = null;
    +        String downloadUrl = getConfig(DOWNLOAD_URL);
    +        String command = getConfig(COMMAND);
    +
    +        String downloadName = DOWNLOAD_URL.getName();
    +        String commandName = COMMAND.getName();
    +
    +        if (!(isNonBlank(downloadUrl) ^ isNonBlank(command))) {
    +            throw illegal("Must specify exactly one of", downloadName, "and", commandName);
    +        }
    +
    +        if (isNonBlank(downloadUrl)) {
    +            String scriptDir = getConfig(SCRIPT_DIR);
    +            String scriptPath = calculateDestPath(downloadUrl, scriptDir);
    +            result = executeDownloadedScript(machineLocation, downloadUrl, scriptPath);
    --- End diff --
    
    oh i see, there are two modes, `DOWNLOAD_URL` or `COMMAND`, and the latter does what i suggested.  ignore that comment!
    
    maybe a bit simpler to do something like:
    
         if (downloadUrl!=null) {
             command = `/tmp/brooklyn-`+Identifiers.makeRandomId(8);
             queue(installFromUrl(...., /* to file: */ command));
             // or even: command="curl URL | bash";
         }
         executeShellCommand(command);
    
    but fine as is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859977
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    +
    +    @Override
    +    public SimpleCommandDriver getDriver() {
    +        return driver;
    +    }
    +
    +    @Override
    +    public Class<SimpleCommandDriver> getDriverInterface() {
    +        return SimpleCommandDriver.class;
    +    }
    +
    +    /**
    +     * Gives the opportunity to sub-classes to do additional work based on the result of the command.
    +     */
    +    protected void handle(SimpleCommandDriver.Result result) {
    +        LOG.debug("Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +                result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    /**
    +     * Does nothing in this class but gives sub-classes the opportunity to filter locations according to some criterion.
    +     */
    +    public Collection<? extends Location> filterLocations(Collection<? extends Location> locations) {
    +        return locations;
    +    }
    +
    +
    +    @Override
    +    public void init() {
    +        super.init();
    +        getLifecycleEffectorTasks().attachLifecycleEffectors(this);
    +    }
    +
    +
    +    protected void initDriver(MachineLocation machine) {
    +        LOG.debug("Initializing simple command driver");
    +        SimpleCommandDriver newDriver = doInitDriver(machine);
    +        if (newDriver == null) {
    +            throw new UnsupportedOperationException("cannot start "+this+" on "+machine+": no driver available");
    +        }
    +        driver = newDriver;
    +    }
    +
    +    protected SimpleCommandDriver doInitDriver(MachineLocation machine) {
    +        if (driver!=null) {
    +            if (machine.equals(driver.getLocation())) {
    +                return driver; //just reuse
    +            } else {
    +                LOG.warn("driver/location change is untested for {} at {}; changing driver and continuing", this, machine);
    +                return newDriver(machine);
    +            }
    +        } else {
    +            return newDriver(machine);
    +        }
    +    }
    +
    +    protected SimpleCommandDriver newDriver(MachineLocation machine) {
    +        LOG.debug("Creating new simple command driver for {} from management context", machine);
    +        EntityDriverManager entityDriverManager = getManagementContext().getEntityDriverManager();
    +        return entityDriverManager.build(this, machine);
    +    }
    +
    +    @Override
    +    public void start(@EffectorParam(name = "locations") Collection<? extends Location> locations) {
    +        this.locations = locations;
    +        startOnLocations();
    +    }
    +
    +    protected void startOnLocations() {
    +        setExpectedState(this, STARTING);
    +        int size = locations.size();
    +        LOG.debug("Starting simple command at {} locations{}", size,
    +                size > 0 ? " beginning " + locations.iterator().next() : "");
    +        try {
    +            execute(locations);
    +            setUpAndRunState(true, RUNNING);
    +
    +        } catch (final Exception e) {
    +            setUpAndRunState(false, ON_FIRE);
    +            throw Exceptions.propagate(e);
    +        }
    +    }
    +
    +    private void execute(Collection<? extends Location> locations) {
    +        SimpleCommandDriver.Result result = null;
    +        String downloadUrl = getConfig(DOWNLOAD_URL);
    +        if (Strings.isNonBlank(downloadUrl)) {
    +            String scriptDir = getConfig(SCRIPT_DIR);
    +            result = getDriver().executeDownloadedScript(locations, downloadUrl, scriptDir);
    +
    +        } else {
    +            String command = getConfig(DEFAULT_COMMAND);
    +            if (Strings.isBlank(command)) {
    +                throw new IllegalArgumentException("No default command and no downloadUrl provided");
    +            }
    +
    +            result = getDriver().execute(locations, command);
    +        }
    +        handle(result);
    +    }
    +
    +
    +    @Override
    +    public void stop() {
    +        LOG.debug("Stopping simple command");
    +        setUpAndRunState(false, STOPPED);
    +    }
    +
    +    @Override
    +    public void restart() {
    +        LOG.debug("Restarting simple command");
    +        setUpAndRunState(true, RUNNING);
    +    }
    +
    +    private void setUpAndRunState(boolean up, Lifecycle status) {
    +        sensors().set(SERVICE_UP, up);
    +        setExpectedState(this, status);
    +    }
    +
    +    protected SimpleCommandLifecycleEffectorTasks getLifecycleEffectorTasks () {
    --- End diff --
    
    Extra space before brackets.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy closed the pull request at:

    https://github.com/apache/incubator-brooklyn/pull/1030


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45034307
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommand.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.ImplementedBy;
    +import org.apache.brooklyn.config.ConfigKey;
    +import org.apache.brooklyn.core.config.ConfigKeys;
    +import org.apache.brooklyn.core.entity.trait.Startable;
    +import org.apache.brooklyn.core.sensor.AttributeSensorAndConfigKey;
    +import org.apache.brooklyn.entity.software.base.SoftwareProcess;
    +import org.apache.brooklyn.util.core.flags.SetFromFlag;
    +import org.apache.brooklyn.util.os.Os;
    +
    +import static org.apache.brooklyn.core.config.ConfigKeys.newConfigKey;
    +
    +/**
    + * Entity to invoke on a node a simple command that will immediately succeed or fail.
    + *
    + * Invokes the command in the start operation, and declares itself RUNNING.
    + */
    +@ImplementedBy(SimpleCommandImpl.class)
    +public interface SimpleCommand extends Entity, Startable {
    +
    +    @SetFromFlag(nullable = false)
    +    ConfigKey<String> DEFAULT_COMMAND = ConfigKeys.newConfigKey(String.class, "defaultCommand",
    +            "Command to invoke if no script is provided via a downloadUrl");
    +
    +    @SetFromFlag("downloadUrl")
    +    AttributeSensorAndConfigKey<String, String> DOWNLOAD_URL = SoftwareProcess.DOWNLOAD_URL;
    +
    +    @SetFromFlag("scriptDir")
    +    ConfigKey<String> SCRIPT_DIR = newConfigKey("scriptDir", "directory where downloaded scripts should be put", Os.tmp());
    --- End diff --
    
    Why not leave it in the root dir, or perhaps better, switch to `getConfig(RUN_DIR)` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860179
  
    --- Diff: usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleCommandImplIntegrationTest.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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.test.framework;
    +
    +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.LocationSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
    +import org.apache.brooklyn.core.test.entity.TestApplication;
    +import org.apache.brooklyn.core.test.entity.TestEntity;
    +import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.AfterMethod;
    +import org.testng.annotations.BeforeMethod;
    +import org.testng.annotations.Test;
    +
    +import java.util.UUID;
    +
    +import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
    +import static org.apache.brooklyn.test.framework.SimpleCommand.DEFAULT_COMMAND;
    +import static org.apache.brooklyn.test.framework.SimpleCommandTest.*;
    +import static org.assertj.core.api.Assertions.assertThat;
    +
    +public class SimpleCommandImplIntegrationTest {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImplIntegrationTest.class);
    +
    +    private static final String UP = "up";
    +    private TestApplication app;
    +    private ManagementContext managementContext;
    +    private LocalhostMachineProvisioningLocation localhost;
    +    private String testId;
    +
    +
    +    @BeforeMethod
    --- End diff --
    
    Use `@BeforeMethod(alwaysRun=true)`. Otherwise when you run it at the mvn command line it may well skip the method when running integration tests (because the method is not in that group).
    
    Even if your class doesn't have integration tests, still always include the `alwaysRun=true` - otherwise when someone else adds an integration test annotation, they may well hit that problem.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-158995925
  
    I had a look today at using the new general purpose TestFrameworkAssertions as per Alex’s comment [above](https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45317688).
    
    One issue with that is that the general purpose code doesn’t give the same level of detail in the error message if the assertions fail.   For instance with a test with several failing assertions:
    ```
        - type: org.apache.brooklyn.test.framework.SimpleShellCommandTest
          targetId: testprocess5
          command: hostname
          assertStatus:
            - equals: 1
          assertOut: 
            - equals: bogus
            - contains: porkpie
          assertErr: 
            - equals: explosion
    ```
    The general purpose class gives the following: 
    ```
    Failure running task invoking start[locations] on 1 node (JnQQSTYi): org.apache.brooklyn.util.exceptions.PropagatedRuntimeException: org.apache.brooklyn.util.exceptions.PropagatedRuntimeException: Error invoking start at TestCaseImpl{id=UwYqp5yt}: org.apache.brooklyn.util.exceptions.PropagatedRuntimeException: java.lang.AssertionError: expected [true] but found [false]
    ```
    While the current code gives
    ```
    Failure running task start (Lbltxgqq): java.lang.AssertionError: Assertion Failures: exit code equals 1 stdout equals bogus stdout contains porkpie stderr equals explosion
    ```
    It may be valuable to extend the general purpose code to give more detail but I think that’s a separate issue from 1030.  I think the other comments are all done, so I would like to propose leaving the assertion handling as it is for the moment, in which case I hope this PR could be merged?
    
    Let me know what you think,
    
    regards
    Geoff


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45322611
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.test.framework;
    +
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Splitter;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.TaskFactory;
    +import org.apache.brooklyn.core.effector.ssh.SshEffectorTasks;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.location.Machines;
    +import org.apache.brooklyn.location.ssh.SshMachineLocation;
    +import org.apache.brooklyn.test.Asserts;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.core.task.DynamicTasks;
    +import org.apache.brooklyn.util.core.task.ssh.SshTasks;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskWrapper;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Identifiers;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.*;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +import static org.apache.brooklyn.util.groovy.GroovyJavaMethods.truth;
    +import static org.apache.brooklyn.util.text.Strings.isBlank;
    +import static org.apache.brooklyn.util.text.Strings.isNonBlank;
    +import static org.apache.commons.collections.MapUtils.isEmpty;
    +
    +public class SimpleShellCommandTestImpl extends AbstractTest implements SimpleShellCommandTest {
    +
    +    public static final int SUCCESS = 0;
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleShellCommandTestImpl.class);
    +    private static final int A_LINE = 80;
    +    public static final String DEFAULT_NAME = "download.sh";
    +    private static final String CD = "cd";
    +
    +    @Override
    +    public void start(Collection<? extends Location> locations) {
    +        setExpectedState(this, STARTING);
    +        execute();
    +    }
    +
    +    @Override
    +    public void stop() {
    +        LOG.debug("{} Stopping simple command", this);
    +        setUpAndRunState(false, STOPPED);
    +    }
    +
    +    @Override
    +    public void restart() {
    +        LOG.debug("{} Restarting simple command", this);
    +        execute();
    +    }
    +
    +    private void setUpAndRunState(boolean up, Lifecycle status) {
    +        sensors().set(SERVICE_UP, up);
    +        setExpectedState(this, status);
    +    }
    +
    +    private static class Result {
    +        int exitCode;
    +        String stdout;
    +        String stderr;
    +        public Result(final ProcessTaskWrapper<Integer> job) {
    +            exitCode = job.get();
    +            stdout = job.getStdout().trim();
    +            stderr = job.getStderr().trim();
    +        }
    +        public int getExitCode() {
    +            return exitCode;
    +        }
    +        public String getStdout() {
    +            return stdout;
    +        }
    +        public String getStderr() {
    +            return stderr;
    +        }
    +    }
    +
    +    protected void handle(Result result) {
    +        LOG.debug("{}, Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +            this, result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +        AssertionSupport support = new AssertionSupport();
    +        checkAssertions(support, exitCodeAssertions(), "exit code", result.getExitCode());
    +        checkAssertions(support, getConfig(ASSERT_OUT), "stdout", result.getStdout());
    +        checkAssertions(support, getConfig(ASSERT_ERR), "stderr", result.getStderr());
    +        support.validate();
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    public void execute() {
    +        try {
    +            SshMachineLocation machineLocation =
    +                Machines.findUniqueMachineLocation(resolveTarget().getLocations(), SshMachineLocation.class).get();
    +            executeCommand(machineLocation);
    +            setUpAndRunState(true, RUNNING);
    +        } catch (Throwable t) {
    +            setUpAndRunState(false, ON_FIRE);
    +            throw Exceptions.propagate(t);
    +        }
    +    }
    +
    +    private void executeCommand(SshMachineLocation machineLocation) {
    +
    +        Result result = null;
    +        String downloadUrl = getConfig(DOWNLOAD_URL);
    +        String command = getConfig(COMMAND);
    +
    +        String downloadName = DOWNLOAD_URL.getName();
    +        String commandName = COMMAND.getName();
    +
    +        if (!(isNonBlank(downloadUrl) ^ isNonBlank(command))) {
    +            throw illegal("Must specify exactly one of", downloadName, "and", commandName);
    +        }
    +
    +        if (isNonBlank(downloadUrl)) {
    +            String scriptDir = getConfig(SCRIPT_DIR);
    +            String scriptPath = calculateDestPath(downloadUrl, scriptDir);
    +            result = executeDownloadedScript(machineLocation, downloadUrl, scriptPath);
    +        }
    +
    +        if (isNonBlank(command)) {
    +            result = executeShellCommand(machineLocation, command);
    +        }
    +
    +        handle(result);
    +    }
    +
    +    private Result executeDownloadedScript(SshMachineLocation machineLocation, String url, String scriptPath) {
    +
    +        TaskFactory<?> install = SshTasks.installFromUrl(ImmutableMap.<String, Object>of(), machineLocation, url, scriptPath);
    +        DynamicTasks.queue(install);
    +        DynamicTasks.waitForLast();
    +
    +        List<String> commands = new ArrayList<>();
    +        commands.add("chmod u+x " + scriptPath);
    +        maybeCdToRunDir(commands);
    +        commands.add(scriptPath);
    +
    +        return runCommands(machineLocation, commands);
    +    }
    +
    +    private Result executeShellCommand(SshMachineLocation machineLocation, String command) {
    +
    +        List<String> commands = new ArrayList<>();
    +        maybeCdToRunDir(commands);
    +        commands.add(command);
    +
    +        return runCommands(machineLocation, commands);
    +    }
    +
    +    private void maybeCdToRunDir(List<String> commands) {
    +        String runDir = getConfig(RUN_DIR);
    +        if (!isBlank(runDir)) {
    +            commands.add(CD + " " + runDir);
    +        }
    +    }
    +
    +    private Result runCommands(SshMachineLocation machine, List<String> commands) {
    +        SshEffectorTasks.SshEffectorTaskFactory<Integer> etf = SshEffectorTasks.ssh(commands.toArray(new String[]{}))
    +            .machine(machine);
    +
    +        ProcessTaskWrapper<Integer> job = DynamicTasks.queue(etf);
    +        job.asTask().blockUntilEnded();
    +        return new Result(job);
    +    }
    +
    +
    +
    +    private IllegalArgumentException illegal(String message, String ...messages) {
    +        return new IllegalArgumentException(Joiner.on(' ').join(this.toString() + ":", message, messages));
    +    }
    +
    +    private String calculateDestPath(String url, String directory) {
    +        try {
    +            URL asUrl = new URL(url);
    +            Iterable<String> path = Splitter.on("/").split(asUrl.getPath());
    +            String scriptName = getLastPartOfPath(path, DEFAULT_NAME);
    +            return Joiner.on("/").join(directory, "test-" + Identifiers.makeRandomId(8), scriptName);
    +        } catch (MalformedURLException e) {
    +            throw illegal("Malformed URL:", url);
    +        }
    +    }
    +
    +    private static String getLastPartOfPath(Iterable<String> path, String defaultName) {
    +        MutableList<String> parts = MutableList.copyOf(path);
    +        Collections.reverse(parts);
    +        Iterator<String> it = parts.iterator();
    +        String scriptName = null;
    +
    +        // strip any trailing "/" parts of URL
    +        while (isBlank(scriptName) && it.hasNext()) {
    +            scriptName = it.next();
    +        }
    +        if (isBlank(scriptName)) {
    +            scriptName = defaultName;
    +        }
    +        return scriptName;
    +    }
    +    
    +    private <T> void checkAssertions(AssertionSupport support, Map<?, ?> assertions, String target, T actual) {
    --- End diff --
    
    Yes, this was written before we started pulling them out, but I do plan to do that as soon as they are ready. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44902393
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommand.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.ImplementedBy;
    +import org.apache.brooklyn.config.ConfigKey;
    +import org.apache.brooklyn.core.config.ConfigKeys;
    +import org.apache.brooklyn.core.entity.trait.Startable;
    +import org.apache.brooklyn.core.sensor.AttributeSensorAndConfigKey;
    +import org.apache.brooklyn.entity.software.base.SoftwareProcess;
    +import org.apache.brooklyn.util.core.flags.SetFromFlag;
    +import org.apache.brooklyn.util.os.Os;
    +
    +import static org.apache.brooklyn.core.config.ConfigKeys.newConfigKey;
    +
    +/**
    + * Entity to invoke on a node a simple command that will immediately succeed or fail.
    + *
    + * Invokes the command in the start operation, and declares itself RUNNING.
    + */
    +@ImplementedBy(SimpleCommandImpl.class)
    +public interface SimpleCommand extends Entity, Startable {
    +
    +    @SetFromFlag(nullable = false)
    +    ConfigKey<String> DEFAULT_COMMAND = ConfigKeys.newConfigKey(String.class, "defaultCommand",
    +            "Command to invoke if no script is provided via a downloadUrl");
    +
    +    @SetFromFlag("downloadUrl")
    +    AttributeSensorAndConfigKey<String, String> DOWNLOAD_URL = SoftwareProcess.DOWNLOAD_URL;
    +
    +    @SetFromFlag("scriptDir")
    +    ConfigKey<String> SCRIPT_DIR = newConfigKey("scriptDir", "directory where downloaded scripts should be put", Os.tmp());
    --- End diff --
    
    Thanks for the detailed reviews.  Regarding the 'step back' comment on the amount of code, I couldn't agree more!  I followed the VanillaSoftwareProcess model, and while that was probably a valuable learning exercise, it leaves what does seem like an awful lot of code to do something simple. If it's ok to do the work as tasks directly in the ```start()``` of the entity then I'll rework it that way.  (I wasn't sure if there might be some non-obvious dependencies elsewhere in the code that meant you had to have a driver or lifecycle classes?) 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45050093
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommand.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.ImplementedBy;
    +import org.apache.brooklyn.config.ConfigKey;
    +import org.apache.brooklyn.core.config.ConfigKeys;
    +import org.apache.brooklyn.core.entity.trait.Startable;
    +import org.apache.brooklyn.core.sensor.AttributeSensorAndConfigKey;
    +import org.apache.brooklyn.entity.software.base.SoftwareProcess;
    +import org.apache.brooklyn.util.core.flags.SetFromFlag;
    +import org.apache.brooklyn.util.os.Os;
    +
    +import static org.apache.brooklyn.core.config.ConfigKeys.newConfigKey;
    +
    +/**
    + * Entity to invoke on a node a simple command that will immediately succeed or fail.
    + *
    + * Invokes the command in the start operation, and declares itself RUNNING.
    + */
    +@ImplementedBy(SimpleCommandImpl.class)
    +public interface SimpleCommand extends Entity, Startable {
    +
    +    @SetFromFlag(nullable = false)
    +    ConfigKey<String> DEFAULT_COMMAND = ConfigKeys.newConfigKey(String.class, "defaultCommand",
    +            "Command to invoke if no script is provided via a downloadUrl");
    +
    +    @SetFromFlag("downloadUrl")
    +    AttributeSensorAndConfigKey<String, String> DOWNLOAD_URL = SoftwareProcess.DOWNLOAD_URL;
    +
    +    @SetFromFlag("scriptDir")
    +    ConfigKey<String> SCRIPT_DIR = newConfigKey("scriptDir", "directory where downloaded scripts should be put", Os.tmp());
    --- End diff --
    
    I need a temporary location on the remote machine to put the script before I execute it.  A subdir of /tmp seems the best choice.  I like the RUN_DIR idea and will have a look at adding that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-158143546
  
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45317862
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.test.framework;
    +
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Splitter;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.TaskFactory;
    +import org.apache.brooklyn.core.effector.ssh.SshEffectorTasks;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.location.Machines;
    +import org.apache.brooklyn.location.ssh.SshMachineLocation;
    +import org.apache.brooklyn.test.Asserts;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.core.task.DynamicTasks;
    +import org.apache.brooklyn.util.core.task.ssh.SshTasks;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskWrapper;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Identifiers;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.*;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +import static org.apache.brooklyn.util.groovy.GroovyJavaMethods.truth;
    +import static org.apache.brooklyn.util.text.Strings.isBlank;
    +import static org.apache.brooklyn.util.text.Strings.isNonBlank;
    +import static org.apache.commons.collections.MapUtils.isEmpty;
    +
    +public class SimpleShellCommandTestImpl extends AbstractTest implements SimpleShellCommandTest {
    +
    +    public static final int SUCCESS = 0;
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleShellCommandTestImpl.class);
    +    private static final int A_LINE = 80;
    +    public static final String DEFAULT_NAME = "download.sh";
    +    private static final String CD = "cd";
    +
    +    @Override
    +    public void start(Collection<? extends Location> locations) {
    +        setExpectedState(this, STARTING);
    +        execute();
    +    }
    +
    +    @Override
    +    public void stop() {
    +        LOG.debug("{} Stopping simple command", this);
    +        setUpAndRunState(false, STOPPED);
    +    }
    +
    +    @Override
    +    public void restart() {
    +        LOG.debug("{} Restarting simple command", this);
    +        execute();
    +    }
    +
    +    private void setUpAndRunState(boolean up, Lifecycle status) {
    +        sensors().set(SERVICE_UP, up);
    +        setExpectedState(this, status);
    +    }
    +
    +    private static class Result {
    +        int exitCode;
    +        String stdout;
    +        String stderr;
    +        public Result(final ProcessTaskWrapper<Integer> job) {
    +            exitCode = job.get();
    +            stdout = job.getStdout().trim();
    +            stderr = job.getStderr().trim();
    +        }
    +        public int getExitCode() {
    +            return exitCode;
    +        }
    +        public String getStdout() {
    +            return stdout;
    +        }
    +        public String getStderr() {
    +            return stderr;
    +        }
    +    }
    +
    +    protected void handle(Result result) {
    +        LOG.debug("{}, Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +            this, result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +        AssertionSupport support = new AssertionSupport();
    +        checkAssertions(support, exitCodeAssertions(), "exit code", result.getExitCode());
    +        checkAssertions(support, getConfig(ASSERT_OUT), "stdout", result.getStdout());
    +        checkAssertions(support, getConfig(ASSERT_ERR), "stderr", result.getStderr());
    +        support.validate();
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    public void execute() {
    +        try {
    +            SshMachineLocation machineLocation =
    +                Machines.findUniqueMachineLocation(resolveTarget().getLocations(), SshMachineLocation.class).get();
    +            executeCommand(machineLocation);
    +            setUpAndRunState(true, RUNNING);
    +        } catch (Throwable t) {
    +            setUpAndRunState(false, ON_FIRE);
    +            throw Exceptions.propagate(t);
    +        }
    +    }
    +
    +    private void executeCommand(SshMachineLocation machineLocation) {
    +
    +        Result result = null;
    +        String downloadUrl = getConfig(DOWNLOAD_URL);
    +        String command = getConfig(COMMAND);
    +
    +        String downloadName = DOWNLOAD_URL.getName();
    +        String commandName = COMMAND.getName();
    +
    +        if (!(isNonBlank(downloadUrl) ^ isNonBlank(command))) {
    +            throw illegal("Must specify exactly one of", downloadName, "and", commandName);
    +        }
    +
    +        if (isNonBlank(downloadUrl)) {
    +            String scriptDir = getConfig(SCRIPT_DIR);
    +            String scriptPath = calculateDestPath(downloadUrl, scriptDir);
    +            result = executeDownloadedScript(machineLocation, downloadUrl, scriptPath);
    --- End diff --
    
    why copy file and run instead of the code i suggested which just runs the commands?  (i think it has built-in support to write to a file and execute the file.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-157683343
  
    I’ve made most of the changes to the simple command classes from the review comments.  The driver is gone, but I have had trouble trying to get rid of the lifecycle effector stuff entirely (thought the class is simpler) — without it my unit tests fail to get the MachineLocation. 
    
    Pushing back the changes so far.  Maybe it’s now simple enough to be ok as-is.  
    
    I meant to mention before the reason there’s a separate SimpleShellCommand and a SimpleShellCommandTest subclass was that I tried to keep a “separation of concerns” between invoking the command and testing the results (with a notion that SimpleShellCommand might be useful in its own right apart from tests).   But I could collapse those together to cut down the number of classes.  What do you think?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860563
  
    --- Diff: usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleCommandScriptIntegrationTest.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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.test.framework;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Splitter;
    +import com.google.common.base.Strings;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Iterables;
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.location.LocationSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.Task;
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
    +import org.apache.brooklyn.core.test.entity.TestApplication;
    +import org.apache.brooklyn.core.test.entity.TestEntity;
    +import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
    +import org.apache.brooklyn.test.http.TestHttpRequestHandler;
    +import org.apache.brooklyn.test.http.TestHttpServer;
    +import org.apache.brooklyn.util.core.task.Tasks;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.FatalRuntimeException;
    +import org.apache.brooklyn.util.http.HttpAsserts;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.*;
    +
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.Iterator;
    +import java.util.UUID;
    +
    +import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
    +import static org.apache.brooklyn.test.framework.SimpleCommand.DEFAULT_COMMAND;
    +import static org.apache.brooklyn.test.framework.SimpleCommandTest.*;
    +import static org.assertj.core.api.Assertions.assertThat;
    +
    +public class SimpleCommandScriptIntegrationTest {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandScriptIntegrationTest.class);
    +
    +    private static final String UP = "up";
    +    private static final String SCRIPT_NAME = "script.sh";
    +    private static final String TEXT = "hello world";
    +    private TestApplication app;
    +    private ManagementContext managementContext;
    +    private LocalhostMachineProvisioningLocation localhost;
    +    private TestHttpServer server;
    +    private String testId;
    +
    +
    +    @BeforeClass
    +    public void setUpTests() {
    +        server = initializeServer();
    +    }
    +
    +    @AfterClass
    +    public void tearDownTests() {
    +        if (null != server) {
    +            server.stop();
    +        }
    +    }
    +
    +    @BeforeMethod
    +    public void setUp() {
    +
    +        testId = UUID.randomUUID().toString();
    +
    +        app = TestApplication.Factory.newManagedInstanceForTests();
    +        managementContext = app.getManagementContext();
    +
    +        localhost = managementContext.getLocationManager()
    +            .createLocation(LocationSpec.create(LocalhostMachineProvisioningLocation.class)
    +                .configure("name", testId));
    +    }
    +
    +    @AfterMethod(alwaysRun = true)
    +    public void tearDown() throws Exception {
    +        if (app != null) Entities.destroyAll(app.getManagementContext());
    +    }
    +
    +
    +    private TestHttpServer initializeServerUnstarted() {
    --- End diff --
    
    Interesting, and simpler than I thought it would be!
    
    I'd recommend just passing in a file URL for something on localhost (i.e. `file://..`). In your setup, you can create the file on localhost with the desired script contents, and then in tearDown you can delete it.
    
    For an example, see `VanillaSoftwareProcessIntegrationTest.testDownloadOnlyCmd`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44902605
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    --- End diff --
    
    I'll see if I can do that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-156714424
  
    Taking a step back from the minor comments I've been making so far... This feels like a lot of code we have to write for the SimpleCommand entity. I wonder if there is another way of doing it that does not involve copying the pattern used in the `SoftwareProcess` entity.
    
    The "driver pattern" was really introduced to have different implementations depending on whether it was ssh or something else - and it's never used for anything but ssh. The "LifecycleEffectorTasks pattern" is mostly for ensuring that all the different parts of the entity deployment/actions are executed correctly as tasks that are tracked in the activity view. It includes support for many tasks being executed (e.g. for pre-install, install, post-install, customize, etc). We don't need all that complexity for `SimpleCommand`.
    
    All we really need is to get hold of the right SshMachineLocation and to execute a command on it using `sshMachine.execScript()`.
    
    For the download-url case, we could download it on Brooklyn, and just pass the string contents to `sshMachine.execScript`, or if it might be binary then we could use `machine.copyTo(...); machine.execScript(...)`.
    
    To ensure the command's stdout/stderr etc appears in the activites view correctly, we can go through the `SshEffectorTasks` stuff rather than calling methods on machine directly.
    
    My gut feel is that we could put that code directly in the start() method of the `SimpleCommandImpl` (or more accurately in a method that it delegates to).
    
    Does that sound like it would work, and does it sound like it would reduce massively the amount of code required?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859871
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    --- End diff --
    
    Can we just use `AbstractEntity.locations`? i.e. `start` or `startOnLocations` would call `addLocation()`, and other things would call `getLocations()` as needed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859968
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    +
    +    @Override
    +    public SimpleCommandDriver getDriver() {
    +        return driver;
    +    }
    +
    +    @Override
    +    public Class<SimpleCommandDriver> getDriverInterface() {
    +        return SimpleCommandDriver.class;
    +    }
    +
    +    /**
    +     * Gives the opportunity to sub-classes to do additional work based on the result of the command.
    +     */
    +    protected void handle(SimpleCommandDriver.Result result) {
    +        LOG.debug("Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +                result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    /**
    +     * Does nothing in this class but gives sub-classes the opportunity to filter locations according to some criterion.
    +     */
    +    public Collection<? extends Location> filterLocations(Collection<? extends Location> locations) {
    +        return locations;
    +    }
    +
    +
    +    @Override
    +    public void init() {
    +        super.init();
    +        getLifecycleEffectorTasks().attachLifecycleEffectors(this);
    +    }
    +
    +
    +    protected void initDriver(MachineLocation machine) {
    +        LOG.debug("Initializing simple command driver");
    +        SimpleCommandDriver newDriver = doInitDriver(machine);
    +        if (newDriver == null) {
    +            throw new UnsupportedOperationException("cannot start "+this+" on "+machine+": no driver available");
    +        }
    +        driver = newDriver;
    +    }
    +
    +    protected SimpleCommandDriver doInitDriver(MachineLocation machine) {
    +        if (driver!=null) {
    +            if (machine.equals(driver.getLocation())) {
    +                return driver; //just reuse
    +            } else {
    +                LOG.warn("driver/location change is untested for {} at {}; changing driver and continuing", this, machine);
    +                return newDriver(machine);
    +            }
    +        } else {
    +            return newDriver(machine);
    +        }
    +    }
    +
    +    protected SimpleCommandDriver newDriver(MachineLocation machine) {
    +        LOG.debug("Creating new simple command driver for {} from management context", machine);
    +        EntityDriverManager entityDriverManager = getManagementContext().getEntityDriverManager();
    +        return entityDriverManager.build(this, machine);
    +    }
    +
    +    @Override
    +    public void start(@EffectorParam(name = "locations") Collection<? extends Location> locations) {
    +        this.locations = locations;
    +        startOnLocations();
    +    }
    +
    +    protected void startOnLocations() {
    +        setExpectedState(this, STARTING);
    +        int size = locations.size();
    +        LOG.debug("Starting simple command at {} locations{}", size,
    +                size > 0 ? " beginning " + locations.iterator().next() : "");
    +        try {
    +            execute(locations);
    +            setUpAndRunState(true, RUNNING);
    +
    +        } catch (final Exception e) {
    +            setUpAndRunState(false, ON_FIRE);
    +            throw Exceptions.propagate(e);
    +        }
    +    }
    +
    +    private void execute(Collection<? extends Location> locations) {
    +        SimpleCommandDriver.Result result = null;
    +        String downloadUrl = getConfig(DOWNLOAD_URL);
    +        if (Strings.isNonBlank(downloadUrl)) {
    +            String scriptDir = getConfig(SCRIPT_DIR);
    +            result = getDriver().executeDownloadedScript(locations, downloadUrl, scriptDir);
    +
    +        } else {
    +            String command = getConfig(DEFAULT_COMMAND);
    +            if (Strings.isBlank(command)) {
    +                throw new IllegalArgumentException("No default command and no downloadUrl provided");
    --- End diff --
    
    I'd include the context in the message - to say which entity it was. That can be very helpful when looking through a log file, to find out what the exception relates to.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860541
  
    --- Diff: usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleCommandImplIntegrationTest.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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.test.framework;
    +
    +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.LocationSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
    +import org.apache.brooklyn.core.test.entity.TestApplication;
    +import org.apache.brooklyn.core.test.entity.TestEntity;
    +import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.AfterMethod;
    +import org.testng.annotations.BeforeMethod;
    +import org.testng.annotations.Test;
    +
    +import java.util.UUID;
    +
    +import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
    +import static org.apache.brooklyn.test.framework.SimpleCommand.DEFAULT_COMMAND;
    +import static org.apache.brooklyn.test.framework.SimpleCommandTest.*;
    +import static org.assertj.core.api.Assertions.assertThat;
    +
    +public class SimpleCommandImplIntegrationTest {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImplIntegrationTest.class);
    +
    +    private static final String UP = "up";
    +    private TestApplication app;
    +    private ManagementContext managementContext;
    +    private LocalhostMachineProvisioningLocation localhost;
    +    private String testId;
    +
    +
    +    @BeforeMethod
    +    public void setUp() {
    +
    +        testId = UUID.randomUUID().toString();
    +
    +        app = TestApplication.Factory.newManagedInstanceForTests();
    +        managementContext = app.getManagementContext();
    +
    +        localhost = managementContext.getLocationManager()
    +            .createLocation(LocationSpec.create(LocalhostMachineProvisioningLocation.class)
    +                .configure("name", testId));
    --- End diff --
    
    You shouldn't have to give it a name - it will auto-generate a displayname if you don't supply one (it will concatenate the location's simple name with the first 4 chars of the location id, or something like that). Worth just deleting the few extra lines from each test, I think.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-156983714
  
    Thanks for all the comments Aled, if I haven't responded to any particular one it's because I'll just go ahead with it.  Will get on to making the updates accordingly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859887
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    +
    +    @Override
    +    public SimpleCommandDriver getDriver() {
    +        return driver;
    +    }
    +
    +    @Override
    +    public Class<SimpleCommandDriver> getDriverInterface() {
    +        return SimpleCommandDriver.class;
    +    }
    +
    +    /**
    +     * Gives the opportunity to sub-classes to do additional work based on the result of the command.
    +     */
    +    protected void handle(SimpleCommandDriver.Result result) {
    +        LOG.debug("Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +                result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    /**
    +     * Does nothing in this class but gives sub-classes the opportunity to filter locations according to some criterion.
    +     */
    +    public Collection<? extends Location> filterLocations(Collection<? extends Location> locations) {
    +        return locations;
    +    }
    +
    +
    +    @Override
    +    public void init() {
    +        super.init();
    +        getLifecycleEffectorTasks().attachLifecycleEffectors(this);
    +    }
    +
    +
    +    protected void initDriver(MachineLocation machine) {
    +        LOG.debug("Initializing simple command driver");
    --- End diff --
    
    For debug messages like this, prefer to include some context such as a toString of this entity. Imagine if it's running in a Brooklyn server with a dozen other tests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-brooklyn/pull/1030


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45317585
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensor.java ---
    @@ -21,6 +21,7 @@
     import org.apache.brooklyn.api.entity.ImplementedBy;
     import org.apache.brooklyn.config.ConfigKey;
     import org.apache.brooklyn.core.config.ConfigKeys;
    +import org.apache.brooklyn.core.entity.trait.Startable;
    --- End diff --
    
    is this import needed?  (it's the only changed line in the file!)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45317465
  
    --- Diff: usage/test-framework/src/test/resources/brooklyn/logback-appender-stdout.xml ---
    @@ -0,0 +1,37 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +    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.
    +-->
    +<included>
    --- End diff --
    
    normal practice is to include the project `brooklyn-logback-includes` in the pom instead of redeclaring the log config here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44902697
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    +
    +    @Override
    +    public SimpleCommandDriver getDriver() {
    +        return driver;
    +    }
    +
    +    @Override
    +    public Class<SimpleCommandDriver> getDriverInterface() {
    +        return SimpleCommandDriver.class;
    +    }
    +
    +    /**
    +     * Gives the opportunity to sub-classes to do additional work based on the result of the command.
    +     */
    +    protected void handle(SimpleCommandDriver.Result result) {
    +        LOG.debug("Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +                result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    /**
    +     * Does nothing in this class but gives sub-classes the opportunity to filter locations according to some criterion.
    +     */
    +    public Collection<? extends Location> filterLocations(Collection<? extends Location> locations) {
    +        return locations;
    +    }
    +
    +
    +    @Override
    +    public void init() {
    +        super.init();
    +        getLifecycleEffectorTasks().attachLifecycleEffectors(this);
    +    }
    +
    +
    +    protected void initDriver(MachineLocation machine) {
    +        LOG.debug("Initializing simple command driver");
    +        SimpleCommandDriver newDriver = doInitDriver(machine);
    +        if (newDriver == null) {
    +            throw new UnsupportedOperationException("cannot start "+this+" on "+machine+": no driver available");
    +        }
    +        driver = newDriver;
    +    }
    +
    +    protected SimpleCommandDriver doInitDriver(MachineLocation machine) {
    +        if (driver!=null) {
    +            if (machine.equals(driver.getLocation())) {
    +                return driver; //just reuse
    +            } else {
    +                LOG.warn("driver/location change is untested for {} at {}; changing driver and continuing", this, machine);
    +                return newDriver(machine);
    +            }
    +        } else {
    +            return newDriver(machine);
    +        }
    +    }
    +
    +    protected SimpleCommandDriver newDriver(MachineLocation machine) {
    +        LOG.debug("Creating new simple command driver for {} from management context", machine);
    +        EntityDriverManager entityDriverManager = getManagementContext().getEntityDriverManager();
    +        return entityDriverManager.build(this, machine);
    +    }
    +
    +    @Override
    +    public void start(@EffectorParam(name = "locations") Collection<? extends Location> locations) {
    +        this.locations = locations;
    +        startOnLocations();
    +    }
    +
    +    protected void startOnLocations() {
    +        setExpectedState(this, STARTING);
    +        int size = locations.size();
    +        LOG.debug("Starting simple command at {} locations{}", size,
    +                size > 0 ? " beginning " + locations.iterator().next() : "");
    +        try {
    +            execute(locations);
    +            setUpAndRunState(true, RUNNING);
    +
    +        } catch (final Exception e) {
    --- End diff --
    
    Thanks - still following the rules from my previous role :-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-157727948
  
    Forgot to mark tests as integration, will do so now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44902571
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommand.java ---
    @@ -0,0 +1,50 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.Entity;
    +import org.apache.brooklyn.api.entity.ImplementedBy;
    +import org.apache.brooklyn.config.ConfigKey;
    +import org.apache.brooklyn.core.config.ConfigKeys;
    +import org.apache.brooklyn.core.entity.trait.Startable;
    +import org.apache.brooklyn.core.sensor.AttributeSensorAndConfigKey;
    +import org.apache.brooklyn.entity.software.base.SoftwareProcess;
    +import org.apache.brooklyn.util.core.flags.SetFromFlag;
    +import org.apache.brooklyn.util.os.Os;
    +
    +import static org.apache.brooklyn.core.config.ConfigKeys.newConfigKey;
    +
    +/**
    + * Entity to invoke on a node a simple command that will immediately succeed or fail.
    + *
    + * Invokes the command in the start operation, and declares itself RUNNING.
    + */
    +@ImplementedBy(SimpleCommandImpl.class)
    +public interface SimpleCommand extends Entity, Startable {
    +
    +    @SetFromFlag(nullable = false)
    +    ConfigKey<String> DEFAULT_COMMAND = ConfigKeys.newConfigKey(String.class, "defaultCommand",
    +            "Command to invoke if no script is provided via a downloadUrl");
    +
    +    @SetFromFlag("downloadUrl")
    +    AttributeSensorAndConfigKey<String, String> DOWNLOAD_URL = SoftwareProcess.DOWNLOAD_URL;
    +
    +    @SetFromFlag("scriptDir")
    +    ConfigKey<String> SCRIPT_DIR = newConfigKey("scriptDir", "directory where downloaded scripts should be put", Os.tmp());
    --- End diff --
    
    Re. Os.tmp(), it is intended to be the dir on the remote machine, I'll change that to "/tmp".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860460
  
    --- Diff: usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleCommandImplIntegrationTest.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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.test.framework;
    +
    +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.LocationSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
    +import org.apache.brooklyn.core.test.entity.TestApplication;
    +import org.apache.brooklyn.core.test.entity.TestEntity;
    +import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.AfterMethod;
    +import org.testng.annotations.BeforeMethod;
    +import org.testng.annotations.Test;
    +
    +import java.util.UUID;
    +
    +import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
    +import static org.apache.brooklyn.test.framework.SimpleCommand.DEFAULT_COMMAND;
    +import static org.apache.brooklyn.test.framework.SimpleCommandTest.*;
    +import static org.assertj.core.api.Assertions.assertThat;
    +
    +public class SimpleCommandImplIntegrationTest {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImplIntegrationTest.class);
    +
    +    private static final String UP = "up";
    +    private TestApplication app;
    +    private ManagementContext managementContext;
    +    private LocalhostMachineProvisioningLocation localhost;
    +    private String testId;
    +
    +
    +    @BeforeMethod
    +    public void setUp() {
    +
    +        testId = UUID.randomUUID().toString();
    +
    +        app = TestApplication.Factory.newManagedInstanceForTests();
    +        managementContext = app.getManagementContext();
    +
    +        localhost = managementContext.getLocationManager()
    +            .createLocation(LocationSpec.create(LocalhostMachineProvisioningLocation.class)
    +                .configure("name", testId));
    +    }
    +
    +    @AfterMethod(alwaysRun = true)
    +    public void tearDown() throws Exception {
    +        if (app != null) Entities.destroyAll(app.getManagementContext());
    +    }
    +
    +    @Test(groups = "Integration")
    +    public void shouldInvokeCommand() {
    +        TestEntity testEntity = app.createAndManageChild(EntitySpec.create(TestEntity.class));
    +
    +        SimpleCommandTest uptime = app.createAndManageChild(EntitySpec.create(SimpleCommandTest.class)
    +            .configure(TARGET_ENTITY, testEntity)
    +            .configure(DEFAULT_COMMAND, "uptime")
    +            .configure(ASSERT_STATUS, ImmutableMap.of(EQUALS, 0))
    +            .configure(ASSERT_OUT, ImmutableMap.of(CONTAINS, UP)));
    +
    +        app.start(ImmutableList.of(localhost));
    --- End diff --
    
    I don't follow how this test works. The `TestEntity` doesn't actually "provision" a VM (see https://github.com/apache/incubator-brooklyn/blob/master/core/src/test/java/org/apache/brooklyn/core/test/entity/TestEntityImpl.java#L122-L129). The only location it will have is the `LocalhostMachineProvisioningLocation`. Therefore what does the `uptime` entity execute its command on?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
GitHub user geomacy reopened a pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030

    SimpleCommand addition to brooklyn-test-framework

    ### Description
    
    An additional feature for the brooklyn-test-framework.
    
    Supports a test that runs a simple command (expected to return quickly) 
    on the host of the target entity and allows assertions to be made on the
    exit code, standard out and standard error of the command.  
    
    This is to support the use of arbitrary 'entities' (in the general sense)
    in tests using the brooklyn-test-framework.  Tests are regular blueprints
    so already allow the deployment of arbitrary Brooklyn entities. Additionally,
    to deploy arbitrary server type processes (expected to continue running),
    one can use the VanillaSoftwareProcess, which runs an arbitrary script.
    
    The SimpleCommandTest in this PR is to support a similar case to that of the 
    VanillaSoftwareProcess, except the command is expected to "do something", 
    finishing quickly, and return a result (process exit code), along with its 
    standard out and error streams, which can then be tested using assertions
    provided by the test-framework.
    
    See below for example YAML.
    
    ### Usage
    - The type is type: org.apache.brooklyn.test.framework.SimpleCommandTest.
    - The command can be specified as a script to be downloaded, by default
    to /tmp, using the "downloadUrl" flag,
    - Download directory can be changed using flag scriptDir.
    - Alternatively the command can be specified as a simple command
    to be executed, using the "defaultCommand" flag. This command can have
    bash style command syntax including pipes and redirects.
    - The command is run on the node of the target entity of the test via
    an SSH connection.
    - Assertions are made on the return code of the command using the 
    assertStatus flag, and on the standard output and error using assertOut
    and assertErr.
    
    ### TODO
    - The currently supported exceptions are equals, contains, matches and 
    isEmpty.  A TODO is to merge this with the refactored standalone assertion 
    class currently being developed for the test-framework.
    - Make the command execute against members of a cluster.  At present
    the command will only run against one of the cluster members.
    
    ### Example YAML
    ```
    name: simpleCommandTest 
    location: byon4
    services:
    - type: org.apache.brooklyn.test.framework.TestCase
      name: testcase1
      targetId: testprocess
      brooklyn.children:
        - type: org.apache.brooklyn.entity.software.base.EmptySoftwareProcess
          id: testprocess
    
        - type: org.apache.brooklyn.test.framework.SimpleCommandTest
          defaultCommand: hostname
          assertStatus:
            equals: 0
          assertOut: 
            equals: byon4
          assertErr: 
            isEmpty: true
    
        - type: org.apache.brooklyn.test.framework.SimpleCommandTest
          downloadUrl: http://localhost:8080/script1.sh
          assertStatus:
            equals: 0
          assertOut: 
            equals: hello world
          assertErr: 
            isEmpty: true
    ```

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/geomacy/incubator-brooklyn simplecommand

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-brooklyn/pull/1030.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1030
    
----
commit b4e3ac7b68ed26f3efa74be7285de013d1f8c02a
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-05T15:06:45Z

    Initial draft of SimpleCommand.

commit 6cd62e7d3b58876f40bbea9efb49426c9464589f
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-13T15:31:13Z

    Disable failing tests while I inquire about proper use of tests that require SSH access on Jenkins build machine.

commit 014b6bf162c2ca5df7cedf55c63232eed8028e22
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-13T15:44:46Z

    Make SimpleCommand tests be Integration tests as they use SSH.

commit b30489ed043da00162f5102dbb6ccfd8582d05df
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T10:41:42Z

    Initial restructuring due to review comments.
    
    Removes as much as possible of the driver/lifecycle material, in response to
    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-156714424
    
    Driver related classes removed.
    
    SshEffectorTasks used where possible.
    
    The lifecycle class is slimmed down but retained as it takes care of getting
    hold of the MachineLocation.

commit 93039184a736a9fc698758d014e545a010438e42
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T11:41:00Z

    Use /tmp rather than Os.tmp()
    
    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859808

commit cff1961fa442f6695ebc3b891f87a3235c3988c5
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T12:18:52Z

    Add RUN_DIR.
    
    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45050093

commit f6801227a90ebe0710a4f98509e61036606a2e81
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T12:23:46Z

    Add 'this' to loggers for context.
    
    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859887

commit f06fb74fed0b92d97f037efe4b958212df6d3156
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T12:25:14Z

    Remove unnecessary annotation.
    
    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859924

commit f173ad31f2ec08ac3602d92d4c0a14cb71406f4b
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T12:31:58Z

    Rename DEFAULT_COMMAND to COMMAND.
    
    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859961

commit 4467878da3e5f5738c469fe67da2ce544f0082da
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T12:34:22Z

    Format change.
    
    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859977

commit f3a6395e0028d26af473428f9d248a7d917ee7d1
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T13:19:43Z

    Simplify integration test, avoid TestHttpServer and just have one test file.
    
    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860563
    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860570

commit 5f98079e62a7668aec8139fab4848b8e064824b6
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T14:14:57Z

    Rename everything as SimpleShellCommand.
    
    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-157311566

commit 89d69269cad450f342a622c375b92c00c5482670
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T14:18:17Z

    Don't need to configure name in integration test.
    
    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860541

commit 60178809b3c51a429e6871bb68404245baa86269
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-17T16:23:23Z

    Additional tests and improvements.

commit b2713ca4d7382ff1057531a6ef92a8a038a59d39
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Date:   2015-11-18T12:59:25Z

    Fix location handling in integration test and remove unnecessary classes.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859924
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    +
    +    @Override
    +    public SimpleCommandDriver getDriver() {
    +        return driver;
    +    }
    +
    +    @Override
    +    public Class<SimpleCommandDriver> getDriverInterface() {
    +        return SimpleCommandDriver.class;
    +    }
    +
    +    /**
    +     * Gives the opportunity to sub-classes to do additional work based on the result of the command.
    +     */
    +    protected void handle(SimpleCommandDriver.Result result) {
    +        LOG.debug("Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +                result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    /**
    +     * Does nothing in this class but gives sub-classes the opportunity to filter locations according to some criterion.
    +     */
    +    public Collection<? extends Location> filterLocations(Collection<? extends Location> locations) {
    +        return locations;
    +    }
    +
    +
    +    @Override
    +    public void init() {
    +        super.init();
    +        getLifecycleEffectorTasks().attachLifecycleEffectors(this);
    +    }
    +
    +
    +    protected void initDriver(MachineLocation machine) {
    +        LOG.debug("Initializing simple command driver");
    +        SimpleCommandDriver newDriver = doInitDriver(machine);
    +        if (newDriver == null) {
    +            throw new UnsupportedOperationException("cannot start "+this+" on "+machine+": no driver available");
    +        }
    +        driver = newDriver;
    +    }
    +
    +    protected SimpleCommandDriver doInitDriver(MachineLocation machine) {
    +        if (driver!=null) {
    +            if (machine.equals(driver.getLocation())) {
    +                return driver; //just reuse
    +            } else {
    +                LOG.warn("driver/location change is untested for {} at {}; changing driver and continuing", this, machine);
    +                return newDriver(machine);
    +            }
    +        } else {
    +            return newDriver(machine);
    +        }
    +    }
    +
    +    protected SimpleCommandDriver newDriver(MachineLocation machine) {
    +        LOG.debug("Creating new simple command driver for {} from management context", machine);
    +        EntityDriverManager entityDriverManager = getManagementContext().getEntityDriverManager();
    +        return entityDriverManager.build(this, machine);
    +    }
    +
    +    @Override
    +    public void start(@EffectorParam(name = "locations") Collection<? extends Location> locations) {
    --- End diff --
    
    Don't need the `@EffectorParam` annotations on the concrete  class - just on the interface where it's declared as an effector.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-157683009
  
    I think there's no need for a driver or for start to do anything other than the ssh task above (maybe wait for the target entity to stop).
    
    The task returns a `ProcessTaskWrapper` which allows you to get any info on the process.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860015
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandSshDriver.java ---
    @@ -0,0 +1,192 @@
    +/*
    + * 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.test.framework;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Splitter;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.brooklyn.api.entity.EntityLocal;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.TaskAdaptable;
    +import org.apache.brooklyn.api.mgmt.TaskFactory;
    +import org.apache.brooklyn.core.location.Locations;
    +import org.apache.brooklyn.location.ssh.SshMachineLocation;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.core.ResourceUtils;
    +import org.apache.brooklyn.util.core.task.DynamicTasks;
    +import org.apache.brooklyn.util.core.task.ssh.SshTasks;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskFactory;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskWrapper;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.FatalRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.Random;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static org.apache.brooklyn.util.text.Strings.isBlank;
    +
    +/**
    + * Driver for {@link SimpleCommand}.
    + */
    +public class SimpleCommandSshDriver implements SimpleCommandDriver {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandSshDriver.class);
    +    public static final String DEFAULT_NAME = "download.sh";
    +
    +    protected final EntityLocal entity;
    +    protected final ResourceUtils resource;
    +    protected final Location location;
    +
    +    public SimpleCommandSshDriver(EntityLocal entity, SshMachineLocation location) {
    +        LOG.debug("Constructing SSH driver for simple command for {} at {}", entity, location);
    +        this.entity = checkNotNull(entity, "entity");
    +        this.location = checkNotNull(location, "location");
    +        this.resource = ResourceUtils.create(entity);
    +    }
    +
    +    @Override
    +    public EntityLocal getEntity() {
    +        return entity;
    +    }
    +
    +    @Override
    +    public void start() {
    +        LOG.debug("Performing start in SSH driver for simple command");
    +        invoke();
    +    }
    +
    +    private void invoke() {
    +        SimpleCommand simpleCommand = (SimpleCommand) getEntity();
    +        simpleCommand.start(ImmutableList.of(location));
    --- End diff --
    
    This looks surprising, that the driver's `start()` calls into the entity's `start()`. I'd have thought that the entity's `start()` would have caused the driver to be invoked rather than the other way round.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44902950
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    +
    +    @Override
    +    public SimpleCommandDriver getDriver() {
    +        return driver;
    +    }
    +
    +    @Override
    +    public Class<SimpleCommandDriver> getDriverInterface() {
    +        return SimpleCommandDriver.class;
    +    }
    +
    +    /**
    +     * Gives the opportunity to sub-classes to do additional work based on the result of the command.
    +     */
    +    protected void handle(SimpleCommandDriver.Result result) {
    +        LOG.debug("Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +                result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    /**
    +     * Does nothing in this class but gives sub-classes the opportunity to filter locations according to some criterion.
    +     */
    +    public Collection<? extends Location> filterLocations(Collection<? extends Location> locations) {
    +        return locations;
    +    }
    +
    +
    +    @Override
    +    public void init() {
    +        super.init();
    +        getLifecycleEffectorTasks().attachLifecycleEffectors(this);
    +    }
    +
    +
    +    protected void initDriver(MachineLocation machine) {
    +        LOG.debug("Initializing simple command driver");
    +        SimpleCommandDriver newDriver = doInitDriver(machine);
    +        if (newDriver == null) {
    +            throw new UnsupportedOperationException("cannot start "+this+" on "+machine+": no driver available");
    +        }
    +        driver = newDriver;
    +    }
    +
    +    protected SimpleCommandDriver doInitDriver(MachineLocation machine) {
    +        if (driver!=null) {
    +            if (machine.equals(driver.getLocation())) {
    +                return driver; //just reuse
    +            } else {
    +                LOG.warn("driver/location change is untested for {} at {}; changing driver and continuing", this, machine);
    +                return newDriver(machine);
    +            }
    +        } else {
    +            return newDriver(machine);
    +        }
    +    }
    +
    +    protected SimpleCommandDriver newDriver(MachineLocation machine) {
    +        LOG.debug("Creating new simple command driver for {} from management context", machine);
    +        EntityDriverManager entityDriverManager = getManagementContext().getEntityDriverManager();
    +        return entityDriverManager.build(this, machine);
    +    }
    +
    +    @Override
    +    public void start(@EffectorParam(name = "locations") Collection<? extends Location> locations) {
    +        this.locations = locations;
    +        startOnLocations();
    +    }
    +
    +    protected void startOnLocations() {
    +        setExpectedState(this, STARTING);
    +        int size = locations.size();
    +        LOG.debug("Starting simple command at {} locations{}", size,
    +                size > 0 ? " beginning " + locations.iterator().next() : "");
    +        try {
    +            execute(locations);
    +            setUpAndRunState(true, RUNNING);
    +
    +        } catch (final Exception e) {
    +            setUpAndRunState(false, ON_FIRE);
    +            throw Exceptions.propagate(e);
    +        }
    +    }
    +
    +    private void execute(Collection<? extends Location> locations) {
    +        SimpleCommandDriver.Result result = null;
    +        String downloadUrl = getConfig(DOWNLOAD_URL);
    +        if (Strings.isNonBlank(downloadUrl)) {
    --- End diff --
    
    COMMAND would be better, right enough


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44907503
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandSshDriver.java ---
    @@ -0,0 +1,192 @@
    +/*
    + * 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.test.framework;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Splitter;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.brooklyn.api.entity.EntityLocal;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.TaskAdaptable;
    +import org.apache.brooklyn.api.mgmt.TaskFactory;
    +import org.apache.brooklyn.core.location.Locations;
    +import org.apache.brooklyn.location.ssh.SshMachineLocation;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.core.ResourceUtils;
    +import org.apache.brooklyn.util.core.task.DynamicTasks;
    +import org.apache.brooklyn.util.core.task.ssh.SshTasks;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskFactory;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskWrapper;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.FatalRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.Random;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static org.apache.brooklyn.util.text.Strings.isBlank;
    +
    +/**
    + * Driver for {@link SimpleCommand}.
    + */
    +public class SimpleCommandSshDriver implements SimpleCommandDriver {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandSshDriver.class);
    +    public static final String DEFAULT_NAME = "download.sh";
    +
    +    protected final EntityLocal entity;
    +    protected final ResourceUtils resource;
    +    protected final Location location;
    +
    +    public SimpleCommandSshDriver(EntityLocal entity, SshMachineLocation location) {
    +        LOG.debug("Constructing SSH driver for simple command for {} at {}", entity, location);
    +        this.entity = checkNotNull(entity, "entity");
    +        this.location = checkNotNull(location, "location");
    +        this.resource = ResourceUtils.create(entity);
    +    }
    +
    +    @Override
    +    public EntityLocal getEntity() {
    +        return entity;
    +    }
    +
    +    @Override
    +    public void start() {
    +        LOG.debug("Performing start in SSH driver for simple command");
    +        invoke();
    +    }
    +
    +    private void invoke() {
    +        SimpleCommand simpleCommand = (SimpleCommand) getEntity();
    +        simpleCommand.start(ImmutableList.of(location));
    --- End diff --
    
    Yes, this came from aiming for a simpler copy of the VanillaSoftwareProcess/SoftwareProcessDriver/SoftwareProcessDriverLifecycleEffectorTasks structure, but it's probably one indirection too many. Thankfully it can go away if we simplify things as discussed [above](https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44902393).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859933
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    +
    +    @Override
    +    public SimpleCommandDriver getDriver() {
    +        return driver;
    +    }
    +
    +    @Override
    +    public Class<SimpleCommandDriver> getDriverInterface() {
    +        return SimpleCommandDriver.class;
    +    }
    +
    +    /**
    +     * Gives the opportunity to sub-classes to do additional work based on the result of the command.
    +     */
    +    protected void handle(SimpleCommandDriver.Result result) {
    +        LOG.debug("Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +                result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    /**
    +     * Does nothing in this class but gives sub-classes the opportunity to filter locations according to some criterion.
    +     */
    +    public Collection<? extends Location> filterLocations(Collection<? extends Location> locations) {
    +        return locations;
    +    }
    +
    +
    +    @Override
    +    public void init() {
    +        super.init();
    +        getLifecycleEffectorTasks().attachLifecycleEffectors(this);
    +    }
    +
    +
    +    protected void initDriver(MachineLocation machine) {
    +        LOG.debug("Initializing simple command driver");
    +        SimpleCommandDriver newDriver = doInitDriver(machine);
    +        if (newDriver == null) {
    +            throw new UnsupportedOperationException("cannot start "+this+" on "+machine+": no driver available");
    +        }
    +        driver = newDriver;
    +    }
    +
    +    protected SimpleCommandDriver doInitDriver(MachineLocation machine) {
    +        if (driver!=null) {
    +            if (machine.equals(driver.getLocation())) {
    +                return driver; //just reuse
    +            } else {
    +                LOG.warn("driver/location change is untested for {} at {}; changing driver and continuing", this, machine);
    +                return newDriver(machine);
    +            }
    +        } else {
    +            return newDriver(machine);
    +        }
    +    }
    +
    +    protected SimpleCommandDriver newDriver(MachineLocation machine) {
    +        LOG.debug("Creating new simple command driver for {} from management context", machine);
    +        EntityDriverManager entityDriverManager = getManagementContext().getEntityDriverManager();
    +        return entityDriverManager.build(this, machine);
    +    }
    +
    +    @Override
    +    public void start(@EffectorParam(name = "locations") Collection<? extends Location> locations) {
    +        this.locations = locations;
    +        startOnLocations();
    +    }
    +
    +    protected void startOnLocations() {
    +        setExpectedState(this, STARTING);
    +        int size = locations.size();
    +        LOG.debug("Starting simple command at {} locations{}", size,
    +                size > 0 ? " beginning " + locations.iterator().next() : "");
    +        try {
    +            execute(locations);
    +            setUpAndRunState(true, RUNNING);
    +
    +        } catch (final Exception e) {
    --- End diff --
    
    Personal preference: not including `final` in things like this. Generally I love `final` on fields, and only bother with it on parameters if it's necessary for an anonymous inner class or some such.
    
    The rest of our code doesn't declare `final` for the exceptions in catch blocks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859836
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandDriver.java ---
    @@ -0,0 +1,70 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.EntityLocal;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriver;
    +import org.apache.brooklyn.api.location.Location;
    +
    +import java.util.Collection;
    +
    +/**
    + * Driver to invoke a command on a node.
    + */
    +public interface SimpleCommandDriver extends EntityDriver {
    +
    +    /**
    +     * Result of the command invocation.
    +     */
    +    interface Result {
    --- End diff --
    
    Could think about the relationship between this and `org.apache.brooklyn.feed.ssh.SshPollValue`. Feels like at some point we should extract a common interface/class, but not sure exactly where yet (so don't worry about it for this PR).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-157748032
  
    Commit includes updates as suggested by Aled, and also collapses the test classes so that there is now just the one interface SimpleShellCommandTest and its impl.  I found the cause of the problem I was having with the integration tests, where I wasn’t initialising the machine location properly, so I’ve been able to get rid of the lifecycle effector class too. 
    
    The build broke when I pushed it with what I *think* is an unrelated and hopefully transient issue, before it even gets to the test-framework - 
    ```
    org.apache.brooklyn.feed.jmx.JmxFeedTest.setUp (from TestSuite)
    Error Message
    Port already in use: 40129; nested exception is: 
     java.net.BindException: Address already in use
    ```
    Closing the PR in order to re-open to force a new build. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-157388335
  
    I want to be able to make assertions on the exit code, standard out and error, so .requiringZeroAndReturningStandardOut wouldn't cover that.  
    
    I have made updates based on most of the comments above and shall push them back shortly.  Things are simpler than before, and hopefully simple enough.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-158996965
  
    there are merge conflicts that have to be resolved.  personally i'd like to see the code improved before merging; if that means improving test assertion messages that seems to fit, it's not so large that it's worth the overhead of doing as a separate PR, but if you prefer to do a separate PR i'm happy to merge this once the conflicts are resolved.  and pls just put in a code comment such as:
    
        // TODO assertions below should use TestFrameworkAssertions but that class needs to be improved to give better error messages


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-157311566
  
    maybe call it `SimpleShellCommandTest` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860570
  
    --- Diff: usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleCommandScriptIntegrationTest.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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.test.framework;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Splitter;
    +import com.google.common.base.Strings;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Iterables;
    +import org.apache.brooklyn.api.entity.EntitySpec;
    +import org.apache.brooklyn.api.location.LocationSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.api.mgmt.Task;
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
    +import org.apache.brooklyn.core.test.entity.TestApplication;
    +import org.apache.brooklyn.core.test.entity.TestEntity;
    +import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
    +import org.apache.brooklyn.test.http.TestHttpRequestHandler;
    +import org.apache.brooklyn.test.http.TestHttpServer;
    +import org.apache.brooklyn.util.core.task.Tasks;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.FatalRuntimeException;
    +import org.apache.brooklyn.util.http.HttpAsserts;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.*;
    +
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.Iterator;
    +import java.util.UUID;
    +
    +import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
    +import static org.apache.brooklyn.test.framework.SimpleCommand.DEFAULT_COMMAND;
    +import static org.apache.brooklyn.test.framework.SimpleCommandTest.*;
    +import static org.assertj.core.api.Assertions.assertThat;
    +
    +public class SimpleCommandScriptIntegrationTest {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandScriptIntegrationTest.class);
    +
    +    private static final String UP = "up";
    +    private static final String SCRIPT_NAME = "script.sh";
    +    private static final String TEXT = "hello world";
    +    private TestApplication app;
    +    private ManagementContext managementContext;
    +    private LocalhostMachineProvisioningLocation localhost;
    +    private TestHttpServer server;
    +    private String testId;
    +
    +
    +    @BeforeClass
    +    public void setUpTests() {
    +        server = initializeServer();
    +    }
    +
    +    @AfterClass
    +    public void tearDownTests() {
    +        if (null != server) {
    +            server.stop();
    +        }
    +    }
    +
    +    @BeforeMethod
    +    public void setUp() {
    +
    +        testId = UUID.randomUUID().toString();
    +
    +        app = TestApplication.Factory.newManagedInstanceForTests();
    +        managementContext = app.getManagementContext();
    +
    +        localhost = managementContext.getLocationManager()
    +            .createLocation(LocationSpec.create(LocalhostMachineProvisioningLocation.class)
    +                .configure("name", testId));
    +    }
    +
    +    @AfterMethod(alwaysRun = true)
    +    public void tearDown() throws Exception {
    +        if (app != null) Entities.destroyAll(app.getManagementContext());
    +    }
    +
    +
    +    private TestHttpServer initializeServerUnstarted() {
    --- End diff --
    
    If you use the file url, you can probably just move this test into `SimpleCommandImplIntegrationTest.java`, and delete this class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44859961
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandImpl.java ---
    @@ -0,0 +1,177 @@
    +/*
    + * 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.test.framework;
    +
    +import org.apache.brooklyn.api.entity.drivers.DriverDependentEntity;
    +import org.apache.brooklyn.api.entity.drivers.EntityDriverManager;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.location.MachineLocation;
    +import org.apache.brooklyn.core.annotation.EffectorParam;
    +import org.apache.brooklyn.core.entity.AbstractEntity;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +
    +/**
    + * Implementation for {@link SimpleCommand}.
    + */
    +public class SimpleCommandImpl extends AbstractEntity
    +        implements SimpleCommand, DriverDependentEntity<SimpleCommandDriver> {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImpl.class);
    +    private static final int A_LINE = 80;
    +    private transient SimpleCommandDriver driver;
    +
    +    private Collection<? extends Location> locations;
    +
    +    @Override
    +    public SimpleCommandDriver getDriver() {
    +        return driver;
    +    }
    +
    +    @Override
    +    public Class<SimpleCommandDriver> getDriverInterface() {
    +        return SimpleCommandDriver.class;
    +    }
    +
    +    /**
    +     * Gives the opportunity to sub-classes to do additional work based on the result of the command.
    +     */
    +    protected void handle(SimpleCommandDriver.Result result) {
    +        LOG.debug("Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +                result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    /**
    +     * Does nothing in this class but gives sub-classes the opportunity to filter locations according to some criterion.
    +     */
    +    public Collection<? extends Location> filterLocations(Collection<? extends Location> locations) {
    +        return locations;
    +    }
    +
    +
    +    @Override
    +    public void init() {
    +        super.init();
    +        getLifecycleEffectorTasks().attachLifecycleEffectors(this);
    +    }
    +
    +
    +    protected void initDriver(MachineLocation machine) {
    +        LOG.debug("Initializing simple command driver");
    +        SimpleCommandDriver newDriver = doInitDriver(machine);
    +        if (newDriver == null) {
    +            throw new UnsupportedOperationException("cannot start "+this+" on "+machine+": no driver available");
    +        }
    +        driver = newDriver;
    +    }
    +
    +    protected SimpleCommandDriver doInitDriver(MachineLocation machine) {
    +        if (driver!=null) {
    +            if (machine.equals(driver.getLocation())) {
    +                return driver; //just reuse
    +            } else {
    +                LOG.warn("driver/location change is untested for {} at {}; changing driver and continuing", this, machine);
    +                return newDriver(machine);
    +            }
    +        } else {
    +            return newDriver(machine);
    +        }
    +    }
    +
    +    protected SimpleCommandDriver newDriver(MachineLocation machine) {
    +        LOG.debug("Creating new simple command driver for {} from management context", machine);
    +        EntityDriverManager entityDriverManager = getManagementContext().getEntityDriverManager();
    +        return entityDriverManager.build(this, machine);
    +    }
    +
    +    @Override
    +    public void start(@EffectorParam(name = "locations") Collection<? extends Location> locations) {
    +        this.locations = locations;
    +        startOnLocations();
    +    }
    +
    +    protected void startOnLocations() {
    +        setExpectedState(this, STARTING);
    +        int size = locations.size();
    +        LOG.debug("Starting simple command at {} locations{}", size,
    +                size > 0 ? " beginning " + locations.iterator().next() : "");
    +        try {
    +            execute(locations);
    +            setUpAndRunState(true, RUNNING);
    +
    +        } catch (final Exception e) {
    +            setUpAndRunState(false, ON_FIRE);
    +            throw Exceptions.propagate(e);
    +        }
    +    }
    +
    +    private void execute(Collection<? extends Location> locations) {
    +        SimpleCommandDriver.Result result = null;
    +        String downloadUrl = getConfig(DOWNLOAD_URL);
    +        if (Strings.isNonBlank(downloadUrl)) {
    --- End diff --
    
    I'd also log.warn if downloadUrl and defaultCommand are both set (because in that case we'll ignore their command).
    
    Is DEFAULT_COMMAND the right name? It feels more like *the* command if the user is just supplying a simple command, rather than a URL to download a full script. Maybe this should just be "command"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-157387018
  
    Agree with @aledsage this seems overly complicated; rather than having it be a driver it can be just like the other simple test cases, and just do
    
                SshEffectorTasks.SshEffectorTaskFactory<String> t = SshEffectorTasks.ssh(command)
                    .requiringZeroAndReturningStdout()
                    .summary("effector "+effector.getName())
                    .environmentVariables(env);
                return queue(t).get();
    
    only extra thing is to add the `machine` of the `target` and switch to its `RUN_DIR`, i think


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860159
  
    --- Diff: usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleCommandImplIntegrationTest.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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.test.framework;
    +
    +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.LocationSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
    +import org.apache.brooklyn.core.test.entity.TestApplication;
    +import org.apache.brooklyn.core.test.entity.TestEntity;
    +import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.AfterMethod;
    +import org.testng.annotations.BeforeMethod;
    +import org.testng.annotations.Test;
    +
    +import java.util.UUID;
    +
    +import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
    +import static org.apache.brooklyn.test.framework.SimpleCommand.DEFAULT_COMMAND;
    +import static org.apache.brooklyn.test.framework.SimpleCommandTest.*;
    +import static org.assertj.core.api.Assertions.assertThat;
    +
    +public class SimpleCommandImplIntegrationTest {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImplIntegrationTest.class);
    +
    +    private static final String UP = "up";
    +    private TestApplication app;
    +    private ManagementContext managementContext;
    +    private LocalhostMachineProvisioningLocation localhost;
    +    private String testId;
    +
    +
    +    @BeforeMethod
    +    public void setUp() {
    --- End diff --
    
    Recommend extending `BrooklynAppUnitTestSupport` instead - it gives you the setUp and tearDown methods.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#issuecomment-158002524
  
    looks real good.  quite a few pretty minor comments, worth addressing and should give you more familiarity with the codebase.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by ahgittin <gi...@git.apache.org>.
Github user ahgittin commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r45317688
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java ---
    @@ -0,0 +1,315 @@
    +/*
    + * 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.test.framework;
    +
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Splitter;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.TaskFactory;
    +import org.apache.brooklyn.core.effector.ssh.SshEffectorTasks;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.location.Machines;
    +import org.apache.brooklyn.location.ssh.SshMachineLocation;
    +import org.apache.brooklyn.test.Asserts;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.core.task.DynamicTasks;
    +import org.apache.brooklyn.util.core.task.ssh.SshTasks;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskWrapper;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.text.Identifiers;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.*;
    +
    +import static org.apache.brooklyn.core.entity.lifecycle.Lifecycle.*;
    +import static org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.setExpectedState;
    +import static org.apache.brooklyn.util.groovy.GroovyJavaMethods.truth;
    +import static org.apache.brooklyn.util.text.Strings.isBlank;
    +import static org.apache.brooklyn.util.text.Strings.isNonBlank;
    +import static org.apache.commons.collections.MapUtils.isEmpty;
    +
    +public class SimpleShellCommandTestImpl extends AbstractTest implements SimpleShellCommandTest {
    +
    +    public static final int SUCCESS = 0;
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleShellCommandTestImpl.class);
    +    private static final int A_LINE = 80;
    +    public static final String DEFAULT_NAME = "download.sh";
    +    private static final String CD = "cd";
    +
    +    @Override
    +    public void start(Collection<? extends Location> locations) {
    +        setExpectedState(this, STARTING);
    +        execute();
    +    }
    +
    +    @Override
    +    public void stop() {
    +        LOG.debug("{} Stopping simple command", this);
    +        setUpAndRunState(false, STOPPED);
    +    }
    +
    +    @Override
    +    public void restart() {
    +        LOG.debug("{} Restarting simple command", this);
    +        execute();
    +    }
    +
    +    private void setUpAndRunState(boolean up, Lifecycle status) {
    +        sensors().set(SERVICE_UP, up);
    +        setExpectedState(this, status);
    +    }
    +
    +    private static class Result {
    +        int exitCode;
    +        String stdout;
    +        String stderr;
    +        public Result(final ProcessTaskWrapper<Integer> job) {
    +            exitCode = job.get();
    +            stdout = job.getStdout().trim();
    +            stderr = job.getStderr().trim();
    +        }
    +        public int getExitCode() {
    +            return exitCode;
    +        }
    +        public String getStdout() {
    +            return stdout;
    +        }
    +        public String getStderr() {
    +            return stderr;
    +        }
    +    }
    +
    +    protected void handle(Result result) {
    +        LOG.debug("{}, Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
    +            this, result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
    +        });
    +        AssertionSupport support = new AssertionSupport();
    +        checkAssertions(support, exitCodeAssertions(), "exit code", result.getExitCode());
    +        checkAssertions(support, getConfig(ASSERT_OUT), "stdout", result.getStdout());
    +        checkAssertions(support, getConfig(ASSERT_ERR), "stderr", result.getStderr());
    +        support.validate();
    +    }
    +
    +    private String shorten(String text) {
    +        return Strings.maxlenWithEllipsis(text, A_LINE);
    +    }
    +
    +    public void execute() {
    +        try {
    +            SshMachineLocation machineLocation =
    +                Machines.findUniqueMachineLocation(resolveTarget().getLocations(), SshMachineLocation.class).get();
    +            executeCommand(machineLocation);
    +            setUpAndRunState(true, RUNNING);
    +        } catch (Throwable t) {
    +            setUpAndRunState(false, ON_FIRE);
    +            throw Exceptions.propagate(t);
    +        }
    +    }
    +
    +    private void executeCommand(SshMachineLocation machineLocation) {
    +
    +        Result result = null;
    +        String downloadUrl = getConfig(DOWNLOAD_URL);
    +        String command = getConfig(COMMAND);
    +
    +        String downloadName = DOWNLOAD_URL.getName();
    +        String commandName = COMMAND.getName();
    +
    +        if (!(isNonBlank(downloadUrl) ^ isNonBlank(command))) {
    +            throw illegal("Must specify exactly one of", downloadName, "and", commandName);
    +        }
    +
    +        if (isNonBlank(downloadUrl)) {
    +            String scriptDir = getConfig(SCRIPT_DIR);
    +            String scriptPath = calculateDestPath(downloadUrl, scriptDir);
    +            result = executeDownloadedScript(machineLocation, downloadUrl, scriptPath);
    +        }
    +
    +        if (isNonBlank(command)) {
    +            result = executeShellCommand(machineLocation, command);
    +        }
    +
    +        handle(result);
    +    }
    +
    +    private Result executeDownloadedScript(SshMachineLocation machineLocation, String url, String scriptPath) {
    +
    +        TaskFactory<?> install = SshTasks.installFromUrl(ImmutableMap.<String, Object>of(), machineLocation, url, scriptPath);
    +        DynamicTasks.queue(install);
    +        DynamicTasks.waitForLast();
    +
    +        List<String> commands = new ArrayList<>();
    +        commands.add("chmod u+x " + scriptPath);
    +        maybeCdToRunDir(commands);
    +        commands.add(scriptPath);
    +
    +        return runCommands(machineLocation, commands);
    +    }
    +
    +    private Result executeShellCommand(SshMachineLocation machineLocation, String command) {
    +
    +        List<String> commands = new ArrayList<>();
    +        maybeCdToRunDir(commands);
    +        commands.add(command);
    +
    +        return runCommands(machineLocation, commands);
    +    }
    +
    +    private void maybeCdToRunDir(List<String> commands) {
    +        String runDir = getConfig(RUN_DIR);
    +        if (!isBlank(runDir)) {
    +            commands.add(CD + " " + runDir);
    +        }
    +    }
    +
    +    private Result runCommands(SshMachineLocation machine, List<String> commands) {
    +        SshEffectorTasks.SshEffectorTaskFactory<Integer> etf = SshEffectorTasks.ssh(commands.toArray(new String[]{}))
    +            .machine(machine);
    +
    +        ProcessTaskWrapper<Integer> job = DynamicTasks.queue(etf);
    +        job.asTask().blockUntilEnded();
    +        return new Result(job);
    +    }
    +
    +
    +
    +    private IllegalArgumentException illegal(String message, String ...messages) {
    +        return new IllegalArgumentException(Joiner.on(' ').join(this.toString() + ":", message, messages));
    +    }
    +
    +    private String calculateDestPath(String url, String directory) {
    +        try {
    +            URL asUrl = new URL(url);
    +            Iterable<String> path = Splitter.on("/").split(asUrl.getPath());
    +            String scriptName = getLastPartOfPath(path, DEFAULT_NAME);
    +            return Joiner.on("/").join(directory, "test-" + Identifiers.makeRandomId(8), scriptName);
    +        } catch (MalformedURLException e) {
    +            throw illegal("Malformed URL:", url);
    +        }
    +    }
    +
    +    private static String getLastPartOfPath(Iterable<String> path, String defaultName) {
    +        MutableList<String> parts = MutableList.copyOf(path);
    +        Collections.reverse(parts);
    +        Iterator<String> it = parts.iterator();
    +        String scriptName = null;
    +
    +        // strip any trailing "/" parts of URL
    +        while (isBlank(scriptName) && it.hasNext()) {
    +            scriptName = it.next();
    +        }
    +        if (isBlank(scriptName)) {
    +            scriptName = defaultName;
    +        }
    +        return scriptName;
    +    }
    +    
    +    private <T> void checkAssertions(AssertionSupport support, Map<?, ?> assertions, String target, T actual) {
    --- End diff --
    
    can you use the shared assertions routines that have been written for the other test cases?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by aledsage <gi...@git.apache.org>.
Github user aledsage commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44860007
  
    --- Diff: usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleCommandSshDriver.java ---
    @@ -0,0 +1,192 @@
    +/*
    + * 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.test.framework;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.base.Splitter;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.ImmutableMap;
    +import org.apache.brooklyn.api.entity.EntityLocal;
    +import org.apache.brooklyn.api.location.Location;
    +import org.apache.brooklyn.api.mgmt.TaskAdaptable;
    +import org.apache.brooklyn.api.mgmt.TaskFactory;
    +import org.apache.brooklyn.core.location.Locations;
    +import org.apache.brooklyn.location.ssh.SshMachineLocation;
    +import org.apache.brooklyn.util.collections.MutableList;
    +import org.apache.brooklyn.util.core.ResourceUtils;
    +import org.apache.brooklyn.util.core.task.DynamicTasks;
    +import org.apache.brooklyn.util.core.task.ssh.SshTasks;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskFactory;
    +import org.apache.brooklyn.util.core.task.system.ProcessTaskWrapper;
    +import org.apache.brooklyn.util.exceptions.Exceptions;
    +import org.apache.brooklyn.util.exceptions.FatalRuntimeException;
    +import org.apache.brooklyn.util.guava.Maybe;
    +import org.apache.brooklyn.util.text.Strings;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.net.MalformedURLException;
    +import java.net.URL;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.Random;
    +
    +import static com.google.common.base.Preconditions.checkNotNull;
    +import static org.apache.brooklyn.util.text.Strings.isBlank;
    +
    +/**
    + * Driver for {@link SimpleCommand}.
    + */
    +public class SimpleCommandSshDriver implements SimpleCommandDriver {
    +
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandSshDriver.class);
    +    public static final String DEFAULT_NAME = "download.sh";
    +
    +    protected final EntityLocal entity;
    +    protected final ResourceUtils resource;
    +    protected final Location location;
    +
    +    public SimpleCommandSshDriver(EntityLocal entity, SshMachineLocation location) {
    +        LOG.debug("Constructing SSH driver for simple command for {} at {}", entity, location);
    +        this.entity = checkNotNull(entity, "entity");
    +        this.location = checkNotNull(location, "location");
    +        this.resource = ResourceUtils.create(entity);
    +    }
    +
    +    @Override
    +    public EntityLocal getEntity() {
    +        return entity;
    +    }
    +
    +    @Override
    +    public void start() {
    +        LOG.debug("Performing start in SSH driver for simple command");
    +        invoke();
    +    }
    +
    +    private void invoke() {
    +        SimpleCommand simpleCommand = (SimpleCommand) getEntity();
    +        simpleCommand.start(ImmutableList.of(location));
    +    }
    +
    +    @Override
    +    public void restart() {
    +        LOG.debug("Performing restart in SSH driver for simple command");
    +        invoke();
    --- End diff --
    
    Should this not call `simpleCommand.restart`, rather than `.start()`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-brooklyn pull request: SimpleCommand addition to brookly...

Posted by geomacy <gi...@git.apache.org>.
Github user geomacy commented on a diff in the pull request:

    https://github.com/apache/incubator-brooklyn/pull/1030#discussion_r44907814
  
    --- Diff: usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleCommandImplIntegrationTest.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * 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.test.framework;
    +
    +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.LocationSpec;
    +import org.apache.brooklyn.api.mgmt.ManagementContext;
    +import org.apache.brooklyn.core.entity.Entities;
    +import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
    +import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
    +import org.apache.brooklyn.core.test.entity.TestApplication;
    +import org.apache.brooklyn.core.test.entity.TestEntity;
    +import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.testng.annotations.AfterMethod;
    +import org.testng.annotations.BeforeMethod;
    +import org.testng.annotations.Test;
    +
    +import java.util.UUID;
    +
    +import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
    +import static org.apache.brooklyn.test.framework.SimpleCommand.DEFAULT_COMMAND;
    +import static org.apache.brooklyn.test.framework.SimpleCommandTest.*;
    +import static org.assertj.core.api.Assertions.assertThat;
    +
    +public class SimpleCommandImplIntegrationTest {
    +    private static final Logger LOG = LoggerFactory.getLogger(SimpleCommandImplIntegrationTest.class);
    +
    +    private static final String UP = "up";
    +    private TestApplication app;
    +    private ManagementContext managementContext;
    +    private LocalhostMachineProvisioningLocation localhost;
    +    private String testId;
    +
    +
    +    @BeforeMethod
    +    public void setUp() {
    +
    +        testId = UUID.randomUUID().toString();
    +
    +        app = TestApplication.Factory.newManagedInstanceForTests();
    +        managementContext = app.getManagementContext();
    +
    +        localhost = managementContext.getLocationManager()
    +            .createLocation(LocationSpec.create(LocalhostMachineProvisioningLocation.class)
    +                .configure("name", testId));
    +    }
    +
    +    @AfterMethod(alwaysRun = true)
    +    public void tearDown() throws Exception {
    +        if (app != null) Entities.destroyAll(app.getManagementContext());
    +    }
    +
    +    @Test(groups = "Integration")
    +    public void shouldInvokeCommand() {
    +        TestEntity testEntity = app.createAndManageChild(EntitySpec.create(TestEntity.class));
    +
    +        SimpleCommandTest uptime = app.createAndManageChild(EntitySpec.create(SimpleCommandTest.class)
    +            .configure(TARGET_ENTITY, testEntity)
    +            .configure(DEFAULT_COMMAND, "uptime")
    +            .configure(ASSERT_STATUS, ImmutableMap.of(EQUALS, 0))
    +            .configure(ASSERT_OUT, ImmutableMap.of(CONTAINS, UP)));
    +
    +        app.start(ImmutableList.of(localhost));
    --- End diff --
    
    It's using localhost.  Can you point me to an example integration test that does provisioning and I'll rework it accordingly?  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---