You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by al...@apache.org on 2015/11/30 23:05:24 UTC

[1/3] incubator-brooklyn git commit: Updates to Test framework assertions to support multiple assertion failures.

Repository: incubator-brooklyn
Updated Branches:
  refs/heads/master aeb56fdf1 -> d20e41d6d


Updates to Test framework assertions to support multiple assertion failures.

Merges the assertion code from SimpleShellCommand and TestFrameworkAssertions to give:

Updates test framework assertions so that multiple failures are described
in the assertion failure message, such as

    Assertions failed: exit code equals 1 stdout equals bogus stdout contains porkpie stderr

Supports generic 'actual' parameter, so we can avoid doing type coercions to String in
order to check assertions.

Adds "hasTruthValue" condition.


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

Branch: refs/heads/master
Commit: d3d120c5c0ed96231cbc7a4034ada8ae387cc9d1
Parents: 9fa0e3b
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Authored: Tue Nov 24 09:05:08 2015 +0000
Committer: Geoff Macartney <ge...@cloudsoftcorp.com>
Committed: Tue Nov 24 15:42:28 2015 +0000

----------------------------------------------------------------------
 .../test/framework/SimpleShellCommandTest.java  |  18 +-
 .../framework/SimpleShellCommandTestImpl.java   | 107 ++------
 .../test/framework/TestFrameworkAssertions.java | 253 +++++++++++--------
 .../test/framework/TestHttpCallImpl.java        |  34 ++-
 .../brooklyn/test/framework/TestSensorImpl.java |  15 +-
 .../SimpleShellCommandIntegrationTest.java      | 102 +++++++-
 .../framework/TestFrameworkAssertionsTest.java  |  73 ++++--
 .../test/framework/TestHttpCallTest.java        |   9 +-
 .../brooklyn/test/framework/TestSensorTest.java |  21 +-
 .../testhttpcall-examples.yml                   |  11 +-
 10 files changed, 376 insertions(+), 267 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java
index 0b71f2f..de1f176 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java
@@ -19,6 +19,7 @@
 package org.apache.brooklyn.test.framework;
 
 import com.google.common.collect.Maps;
+import com.google.common.reflect.TypeToken;
 import org.apache.brooklyn.api.entity.ImplementedBy;
 import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.core.config.ConfigKeys;
@@ -26,6 +27,8 @@ import org.apache.brooklyn.core.sensor.AttributeSensorAndConfigKey;
 import org.apache.brooklyn.entity.software.base.SoftwareProcess;
 import org.apache.brooklyn.util.core.flags.SetFromFlag;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 
 import static org.apache.brooklyn.core.config.ConfigKeys.newConfigKey;
@@ -87,21 +90,24 @@ public interface SimpleShellCommandTest extends BaseTest {
      * If not explicitly configured, the default assertion is a non-zero exit code.
      */
     @SetFromFlag("assertStatus")
-    ConfigKey<Map> ASSERT_STATUS = ConfigKeys.newConfigKey(Map.class, "assert.status",
-            "Assertions on command exit code", Maps.newLinkedHashMap());
+    ConfigKey<List<Map<String, Object>>> ASSERT_STATUS = ConfigKeys.newConfigKey(
+        new TypeToken<List<Map<String, Object>>>() {},
+        "assert.status", "Assertions on command exit code", new ArrayList<Map<String, Object>>());
 
     /**
      * Assertions on the standard output of the command as a String.
      */
     @SetFromFlag("assertOut")
-    ConfigKey<Map> ASSERT_OUT = ConfigKeys.newConfigKey(Map.class, "assert.out",
-            "Assertions on command standard output", Maps.newLinkedHashMap());
+    ConfigKey<List<Map<String, Object>>> ASSERT_OUT = ConfigKeys.newConfigKey(
+        new TypeToken<List<Map<String, Object>>>() {},
+        "assert.out", "Assertions on command standard output", new ArrayList<Map<String, Object>>());
 
     /**
      * Assertions on the standard error of the command as a String.
      */
     @SetFromFlag("assertErr")
-    ConfigKey<Map> ASSERT_ERR = ConfigKeys.newConfigKey(Map.class, "assert.err",
-            "Assertions on command standard error", Maps.newLinkedHashMap());
+    ConfigKey<List<Map<String, Object>>> ASSERT_ERR = ConfigKeys.newConfigKey(
+        new TypeToken<List<Map<String, Object>>>() {},
+        "assert.err", "Assertions on command standard error", new ArrayList<Map<String, Object>>());
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java
index b8cceb8..5a709e8 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java
@@ -21,6 +21,7 @@ package org.apache.brooklyn.test.framework;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Splitter;
+import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableMap;
 import org.apache.brooklyn.api.location.Location;
 import org.apache.brooklyn.api.mgmt.TaskFactory;
@@ -28,7 +29,7 @@ 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.test.framework.TestFrameworkAssertions.AssertionSupport;
 import org.apache.brooklyn.util.collections.MutableList;
 import org.apache.brooklyn.util.core.task.DynamicTasks;
 import org.apache.brooklyn.util.core.task.ssh.SshTasks;
@@ -36,6 +37,7 @@ 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.apache.brooklyn.util.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,10 +47,8 @@ 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;
 
 // TODO assertions below should use TestFrameworkAssertions but that class needs to be improved to give better error messages
 public class SimpleShellCommandTestImpl extends AbstractTest implements SimpleShellCommandTest {
@@ -107,13 +107,23 @@ public class SimpleShellCommandTestImpl extends AbstractTest implements SimpleSh
         LOG.debug("{}, Result is {}\nwith output [\n{}\n] and error [\n{}\n]", new Object[] {
             this, result.getExitCode(), shorten(result.getStdout()), shorten(result.getStderr())
         });
+        ImmutableMap<String, Duration> flags = ImmutableMap.of("timeout", getConfig(TIMEOUT));
         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());
+        TestFrameworkAssertions.checkAssertions(support, flags, exitCodeAssertions(), "exit code", supply(result.getExitCode()));
+        TestFrameworkAssertions.checkAssertions(support, flags, getConfig(ASSERT_OUT), "stdout", supply(result.getStdout()));
+        TestFrameworkAssertions.checkAssertions(support, flags, getConfig(ASSERT_ERR), "stderr", supply(result.getStderr()));
         support.validate();
     }
 
+    private static <T> Supplier<T> supply(final T t) {
+        return new Supplier<T>() {
+            @Override
+            public T get() {
+                return t;
+            }
+        };
+    }
+
     private String shorten(String text) {
         return Strings.maxlenWithEllipsis(text, A_LINE);
     }
@@ -228,89 +238,14 @@ public class SimpleShellCommandTestImpl extends AbstractTest implements SimpleSh
         return scriptName;
     }
     
-    private <T> void checkAssertions(AssertionSupport support, Map<?, ?> assertions, String target, T actual) {
-        if (null == assertions) {
-            return;
-        }
-        if (null == actual) {
-            support.fail(target, "no actual value", "");
-            return;
-        }
-        for (Map.Entry<?, ?> assertion : assertions.entrySet()) {
-            String condition = assertion.getKey().toString();
-            Object expected = assertion.getValue();
-            switch (condition) {
-                case EQUALS :
-                    if (!actual.equals(expected)) {
-                        support.fail(target, EQUALS, expected);
-                    }
-                    break;
-                case CONTAINS :
-                    if (!actual.toString().contains(expected.toString())) {
-                        support.fail(target, CONTAINS, expected);
-                    }
-                    break;
-                case IS_EMPTY:
-                    if (!actual.toString().isEmpty() && truth(expected)) {
-                        support.fail(target, IS_EMPTY, expected);
-                    }
-                    break;
-                case MATCHES :
-                    if (!actual.toString().matches(expected.toString())) {
-                        support.fail(target, MATCHES, expected);
-                    }
-                    break;
-                default:
-                    support.fail(target, "unknown condition", condition);
-            }
-        }
-    }
 
