You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@brooklyn.apache.org by he...@apache.org on 2015/05/07 12:57:17 UTC

[2/7] incubator-brooklyn git commit: add ability to collect startup errors and check healthy through rest api

add ability to collect startup errors and check healthy through rest api

changes defaults to be to continue on error, and extends (and optimizes) the BrooklynLauncherTest which checks this


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

Branch: refs/heads/master
Commit: ddce439f17143e838291c8f60ab6b69610ed9249
Parents: 20810ac
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Thu Apr 23 11:52:53 2015 +0100
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Mon Apr 27 08:34:45 2015 +0100

----------------------------------------------------------------------
 .../internal/AbstractManagementContext.java     |  11 +-
 .../internal/ManagementContextInternal.java     |   9 ++
 .../NonDeploymentManagementContext.java         |   6 ++
 usage/cli/src/main/java/brooklyn/cli/Main.java  |  15 +--
 .../brooklyn/launcher/BrooklynLauncher.java     |  24 +++--
 .../BrooklynLauncherHighAvailabilityTest.java   |   2 +
 .../brooklyn/launcher/BrooklynLauncherTest.java | 108 ++++++++++++++-----
 .../brooklyn/rest/resources/ServerResource.java |   2 +-
 8 files changed, 138 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ddce439f/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java b/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java
index 501f4e4..0f20f6b 100644
--- a/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java
+++ b/core/src/main/java/brooklyn/management/internal/AbstractManagementContext.java
@@ -23,6 +23,8 @@ import static java.lang.String.format;
 
 import java.net.URI;
 import java.net.URL;
+import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -72,14 +74,15 @@ import brooklyn.management.ha.HighAvailabilityManager;
 import brooklyn.management.ha.HighAvailabilityManagerImpl;
 import brooklyn.util.GroovyJavaMethods;
 import brooklyn.util.ResourceUtils;
+import brooklyn.util.collections.MutableList;
 import brooklyn.util.collections.MutableMap;
 import brooklyn.util.config.ConfigBag;
 import brooklyn.util.guava.Maybe;
 import brooklyn.util.task.BasicExecutionContext;
 import brooklyn.util.task.Tasks;
 
-import com.google.common.base.Objects;
 import com.google.common.base.Function;
+import com.google.common.base.Objects;
 
 public abstract class AbstractManagementContext implements ManagementContextInternal {
     private static final Logger log = LoggerFactory.getLogger(AbstractManagementContext.class);
@@ -164,6 +167,7 @@ public abstract class AbstractManagementContext implements ManagementContextInte
 
     private volatile boolean running = true;
     protected boolean startupComplete = false;
+    protected final List<Throwable> errors = Collections.synchronizedList(MutableList.<Throwable>of()); 
 
     protected Maybe<URI> uri = Maybe.absent();
 
@@ -457,4 +461,9 @@ public abstract class AbstractManagementContext implements ManagementContextInte
         return null;
     }
 
