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 2014/11/19 14:52:33 UTC

[1/7] incubator-brooklyn git commit: service_process_is_running absence will not trump service_up even on rebind

Repository: incubator-brooklyn
Updated Branches:
  refs/heads/master fbfd78fe5 -> 1241ee175


service_process_is_running absence will not trump service_up even on rebind

modify service_process_is_running enricher to be better about respecting entities which manually set service_up;
previously it did not change service_up *except* on a rebind; now it should not even change them on a rebind

also switch mysql to support service_process_is_running


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

Branch: refs/heads/master
Commit: 7503386fb8f1ddad7b43ed3b6f043ac67a21d5f6
Parents: 4ced56e
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Tue Nov 18 23:41:32 2014 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Tue Nov 18 23:44:56 2014 +0000

----------------------------------------------------------------------
 .../entity/basic/SoftwareProcessImpl.java       | 56 +++++++++++++++++---
 .../entity/database/mysql/MySqlNodeImpl.java    |  2 +-
 2 files changed, 49 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/7503386f/software/base/src/main/java/brooklyn/entity/basic/SoftwareProcessImpl.java
----------------------------------------------------------------------
diff --git a/software/base/src/main/java/brooklyn/entity/basic/SoftwareProcessImpl.java b/software/base/src/main/java/brooklyn/entity/basic/SoftwareProcessImpl.java
index aad9fa5..06a45d6 100644
--- a/software/base/src/main/java/brooklyn/entity/basic/SoftwareProcessImpl.java
+++ b/software/base/src/main/java/brooklyn/entity/basic/SoftwareProcessImpl.java
@@ -32,12 +32,14 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import brooklyn.config.ConfigKey;
-import brooklyn.enricher.Enrichers;
+import brooklyn.enricher.basic.AbstractEnricher;
 import brooklyn.entity.Entity;
 import brooklyn.entity.basic.ServiceStateLogic.ServiceNotUpLogic;
 import brooklyn.entity.drivers.DriverDependentEntity;
 import brooklyn.entity.drivers.EntityDriverManager;
 import brooklyn.entity.effector.EffectorBody;
+import brooklyn.event.SensorEvent;
+import brooklyn.event.SensorEventListener;
 import brooklyn.event.feed.function.FunctionFeed;
 import brooklyn.event.feed.function.FunctionPollConfig;
 import brooklyn.location.Location;
@@ -48,11 +50,11 @@ import brooklyn.location.basic.LocationConfigKeys;
 import brooklyn.location.basic.Machines;
 import brooklyn.location.cloud.CloudLocationConfig;
 import brooklyn.management.Task;
+import brooklyn.policy.EnricherSpec;
 import brooklyn.util.collections.MutableMap;
 import brooklyn.util.collections.MutableSet;
 import brooklyn.util.config.ConfigBag;
 import brooklyn.util.exceptions.Exceptions;
-import brooklyn.util.guava.Functionals;
 import brooklyn.util.task.DynamicTasks;
 import brooklyn.util.task.Tasks;
 import brooklyn.util.time.CountdownTimer;