-    private Map<?, ?> exitCodeAssertions() {
-        Map<?, ?> assertStatus = getConfig(ASSERT_STATUS);
-        if (isEmpty(assertStatus)) {
-            assertStatus = ImmutableMap.of(EQUALS, SUCCESS);
+    private List<Map<String, Object>> exitCodeAssertions() {
+        List<Map<String, Object>> assertStatus = getConfig(ASSERT_STATUS);
+        if (assertStatus.isEmpty()) {
+            Map<String, Object> shouldSucceed = ImmutableMap.<String,Object>of(EQUALS, SUCCESS);
+            assertStatus.add(shouldSucceed);
         }
         return assertStatus;
     }
 
-    public static class FailedAssertion {
-        String target;
-        String assertion;
-        String expected;
-
-        public FailedAssertion(String target, String assertion, String expected) {
-            this.target = target;
-            this.assertion = assertion;
-            this.expected = expected;
-        }
-        public String description() {
-            return Joiner.on(' ').join(target, assertion, expected);
-        }
-    }
-
-    /**
-     * A convenience to collect and validate any assertion failures.
-     */
-    public static class AssertionSupport {
-        private List<FailedAssertion> failures = new ArrayList<>();
-
-        public void fail(String target, String assertion, Object expected) {
-            failures.add(new FailedAssertion(target, assertion, expected.toString()));
-        }
-
-        /**
-         * @throws AssertionError if any failures were collected.
-         */
-        public void validate() {
-            if (0 < failures.size()) {
-                StringBuilder summary = new StringBuilder();
-                summary.append("Assertion Failures: \n");
-                for (FailedAssertion fail : failures) {
-                    summary.append(fail.description()).append("\n");
-                }
-                Asserts.fail(summary.toString());
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java
index 9851ca2..bd955fa 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java
@@ -18,20 +18,16 @@
  */
 package org.apache.brooklyn.test.framework;
 
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
+import com.google.common.base.Joiner;
 import com.google.common.base.Supplier;
 import org.apache.brooklyn.test.Asserts;
-import org.apache.brooklyn.util.core.flags.TypeCoercions;
-import org.apache.brooklyn.util.guava.Maybe;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.brooklyn.util.exceptions.CompoundRuntimeException;
+import org.apache.brooklyn.util.text.Strings;
 
-import javax.annotation.Nullable;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.regex.Pattern;
+
 
 /**
  * Utility class to evaluate test-framework assertions
@@ -39,122 +35,179 @@ import java.util.regex.Pattern;
  * @author m4rkmckenna on 11/11/2015.
  */
 public class TestFrameworkAssertions {
-    private static final Logger LOG = LoggerFactory.getLogger(TestFrameworkAssertions.class);
+
+    public static final String IS_NULL = "isNull";
+    public static final String NOT_NULL = "notNull";
+    public static final String IS_EQUAL_TO = "isEqualTo";
+    public static final String EQUAL_TO = "equalTo";
+    public static final String EQUALS = "equals";
+    public static final String MATCHES = "matches";
+    public static final String CONTAINS = "contains";
+    public static final String IS_EMPTY = "isEmpty";
+    public static final String NOT_EMPTY = "notEmpty";
+    public static final String HAS_TRUTH_VALUE = "hasTruthValue";
+    public static final String UNKNOWN_CONDITION = "unknown condition";
+
 
     private TestFrameworkAssertions() {
     }
 
-    /**
-     * Evaluates all assertions against dataSupplier
-     *
-     * @param dataSupplier
-     * @param flags
-     * @param assertions
-     */
-    public static void checkAssertions(final Supplier<String> dataSupplier, final Map flags, final List<Map<String, Object>> assertions) {
-        //Iterate through assert array
-        for (final Map<String, Object> assertionsMap : assertions) {
-            checkAssertions(dataSupplier, flags, assertionsMap);
+    public static <T> void checkAssertions(Map<String,?> flags,
+                                           Map<String, Object> assertions,
+                                           String target,
+                                           final Supplier<T> actualSupplier) {
+
+        AssertionSupport support = new AssertionSupport();
+        checkAssertions(support, flags, assertions, target, actualSupplier);
+        support.validate();
+    }
+
+
+    public static <T> void checkAssertions(Map<String,?> flags,
+                                           List<Map<String, Object>> assertions,
+                                           String target,
+                                           final Supplier<T> actualSupplier) {
+
+        AssertionSupport support = new AssertionSupport();
+        for (Map<String, Object> assertionMap : assertions) {
+            checkAssertions(support, flags, assertionMap, target, actualSupplier);
         }
+        support.validate();
     }
 
-    /**
-     * Evaluates all assertions against dataSupplier
-     *
-     * @param dataSupplier
-     * @param flags
-     * @param assertionsMap
-     */
-    public static void checkAssertions(final Supplier<String> dataSupplier, final Map flags, final Map<String, Object> assertionsMap) {
-        for (final Map.Entry<String, Object> assertion : assertionsMap.entrySet()) {
-            final Maybe<Predicate<String>> optionalPredicate = getPredicate(assertion.getKey(), assertion.getValue());
-            Asserts.succeedsEventually(flags, new PredicateChecker(dataSupplier, optionalPredicate.get()));
+    public static <T> void checkAssertions(final AssertionSupport support,
+                                           Map<String,?> flags,
+                                           final List<Map<String, Object>> assertions,
+                                           final String target,
+                                           final Supplier<T> actualSupplier) {
+
+        for (Map<String, Object> assertionMap : assertions) {
+            checkAssertions(support, flags, assertionMap, target, actualSupplier);
         }
     }
 
-    /**
-     * Returns the predicate associated with the predicateKey if one exists
-     *
-     * @param predicateKey
-     * @param predicateTarget
-     * @return {@link Maybe} of {@Link Predicate}
-     */
-    public static Maybe<Predicate<String>> getPredicate(final String predicateKey, final Object predicateTarget) {
-        if (StringUtils.equalsIgnoreCase("isNull", predicateKey)) {
-            return Maybe.of(Predicates.<String>isNull());
-        } else if (StringUtils.equalsIgnoreCase("notNull", predicateKey)) {
-            return Maybe.of(Predicates.<String>notNull());
-        } else if (StringUtils.equalsIgnoreCase("isEqualTo", predicateKey)
-                || StringUtils.equalsIgnoreCase("equalTo", predicateKey)
-                || StringUtils.equalsIgnoreCase("equals", predicateKey)) {
-            return Maybe.of(Predicates.equalTo(TypeCoercions.coerce(predicateTarget.toString(), String.class)));
-        } else if (StringUtils.equalsIgnoreCase("matches", predicateKey)) {
-            return Maybe.of(buildMatchesPredicate(TypeCoercions.coerce(predicateTarget, String.class)));
-        } else if (StringUtils.equalsIgnoreCase("contains", predicateKey)) {
-            return Maybe.of(buildContainsPredicate(TypeCoercions.coerce(predicateTarget, String.class)));
+    public static <T> void checkAssertions(final AssertionSupport support,
+                                           Map<String,?> flags,
+                                           final Map<String, Object> assertions,
+                                           final String target,
+                                           final Supplier<T> actualSupplier) {
+
+        if (null == assertions) {
+            return;
+        }
+        try {
+            Asserts.succeedsEventually(flags, new Runnable() {
+                @Override
+                public void run() {
+                    T actual = actualSupplier.get();
+                    checkActualAgainstAssertions(support, assertions, target, actual);
+                }
+            });
+        } catch (Throwable t) {
+            support.fail(t);
         }
-        return Maybe.absent(String.format("No predicate found with signature [%s]", predicateKey));
     }
 
-    /**
-     * Builds a predicate that checks if a string contains the supplied value
-     *
-     * @param predicateTarget
-     * @return {@link Predicate}
-     */
-    private static Predicate<String> buildContainsPredicate(final String predicateTarget) {
-        return new Predicate<String>() {
+    private static <T> void checkActualAgainstAssertions(AssertionSupport support, Map<String, Object> assertions,
+                                                         String target, T actual) {
+        for (Map.Entry<String, Object> assertion : assertions.entrySet()) {
+            String condition = assertion.getKey().toString();
+            Object expected = assertion.getValue();
+            switch (condition) {
 
-            @Override
-            public boolean apply(@Nullable final String input) {
-                return StringUtils.contains(input, predicateTarget);
-            }
+                case IS_EQUAL_TO :
+                case EQUAL_TO :
+                case EQUALS :
+                    if (null == actual || !actual.equals(expected)) {
+                        support.fail(target, EQUALS, expected);
+                    }
+                    break;
 
-            @Override
-            public String toString() {
-                return String.format("TestFrameworkAssertions.contains(%s)", predicateTarget);
-            }
-        };
-    }
+                case IS_NULL :
+                    if (isTrue(expected) != (null == actual)) {
+                        support.fail(target, IS_NULL, expected);
+                    }
+                    break;
 
-    /**
-     * Builds a predicate that checks if a string matches the supplied pattern
-     *
-     * @param predicateTarget The pattern to check
-     * @return {@link Predicate}
-     */
-    private static Predicate<String> buildMatchesPredicate(final String predicateTarget) {
-        final Pattern pattern = Pattern.compile(predicateTarget);
-        return new Predicate<String>() {
-            public boolean apply(final String input) {
-                return (input != null) && pattern.matcher(input.toString()).matches();
-            }
+                case NOT_NULL :
+                    if (isTrue(expected) != (null != actual)) {
+                        support.fail(target, NOT_NULL, expected);
+                    }
+                    break;
+
+                case CONTAINS :
+                    if (null == actual || !actual.toString().contains(expected.toString())) {
+                        support.fail(target, CONTAINS, expected);
+                    }
+                    break;
+
+                case IS_EMPTY :
+                    if (isTrue(expected) != (null == actual || Strings.isEmpty(actual.toString()))) {
+                        support.fail(target, IS_EMPTY, expected);
+                    }
+                    break;
+
+                case NOT_EMPTY :
+                    if (isTrue(expected) != ((null != actual && Strings.isNonEmpty(actual.toString())))) {
+                        support.fail(target, NOT_EMPTY, expected);
+                    }
+                    break;
+
+                case MATCHES :
+                    if (null == actual || !actual.toString().matches(expected.toString())) {
+                        support.fail(target, MATCHES, expected);
+                    }
+                    break;
 
-            @Override
-            public String toString() {
-                return String.format("TestFrameworkAssertions.matches(%s)", predicateTarget);
+                case HAS_TRUTH_VALUE :
+                    if (isTrue(expected) != isTrue(actual)) {
+                        support.fail(target, HAS_TRUTH_VALUE, expected);
+                    }
+                    break;
+
+                default:
+                    support.fail(target, UNKNOWN_CONDITION, condition);
             }
-        };
+        }
+    }
+
+    private static boolean isTrue(Object object) {
+        return null != object && Boolean.valueOf(object.toString());
     }
 
     /**
-     * Runnable that will be passed to {@link Asserts#succeedsEventually}
+     * A convenience to collect multiple assertion failures.
      */
-    private static class PredicateChecker implements Runnable {
-        private final Supplier<String> dataSupplier;
-        private final Predicate<String> predicate;
+    public static class AssertionSupport {
+        private List<AssertionError> failures = new ArrayList<>();
 
-        public PredicateChecker(final Supplier<String> dataSupplier, final Predicate<String> predicate) {
-            this.dataSupplier = dataSupplier;
-            this.predicate = predicate;
+        public void fail(String target, String assertion, Object expected) {
+            failures.add(new AssertionError(Joiner.on(' ').join(
+                null != target ? target : "null",
+                null != assertion ? assertion : "null",
+                null != expected ? expected : "null")));
         }
 
-        @Override
-        public void run() {
-            final String value = dataSupplier.get();
-            LOG.debug("Evaluating predicate [{}] with value [{}]", predicate.toString(), value);
-            Asserts.assertEquals(predicate.apply(value), true);
-            LOG.debug("Evaluation of predicate [{}] with value [{}] ... PASSED", predicate.toString(), value);
+        public void fail(Throwable throwable) {
+            failures.add(new AssertionError(throwable.getMessage(), throwable));
+        }
+
+        /**
+         * @throws AssertionError if any failures were collected.
+         */
+        public void validate() {
+            if (0 < failures.size()) {
+
+                if (1 == failures.size()) {
+                    throw failures.get(0);
+                }
+
+                StringBuilder builder = new StringBuilder();
+                for (AssertionError assertionError : failures) {
+                    builder.append(assertionError.getMessage()).append("\n");
+                }
+                throw new AssertionError("Assertions failed:\n" + builder, new CompoundRuntimeException("Assertions", failures));
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java
index df1104d..92f2885 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java
@@ -27,6 +27,7 @@ import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
 import org.apache.brooklyn.util.exceptions.Exceptions;
 import org.apache.brooklyn.util.http.HttpTool;
 import org.apache.brooklyn.util.time.Duration;
+import org.apache.http.HttpStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,12 +53,13 @@ public class TestHttpCallImpl extends AbstractTest implements TestHttpCall {
         final String url = getConfig(TARGET_URL);
         final List<Map<String, Object>> assertions = getConfig(ASSERTIONS);
         final Duration timeout = getConfig(TIMEOUT);
-        final HttpAssertionTarget httpAssertionTarget = getConfig(ASSERTION_TARGET);
+        final HttpAssertionTarget target = getConfig(ASSERTION_TARGET);
+
         try {
-            TestFrameworkAssertions.checkAssertions(buildDataSupplier(httpAssertionTarget, url),
-                    ImmutableMap.of("timeout", timeout), assertions);
+            doRequestAndCheckAssertions(ImmutableMap.of("timeout", timeout), assertions, target, url);
             sensors().set(SERVICE_UP, true);
             ServiceStateLogic.setExpectedState(this, Lifecycle.RUNNING);
+
         } catch (Throwable t) {
             LOG.info("{} Url [{}] test failed", this, url);
             sensors().set(SERVICE_UP, false);
@@ -66,31 +68,35 @@ public class TestHttpCallImpl extends AbstractTest implements TestHttpCall {
         }
     }
 
-    private Supplier<String> buildDataSupplier(final HttpAssertionTarget httpAssertionTarget, final String url) {
-
-        switch (httpAssertionTarget) {
+    private void doRequestAndCheckAssertions(Map<String, Duration> flags, List<Map<String, Object>> assertions,
+                                             HttpAssertionTarget target, final String url) {
+        switch (target) {
             case body:
-                return new Supplier<String>() {
+                Supplier<String> getBody = new Supplier<String>() {
                     @Override
                     public String get() {
                         return HttpTool.getContent(url);
                     }
                 };
+                TestFrameworkAssertions.checkAssertions(flags, assertions, target.toString(), getBody);
+                break;
             case status:
-                return new Supplier<String>() {
+                Supplier<Integer> getStatusCode = new Supplier<Integer>() {
                     @Override
-                    public String get() {
+                    public Integer get() {
                         try {
-                            return String.valueOf(HttpTool.getHttpStatusCode(url));
+                            return HttpTool.getHttpStatusCode(url);
                         } catch (Exception e) {
-                            LOG.error("HTTP call to [{}] failed due to [{}] ... returning Status code [0 - Unreachable]", url, e.getMessage());
-                            return "0";
+                            LOG.error("HTTP call to [{}] failed due to [{}] ... returning Status code [500]",
+                                url, e.getMessage());
+                            return HttpStatus.SC_INTERNAL_SERVER_ERROR;
                         }
-
                     }
                 };
+                TestFrameworkAssertions.checkAssertions(flags, assertions, target.toString(), getStatusCode);
+                break;
             default:
-                throw new RuntimeException("Unable to build a data supplier to target assertion [" + httpAssertionTarget + "]");
+                throw new RuntimeException("Unexpected assertion target (" + target + ")");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java
index 3dc6cff..f5ed913 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java
@@ -28,7 +28,6 @@ import org.apache.brooklyn.api.location.Location;
 import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
 import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
 import org.apache.brooklyn.core.sensor.Sensors;
-import org.apache.brooklyn.util.core.flags.ClassCoercionException;
 import org.apache.brooklyn.util.core.flags.TypeCoercions;
 import org.apache.brooklyn.util.exceptions.Exceptions;
 import org.apache.brooklyn.util.time.Duration;
@@ -59,18 +58,14 @@ public class TestSensorImpl extends AbstractTest implements TestSensor {
         final Duration timeout = getConfig(TIMEOUT);
         final List<Map<String, Object>> assertions = getConfig(ASSERTIONS);
         try {
-            TestFrameworkAssertions.checkAssertions(new Supplier<String>() {
+            TestFrameworkAssertions.checkAssertions(ImmutableMap.of("timeout", timeout), assertions, sensor,
+                new Supplier<Object>() {
                 @Override
-                public String get() {
+                public Object get() {
                     final Object sensorValue = target.sensors().get(Sensors.newSensor(Object.class, sensor));
-                    try {
-                        return TypeCoercions.coerce(sensorValue, String.class);
-                    } catch (ClassCoercionException cce) {
-                        LOG.debug("Could mot coerce sensor value to a string ... invoking toString() instead");
-                        return (sensorValue != null) ? sensorValue.toString() : null;
-                    }
+                    return sensorValue;
                 }
-            }, ImmutableMap.of("timeout", timeout), assertions);
+            });
 
             sensors().set(SERVICE_UP, true);
             ServiceStateLogic.setExpectedState(this, Lifecycle.RUNNING);

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java
index c6b5b87..9e6a27a 100644
--- a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java
+++ b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java
@@ -28,7 +28,9 @@ import org.apache.brooklyn.core.test.entity.TestEntity;
 import org.apache.brooklyn.location.localhost.LocalhostMachineProvisioningLocation;
 import org.apache.brooklyn.location.ssh.SshMachineLocation;
 import org.apache.brooklyn.test.Asserts;
+import org.apache.brooklyn.util.collections.MutableMap;
 import org.apache.brooklyn.util.exceptions.Exceptions;
+import org.apache.brooklyn.util.text.Identifiers;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
@@ -36,6 +38,9 @@ import org.testng.annotations.Test;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 
 import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
 import static org.apache.brooklyn.test.framework.SimpleShellCommandTest.*;
@@ -96,15 +101,70 @@ public class SimpleShellCommandIntegrationTest extends BrooklynAppUnitTestSuppor
         }
     }
 
+    private List<Map<String, Object>> makeAssertions(Map<String, Object> ...maps) {
+        ArrayList<Map<String, Object>> assertions = new ArrayList<>();
+        for (Map<String, Object> map : maps) {
+            assertions.add(map);
+        }
+        return assertions;
+    }
+
+
+
+    @Test(groups = "Integration")
+    public void shouldSucceedUsingSuccessfulExitAsDefaultCondition() {
+        TestEntity testEntity = app.createAndManageChild(EntitySpec.create(TestEntity.class).location(machine));
+
+        SimpleShellCommandTest uptime = app.createAndManageChild(EntitySpec.create(SimpleShellCommandTest.class)
+            .configure(TARGET_ENTITY, testEntity)
+            .configure(COMMAND, "uptime"));
+
+        app.start(ImmutableList.of(loc));
+
+        assertThat(uptime.sensors().get(SERVICE_UP)).isTrue()
+            .withFailMessage("Service should be up");
+        assertThat(ServiceStateLogic.getExpectedState(uptime)).isEqualTo(Lifecycle.RUNNING)
+            .withFailMessage("Service should be marked running");
+    }
+
+
+    @Test(groups = "Integration")
+    public void shouldFailUsingSuccessfulExitAsDefaultCondition() {
+        TestEntity testEntity = app.createAndManageChild(EntitySpec.create(TestEntity.class).location(machine));
+
+        SimpleShellCommandTest uptime = app.createAndManageChild(EntitySpec.create(SimpleShellCommandTest.class)
+            .configure(TARGET_ENTITY, testEntity)
+            .configure(COMMAND, "ls /tmp/bogus-" + Identifiers.randomLong()));
+
+        try {
+            app.start(ImmutableList.of(loc));
+        } catch (Throwable t) {
+            Asserts.expectedFailureContains(t, "exit code equals 0");
+        }
+
+        assertThat(uptime.sensors().get(SERVICE_UP)).isFalse()
+            .withFailMessage("Service should be down");
+        assertThat(ServiceStateLogic.getExpectedState(uptime)).isEqualTo(Lifecycle.ON_FIRE)
+            .withFailMessage("Service should be marked on fire");
+    }
+
+
+
     @Test(groups = "Integration")
     public void shouldInvokeCommand() {
         TestEntity testEntity = app.createAndManageChild(EntitySpec.create(TestEntity.class).location(machine));
 
+        Map<String, Object> equalsZero = MutableMap.of();
+        equalsZero.put(EQUALS, 0);
+
+        Map<String, Object> containsUp = MutableMap.of();
+        containsUp.put(CONTAINS, UP);
+
         SimpleShellCommandTest uptime = app.createAndManageChild(EntitySpec.create(SimpleShellCommandTest.class)
             .configure(TARGET_ENTITY, testEntity)
             .configure(COMMAND, "uptime")
-            .configure(ASSERT_STATUS, ImmutableMap.of(EQUALS, 0))
-            .configure(ASSERT_OUT, ImmutableMap.of(CONTAINS, UP)));
+            .configure(ASSERT_STATUS, makeAssertions(equalsZero))
+            .configure(ASSERT_OUT, makeAssertions(containsUp)));
 
         app.start(ImmutableList.of(loc));
 
@@ -116,18 +176,24 @@ public class SimpleShellCommandIntegrationTest extends BrooklynAppUnitTestSuppor
     }
 
     @Test(groups = "Integration")
-    public void shouldNotBeUpIfAssertionFails() {
+    public void shouldNotBeUpIfAssertionsFail() {
         TestEntity testEntity = app.createAndManageChild(EntitySpec.create(TestEntity.class).location(machine));
 
+        Map<String, Object> equalsOne = MutableMap.of();
+        equalsOne.put(EQUALS, 1);
+
+        Map<String, Object> equals255 = MutableMap.of();
+        equals255.put(EQUALS, 255);
+
         SimpleShellCommandTest uptime = app.createAndManageChild(EntitySpec.create(SimpleShellCommandTest.class)
             .configure(TARGET_ENTITY, testEntity)
             .configure(COMMAND, "uptime")
-            .configure(ASSERT_STATUS, ImmutableMap.of(EQUALS, 1)));
+            .configure(ASSERT_STATUS, makeAssertions(equalsOne, equals255)));
 
         try {
             app.start(ImmutableList.of(loc));
         } catch (Exception e) {
-            assertThat(e.getCause().getMessage().contains("exit code equals 1"));
+            Asserts.expectedFailureContains(e, "exit code equals 1", "exit code equals 255");
         }
 
         assertThat(ServiceStateLogic.getExpectedState(uptime)).isEqualTo(Lifecycle.ON_FIRE)
@@ -143,11 +209,18 @@ public class SimpleShellCommandIntegrationTest extends BrooklynAppUnitTestSuppor
         Path testScript = createTempScript("script", "echo " + text);
 
         try {
+
+            Map<String, Object> equalsZero = MutableMap.of();
+            equalsZero.put(EQUALS, 0);
+
+            Map<String, Object> containsText = MutableMap.of();
+            containsText.put(CONTAINS, text);
+
             SimpleShellCommandTest uptime = app.createAndManageChild(EntitySpec.create(SimpleShellCommandTest.class)
                 .configure(TARGET_ENTITY, testEntity)
                 .configure(DOWNLOAD_URL, "file:" + testScript)
-                .configure(ASSERT_STATUS, ImmutableMap.of(EQUALS, 0))
-                .configure(ASSERT_OUT, ImmutableMap.of(CONTAINS, text)));
+                .configure(ASSERT_STATUS, makeAssertions(equalsZero))
+                .configure(ASSERT_OUT, makeAssertions(containsText)));
 
             app.start(ImmutableList.of(loc));
 
@@ -168,20 +241,27 @@ public class SimpleShellCommandIntegrationTest extends BrooklynAppUnitTestSuppor
         Path pwdPath = createTempScript("pwd", "pwd");
 
         try {
+
+            Map<String, Object> equalsZero = MutableMap.of();
+            equalsZero.put(EQUALS, 0);
+
+            Map<String, Object> containsTmp = MutableMap.of();
+            containsTmp.put(CONTAINS, "/tmp");
+
             SimpleShellCommandTest pwd = app.createAndManageChild(EntitySpec.create(SimpleShellCommandTest.class)
                 .configure(TARGET_ENTITY, testEntity)
                 .configure(DOWNLOAD_URL, "file:" + pwdPath)
                 .configure(RUN_DIR, "/tmp")
-                .configure(ASSERT_STATUS, ImmutableMap.of(EQUALS, 0))
-                .configure(ASSERT_OUT, ImmutableMap.of(CONTAINS, "/tmp")));
+                .configure(ASSERT_STATUS, makeAssertions(equalsZero))
+                .configure(ASSERT_OUT, makeAssertions(containsTmp)));
 
 
             SimpleShellCommandTest alsoPwd = app.createAndManageChild(EntitySpec.create(SimpleShellCommandTest.class)
                 .configure(TARGET_ENTITY, testEntity)
                 .configure(COMMAND, "pwd")
                 .configure(RUN_DIR, "/tmp")
-                .configure(ASSERT_STATUS, ImmutableMap.of(EQUALS, 0))
-                .configure(ASSERT_OUT, ImmutableMap.of(CONTAINS, "/tmp")));
+                .configure(ASSERT_STATUS, makeAssertions(equalsZero))
+                .configure(ASSERT_OUT, makeAssertions(containsTmp)));
 
             app.start(ImmutableList.of(loc));
 

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java
index eb39735..60d452c 100644
--- a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java
+++ b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java
@@ -19,6 +19,7 @@
 package org.apache.brooklyn.test.framework;
 
 import com.google.common.base.Supplier;
+import org.apache.brooklyn.test.Asserts;
 import org.apache.brooklyn.util.text.Identifiers;
 import org.apache.brooklyn.util.time.Duration;
 import org.python.google.common.collect.ImmutableMap;
@@ -51,12 +52,22 @@ public class TestFrameworkAssertionsTest {
     public Object[][] positiveTestsDP() {
         return new Object[][]{
                 {"some-sensor-value", Arrays.asList(ImmutableMap.of("isEqualTo", "some-sensor-value"))},
+                {"some-sensor-value", Arrays.asList(ImmutableMap.of("equalTo", "some-sensor-value"))},
                 {"some-sensor-value", Arrays.asList(ImmutableMap.of("equals", "some-sensor-value"))},
                 {"some-regex-value-to-match", Arrays.asList(ImmutableMap.of("matches", "some.*match", "isEqualTo", "some-regex-value-to-match"))},
-                {null, Arrays.asList(ImmutableMap.of("isNUll", ""))},
-                {"some-non-null-value", Arrays.asList(ImmutableMap.of("notNull", ""))},
+                {null, Arrays.asList(ImmutableMap.of("isNull", Boolean.TRUE))},
+                {"some-non-null-value", Arrays.asList(ImmutableMap.of("isNull", Boolean.FALSE))},
+                {null, Arrays.asList(ImmutableMap.of("notNull", Boolean.FALSE))},
+                {"some-non-null-value", Arrays.asList(ImmutableMap.of("notNull", Boolean.TRUE))},
                 {"<html><body><h1>Im a H1 tag!</h1></body></html>", Arrays.asList(ImmutableMap.of("contains", "Im a H1 tag!"))},
-                {"{\"a\":\"b\",\"c\":\"d\",\"e\":123,\"g\":false}", Arrays.asList(ImmutableMap.of("contains", "false"))}
+                {"{\"a\":\"b\",\"c\":\"d\",\"e\":123,\"g\":false}", Arrays.asList(ImmutableMap.of("contains", "false"))},
+                {"", Arrays.asList(ImmutableMap.of("isEmpty", Boolean.TRUE))},
+                {"some-non-null-value", Arrays.asList(ImmutableMap.of("isEmpty", Boolean.FALSE))},
+                {null, Arrays.asList(ImmutableMap.of("notEmpty", Boolean.FALSE))},
+                {"some-non-null-value", Arrays.asList(ImmutableMap.of("notEmpty", Boolean.TRUE))},
+                {"true", Arrays.asList(ImmutableMap.of("hasTruthValue", Boolean.TRUE))},
+                {"false", Arrays.asList(ImmutableMap.of("hasTruthValue", Boolean.FALSE))},
+                {"some-non-null-value", Arrays.asList(ImmutableMap.of("hasTruthValue", Boolean.FALSE))},
         };
     }
 
@@ -69,24 +80,40 @@ public class TestFrameworkAssertionsTest {
                 return data;
             }
         };
-        TestFrameworkAssertions.checkAssertions(supplier, ImmutableMap.of("timeout", new Duration(2L, TimeUnit.SECONDS)), assertions);
+        TestFrameworkAssertions.checkAssertions(ImmutableMap.of("timeout", new Duration(2L, TimeUnit.SECONDS)), assertions, data, supplier);
     }
 
     @DataProvider
     public Object[][] negativeTestsDP() {
+        String arbitrary = Identifiers.makeRandomId(8);
         return new Object[][]{
-                {"some-sensor-value", Arrays.asList(ImmutableMap.of("isEqualTo", Identifiers.makeRandomId(8)))},
-                {"some-sensor-value", Arrays.asList(ImmutableMap.of("equals", Identifiers.makeRandomId(8)))},
-                {"some-regex-value-to-match", Arrays.asList(ImmutableMap.of("matches", "some.*not-match", "isEqualTo", "oink"))},
-                {null, Arrays.asList(ImmutableMap.of("notNull", ""))},
-                {"some-non-null-value", Arrays.asList(ImmutableMap.of("isNull", ""))},
-                {"<html><body><h1>Im a H1 tag!</h1></body></html>", Arrays.asList(ImmutableMap.of("contains", "quack"))},
-                {"{\"a\":\"b\",\"c\":\"d\",\"e\":123,\"g\":false}", Arrays.asList(ImmutableMap.of("contains", "moo"))}
+                {"some-sensor-value", "equals", arbitrary, Arrays.asList(ImmutableMap.of("isEqualTo", arbitrary))},
+                {"some-sensor-value", "equals", arbitrary, Arrays.asList(ImmutableMap.of("equalTo", arbitrary))},
+                {"some-sensor-value", "equals", arbitrary, Arrays.asList(ImmutableMap.of("equals", arbitrary))},
+
+                {"some-regex-value-to-match", "matches", "some.*not-match", Arrays.asList(ImmutableMap.of("matches", "some.*not-match", "isEqualTo", "oink"))},
+
+                {null, "notNull", Boolean.TRUE, Arrays.asList(ImmutableMap.of("notNull", Boolean.TRUE))},
+                {"some-not-null-value", "notNull", Boolean.FALSE, Arrays.asList(ImmutableMap.of("notNull", Boolean.FALSE))},
+                {"some-non-null-value", "isNull", Boolean.TRUE, Arrays.asList(ImmutableMap.of("isNull", Boolean.TRUE))},
+                {null, "isNull", Boolean.FALSE, Arrays.asList(ImmutableMap.of("isNull", Boolean.FALSE))},
+
+                {null, "notEmpty", Boolean.TRUE, Arrays.asList(ImmutableMap.of("notEmpty", Boolean.TRUE))},
+                {"some-not-null-value", "notEmpty", Boolean.FALSE, Arrays.asList(ImmutableMap.of("notEmpty", Boolean.FALSE))},
+                {"some-non-null-value", "isEmpty", Boolean.TRUE, Arrays.asList(ImmutableMap.of("isEmpty", Boolean.TRUE))},
+                {null, "isEmpty", Boolean.FALSE, Arrays.asList(ImmutableMap.of("isEmpty", Boolean.FALSE))},
+
+                {"<html><body><h1>Im a H1 tag!</h1></body></html>", "contains", "quack", Arrays.asList(ImmutableMap.of("contains", "quack"))},
+                {"{\"a\":\"b\",\"c\":\"d\",\"e\":123,\"g\":false}", "contains", "moo", Arrays.asList(ImmutableMap.of("contains", "moo"))},
+
+                {"true", "hasTruthValue", Boolean.FALSE, Arrays.asList(ImmutableMap.of("hasTruthValue", Boolean.FALSE))},
+                {"false", "hasTruthValue", Boolean.TRUE, Arrays.asList(ImmutableMap.of("hasTruthValue", Boolean.TRUE))},
+                {"some-not-null-value", "hasTruthValue", Boolean.TRUE, Arrays.asList(ImmutableMap.of("hasTruthValue", Boolean.TRUE))}
         };
     }
 
     @Test(dataProvider = "negativeTestsDP")
-    public void negativeTests(final String data, final List<Map<String, Object>> assertions) {
+    public void negativeTests(final String data, String condition, Object expected, final List<Map<String, Object>> assertions) {
         final Supplier<String> supplier = new Supplier<String>() {
             @Override
             public String get() {
@@ -94,14 +121,11 @@ public class TestFrameworkAssertionsTest {
                 return data;
             }
         };
-        boolean assertionErrorCaught = false;
         try {
-            TestFrameworkAssertions.checkAssertions(supplier, ImmutableMap.of("timeout", new Duration(2L, TimeUnit.SECONDS)), assertions);
+            TestFrameworkAssertions.checkAssertions(ImmutableMap.of("timeout", new Duration(2L, TimeUnit.SECONDS)), assertions, data, supplier);
+            Asserts.shouldHaveFailedPreviously();
         } catch (AssertionError e) {
-            assertionErrorCaught = true;
-            assertThat(e).hasMessage("expected [true] but found [false]");
-        } finally {
-            assertThat(assertionErrorCaught).isTrue().as("An assertion error should have been thrown");
+            Asserts.expectedFailureContains(e, data, condition, expected.toString());
         }
 
     }
@@ -119,15 +143,12 @@ public class TestFrameworkAssertionsTest {
                 return randomId;
             }
         };
-        boolean illegalStateExceptionThrown = false;
         try {
-            TestFrameworkAssertions.checkAssertions(supplier, ImmutableMap.of("timeout", new Duration(2L, TimeUnit.SECONDS)), assertions);
-        } catch (Exception e) {
-            assertThat(e).isInstanceOf(IllegalStateException.class);
-            assertThat(e).hasMessage("No predicate found with signature [" + randomId + "]");
-            illegalStateExceptionThrown = true;
-        } finally {
-            assertThat(illegalStateExceptionThrown).isTrue().as("An illegal state exception should have been thrown");
+            TestFrameworkAssertions.checkAssertions(ImmutableMap.of("timeout", new Duration(2L, TimeUnit.SECONDS)), assertions, "anyTarget", supplier);
+            Asserts.shouldHaveFailedPreviously();
+        } catch (Throwable e) {
+            Asserts.expectedFailureOfType(e, AssertionError.class);
+            Asserts.expectedFailureContains(e, TestFrameworkAssertions.UNKNOWN_CONDITION);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java
index ffa4001..f3552e6 100644
--- a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java
+++ b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java
@@ -29,6 +29,7 @@ import org.apache.brooklyn.test.http.TestHttpRequestHandler;
 import org.apache.brooklyn.test.http.TestHttpServer;
 import org.apache.brooklyn.util.text.Identifiers;
 import org.apache.brooklyn.util.time.Duration;
+import org.apache.http.HttpStatus;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
@@ -71,7 +72,7 @@ public class TestHttpCallTest {
     }
 
 
-    @Test
+    @Test(groups = "Integration")
     public void testHttpBodyAssertions() {
         app.createAndManageChild(EntitySpec.create(TestHttpCall.class)
                 .configure(TestHttpCall.TARGET_URL, server.getUrl() + "/201")
@@ -92,18 +93,18 @@ public class TestHttpCallTest {
         app.start(ImmutableList.of(loc));
     }
 
-    @Test
+    @Test(groups = "Integration")
     public void testHttpStatusAssertions() {
         app.createAndManageChild(EntitySpec.create(TestHttpCall.class)
                 .configure(TestHttpCall.TARGET_URL, server.getUrl() + "/201")
                 .configure(TestHttpCall.TIMEOUT, new Duration(10L, TimeUnit.SECONDS))
                 .configure(TestHttpCall.ASSERTION_TARGET, TestHttpCall.HttpAssertionTarget.status)
-                .configure(TestSensor.ASSERTIONS, newAssertion("notNull", "")));
+                .configure(TestSensor.ASSERTIONS, newAssertion("notNull", Boolean.TRUE)));
         app.createAndManageChild(EntitySpec.create(TestHttpCall.class)
                 .configure(TestHttpCall.TARGET_URL, server.getUrl() + "/204")
                 .configure(TestHttpCall.TIMEOUT, new Duration(10L, TimeUnit.SECONDS))
                 .configure(TestHttpCall.ASSERTION_TARGET, TestHttpCall.HttpAssertionTarget.status)
-                .configure(TestSensor.ASSERTIONS, newAssertion("isEqualTo", "204")));
+                .configure(TestSensor.ASSERTIONS, newAssertion("isEqualTo", HttpStatus.SC_NO_CONTENT)));
         app.createAndManageChild(EntitySpec.create(TestHttpCall.class)
                 .configure(TestHttpCall.TARGET_URL, server.getUrl() + "/body.json")
                 .configure(TestHttpCall.TIMEOUT, new Duration(10L, TimeUnit.SECONDS))

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java
index 83634fa..9fea7a5 100644
--- a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java
+++ b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java
@@ -50,6 +50,7 @@ public class TestSensorTest {
 
     private static final AttributeSensorAndConfigKey<Boolean, Boolean> BOOLEAN_SENSOR = ConfigKeys.newSensorAndConfigKey(Boolean.class, "boolean-sensor", "Boolean Sensor");
     private static final AttributeSensorAndConfigKey<String, String> STRING_SENSOR = ConfigKeys.newSensorAndConfigKey(String.class, "string-sensor", "String Sensor");
+    private static final AttributeSensorAndConfigKey<Integer, Integer> INTEGER_SENSOR = ConfigKeys.newIntegerSensorAndConfigKey("integer-sensor", "Integer Sensor");
     private static final AttributeSensorAndConfigKey<Object, Object> OBJECT_SENSOR = ConfigKeys.newSensorAndConfigKey(Object.class, "object-sensor", "Object Sensor");
 
     private TestApplication app;
@@ -73,6 +74,8 @@ public class TestSensorTest {
 
     @Test
     public void testAssertEqual() {
+        int testInteger = 100;
+
         //Add Sensor Test for BOOLEAN sensor
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
@@ -83,13 +86,21 @@ public class TestSensorTest {
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, STRING_SENSOR.getName())
                 .configure(TestSensor.ASSERTIONS, newAssertion("equals", testId)));
+        //Add Sensor Test for INTEGER sensor
+        app.createAndManageChild(EntitySpec.create(TestSensor.class)
+                .configure(TestSensor.TARGET_ENTITY, app)
+                .configure(TestSensor.SENSOR_NAME, INTEGER_SENSOR.getName())
+                .configure(TestSensor.ASSERTIONS, newAssertion("equals", testInteger)));
 
         //Set BOOLEAN Sensor to true
         app.sensors().set(BOOLEAN_SENSOR, Boolean.TRUE);
+
+        // Give a value to INTEGER sensor
+        app.sensors().set(INTEGER_SENSOR, testInteger);
+
         //Set STRING sensor to random string
         app.sensors().set(STRING_SENSOR, testId);
 
-
         app.start(ImmutableList.of(loc));
 
     }
@@ -119,7 +130,7 @@ public class TestSensorTest {
     }
 
     @Test
-    public void testAssertEqualOnNullSenor() {
+    public void testAssertEqualOnNullSensor() {
         boolean booleanAssertFailed = false;
 
         //Add Sensor Test for BOOLEAN sensor
@@ -145,12 +156,12 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, BOOLEAN_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("isNull", "")));
+                .configure(TestSensor.ASSERTIONS,  newAssertion("isNull", true)));
         //Add Sensor Test for STRING sensor
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, STRING_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("notNUll", "")));
+                .configure(TestSensor.ASSERTIONS, newAssertion("notNull", true)));
 
         //Set STRING sensor to random string
         app.sensors().set(STRING_SENSOR, testId);
@@ -167,7 +178,7 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, STRING_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("isNull", "true")));
+                .configure(TestSensor.ASSERTIONS, newAssertion("isNull", true)));
 
         //Set STRING sensor to random string
         app.sensors().set(STRING_SENSOR, testId);

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/d3d120c5/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml b/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml
index 738f787..9844d20 100644
--- a/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml
+++ b/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml
@@ -31,29 +31,30 @@ services:
     url: $brooklyn:component("tomcat").attributeWhenReady("webapp.url")
     applyAssertionTo: status
     assert:
-    - equalTo: 200
+      - equalTo: 200
   - type: org.apache.brooklyn.test.framework.TestHttpCall
     name: Status Code 404
     url: $brooklyn:formatString("%s/invalidpath/", component("tomcat").attributeWhenReady("webapp.url"))
     timeout: 10s
     applyAssertionTo: status
     assert:
-      status: 404
+      - equals: 404
   - type: org.apache.brooklyn.test.framework.TestHttpCall
     name: String match
     url: $brooklyn:component("tomcat").attributeWhenReady("webapp.url")
     applyAssertionTo: body
     assert:
-    - contains: Sample Brooklyn Deployed
+      - contains: Sample Brooklyn Deployed
   - type: org.apache.brooklyn.test.framework.TestHttpCall
     name: Regex match
     url: $brooklyn:component("tomcat").attributeWhenReady("webapp.url")
     applyAssertionTo: body
-    # the regex assert uses java.lang.String under the hood so if the url is expected to returns
+    # the matches assert uses java.lang.String under the hood so if the url is expected to returns
     # a multi-line response you should use the embedded dotall flag expression `(?s)` in your regex.
     # See: http://docs.oracle.com/javase/7/docs/api/java/util/regex/Pattern.html
     assert:
-      regex: "(?s).*illustrate(\\s)*how(\\s)*web(\\s)*applications.*"
+      - matches: "(?s).*illustrate(\\s)*how(\\s)*web(\\s)*applications.*"
+
 ...
 
 ---


[3/3] incubator-brooklyn git commit: This closes #1066

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


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

Branch: refs/heads/master
Commit: d20e41d6ddbab4f19c8bd2a2e6695f86c50b6b34
Parents: aeb56fd e1412e8
Author: Aled Sage <al...@gmail.com>
Authored: Mon Nov 30 22:05:04 2015 +0000
Committer: Aled Sage <al...@gmail.com>
Committed: Mon Nov 30 22:05:04 2015 +0000

----------------------------------------------------------------------
 .../brooklyn/test/framework/BaseTest.java       |  12 +-
 .../test/framework/SimpleShellCommandTest.java  |  47 ++-
 .../framework/SimpleShellCommandTestImpl.java   | 106 ++-----
 .../test/framework/TestFrameworkAssertions.java | 289 +++++++++++++------
 .../test/framework/TestHttpCallImpl.java        |  36 ++-
 .../brooklyn/test/framework/TestSensorImpl.java |  19 +-
 .../SimpleShellCommandIntegrationTest.java      | 105 ++++++-
 .../framework/TestFrameworkAssertionsTest.java  |  74 +++--
 .../test/framework/TestHttpCallTest.java        |  11 +-
 .../brooklyn/test/framework/TestSensorTest.java |  45 ++-
 .../testhttpcall-examples.yml                   |  20 +-
 11 files changed, 455 insertions(+), 309 deletions(-)
----------------------------------------------------------------------



[2/3] incubator-brooklyn git commit: Code review comments from https://github.com/apache/incubator-brooklyn/pull/1066/.

Posted by al...@apache.org.
Code review comments from https://github.com/apache/incubator-brooklyn/pull/1066/.

Unused Imports
https://github.com/apache/incubator-brooklyn/pull/1066/files#r45793815
https://github.com/apache/incubator-brooklyn/pull/1066/files#r45934555

Use 'Suppliers.ofInstance' where appropriate.
https://github.com/apache/incubator-brooklyn/pull/1066/files#r45842448

Add member variable for default exit code assertion.
https://github.com/apache/incubator-brooklyn/pull/1066/files#r45933970
Note the logic has been corrected, the default exit status assertion
is meant to be made only if you don't supply any assertions.  If you
supply for example just an assertion on stdout then this should mean
you don't care what the exit code is.

Throw exception instead of reporting inaccurate status code.
https://github.com/apache/incubator-brooklyn/pull/1066/files#r45843346

Make the assertions tolerant to definition as maps or as lists of maps
https://github.com/apache/incubator-brooklyn/pull/1066/files#r45842311

Also:

remove some redundant duplicated declarations (EQUALS...)


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

Branch: refs/heads/master
Commit: e1412e8078c76710f3aedd47191783ab77160553
Parents: d3d120c
Author: Geoff Macartney <ge...@cloudsoftcorp.com>
Authored: Thu Nov 26 11:23:30 2015 +0000
Committer: Geoff Macartney <ge...@cloudsoftcorp.com>
Committed: Thu Nov 26 13:07:27 2015 +0000

----------------------------------------------------------------------
 .../brooklyn/test/framework/BaseTest.java       | 12 ++---
 .../test/framework/SimpleShellCommandTest.java  | 47 ++++++++------------
 .../framework/SimpleShellCommandTestImpl.java   | 31 ++++++-------
 .../test/framework/TestFrameworkAssertions.java | 44 ++++++++++++++++++
 .../test/framework/TestHttpCallImpl.java        | 10 ++---
 .../brooklyn/test/framework/TestSensorImpl.java |  4 +-
 .../SimpleShellCommandIntegrationTest.java      |  3 +-
 .../framework/TestFrameworkAssertionsTest.java  |  1 -
 .../test/framework/TestHttpCallTest.java        |  2 +-
 .../brooklyn/test/framework/TestSensorTest.java | 32 +++++++------
 .../testhttpcall-examples.yml                   | 17 ++++---
 11 files changed, 120 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/BaseTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/BaseTest.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/BaseTest.java
index 19043e7..d316268 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/BaseTest.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/BaseTest.java
@@ -47,16 +47,16 @@ public interface BaseTest extends Entity, Startable {
     ConfigKey<String> TARGET_ID = ConfigKeys.newStringConfigKey("targetId", "Id of the entity under test");
 
     /**
-     * The assertions to be made
+     * The assertions to be made.
      */
-    ConfigKey<List<Map<String, Object>>> ASSERTIONS = ConfigKeys.newConfigKey(
-            new TypeToken<List<Map<String, Object>>>() {},
-            "assert", "Assertions to be evaluated", new ArrayList<Map<String, Object>>());
+    ConfigKey<Object> ASSERTIONS = ConfigKeys.newConfigKey(Object.class, "assert", "Assertions to be evaluated",
+        new ArrayList<Map<String, Object>>());
 
     /**
-     * THe duration to wait
+     * THe duration to wait for an assertion to succeed or fail before throwing an exception.
      */
-    ConfigKey<Duration> TIMEOUT = ConfigKeys.newConfigKey(Duration.class, "timeout", "Time to wait on result", new Duration(1L, TimeUnit.SECONDS));
+    ConfigKey<Duration> TIMEOUT = ConfigKeys.newConfigKey(Duration.class, "timeout", "Time to wait on result",
+        new Duration(1L, TimeUnit.SECONDS));
 
     /**
      * Get the target of the test.

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java
index de1f176..4fb2def 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTest.java
@@ -18,7 +18,8 @@
  */
 package org.apache.brooklyn.test.framework;
 
-import com.google.common.collect.Maps;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.reflect.TypeToken;
 import org.apache.brooklyn.api.entity.ImplementedBy;
 import org.apache.brooklyn.config.ConfigKey;
@@ -39,27 +40,10 @@ import static org.apache.brooklyn.core.config.ConfigKeys.newConfigKey;
 @ImplementedBy(SimpleShellCommandTestImpl.class)
 public interface SimpleShellCommandTest extends BaseTest {
 
-    String TMP_DEFAULT = "/tmp";
-
-    /**
-     * Equals assertion on command result.
-     */
-    String EQUALS = "equals";
-
     /**
-     * String contains assertion on command result.
+     * Default location for temporary files.
      */
-    String CONTAINS = "contains";
-
-    /**
-     * Regex match assertion on command result.
-     */
-    String MATCHES = "matches";
-
-    /**
-     * Is-empty match assertion on command result.
-     */
-    String IS_EMPTY = "isEmpty";
+    String TMP_DEFAULT = "/tmp";
 
     /**
      * Supply the command to invoke directly. Cannot be used together with {@link #DOWNLOAD_URL}.
@@ -84,30 +68,35 @@ public interface SimpleShellCommandTest extends BaseTest {
      */
     @SetFromFlag("runDir")
     ConfigKey<String> RUN_DIR = newConfigKey(String.class, "run.dir", "directory where downloaded scripts should be run from");
+
+
+    /**
+     * If no assertions are configured in the test then the default is this assertion that exit status of the command
+     * is zero (successful).
+     */
+    Map<String, Object> DEFAULT_ASSERTION = ImmutableMap.<String,Object>of(TestFrameworkAssertions.EQUALS, 0);
+
     /**
      * Assertions on the exit code of the simple command.
      *
      * If not explicitly configured, the default assertion is a non-zero exit code.
      */
     @SetFromFlag("assertStatus")
-    ConfigKey<List<Map<String, Object>>> ASSERT_STATUS = ConfigKeys.newConfigKey(
-        new TypeToken<List<Map<String, Object>>>() {},
-        "assert.status", "Assertions on command exit code", new ArrayList<Map<String, Object>>());
+    ConfigKey<Object> ASSERT_STATUS = ConfigKeys.newConfigKey(Object.class, "assert.status", "Assertions on command exit code",
+        new ArrayList<Map<String, Object>>());
 
     /**
      * Assertions on the standard output of the command as a String.
      */
     @SetFromFlag("assertOut")
-    ConfigKey<List<Map<String, Object>>> ASSERT_OUT = ConfigKeys.newConfigKey(
-        new TypeToken<List<Map<String, Object>>>() {},
-        "assert.out", "Assertions on command standard output", new ArrayList<Map<String, Object>>());
+    ConfigKey<Object> ASSERT_OUT = ConfigKeys.newConfigKey(Object.class, "assert.out", "Assertions on command standard output",
+        new ArrayList<Map<String, Object>>());
 
     /**
      * Assertions on the standard error of the command as a String.
      */
     @SetFromFlag("assertErr")
-    ConfigKey<List<Map<String, Object>>> ASSERT_ERR = ConfigKeys.newConfigKey(
-        new TypeToken<List<Map<String, Object>>>() {},
-        "assert.err", "Assertions on command standard error", new ArrayList<Map<String, Object>>());
+    ConfigKey<Object> ASSERT_ERR = ConfigKeys.newConfigKey(Object.class, "assert.err", "Assertions on command standard error",
+        new ArrayList<Map<String, Object>>());
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java
index 5a709e8..e01c482 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/SimpleShellCommandTestImpl.java
@@ -21,10 +21,11 @@ package org.apache.brooklyn.test.framework;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Splitter;
-import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableMap;
 import org.apache.brooklyn.api.location.Location;
 import org.apache.brooklyn.api.mgmt.TaskFactory;
+import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.core.effector.ssh.SshEffectorTasks;
 import org.apache.brooklyn.core.entity.lifecycle.Lifecycle;
 import org.apache.brooklyn.core.location.Machines;
@@ -47,14 +48,14 @@ 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.test.framework.TestFrameworkAssertions.checkAssertions;
+import static org.apache.brooklyn.test.framework.TestFrameworkAssertions.getAssertions;
 import static org.apache.brooklyn.util.text.Strings.isBlank;
 import static org.apache.brooklyn.util.text.Strings.isNonBlank;
 
 // TODO assertions below should use TestFrameworkAssertions but that class needs to be improved to give better error messages
 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";
@@ -109,21 +110,12 @@ public class SimpleShellCommandTestImpl extends AbstractTest implements SimpleSh
         });
         ImmutableMap<String, Duration> flags = ImmutableMap.of("timeout", getConfig(TIMEOUT));
         AssertionSupport support = new AssertionSupport();
-        TestFrameworkAssertions.checkAssertions(support, flags, exitCodeAssertions(), "exit code", supply(result.getExitCode()));
-        TestFrameworkAssertions.checkAssertions(support, flags, getConfig(ASSERT_OUT), "stdout", supply(result.getStdout()));
-        TestFrameworkAssertions.checkAssertions(support, flags, getConfig(ASSERT_ERR), "stderr", supply(result.getStderr()));
+        checkAssertions(support, flags, exitCodeAssertions(), "exit code", Suppliers.ofInstance(result.getExitCode()));
+        checkAssertions(support, flags, getAssertions(this, ASSERT_OUT), "stdout", Suppliers.ofInstance(result.getStdout()));
+        checkAssertions(support, flags, getAssertions(this, ASSERT_ERR), "stderr", Suppliers.ofInstance(result.getStderr()));
         support.validate();
     }
 
-    private static <T> Supplier<T> supply(final T t) {
-        return new Supplier<T>() {
-            @Override
-            public T get() {
-                return t;
-            }
-        };
-    }
-
     private String shorten(String text) {
         return Strings.maxlenWithEllipsis(text, A_LINE);
     }
@@ -240,9 +232,12 @@ public class SimpleShellCommandTestImpl extends AbstractTest implements SimpleSh
     
 
     private List<Map<String, Object>> exitCodeAssertions() {
-        List<Map<String, Object>> assertStatus = getConfig(ASSERT_STATUS);
-        if (assertStatus.isEmpty()) {
-            Map<String, Object> shouldSucceed = ImmutableMap.<String,Object>of(EQUALS, SUCCESS);
+        List<Map<String, Object>> assertStatus = getAssertions(this, ASSERT_STATUS);
+        List<Map<String, Object>> assertOut = getAssertions(this, ASSERT_OUT);
+        List<Map<String, Object>> assertErr = getAssertions(this, ASSERT_ERR);
+
+        if (assertStatus.isEmpty() && assertOut.isEmpty() && assertErr.isEmpty()) {
+            Map<String, Object> shouldSucceed = DEFAULT_ASSERTION;
             assertStatus.add(shouldSucceed);
         }
         return assertStatus;

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java
index bd955fa..fee68e6 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestFrameworkAssertions.java
@@ -20,11 +20,18 @@ package org.apache.brooklyn.test.framework;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Supplier;
+import com.google.common.reflect.TypeToken;
+import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.test.Asserts;
+import org.apache.brooklyn.util.core.flags.TypeCoercions;
 import org.apache.brooklyn.util.exceptions.CompoundRuntimeException;
+import org.apache.brooklyn.util.exceptions.FatalConfigurationRuntimeException;
+import org.apache.brooklyn.util.guava.Maybe;
 import org.apache.brooklyn.util.text.Strings;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -52,6 +59,43 @@ public class TestFrameworkAssertions {
     private TestFrameworkAssertions() {
     }
 
+
+    /**
+     *  Get assertions tolerantly from a configuration key.
+     *  This supports either a simple map of assertions, such as
+     *
+     <pre>
+     assertOut:
+       contains: 2 users
+       matches: .*[\d]* days.*
+     </pre>
+     * or a list of such maps, (which allows you to repeat keys):
+     <pre>
+     assertOut:
+     - contains: 2 users
+     - contains: 2 days
+     </pre>
+     or
+    private static List<Map<String,Object>> getAssertions(ConfigKey<Object> key) {
+    }
+    */
+    public static List<Map<String, Object>> getAssertions(Entity entity, ConfigKey<Object> key) {
+        Object config = entity.getConfig(key);
+        Maybe<Map<String, Object>> maybeMap = TypeCoercions.tryCoerce(config, new TypeToken<Map<String, Object>>() {});
+        if (maybeMap.isPresent()) {
+            return Collections.singletonList(maybeMap.get());
+        }
+
+        Maybe<List<Map<String, Object>>> maybeList = TypeCoercions.tryCoerce(config,
+            new TypeToken<List<Map<String, Object>>>() {});
+        if (maybeList.isPresent()) {
+            return maybeList.get();
+        }
+
+        throw new FatalConfigurationRuntimeException(key.getDescription() + " is not a map or list of maps");
+    }
+
+
     public static <T> void checkAssertions(Map<String,?> flags,
                                            Map<String, Object> assertions,
                                            String target,

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java
index 92f2885..67451ed 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestHttpCallImpl.java
@@ -27,7 +27,6 @@ import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
 import org.apache.brooklyn.util.exceptions.Exceptions;
 import org.apache.brooklyn.util.http.HttpTool;
 import org.apache.brooklyn.util.time.Duration;
-import org.apache.http.HttpStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,6 +34,8 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.brooklyn.test.framework.TestFrameworkAssertions.getAssertions;
+
 /**
  * {@inheritDoc}
  */
@@ -51,7 +52,7 @@ public class TestHttpCallImpl extends AbstractTest implements TestHttpCall {
         }
         ServiceStateLogic.setExpectedState(this, Lifecycle.STARTING);
         final String url = getConfig(TARGET_URL);
-        final List<Map<String, Object>> assertions = getConfig(ASSERTIONS);
+        final List<Map<String, Object>> assertions = getAssertions(this, ASSERTIONS);
         final Duration timeout = getConfig(TIMEOUT);
         final HttpAssertionTarget target = getConfig(ASSERTION_TARGET);
 
@@ -87,9 +88,8 @@ public class TestHttpCallImpl extends AbstractTest implements TestHttpCall {
                         try {
                             return HttpTool.getHttpStatusCode(url);
                         } catch (Exception e) {
-                            LOG.error("HTTP call to [{}] failed due to [{}] ... returning Status code [500]",
-                                url, e.getMessage());
-                            return HttpStatus.SC_INTERNAL_SERVER_ERROR;
+                            LOG.info("HTTP call to [{}] failed due to [{}]", url, e.getMessage());
+                            throw Exceptions.propagate(e);
                         }
                     }
                 };

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java
index f5ed913..f368cf5 100644
--- a/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java
+++ b/usage/test-framework/src/main/java/org/apache/brooklyn/test/framework/TestSensorImpl.java
@@ -38,6 +38,8 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.brooklyn.test.framework.TestFrameworkAssertions.getAssertions;
+
 /**
  * {@inheritDoc}
  */
@@ -56,7 +58,7 @@ public class TestSensorImpl extends AbstractTest implements TestSensor {
         final Entity target = resolveTarget();
         final String sensor = getConfig(SENSOR_NAME);
         final Duration timeout = getConfig(TIMEOUT);
-        final List<Map<String, Object>> assertions = getConfig(ASSERTIONS);
+        final List<Map<String, Object>> assertions = getAssertions(this, ASSERTIONS);
         try {
             TestFrameworkAssertions.checkAssertions(ImmutableMap.of("timeout", timeout), assertions, sensor,
                 new Supplier<Object>() {

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java
index 9e6a27a..9a3b339 100644
--- a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java
+++ b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/SimpleShellCommandIntegrationTest.java
@@ -19,7 +19,6 @@
 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.core.entity.lifecycle.Lifecycle;
 import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic;
@@ -44,6 +43,8 @@ import java.util.Map;
 
 import static org.apache.brooklyn.test.framework.BaseTest.TARGET_ENTITY;
 import static org.apache.brooklyn.test.framework.SimpleShellCommandTest.*;
+import static org.apache.brooklyn.test.framework.TestFrameworkAssertions.CONTAINS;
+import static org.apache.brooklyn.test.framework.TestFrameworkAssertions.EQUALS;
 import static org.assertj.core.api.Assertions.assertThat;
 
 public class SimpleShellCommandIntegrationTest extends BrooklynAppUnitTestSupport {

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java
index 60d452c..482cf5e 100644
--- a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java
+++ b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestFrameworkAssertionsTest.java
@@ -35,7 +35,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import static org.assertj.core.api.Assertions.assertThat;
 
 /**
  * @author m4rkmckenna on 11/11/2015.

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java
index f3552e6..055c5fa 100644
--- a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java
+++ b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestHttpCallTest.java
@@ -50,7 +50,7 @@ public class TestHttpCallTest {
     private LocalhostMachineProvisioningLocation loc;
     private String testId;
 
-    @BeforeMethod
+    @BeforeMethod(alwaysRun = true)
     public void setup() {
         testId = Identifiers.makeRandomId(8);
         server = new TestHttpServer()

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java
index 9fea7a5..4a00d75 100644
--- a/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java
+++ b/usage/test-framework/src/test/java/org/apache/brooklyn/test/framework/TestSensorTest.java
@@ -80,17 +80,17 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, BOOLEAN_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("equals", true)));
+                .configure(TestSensor.ASSERTIONS, newMapAssertion("equals", true)));
         //Add Sensor Test for STRING sensor
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, STRING_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("equals", testId)));
+                .configure(TestSensor.ASSERTIONS, newListAssertion("equals", testId)));
         //Add Sensor Test for INTEGER sensor
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, INTEGER_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("equals", testInteger)));
+                .configure(TestSensor.ASSERTIONS, newListAssertion("equals", testInteger)));
 
         //Set BOOLEAN Sensor to true
         app.sensors().set(BOOLEAN_SENSOR, Boolean.TRUE);
@@ -113,7 +113,7 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, BOOLEAN_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("equals", true)));
+                .configure(TestSensor.ASSERTIONS, newMapAssertion("equals", true)));
 
         //Set BOOLEAN Sensor to false
         app.sensors().set(BOOLEAN_SENSOR, Boolean.FALSE);
@@ -137,7 +137,7 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, BOOLEAN_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("equals", false)));
+                .configure(TestSensor.ASSERTIONS, newListAssertion("equals", false)));
 
         try {
             app.start(ImmutableList.of(loc));
@@ -156,12 +156,12 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, BOOLEAN_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS,  newAssertion("isNull", true)));
+                .configure(TestSensor.ASSERTIONS,  newMapAssertion("isNull", true)));
         //Add Sensor Test for STRING sensor
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, STRING_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("notNull", true)));
+                .configure(TestSensor.ASSERTIONS, newListAssertion("notNull", true)));
 
         //Set STRING sensor to random string
         app.sensors().set(STRING_SENSOR, testId);
@@ -178,7 +178,7 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, STRING_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("isNull", true)));
+                .configure(TestSensor.ASSERTIONS, newMapAssertion("isNull", true)));
 
         //Set STRING sensor to random string
         app.sensors().set(STRING_SENSOR, testId);
@@ -205,11 +205,11 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, STRING_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("matches", String.format(".*%s.*", time))));
+                .configure(TestSensor.ASSERTIONS, newListAssertion("matches", String.format(".*%s.*", time))));
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, BOOLEAN_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("matches", "true")));
+                .configure(TestSensor.ASSERTIONS, newMapAssertion("matches", "true")));
 
         //Set STRING sensor
         app.sensors().set(STRING_SENSOR, sensorValue);
@@ -228,7 +228,7 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, STRING_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("matches", String.format(".*%s.*", Identifiers.makeRandomId(8)))));
+                .configure(TestSensor.ASSERTIONS, newListAssertion("matches", String.format(".*%s.*", Identifiers.makeRandomId(8)))));
 
         //Set STRING sensor
         app.sensors().set(STRING_SENSOR, sensorValue);
@@ -250,7 +250,7 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, STRING_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("matches", String.format(".*%s.*", Identifiers.makeRandomId(8)))));
+                .configure(TestSensor.ASSERTIONS, newMapAssertion("matches", String.format(".*%s.*", Identifiers.makeRandomId(8)))));
 
         try {
             app.start(ImmutableList.of(loc));
@@ -270,7 +270,7 @@ public class TestSensorTest {
         app.createAndManageChild(EntitySpec.create(TestSensor.class)
                 .configure(TestSensor.TARGET_ENTITY, app)
                 .configure(TestSensor.SENSOR_NAME, OBJECT_SENSOR.getName())
-                .configure(TestSensor.ASSERTIONS, newAssertion("matches", ".*TestObject.*id=.*")));
+                .configure(TestSensor.ASSERTIONS, newListAssertion("matches", ".*TestObject.*id=.*")));
 
         app.sensors().set(OBJECT_SENSOR, new TestObject());
 
@@ -278,12 +278,16 @@ public class TestSensorTest {
 
     }
 
-    private List<Map<String, Object>> newAssertion(final String assertionKey, final Object assertionValue) {
+    private List<Map<String, Object>> newListAssertion(final String assertionKey, final Object assertionValue) {
         final List<Map<String, Object>> result = new ArrayList<>();
         result.add(ImmutableMap.<String, Object>of(assertionKey, assertionValue));
         return result;
     }
 
+    private Map<String, Object> newMapAssertion(final String assertionKey, final Object assertionValue) {
+        return ImmutableMap.<String, Object>of(assertionKey, assertionValue);
+    }
+
 
     class TestObject {
         private final String id;

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e1412e80/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml
----------------------------------------------------------------------
diff --git a/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml b/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml
index 9844d20..6885679 100644
--- a/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml
+++ b/usage/test-framework/src/test/resources/test-framework-examples/testhttpcall-examples.yml
@@ -31,20 +31,23 @@ services:
     url: $brooklyn:component("tomcat").attributeWhenReady("webapp.url")
     applyAssertionTo: status
     assert:
-      - equalTo: 200
+      equalTo: 200
   - type: org.apache.brooklyn.test.framework.TestHttpCall
     name: Status Code 404
     url: $brooklyn:formatString("%s/invalidpath/", component("tomcat").attributeWhenReady("webapp.url"))
     timeout: 10s
     applyAssertionTo: status
     assert:
-      - equals: 404
+      equals: 404
   - type: org.apache.brooklyn.test.framework.TestHttpCall
     name: String match
     url: $brooklyn:component("tomcat").attributeWhenReady("webapp.url")
     applyAssertionTo: body
+     # This example illustrates the use of a list of maps in order to repeat a key multiple times.
     assert:
-      - contains: Sample Brooklyn Deployed
+      - contains: Sample
+      - contains: Brooklyn
+      - contains: Deployed
   - type: org.apache.brooklyn.test.framework.TestHttpCall
     name: Regex match
     url: $brooklyn:component("tomcat").attributeWhenReady("webapp.url")
@@ -53,7 +56,7 @@ services:
     # a multi-line response you should use the embedded dotall flag expression `(?s)` in your regex.
     # See: http://docs.oracle.com/javase/7/docs/api/java/util/regex/Pattern.html
     assert:
-      - matches: "(?s).*illustrate(\\s)*how(\\s)*web(\\s)*applications.*"
+      matches: "(?s).*illustrate(\\s)*how(\\s)*web(\\s)*applications.*"
 
 ...
 
@@ -72,13 +75,13 @@ services:
     url: $brooklyn:component("tomcat").attributeWhenReady("webapp.url")
     applyAssertionTo: status
     assert:
-    - isEqualTo: 200
+      isEqualTo: 200
   - type: org.apache.brooklyn.test.framework.TestHttpCall
     name: /newcontext Status Code 404
     url: $brooklyn:formatString("%s/newcontext/", component("tomcat").attributeWhenReady("webapp.url"))
     applyAssertionTo: status
     assert:
-    - equalTo: 404
+      equalTo: 404
   - type: org.apache.brooklyn.test.framework.TestEffector
     name: Deploy WAR in /newcontext
     target: $brooklyn:component("tomcat")
@@ -93,7 +96,7 @@ services:
     timeout: 10s
     applyAssertionTo: status
     assert:
-    - equals: 200
+      equals: 200
 ...
 
 ---