+    @Override
+    public List<Throwable> errors() {
+        return errors;
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ddce439f/core/src/main/java/brooklyn/management/internal/ManagementContextInternal.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/management/internal/ManagementContextInternal.java b/core/src/main/java/brooklyn/management/internal/ManagementContextInternal.java
index b8d334c..87bcc98 100644
--- a/core/src/main/java/brooklyn/management/internal/ManagementContextInternal.java
+++ b/core/src/main/java/brooklyn/management/internal/ManagementContextInternal.java
@@ -20,6 +20,7 @@ package brooklyn.management.internal;
 
 import java.net.URI;
 import java.net.URL;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
 
@@ -40,6 +41,8 @@ import brooklyn.management.ha.OsgiManager;
 import brooklyn.util.guava.Maybe;
 import brooklyn.util.task.TaskTags;
 
+import com.google.common.annotations.Beta;
+
 public interface ManagementContextInternal extends ManagementContext {
 
     public static final String SUB_TASK_TAG = TaskTags.SUB_TASK_TAG;
@@ -104,4 +107,10 @@ public interface ManagementContextInternal extends ManagementContext {
      * Registers a location that has been created, but that has not yet begun to be managed.
      */
     void prePreManage(Location location);
+
+    /** Object which allows adding, removing, and clearing errors.
+     * TODO In future this will change to a custom interface with a unique identifier for each error. */
+    @Beta
+    List<Throwable> errors();
+    
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ddce439f/core/src/main/java/brooklyn/management/internal/NonDeploymentManagementContext.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/brooklyn/management/internal/NonDeploymentManagementContext.java b/core/src/main/java/brooklyn/management/internal/NonDeploymentManagementContext.java
index 72d83bd..7f158d5 100644
--- a/core/src/main/java/brooklyn/management/internal/NonDeploymentManagementContext.java
+++ b/core/src/main/java/brooklyn/management/internal/NonDeploymentManagementContext.java
@@ -441,6 +441,12 @@ public class NonDeploymentManagementContext implements ManagementContextInternal
         return initialManagementContext.lookup(id, type);
     }
 
+    @Override
+    public List<Throwable> errors() {
+        checkInitialManagementContextReal();
+        return initialManagementContext.errors();
+    }
+
     /**
      * For when the initial management context is not "real"; the changeListener is a no-op, but everything else forbidden.
      * 

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ddce439f/usage/cli/src/main/java/brooklyn/cli/Main.java
----------------------------------------------------------------------
diff --git a/usage/cli/src/main/java/brooklyn/cli/Main.java b/usage/cli/src/main/java/brooklyn/cli/Main.java
index b66a6eb..45c6128 100644
--- a/usage/cli/src/main/java/brooklyn/cli/Main.java
+++ b/usage/cli/src/main/java/brooklyn/cli/Main.java
@@ -237,21 +237,22 @@ public class Main extends AbstractMain {
         public Boolean noConsoleSecurity = false;
 
         @Option(name = { "--ignoreWebStartupErrors" },
-            description = "Ignore web subsystem failures on startup (default is to abort if it fails to start)")
+            description = "Ignore web subsystem failures on startup (default is to abort if the web API fails to start, as management is not possible)")
         public boolean ignoreWebErrors = false;
 
         @Option(name = { "--ignorePersistenceStartupErrors" },
-            description = "Ignore persistence/HA subsystem failures on startup (default is to abort if it fails to start)")
-        public boolean ignorePersistenceErrors = false;
+            description = "Ignore persistence/HA subsystem failures on startup "
+                + "(default is true, so errors can be viewed via the API)")
+        public boolean ignorePersistenceErrors = true;
 
         @Option(name = { "--ignoreManagedAppsStartupErrors" },
             description = "Ignore failures starting managed applications passed on the command line on startup "
-                + "(default is to abort if they fail to start)")
-        public boolean ignoreAppErrors = false;
+                + "(default is true, so errors can be viewed via the API)")
+        public boolean ignoreAppErrors = true;
 
         @Beta
         @Option(name = { "--startBrooklynNode" },
-                description = "Whether to start a BrooklynNode entity representing this Brooklyn instance")
+                description = "Whether to start a BrooklynNode entity representing this Brooklyn instance (default false)")
         public boolean startBrooklynNode = false;
 
         // Note in some cases, you can get java.util.concurrent.RejectedExecutionException
@@ -259,7 +260,7 @@ public class Main extends AbstractMain {
         // looks like: {@linktourl https://gist.github.com/47066f72d6f6f79b953e}
         @Beta
         @Option(name = { "-sk", "--stopOnKeyPress" },
-                description = "After startup, shutdown on user text entry")
+                description = "After startup, shutdown on user text entry (default false)")
         public boolean stopOnKeyPress = false;
 
         final static String STOP_WHICH_APPS_ON_SHUTDOWN = "--stopOnShutdown";

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ddce439f/usage/launcher/src/main/java/brooklyn/launcher/BrooklynLauncher.java
----------------------------------------------------------------------
diff --git a/usage/launcher/src/main/java/brooklyn/launcher/BrooklynLauncher.java b/usage/launcher/src/main/java/brooklyn/launcher/BrooklynLauncher.java
index 10c9afb..16f4f76 100644
--- a/usage/launcher/src/main/java/brooklyn/launcher/BrooklynLauncher.java
+++ b/usage/launcher/src/main/java/brooklyn/launcher/BrooklynLauncher.java
@@ -45,7 +45,6 @@ import brooklyn.catalog.CatalogLoadMode;
 import brooklyn.config.BrooklynProperties;
 import brooklyn.config.BrooklynServerConfig;
 import brooklyn.config.BrooklynServerPaths;
-import brooklyn.config.BrooklynServiceAttributes;
 import brooklyn.config.ConfigKey;
 import brooklyn.config.ConfigPredicates;
 import brooklyn.entity.Application;
@@ -101,6 +100,7 @@ import brooklyn.util.text.Strings;
 import brooklyn.util.time.Duration;
 import brooklyn.util.time.Time;
 
+import com.google.common.annotations.Beta;
 import com.google.common.base.Function;
 import com.google.common.base.Splitter;
 import com.google.common.base.Stopwatch;
@@ -151,9 +151,9 @@ public class BrooklynLauncher {
     private Map<String, ?> webconsoleFlags = Maps.newLinkedHashMap();
     private Boolean skipSecurityFilter = null;
     
-    private boolean ignorePersistenceErrors = false;
     private boolean ignoreWebErrors = false;
-    private boolean ignoreAppErrors = false;
+    private boolean ignorePersistenceErrors = true;
+    private boolean ignoreAppErrors = true;
     
     private StopWhichAppsOnShutdown stopWhichAppsOnShutdown = StopWhichAppsOnShutdown.THESE_IF_NOT_PERSISTED;
     
@@ -296,13 +296,14 @@ public class BrooklynLauncher {
      */
     public BrooklynLauncher brooklynProperties(BrooklynProperties brooklynProperties){
         if (managementContext != null) throw new IllegalStateException("Cannot set brooklynProperties and managementContext");
-        this.brooklynProperties = checkNotNull(brooklynProperties, "brooklynProperties");
+        if (this.brooklynProperties!=null && brooklynProperties!=null && this.brooklynProperties!=brooklynProperties)
+            LOG.warn("Brooklyn properties being reset in "+this+"; set null first if you wish to clear it", new Throwable("Source of brooklyn properties reset"));
+        this.brooklynProperties = brooklynProperties;
         return this;
     }
     
     /**
-     * Specifies an attribute passed to deployed webapps 
-     * (in addition to {@link BrooklynServiceAttributes#BROOKLYN_MANAGEMENT_CONTEXT}
+     * Specifies a property to be added to the brooklyn properties
      */
     public BrooklynLauncher brooklynProperties(String field, Object value) {
         brooklynAdditionalProperties.put(checkNotNull(field, "field"), value);
@@ -421,7 +422,10 @@ public class BrooklynLauncher {
         return this;
     }
 
+    @Beta
     public BrooklynLauncher customizeInitialCatalog(Function<BrooklynLauncher, Void> customizeInitialCatalog) {
+        if (this.customizeInitialCatalog!=null)
+            throw new IllegalStateException("Initial catalog customization already set.");
         this.customizeInitialCatalog = customizeInitialCatalog;
         return this;
     }
@@ -585,6 +589,8 @@ public class BrooklynLauncher {
         }
 
         try {
+            // TODO currently done *after* above to mirror existing usage, 
+            // but where this runs will likely change
             if (customizeInitialCatalog!=null)
                 customizeInitialCatalog.apply(this);
         } catch (Exception e) {
@@ -660,6 +666,10 @@ public class BrooklynLauncher {
                 }
                 managementContext = new LocalManagementContext(builder, brooklynAdditionalProperties);
             } else {
+                if (globalBrooklynPropertiesFile != null)
+                    LOG.warn("Ignoring globalBrooklynPropertiesFile "+globalBrooklynPropertiesFile+" because explicit brooklynProperties supplied");
+                if (localBrooklynPropertiesFile != null)
+                    LOG.warn("Ignoring localBrooklynPropertiesFile "+localBrooklynPropertiesFile+" because explicit brooklynProperties supplied");
                 managementContext = new LocalManagementContext(brooklynProperties, brooklynAdditionalProperties);
             }
             brooklynProperties = ((ManagementContextInternal)managementContext).getBrooklynProperties();
@@ -711,6 +721,8 @@ public class BrooklynLauncher {
         Exceptions.propagateIfFatal(e);
         if (ignoreSuchErrors) {
             LOG.error("Subsystem for "+system+" had startup error (continuing with startup): "+e, e);
+            if (managementContext!=null)
+                ((ManagementContextInternal)managementContext).errors().add(e);
         } else {
             throw Exceptions.propagate(e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ddce439f/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherHighAvailabilityTest.java
----------------------------------------------------------------------
diff --git a/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherHighAvailabilityTest.java b/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherHighAvailabilityTest.java
index f151d92..c20cae8 100644
--- a/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherHighAvailabilityTest.java
+++ b/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherHighAvailabilityTest.java
@@ -209,6 +209,7 @@ public class BrooklynLauncherHighAvailabilityTest {
                     .persistMode(PersistMode.AUTO)
                     .persistenceDir(persistenceDir)
                     .persistPeriod(Duration.millis(10))
+                    .ignorePersistenceErrors(false)
                     .application(EntitySpec.create(TestApplication.class))
                     .start();
             fail();
@@ -227,6 +228,7 @@ public class BrooklynLauncherHighAvailabilityTest {
                     .persistMode(PersistMode.AUTO)
                     .persistenceDir(persistenceDir)
                     .persistPeriod(Duration.millis(10))
+                    .ignorePersistenceErrors(false)
                     .application(EntitySpec.create(TestApplication.class))
                     .start();
             fail();

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ddce439f/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherTest.java
----------------------------------------------------------------------
diff --git a/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherTest.java b/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherTest.java
index 1c20c62..0eba3d7 100644
--- a/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherTest.java
+++ b/usage/launcher/src/test/java/brooklyn/launcher/BrooklynLauncherTest.java
@@ -25,8 +25,14 @@ import static org.testng.Assert.assertSame;
 import static org.testng.Assert.assertTrue;
 
 import java.io.File;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.Writer;
 import java.net.URI;
+import java.nio.charset.Charset;
+import java.util.Properties;
 
+import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.Test;
 
@@ -40,6 +46,7 @@ import brooklyn.location.Location;
 import brooklyn.location.basic.LocalhostMachineProvisioningLocation;
 import brooklyn.management.ManagementContext;
 import brooklyn.management.internal.LocalManagementContext;
+import brooklyn.management.internal.ManagementContextInternal;
 import brooklyn.test.HttpTestUtils;
 import brooklyn.test.entity.LocalManagementContextForTests;
 import brooklyn.test.entity.TestApplication;
@@ -47,13 +54,18 @@ import brooklyn.test.entity.TestApplicationImpl;
 import brooklyn.test.entity.TestEntity;
 import brooklyn.util.exceptions.FatalRuntimeException;
 import brooklyn.util.io.FileUtil;
+import brooklyn.util.net.Urls;
 import brooklyn.util.os.Os;
+import brooklyn.util.text.StringFunctions;
 import brooklyn.util.text.Strings;
 
+import com.google.api.client.util.Preconditions;
 import com.google.common.base.Charsets;
+import com.google.common.base.Function;
 import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
 import com.google.common.io.Files;
 
 public class BrooklynLauncherTest {
@@ -65,12 +77,13 @@ public class BrooklynLauncherTest {
     public void tearDown() throws Exception {
         if (launcher != null) launcher.terminate();
         if (persistenceDir != null) RebindTestUtils.deleteMementoDir(persistenceDir);
+        launcher = null;
     }
     
     // Integration because takes a few seconds to start web-console
     @Test(groups="Integration")
     public void testStartsWebServerOnExpectectedPort() throws Exception {
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .webconsolePort("10000+")
                 .start();
         
@@ -88,7 +101,7 @@ public class BrooklynLauncherTest {
         String dataDirName = ".brooklyn-foo"+Strings.makeRandomId(4);
         String dataDir = "~/"+dataDirName;
 
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .brooklynProperties(BrooklynServerConfig.MGMT_BASE_DIR, dataDir)
                 .start();
         
@@ -101,17 +114,18 @@ public class BrooklynLauncherTest {
     
     @Test
     public void testCanDisableWebServerStartup() throws Exception {
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .webconsole(false)
                 .start();
         
         assertNull(launcher.getServerDetails().getWebServer());
         assertNull(launcher.getServerDetails().getWebServerUrl());
+        Assert.assertTrue( ((ManagementContextInternal)launcher.getServerDetails().getManagementContext()).errors().isEmpty() );
     }
     
     @Test
     public void testStartsAppInstance() throws Exception {
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .webconsole(false)
                 .application(new TestApplicationImpl())
                 .start();
@@ -121,7 +135,7 @@ public class BrooklynLauncherTest {
     
     @Test
     public void testStartsAppFromSpec() throws Exception {
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .webconsole(false)
                 .application(EntitySpec.create(TestApplication.class))
                 .start();
@@ -131,7 +145,7 @@ public class BrooklynLauncherTest {
     
     @Test
     public void testStartsAppFromBuilder() throws Exception {
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .webconsole(false)
                 .application(new ApplicationBuilder(EntitySpec.create(TestApplication.class)) {
                         @Override protected void doBuild() {
@@ -147,7 +161,7 @@ public class BrooklynLauncherTest {
                 "services:\n" +
                 "- serviceType: brooklyn.test.entity.TestEntity\n" +
                 "  name: test-app";
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .webconsole(false)
                 .application(yaml)
                 .start();
@@ -158,9 +172,9 @@ public class BrooklynLauncherTest {
         assertTrue(Iterables.getOnlyElement(app.getChildren()) instanceof TestEntity);
     }
     
-    @Test
+    @Test  // may take 2s initializing location if running this test case alone, but noise if running suite 
     public void testStartsAppInSuppliedLocations() throws Exception {
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .webconsole(false)
                 .location("localhost")
                 .application(new ApplicationBuilder(EntitySpec.create(TestApplication.class)) {
@@ -175,7 +189,7 @@ public class BrooklynLauncherTest {
     @Test
     public void testUsesSuppliedManagementContext() throws Exception {
         LocalManagementContext myManagementContext = LocalManagementContextForTests.newInstance();
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(false)
                 .webconsole(false)
                 .managementContext(myManagementContext)
                 .start();
@@ -185,9 +199,9 @@ public class BrooklynLauncherTest {
     
     @Test
     public void testUsesSuppliedBrooklynProperties() throws Exception {
-        BrooklynProperties props = BrooklynProperties.Factory.newEmpty();
+        BrooklynProperties props = LocalManagementContextForTests.builder(true).buildProperties();
         props.put("mykey", "myval");
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(false)
                 .webconsole(false)
                 .brooklynProperties(props)
                 .start();
@@ -197,7 +211,7 @@ public class BrooklynLauncherTest {
 
     @Test
     public void testUsesSupplementaryBrooklynProperties() throws Exception {
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .webconsole(false)
                 .brooklynProperties("mykey", "myval")
                 .start();
@@ -207,7 +221,7 @@ public class BrooklynLauncherTest {
     
     @Test
     public void testReloadBrooklynPropertiesRestoresProgrammaticProperties() throws Exception {
-        launcher = BrooklynLauncher.newInstance()
+        launcher = newLauncherForTests(true)
                 .webconsole(false)
                 .brooklynProperties("mykey", "myval")
                 .start();
@@ -225,15 +239,15 @@ public class BrooklynLauncherTest {
         FileUtil.setFilePermissionsTo600(globalPropertiesFile);
         try {
             String property = "mykey=myval";
-            Files.write(property, globalPropertiesFile, Charsets.UTF_8);
-            launcher = BrooklynLauncher.newInstance()
+            Files.append(getMinimalLauncherPropertiesString()+property, globalPropertiesFile, Charsets.UTF_8);
+            launcher = newLauncherForTests(false)
                     .webconsole(false)
                     .globalBrooklynPropertiesFile(globalPropertiesFile.getAbsolutePath())
                     .start();
             LocalManagementContext managementContext = (LocalManagementContext)launcher.getServerDetails().getManagementContext();
             assertEquals(managementContext.getConfig().getFirst("mykey"), "myval");
             property = "mykey=newval";
-            Files.write(property, globalPropertiesFile, Charsets.UTF_8);
+            Files.write(getMinimalLauncherPropertiesString()+property, globalPropertiesFile, Charsets.UTF_8);
             managementContext.reloadBrooklynProperties();
             assertEquals(managementContext.getConfig().getFirst("mykey"), "newval");
         } finally {
@@ -246,12 +260,12 @@ public class BrooklynLauncherTest {
         File propsFile = File.createTempFile("testChecksGlobalBrooklynPropertiesPermissionsX00", ".properties");
         propsFile.setReadable(true, false);
         try {
-            launcher = BrooklynLauncher.newInstance()
+            launcher = newLauncherForTests(false)
                     .webconsole(false)
                     .globalBrooklynPropertiesFile(propsFile.getAbsolutePath())
                     .start();
-            
-            assertEquals(launcher.getServerDetails().getManagementContext().getConfig().getFirst("mykey"), "myval");
+
+            Assert.fail("Should have thrown");
         } catch (FatalRuntimeException e) {
             if (!e.toString().contains("Invalid permissions for file")) throw e;
         } finally {
@@ -264,12 +278,12 @@ public class BrooklynLauncherTest {
         File propsFile = File.createTempFile("testChecksLocalBrooklynPropertiesPermissionsX00", ".properties");
         propsFile.setReadable(true, false);
         try {
-            launcher = BrooklynLauncher.newInstance()
+            launcher = newLauncherForTests(false)
                     .webconsole(false)
                     .localBrooklynPropertiesFile(propsFile.getAbsolutePath())
                     .start();
             
-            assertEquals(launcher.getServerDetails().getManagementContext().getConfig().getFirst("mykey"), "myval");
+            Assert.fail("Should have thrown");
         } catch (FatalRuntimeException e) {
             if (!e.toString().contains("Invalid permissions for file")) throw e;
         } finally {
@@ -280,20 +294,66 @@ public class BrooklynLauncherTest {
     @Test(groups="Integration")
     public void testStartsWithBrooklynPropertiesPermissionsX00() throws Exception {
         File globalPropsFile = File.createTempFile("testChecksLocalBrooklynPropertiesPermissionsX00_global", ".properties");
+        Files.write(getMinimalLauncherPropertiesString()+"key_in_global=1", globalPropsFile, Charset.defaultCharset());
         File localPropsFile = File.createTempFile("testChecksLocalBrooklynPropertiesPermissionsX00_local", ".properties");
+        Files.write("key_in_local=2", localPropsFile, Charset.defaultCharset());
         FileUtil.setFilePermissionsTo600(globalPropsFile);
         FileUtil.setFilePermissionsTo600(localPropsFile);
         try {
-            launcher = BrooklynLauncher.newInstance()
+            launcher = newLauncherForTests(false)
                     .webconsole(false)
-                    .localBrooklynPropertiesFile(globalPropsFile.getAbsolutePath())
+                    .localBrooklynPropertiesFile(localPropsFile.getAbsolutePath())
+                    .globalBrooklynPropertiesFile(globalPropsFile.getAbsolutePath())
                     .start();
+            assertEquals(launcher.getServerDetails().getManagementContext().getConfig().getFirst("key_in_global"), "1");
+            assertEquals(launcher.getServerDetails().getManagementContext().getConfig().getFirst("key_in_local"), "2");
         } finally {
             globalPropsFile.delete();
             localPropsFile.delete();
         }
     }
     
+    @Test  // takes a few seconds because starts webapp, but also tests rest api so useful
+    public void testErrorsCaughtByApiAndRestApiWorks() throws Exception {
+        launcher = newLauncherForTests(true)
+                .customizeInitialCatalog(new Function<BrooklynLauncher, Void>() {
+                    @Override
+                    public Void apply(BrooklynLauncher input) {
+                        throw new RuntimeException("deliberate-exception-for-testing");
+                    }
+                })
+                .start();
+        // such an error should be thrown, then caught in this calling thread
+        ManagementContext mgmt = launcher.getServerDetails().getManagementContext();
+        Assert.assertFalse( ((ManagementContextInternal)mgmt).errors().isEmpty() );
+        Assert.assertTrue( ((ManagementContextInternal)mgmt).errors().get(0).toString().contains("deliberate"), ""+((ManagementContextInternal)mgmt).errors() );
+        HttpTestUtils.assertContentMatches(
+            Urls.mergePaths(launcher.getServerDetails().getWebServerUrl(), "v1/server/up"), 
+            "true");
+        HttpTestUtils.assertContentMatches(
+            Urls.mergePaths(launcher.getServerDetails().getWebServerUrl(), "v1/server/healthy"), 
+            "false");
+        // TODO test errors api?
+    }
+
+    private BrooklynLauncher newLauncherForTests(boolean minimal) {
+        Preconditions.checkArgument(launcher==null, "can only be used if no launcher yet");
+        BrooklynLauncher launcher = BrooklynLauncher.newInstance();
+        if (minimal)
+            launcher.brooklynProperties(LocalManagementContextForTests.builder(true).buildProperties());
+        return launcher;
+    }
+
+    private String getMinimalLauncherPropertiesString() throws IOException {
+        BrooklynProperties p1 = LocalManagementContextForTests.builder(true).buildProperties();
+        Properties p = new Properties();
+        p.putAll(Maps.transformValues(p1.asMapWithStringKeys(), StringFunctions.toStringFunction()));
+        Writer w = new StringWriter();
+        p.store(w, "test");
+        w.close();
+        return w.toString()+"\n";
+    }
+
     private void assertOnlyApp(BrooklynLauncher launcher, Class<? extends Application> expectedType) {
         assertEquals(launcher.getApplications().size(), 1, "apps="+launcher.getApplications());
         assertNotNull(Iterables.find(launcher.getApplications(), Predicates.instanceOf(TestApplication.class), null), "apps="+launcher.getApplications());

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ddce439f/usage/rest-server/src/main/java/brooklyn/rest/resources/ServerResource.java
----------------------------------------------------------------------
diff --git a/usage/rest-server/src/main/java/brooklyn/rest/resources/ServerResource.java b/usage/rest-server/src/main/java/brooklyn/rest/resources/ServerResource.java
index 422e06d..eac52f6 100644
--- a/usage/rest-server/src/main/java/brooklyn/rest/resources/ServerResource.java
+++ b/usage/rest-server/src/main/java/brooklyn/rest/resources/ServerResource.java
@@ -279,7 +279,7 @@ public class ServerResource extends AbstractBrooklynRestResource implements Serv
     @Override
     public boolean isHealthy() {
         if (!isUp()) return false;
-        // TODO errors
+        if (!((ManagementContextInternal)mgmt()).errors().isEmpty()) return false;
         return true;
     }