@@ -128,12 +130,50 @@ public abstract class SoftwareProcessImpl extends AbstractEntity implements Soft
     @Override
     protected void initEnrichers() {
         super.initEnrichers();
-        ServiceNotUpLogic.updateNotUpIndicator(this, SERVICE_PROCESS_IS_RUNNING, "No information yet about whether this service is running");
-        // add an indicator above so that if is_running comes through it clears the map and guarantees an update
-        addEnricher(Enrichers.builder().updatingMap(Attributes.SERVICE_NOT_UP_INDICATORS)
-            .from(SERVICE_PROCESS_IS_RUNNING)
-            .computing(Functionals.ifNotEquals(true).value("The software process for this entity does not appear to be running"))
-            .build());
+        ServiceNotUpLogic.updateNotUpIndicator(this, SERVICE_PROCESS_IS_RUNNING, "No information yet on whether this service is running");
+        // add an indicator above so that if is_running comes through, the map is cleared and an update is guaranteed
+        addEnricher(EnricherSpec.create(UpdatingNotUpFromServiceProcessIsRunning.class).uniqueTag("service-process-is-running-updating-not-up"));
+    }
+    
+    /** subscribes to SERVICE_PROCESS_IS_RUNNING and SERVICE_UP; the latter has no effect if the former is set,
+     * but to support entities which set SERVICE_UP directly we want to make sure that the absence of 
+     * SERVICE_PROCESS_IS_RUNNING does not trigger any not-up indicators */
+    protected static class UpdatingNotUpFromServiceProcessIsRunning extends AbstractEnricher implements SensorEventListener<Object> {
+        public UpdatingNotUpFromServiceProcessIsRunning() {}
+        
+        @Override
+        public void setEntity(EntityLocal entity) {
+            super.setEntity(entity);
+            subscribe(entity, SERVICE_PROCESS_IS_RUNNING, this);
+            subscribe(entity, Attributes.SERVICE_UP, this);
+            onUpdated();
+        }
+
+        @Override
+        public void onEvent(SensorEvent<Object> event) {
+            onUpdated();
+        }
+
+        protected void onUpdated() {
+            Boolean isRunning = entity.getAttribute(SERVICE_PROCESS_IS_RUNNING);
+            if (Boolean.FALSE.equals(isRunning)) {
+                ServiceNotUpLogic.updateNotUpIndicator(entity, SERVICE_PROCESS_IS_RUNNING, "The software process for this entity does not appear to be running");
+                return;
+            }
+            if (Boolean.TRUE.equals(isRunning)) {
+                ServiceNotUpLogic.clearNotUpIndicator(entity, SERVICE_PROCESS_IS_RUNNING);
+                return;
+            }
+            // no info on "isRunning"
+            Boolean isUp = entity.getAttribute(Attributes.SERVICE_UP);
+            if (Boolean.TRUE.equals(isUp)) {
+                // if service explicitly set up, then don't apply our rule
+                ServiceNotUpLogic.clearNotUpIndicator(entity, SERVICE_PROCESS_IS_RUNNING);
+                return;
+            }
+            // service not up, or no info
+            ServiceNotUpLogic.updateNotUpIndicator(entity, SERVICE_PROCESS_IS_RUNNING, "No information on whether this service is running");
+        }
     }
     
       /**

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/7503386f/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java
----------------------------------------------------------------------
diff --git a/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java b/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java
index 7ed69e3..07aacd5 100644
--- a/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java
+++ b/software/database/src/main/java/brooklyn/entity/database/mysql/MySqlNodeImpl.java
@@ -108,7 +108,7 @@ public class MySqlNodeImpl extends SoftwareProcessImpl implements MySqlNode {
                                     return Double.parseDouble(q);
                                 }})
                             .setOnFailureOrException(null) )
-                    .poll(new SshPollConfig<Boolean>(SERVICE_UP)
+                    .poll(new SshPollConfig<Boolean>(SERVICE_PROCESS_IS_RUNNING)
                             .command(cmd)
                             .setOnSuccess(true)
                             .setOnFailureOrException(false))


[2/7] incubator-brooklyn git commit: jmx agent path is set as a sensor to work across versions

Posted by he...@apache.org.
jmx agent path is set as a sensor to work across versions

also deprecate a needless inner class (but keep it around for rebinding)


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

Branch: refs/heads/master
Commit: 2331fbc89c2a9389afab85301b360d2db23eeba3
Parents: fbfd78f
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Wed Nov 19 10:12:54 2014 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Wed Nov 19 10:13:18 2014 +0000

----------------------------------------------------------------------
 .../src/main/java/brooklyn/entity/java/JmxSupport.java   | 10 ++++++++++
 .../base/src/main/java/brooklyn/entity/java/UsesJmx.java | 11 +++++++++--
 2 files changed, 19 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/2331fbc8/software/base/src/main/java/brooklyn/entity/java/JmxSupport.java
----------------------------------------------------------------------
diff --git a/software/base/src/main/java/brooklyn/entity/java/JmxSupport.java b/software/base/src/main/java/brooklyn/entity/java/JmxSupport.java
index 0d93d02..bddd85f 100644
--- a/software/base/src/main/java/brooklyn/entity/java/JmxSupport.java
+++ b/software/base/src/main/java/brooklyn/entity/java/JmxSupport.java
@@ -46,6 +46,7 @@ import brooklyn.util.jmx.jmxrmi.JmxRmiAgent;
 import brooklyn.util.maven.MavenArtifact;
 import brooklyn.util.maven.MavenRetriever;
 import brooklyn.util.net.Urls;
+import brooklyn.util.text.Strings;
 
 import com.google.common.base.Preconditions;
 import com.google.common.net.HostAndPort;
@@ -206,6 +207,15 @@ public class JmxSupport implements UsesJmx {
     }
 
     public String getJmxAgentJarDestinationFilePath() {
+        // cache the local path so we continue to work post-rebind to a different version
+        String result = getEntity().getAttribute(JMX_AGENT_LOCAL_PATH);
+        if (Strings.isNonBlank(result)) return result;
+        result = getJmxAgentJarDestinationFilePathDefault();
+        ((EntityInternal)getEntity()).setAttribute(JMX_AGENT_LOCAL_PATH, result);
+        return result;
+    }
+    
+    public String getJmxAgentJarDestinationFilePathDefault() {
         return Urls.mergePaths(getRunDir(), getJmxAgentJarBasename());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/2331fbc8/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java
----------------------------------------------------------------------
diff --git a/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java b/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java
index ff8164a..778fe1a 100644
--- a/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java
+++ b/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java
@@ -43,7 +43,12 @@ public interface UsesJmx extends UsesJava {
     /** Chosen by Java itself by default, setting this will only have any effect if using an agent. */
     @SetFromFlag("jmxPort")
     PortAttributeSensorAndConfigKey JMX_PORT = new PortAttributeSensorAndConfigKey(
-            "jmx.direct.port", "JMX direct/private port (e.g. JMX RMI server port, or JMXMP port, but not RMI registry port)", PortRanges.fromString("31001+")) {
+        "jmx.direct.port", "JMX direct/private port (e.g. JMX RMI server port, or JMXMP port, but not RMI registry port)", PortRanges.fromString("31001+"));
+    
+    /** @deprecated since 0.7.0, kept for rebinding, use {@link #JMX_PORT} */ @Deprecated
+    PortAttributeSensorAndConfigKey JMX_PORT_LEGACY = new PortAttributeSensorAndConfigKey(
+            "jmx.direct.port.legacy.NOT_USED", "Legacy definition JMX direct/private port (e.g. JMX RMI server port, or JMXMP port, but not RMI registry port)", PortRanges.fromString("31001+")) {
+        private static final long serialVersionUID = 3846846080809179437L;
         @Override protected Integer convertConfigToSensor(PortRange value, Entity entity) {
             // TODO when using JmxAgentModes.NONE we should *not* convert, but leave it null
             // (e.g. to prevent a warning in e.g. ActiveMQIntegrationTest)
@@ -53,7 +58,7 @@ public interface UsesJmx extends UsesJava {
             return super.convertConfigToSensor(value, entity);
         }
     };
-
+    
     /** Well-known port used by Java itself to start the RMI registry where JMX private port can be discovered, ignored if using JMXMP agent. */
     @SetFromFlag("rmiRegistryPort")
     PortAttributeSensorAndConfigKey RMI_REGISTRY_PORT = ConfigKeys.newPortSensorAndConfigKey(
@@ -100,6 +105,8 @@ public interface UsesJmx extends UsesJava {
     /* Currently these are only used to connect, so only applies where systems set this up themselves. */
     AttributeSensorAndConfigKey<String, String> JMX_USER = ConfigKeys.newStringSensorAndConfigKey("jmx.user", "JMX username");
     AttributeSensorAndConfigKey<String, String> JMX_PASSWORD = ConfigKeys.newStringSensorAndConfigKey("jmx.password", "JMX password");
+    
+    AttributeSensorAndConfigKey<String, String> JMX_AGENT_LOCAL_PATH = ConfigKeys.newStringSensorAndConfigKey("jmx.agent.local.path", "Path to JMX driver on the local machine");
 
     /*
      * Synopsis of how the keys work for JMX_SSL:


[7/7] incubator-brooklyn git commit: This closes #347

Posted by he...@apache.org.
This closes #347


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

Branch: refs/heads/master
Commit: 1241ee175488c677a8adcf99b0dcd5ed6b43d1a6
Parents: 3c940ea 7c14906
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Wed Nov 19 13:52:20 2014 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Wed Nov 19 13:52:20 2014 +0000

----------------------------------------------------------------------
 .../src/main/java/brooklyn/entity/java/JmxSupport.java | 10 ++++++++++
 .../src/main/java/brooklyn/entity/java/UsesJmx.java    | 13 ++++++++++---
 2 files changed, 20 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[4/7] incubator-brooklyn git commit: This closes #348

Posted by he...@apache.org.
This closes #348


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

Branch: refs/heads/master
Commit: e3ee1dd41f09161a02fddfe90bd2669a29502273
Parents: fbfd78f 39de027
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Wed Nov 19 13:50:32 2014 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Wed Nov 19 13:50:32 2014 +0000

----------------------------------------------------------------------
 .../JcloudsBlobStoreBasedObjectStore.java       |   6 +-
 .../JcloudsBlobStoreBasedObjectStoreTest.java   | 119 +++++++++++++++++++
 .../main/java/brooklyn/util/text/Strings.java   |   2 +-
 3 files changed, 125 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[3/7] incubator-brooklyn git commit: fix bug *listing* in object store when container name is a virtual path

Posted by he...@apache.org.
fix bug *listing* in object store when container name is a virtual path

previously if container name contained a /, when listing, all items included container name portions after the initial /.
better test coverage now too.


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

Branch: refs/heads/master
Commit: 39de027f5af6f001e405b8b51b34ce845e693bb1
Parents: fbfd78f
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Wed Nov 19 13:00:15 2014 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Wed Nov 19 13:01:53 2014 +0000

----------------------------------------------------------------------
 .../JcloudsBlobStoreBasedObjectStore.java       |   6 +-
 .../JcloudsBlobStoreBasedObjectStoreTest.java   | 119 +++++++++++++++++++
 .../main/java/brooklyn/util/text/Strings.java   |   2 +-
 3 files changed, 125 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/39de027f/locations/jclouds/src/main/java/brooklyn/entity/rebind/persister/jclouds/JcloudsBlobStoreBasedObjectStore.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/main/java/brooklyn/entity/rebind/persister/jclouds/JcloudsBlobStoreBasedObjectStore.java b/locations/jclouds/src/main/java/brooklyn/entity/rebind/persister/jclouds/JcloudsBlobStoreBasedObjectStore.java
index 93759d6..6acaa97 100644
--- a/locations/jclouds/src/main/java/brooklyn/entity/rebind/persister/jclouds/JcloudsBlobStoreBasedObjectStore.java
+++ b/locations/jclouds/src/main/java/brooklyn/entity/rebind/persister/jclouds/JcloudsBlobStoreBasedObjectStore.java
@@ -179,7 +179,10 @@ public class JcloudsBlobStoreBasedObjectStore implements PersistenceObjectStore
                 .transform(new Function<StorageMetadata, String>() {
                     @Override
                     public String apply(@javax.annotation.Nullable StorageMetadata input) {
-                        return input.getName();
+                        String result = input.getName();
+                        result = Strings.removeFromStart(result, containerSubPath);
+                        result = Strings.removeFromStart(result, "/");
+                        return result;
                     }
                 }).toList();
     }
@@ -205,6 +208,7 @@ public class JcloudsBlobStoreBasedObjectStore implements PersistenceObjectStore
         this.mgmt = mgmt;
     }
     
+    @SuppressWarnings("deprecation")
     @Override
     public void prepareForSharedUse(@Nullable PersistMode persistMode, HighAvailabilityMode haMode) {
         if (mgmt==null) throw new NullPointerException("Must inject ManagementContext before preparing "+this);

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/39de027f/locations/jclouds/src/test/java/brooklyn/entity/rebind/persister/jclouds/JcloudsBlobStoreBasedObjectStoreTest.java
----------------------------------------------------------------------
diff --git a/locations/jclouds/src/test/java/brooklyn/entity/rebind/persister/jclouds/JcloudsBlobStoreBasedObjectStoreTest.java b/locations/jclouds/src/test/java/brooklyn/entity/rebind/persister/jclouds/JcloudsBlobStoreBasedObjectStoreTest.java
new file mode 100644
index 0000000..acb5df7
--- /dev/null
+++ b/locations/jclouds/src/test/java/brooklyn/entity/rebind/persister/jclouds/JcloudsBlobStoreBasedObjectStoreTest.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package brooklyn.entity.rebind.persister.jclouds;
+
+
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+import brooklyn.entity.basic.Entities;
+import brooklyn.entity.rebind.persister.BrooklynPersistenceUtils;
+import brooklyn.entity.rebind.persister.PersistenceObjectStore;
+import brooklyn.entity.rebind.persister.PersistenceObjectStore.StoreObjectAccessor;
+import brooklyn.management.internal.LocalManagementContext;
+import brooklyn.test.entity.LocalManagementContextForTests;
+import brooklyn.util.collections.MutableList;
+import brooklyn.util.text.Identifiers;
+import brooklyn.util.time.Duration;
+
+import com.google.common.base.Stopwatch;
+
+/**
+ * @author Andrea Turli
+ */
+@Test(groups={"Live", "Live-sanity"})
+public class JcloudsBlobStoreBasedObjectStoreTest {
+
+    private static final Logger log = LoggerFactory.getLogger(JcloudsBlobStoreBasedObjectStoreTest.class);
+    
+    private List<PersistenceObjectStore> objectStores = MutableList.of();;
+    private LocalManagementContext mgmt;
+
+    @BeforeMethod(alwaysRun=true)
+    public void setUp() throws Exception { 
+        mgmt = LocalManagementContextForTests.builder(true).useDefaultProperties().build();
+    }
+    
+    @AfterMethod(alwaysRun=true)
+    public void tearDown() throws Exception {
+        for (PersistenceObjectStore store: objectStores) store.deleteCompletely();
+        Entities.destroyAll(mgmt);
+        objectStores.clear();
+    }
+
+    public PersistenceObjectStore newObjectStore(String spec, String container) {
+        PersistenceObjectStore newObjectStore = BrooklynPersistenceUtils.newPersistenceObjectStore(mgmt, spec, container);
+        objectStores.add(newObjectStore);
+        return newObjectStore;
+    }
+    
+    @Test(groups={"Integration"})
+    public void testLocalhost() throws Exception {
+        doTestWithStore( newObjectStore(null, 
+            BlobStoreTest.CONTAINER_PREFIX+"-"+Identifiers.makeRandomId(4)) );
+    }
+    
+    @Test(groups={"Integration"})
+    public void testLocalhostWithSubPathInContainerName() throws Exception {
+        doTestWithStore( newObjectStore(null, 
+            BlobStoreTest.CONTAINER_PREFIX+"-"+Identifiers.makeRandomId(4)+"/subpath1/subpath2") );
+    }
+    
+    @Test(groups={"Live", "Live-sanity"})
+    public void testJclouds() throws Exception {
+        doTestWithStore( newObjectStore(BlobStoreTest.PERSIST_TO_OBJECT_STORE_FOR_TEST_SPEC, 
+            BlobStoreTest.CONTAINER_PREFIX+"-"+Identifiers.makeRandomId(4)) );
+    }
+    
+    @Test(groups={"Live", "Live-sanity"})
+    public void testJcloudsWithSubPathInContainerName() throws Exception {
+        doTestWithStore( newObjectStore(BlobStoreTest.PERSIST_TO_OBJECT_STORE_FOR_TEST_SPEC, 
+            BlobStoreTest.CONTAINER_PREFIX+"-"+Identifiers.makeRandomId(4)+"/subpath1/subpath2") );
+    }
+    
+    protected void doTestWithStore(PersistenceObjectStore objectStore) {
+        log.info("testing against "+objectStore.getSummaryName());
+        
+        objectStore.createSubPath("foo");
+        StoreObjectAccessor f = objectStore.newAccessor("foo/file1.txt");
+        Assert.assertFalse(f.exists());
+
+        Stopwatch timer = Stopwatch.createStarted();
+        f.append("Hello world");
+        log.info("created in "+Duration.of(timer));
+        timer.reset();
+        Assert.assertEquals(f.get(), "Hello world");
+        log.info("retrieved in "+Duration.of(timer));
+        Assert.assertTrue(f.exists());
+        
+        timer.reset();
+        List<String> files = objectStore.listContentsWithSubPath("foo");
+        log.info("list retrieved in "+Duration.of(timer)+"; is: "+files);
+        Assert.assertEquals(files, MutableList.of("foo/file1.txt"));
+        
+        f.delete();
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/39de027f/utils/common/src/main/java/brooklyn/util/text/Strings.java
----------------------------------------------------------------------
diff --git a/utils/common/src/main/java/brooklyn/util/text/Strings.java b/utils/common/src/main/java/brooklyn/util/text/Strings.java
index 03b7c67..7145eaa 100644
--- a/utils/common/src/main/java/brooklyn/util/text/Strings.java
+++ b/utils/common/src/main/java/brooklyn/util/text/Strings.java
@@ -150,7 +150,7 @@ public class Strings {
     public static String removeFromStart(String string, String ...prefixes) {
         if (isEmpty(string)) return string;
         for (String prefix : prefixes)
-            if (string.startsWith(prefix)) return string.substring(prefix.length());
+            if (prefix!=null && string.startsWith(prefix)) return string.substring(prefix.length());
         return string;
     }
 


[5/7] incubator-brooklyn git commit: This closes #344

Posted by he...@apache.org.
This closes #344


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

Branch: refs/heads/master
Commit: 3c940ea1f6ebf631931c205ac88deee4e18b8e2b
Parents: e3ee1dd 7503386
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Wed Nov 19 13:50:34 2014 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Wed Nov 19 13:50:34 2014 +0000

----------------------------------------------------------------------
 .../entity/basic/SoftwareProcessImpl.java       | 56 +++++++++++++++++---
 .../entity/database/mysql/MySqlNodeImpl.java    |  2 +-
 2 files changed, 49 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[6/7] incubator-brooklyn git commit: add comment for code review

Posted by he...@apache.org.
add comment for code review


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

Branch: refs/heads/master
Commit: 7c149066ff291639b9177d42b3b66d1d15c04e49
Parents: 2331fbc
Author: Alex Heneveld <al...@cloudsoftcorp.com>
Authored: Wed Nov 19 13:51:59 2014 +0000
Committer: Alex Heneveld <al...@cloudsoftcorp.com>
Committed: Wed Nov 19 13:51:59 2014 +0000

----------------------------------------------------------------------
 software/base/src/main/java/brooklyn/entity/java/UsesJmx.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/7c149066/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java
----------------------------------------------------------------------
diff --git a/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java b/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java
index 778fe1a..fd47eb3 100644
--- a/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java
+++ b/software/base/src/main/java/brooklyn/entity/java/UsesJmx.java
@@ -45,7 +45,7 @@ public interface UsesJmx extends UsesJava {
     PortAttributeSensorAndConfigKey JMX_PORT = new PortAttributeSensorAndConfigKey(
         "jmx.direct.port", "JMX direct/private port (e.g. JMX RMI server port, or JMXMP port, but not RMI registry port)", PortRanges.fromString("31001+"));
     
-    /** @deprecated since 0.7.0, kept for rebinding, use {@link #JMX_PORT} */ @Deprecated
+    /** @deprecated since 0.7.0, kept for rebinding with the anonymous class; code should only ever use {@link #JMX_PORT} */ @Deprecated
     PortAttributeSensorAndConfigKey JMX_PORT_LEGACY = new PortAttributeSensorAndConfigKey(
             "jmx.direct.port.legacy.NOT_USED", "Legacy definition JMX direct/private port (e.g. JMX RMI server port, or JMXMP port, but not RMI registry port)", PortRanges.fromString("31001+")) {
         private static final long serialVersionUID = 3846846080809179437L;
@@ -185,4 +185,4 @@ public interface UsesJmx extends UsesJava {
      *
      * -XX:+DisableAttachMechanism
      */
-}
\ No newline at end of file
+}