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

[1/7] incubator-brooklyn git commit: Apply creation script on all nodes, including slaves, before initializing replication

Repository: incubator-brooklyn
Updated Branches:
  refs/heads/master b118c1194 -> 324bb38be


Apply creation script on all nodes, including slaves, before initializing replication


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

Branch: refs/heads/master
Commit: ac7b9d1522026c5e26a181478fc1155e1254f8c8
Parents: e92dd1e
Author: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Authored: Wed Sep 16 08:49:15 2015 +0300
Committer: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Committed: Wed Sep 16 08:49:15 2015 +0300

----------------------------------------------------------------------
 .../entity/database/mysql/MySqlClusterImpl.java | 22 +++++---------------
 .../database/mysql/MySqlClusterTestHelper.java  |  3 +--
 2 files changed, 6 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ac7b9d15/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java
----------------------------------------------------------------------
diff --git a/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java b/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java
index ae75254..164bac4 100644
--- a/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java
+++ b/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java
@@ -63,7 +63,6 @@ import com.google.common.reflect.TypeToken;
 
 // https://dev.mysql.com/doc/refman/5.7/en/replication-howto.html
 
-// TODO CREATION_SCRIPT_CONTENTS executed before replication setup so it is not replicated to slaves
 // TODO Bootstrap slave from dump for the case where the binary log is purged
 // TODO Promote slave to master
 // TODO SSL connection between master and slave
@@ -150,7 +149,7 @@ public class MySqlClusterImpl extends DynamicClusterImpl implements MySqlCluster
     protected EntitySpec<?> getFirstMemberSpec() {
         final EntitySpec<?> firstMemberSpec = super.getFirstMemberSpec();
         if (firstMemberSpec != null) {
-            return applyDefaults(firstMemberSpec, Suppliers.ofInstance(MASTER_SERVER_ID), MASTER_CONFIG_URL, false);
+            return applyDefaults(firstMemberSpec, Suppliers.ofInstance(MASTER_SERVER_ID), MASTER_CONFIG_URL);
         }
 
         final EntitySpec<?> memberSpec = super.getMemberSpec();
@@ -176,24 +175,19 @@ public class MySqlClusterImpl extends DynamicClusterImpl implements MySqlCluster
 
         EntitySpec<?> spec = super.getMemberSpec();
         if (spec != null) {
-            return applyDefaults(spec, serverIdSupplier, SLAVE_CONFIG_URL, true);
+            return applyDefaults(spec, serverIdSupplier, SLAVE_CONFIG_URL);
         }
 
         return EntitySpec.create(MySqlNode.class)
                 .displayName("MySql Slave")
                 .configure(MySqlNode.MYSQL_SERVER_ID, serverIdSupplier.get())
-                .configure(MySqlNode.TEMPLATE_CONFIGURATION_URL, SLAVE_CONFIG_URL)
-                // block inheritance, only master should execute the creation script
-                .configure(MySqlNode.CREATION_SCRIPT_URL, (String) null)
-                .configure(MySqlNode.CREATION_SCRIPT_CONTENTS, (String) null);
+                .configure(MySqlNode.TEMPLATE_CONFIGURATION_URL, SLAVE_CONFIG_URL);
     }
 
-    private EntitySpec<?> applyDefaults(EntitySpec<?> spec, Supplier<Integer> serverId, String configUrl, boolean resetCreationScript) {
+    private EntitySpec<?> applyDefaults(EntitySpec<?> spec, Supplier<Integer> serverId, String configUrl) {
         boolean needsServerId = !isKeyConfigured(spec, MySqlNode.MYSQL_SERVER_ID);
         boolean needsConfigUrl = !isKeyConfigured(spec, MySqlNode.TEMPLATE_CONFIGURATION_URL.getConfigKey());
-        boolean needsCreationScriptUrl = resetCreationScript && !isKeyConfigured(spec, MySqlNode.CREATION_SCRIPT_URL);
-        boolean needsCreationScriptContents = resetCreationScript && !isKeyConfigured(spec, MySqlNode.CREATION_SCRIPT_CONTENTS);
-        if (needsServerId || needsConfigUrl || needsCreationScriptUrl || needsCreationScriptContents) {
+        if (needsServerId || needsConfigUrl) {
             EntitySpec<?> clonedSpec = EntitySpec.create(spec);
             if (needsServerId) {
                 clonedSpec.configure(MySqlNode.MYSQL_SERVER_ID, serverId.get());
@@ -201,12 +195,6 @@ public class MySqlClusterImpl extends DynamicClusterImpl implements MySqlCluster
             if (needsConfigUrl) {
                 clonedSpec.configure(MySqlNode.TEMPLATE_CONFIGURATION_URL, configUrl);
             }
-            if (needsCreationScriptUrl) {
-                clonedSpec.configure(MySqlNode.CREATION_SCRIPT_URL, (String) null);
-            }
-            if (needsCreationScriptContents) {
-                clonedSpec.configure(MySqlNode.CREATION_SCRIPT_URL, (String) null);
-            }
             return clonedSpec;
         } else {
             return spec;

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ac7b9d15/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java
----------------------------------------------------------------------
diff --git a/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java b/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java
index 0bdf7f6..dc13546 100644
--- a/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java
+++ b/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java
@@ -74,12 +74,11 @@ public class MySqlClusterTestHelper {
     public static void test(TestApplication app, Location location) throws Exception {
         MySqlCluster mysql = app.createAndManageChild(EntitySpec.create(MySqlCluster.class)
                 .configure(MySqlCluster.INITIAL_SIZE, 2)
+                .configure(MySqlNode.CREATION_SCRIPT_CONTENTS, CREATION_SCRIPT)
                 .configure(MySqlNode.MYSQL_SERVER_CONF, MutableMap.<String, Object>of("skip-name-resolve","")));
 
         app.start(ImmutableList.of(location));
         log.info("MySQL started");
-        MySqlNode masterEntity = (MySqlNode) mysql.getAttribute(MySqlCluster.FIRST);
-        masterEntity.invoke(MySqlNode.EXECUTE_SCRIPT, ImmutableMap.of("commands", CREATION_SCRIPT)).asTask().getUnchecked();
 
         VogellaExampleAccess masterDb = new VogellaExampleAccess("com.mysql.jdbc.Driver", mysql.getAttribute(MySqlNode.DATASTORE_URL));
         VogellaExampleAccess slaveDb = new VogellaExampleAccess("com.mysql.jdbc.Driver", Iterables.getOnlyElement(mysql.getAttribute(MySqlCluster.SLAVE_DATASTORE_URL_LIST)));


[2/7] incubator-brooklyn git commit: subscribe to the ROOT_USERNAME sensor from children of replica set

Posted by sj...@apache.org.
subscribe to the ROOT_USERNAME sensor from children of replica set


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

Branch: refs/heads/master
Commit: 3bb9cfc91a5b46a09890aa30dcd24acd3dfbbfab
Parents: 16b8b1e
Author: Robert Moss <ro...@gmail.com>
Authored: Tue Sep 15 15:38:29 2015 +0100
Committer: Robert Moss <ro...@gmail.com>
Committed: Wed Sep 16 15:35:28 2015 +0100

----------------------------------------------------------------------
 .../nosql/mongodb/MongoDBReplicaSetImpl.java       | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/3bb9cfc9/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
----------------------------------------------------------------------
diff --git a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
index c4d675d..59e6524 100644
--- a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
+++ b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
@@ -134,6 +134,23 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
             return Iterables.tryFind(entities, Predicates.not(IS_PRIMARY)).or(Iterables.get(entities, 0));
         }
     };
+    
+    @Override
+    public void init() {
+        addEnricher(Enrichers.builder()
+                .aggregating(MongoDBAuthenticationMixins.ROOT_USERNAME)
+                .publishing(MongoDBAuthenticationMixins.ROOT_USERNAME)
+                .fromMembers()
+                .valueToReportIfNoSensors(null)
+                .computing(new Function<Collection<String>, String>() {
+                        @Override
+                        public String apply(Collection<String> input) {
+                            if (input==null || input.isEmpty()) return null;
+                            // when authentication is used all members have the same value
+                            return Iterables.getFirst(input, null);
+                        }})
+                .build());
+    };
 
     /** @return {@link #NON_PRIMARY_REMOVAL_STRATEGY} */
     @Override


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

Posted by sj...@apache.org.
This closes #900


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

Branch: refs/heads/master
Commit: f8a07abe8b1bde6ddf0d263705a8b5699d7978d2
Parents: b118c11 e4067e5
Author: Sam Corbett <sa...@cloudsoftcorp.com>
Authored: Thu Sep 17 14:27:25 2015 +0100
Committer: Sam Corbett <sa...@cloudsoftcorp.com>
Committed: Thu Sep 17 14:27:25 2015 +0100

----------------------------------------------------------------------
 .../nosql/mongodb/AbstractMongoDBSshDriver.java |  8 +--
 .../entity/nosql/mongodb/MongoDBReplicaSet.java |  4 +-
 .../nosql/mongodb/MongoDBReplicaSetImpl.java    | 69 ++++++++++++++++----
 .../entity/nosql/mongodb/MongoDBServerImpl.java | 50 +++++++-------
 4 files changed, 89 insertions(+), 42 deletions(-)
----------------------------------------------------------------------



[5/7] incubator-brooklyn git commit: Master-only creation script for MySqlCluster

Posted by sj...@apache.org.
Master-only creation script for MySqlCluster


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

Branch: refs/heads/master
Commit: ed4a76e9975173f694381430dbb021f1762a8b58
Parents: ac7b9d1
Author: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Authored: Wed Sep 16 09:42:53 2015 +0300
Committer: Svetoslav Neykov <sv...@cloudsoftcorp.com>
Committed: Thu Sep 17 14:15:46 2015 +0300

----------------------------------------------------------------------
 .../entity/database/mysql/MySqlCluster.java     | 19 +++++++++++-----
 .../entity/database/mysql/MySqlClusterImpl.java | 21 +++++++++++++++++
 .../mysql/MySqlClusterIntegrationTest.java      | 24 ++++++++++++++++++--
 .../database/mysql/MySqlClusterTestHelper.java  | 13 ++++++++++-
 4 files changed, 68 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ed4a76e9/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlCluster.java
----------------------------------------------------------------------
diff --git a/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlCluster.java b/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlCluster.java
index d860d04..de43951 100644
--- a/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlCluster.java
+++ b/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlCluster.java
@@ -25,20 +25,27 @@ import org.apache.brooklyn.api.entity.ImplementedBy;
 import org.apache.brooklyn.api.sensor.AttributeSensor;
 import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.core.config.ConfigKeys;
-import org.apache.brooklyn.core.sensor.Sensors;
 import org.apache.brooklyn.core.sensor.BasicAttributeSensorAndConfigKey.StringAttributeSensorAndConfigKey;
-
-import com.google.common.reflect.TypeToken;
-
+import org.apache.brooklyn.core.sensor.Sensors;
 import org.apache.brooklyn.entity.database.DatastoreMixins.HasDatastoreUrl;
 import org.apache.brooklyn.entity.group.DynamicCluster;
 
+import com.google.common.reflect.TypeToken;
+
 @ImplementedBy(MySqlClusterImpl.class)
 @Catalog(name="MySql Master-Slave cluster", description="Sets up a cluster of MySQL nodes using master-slave relation and binary logging", iconUrl="classpath:///mysql-logo-110x57.png")
 public interface MySqlCluster extends DynamicCluster, HasDatastoreUrl {
     interface MySqlMaster {
-        AttributeSensor<String> MASTER_LOG_FILE = Sensors.newStringSensor("mysql.master.log_file", "The binary log file master is writing to");
-        AttributeSensor<Integer> MASTER_LOG_POSITION = Sensors.newIntegerSensor("mysql.master.log_position", "The position in the log file to start replication");
+        AttributeSensor<String> MASTER_LOG_FILE = Sensors.newStringSensor(
+                "mysql.master.log_file", "The binary log file master is writing to");
+        AttributeSensor<Integer> MASTER_LOG_POSITION = Sensors.newIntegerSensor(
+                "mysql.master.log_position", "The position in the log file to start replication");
+
+        ConfigKey<String> MASTER_CREATION_SCRIPT_CONTENTS = ConfigKeys.newStringConfigKey(
+                "datastore.master.creation.script.contents", "Contents of creation script to initialize the master node after initializing replication");
+
+        ConfigKey<String> MASTER_CREATION_SCRIPT_URL = ConfigKeys.newStringConfigKey(
+                "datastore.master.creation.script.url", "URL of creation script to use to initialize the master node after initializing replication (ignored if creationScriptContents is specified)");
     }
     interface MySqlSlave {
         AttributeSensor<Boolean> SLAVE_HEALTHY = Sensors.newBooleanSensor("mysql.slave.healthy", "Indicates that the replication state of the slave is healthy");

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ed4a76e9/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java
----------------------------------------------------------------------
diff --git a/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java b/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java
index 164bac4..63c5779 100644
--- a/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java
+++ b/software/database/src/main/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterImpl.java
@@ -24,6 +24,8 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import javax.annotation.Nullable;
+
 import org.apache.brooklyn.api.entity.Entity;
 import org.apache.brooklyn.api.entity.EntityLocal;
 import org.apache.brooklyn.api.entity.EntitySpec;
@@ -39,16 +41,19 @@ import org.apache.brooklyn.core.entity.lifecycle.ServiceStateLogic.ServiceNotUpL
 import org.apache.brooklyn.core.sensor.DependentConfiguration;
 import org.apache.brooklyn.core.sensor.Sensors;
 import org.apache.brooklyn.enricher.stock.Enrichers;
+import org.apache.brooklyn.entity.database.DatastoreMixins;
 import org.apache.brooklyn.entity.group.DynamicClusterImpl;
 import org.apache.brooklyn.feed.function.FunctionFeed;
 import org.apache.brooklyn.feed.function.FunctionPollConfig;
 import org.apache.brooklyn.util.collections.CollectionFunctionals;
+import org.apache.brooklyn.util.core.ResourceUtils;
 import org.apache.brooklyn.util.core.task.DynamicTasks;
 import org.apache.brooklyn.util.core.task.TaskBuilder;
 import org.apache.brooklyn.util.guava.Functionals;
 import org.apache.brooklyn.util.guava.IfFunctions;
 import org.apache.brooklyn.util.text.Identifiers;
 import org.apache.brooklyn.util.text.StringPredicates;
+import org.apache.brooklyn.util.text.Strings;
 import org.apache.brooklyn.util.time.Duration;
 
 import com.google.common.base.Function;
@@ -310,8 +315,24 @@ public class MySqlClusterImpl extends DynamicClusterImpl implements MySqlCluster
             if (position != null) {
                 ((EntityInternal)master).setAttribute(MySqlMaster.MASTER_LOG_POSITION, new Integer(position));
             }
+
+            //NOTE: Will be executed on each start, analogously to the standard CREATION_SCRIPT config
+            String creationScript = getDatabaseCreationScriptAsString(master);
+            if (creationScript != null) {
+                master.invoke(MySqlNode.EXECUTE_SCRIPT, ImmutableMap.of("commands", creationScript));
+            }
         }
 
+        @Nullable private static String getDatabaseCreationScriptAsString(Entity entity) {
+            String url = entity.getConfig(MySqlMaster.MASTER_CREATION_SCRIPT_URL);
+            if (!Strings.isBlank(url))
+                return new ResourceUtils(entity).getResourceAsString(url);
+            String contents = entity.getConfig(MySqlMaster.MASTER_CREATION_SCRIPT_CONTENTS);
+            if (!Strings.isBlank(contents))
+                return contents;
+            return null;
+        }
+        
         private void initSlave(MySqlNode slave) {
             MySqlNode master = (MySqlNode) Iterables.find(cluster.getMembers(), IS_MASTER);
             String masterLogFile = validateSqlParam(getAttributeBlocking(master, MySqlMaster.MASTER_LOG_FILE));

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ed4a76e9/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterIntegrationTest.java
----------------------------------------------------------------------
diff --git a/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterIntegrationTest.java b/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterIntegrationTest.java
index 1bf6ccb..c5e12d5 100644
--- a/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterIntegrationTest.java
+++ b/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterIntegrationTest.java
@@ -19,18 +19,20 @@
 package org.apache.brooklyn.entity.database.mysql;
 
 import org.apache.brooklyn.api.entity.Entity;
+import org.apache.brooklyn.api.location.Location;
 import org.apache.brooklyn.core.test.BrooklynAppLiveTestSupport;
 import org.apache.brooklyn.util.os.Os;
 import org.testng.annotations.Test;
 
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 
 public class MySqlClusterIntegrationTest extends BrooklynAppLiveTestSupport {
 
     @Test(groups = {"Integration"})
-    public void test_localhost() throws Exception {
+    public void testAllNodesInit() throws Exception {
         try {
-            MySqlClusterTestHelper.test(app, mgmt.getLocationRegistry().resolve("localhost"));
+            MySqlClusterTestHelper.test(app, getLocation());
         } finally {
             for (Entity member : Iterables.getOnlyElement(app.getChildren()).getChildren()) {
                 String runDir = member.getAttribute(MySqlNode.RUN_DIR);
@@ -40,4 +42,22 @@ public class MySqlClusterIntegrationTest extends BrooklynAppLiveTestSupport {
             }
         }
     }
+
+    @Test(groups = {"Integration"})
+    public void testMasterInit() throws Exception {
+        try {
+            MySqlClusterTestHelper.testMasterInit(app, getLocation());
+        } finally {
+            for (Entity member : Iterables.getOnlyElement(app.getChildren()).getChildren()) {
+                String runDir = member.getAttribute(MySqlNode.RUN_DIR);
+                if (runDir != null) {
+                    Os.deleteRecursively(runDir);
+                }
+            }
+        }
+    }
+
+    private Location getLocation() {
+        return mgmt.getLocationRegistry().resolve("localhost");
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/ed4a76e9/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java
----------------------------------------------------------------------
diff --git a/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java b/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java
index dc13546..43a3b70 100644
--- a/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java
+++ b/software/database/src/test/java/org/apache/brooklyn/entity/database/mysql/MySqlClusterTestHelper.java
@@ -37,6 +37,7 @@ import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterables;
 
 import org.apache.brooklyn.entity.database.VogellaExampleAccess;
+import org.apache.brooklyn.entity.database.mysql.MySqlCluster.MySqlMaster;
 
 /**
  * Runs a slightly modified version of the popular Vogella MySQL tutorial,
@@ -72,11 +73,21 @@ public class MySqlClusterTestHelper {
             ));
 
     public static void test(TestApplication app, Location location) throws Exception {
-        MySqlCluster mysql = app.createAndManageChild(EntitySpec.create(MySqlCluster.class)
+        test(app, location, EntitySpec.create(MySqlCluster.class)
                 .configure(MySqlCluster.INITIAL_SIZE, 2)
                 .configure(MySqlNode.CREATION_SCRIPT_CONTENTS, CREATION_SCRIPT)
                 .configure(MySqlNode.MYSQL_SERVER_CONF, MutableMap.<String, Object>of("skip-name-resolve","")));
+    }
+
+    public static void testMasterInit(TestApplication app, Location location) throws Exception {
+        test(app, location, EntitySpec.create(MySqlCluster.class)
+                .configure(MySqlCluster.INITIAL_SIZE, 2)
+                .configure(MySqlMaster.MASTER_CREATION_SCRIPT_CONTENTS, CREATION_SCRIPT)
+                .configure(MySqlNode.MYSQL_SERVER_CONF, MutableMap.<String, Object>of("skip-name-resolve","")));
+    }
 
+    public static void test(TestApplication app, Location location, EntitySpec<MySqlCluster> clusterSpec) throws Exception {
+        MySqlCluster mysql = app.createAndManageChild(clusterSpec);
         app.start(ImmutableList.of(location));
         log.info("MySQL started");
 


[4/7] incubator-brooklyn git commit: addresses PR review comments

Posted by sj...@apache.org.
addresses PR review comments


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

Branch: refs/heads/master
Commit: e4067e58bb293fea00ad9a60da52b1da1f2667cf
Parents: 3bb9cfc
Author: Robert Moss <ro...@gmail.com>
Authored: Wed Sep 16 15:04:42 2015 +0100
Committer: Robert Moss <ro...@gmail.com>
Committed: Wed Sep 16 15:35:29 2015 +0100

----------------------------------------------------------------------
 .../nosql/mongodb/MongoDBReplicaSetImpl.java    | 60 +++++++++++---------
 1 file changed, 32 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/e4067e58/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
----------------------------------------------------------------------
diff --git a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
index 59e6524..2c95d02 100644
--- a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
+++ b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
@@ -54,7 +54,7 @@ import org.apache.brooklyn.util.text.Strings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.api.client.util.Sets;
+import com.google.common.collect.Sets;
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
@@ -142,15 +142,17 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
                 .publishing(MongoDBAuthenticationMixins.ROOT_USERNAME)
                 .fromMembers()
                 .valueToReportIfNoSensors(null)
-                .computing(new Function<Collection<String>, String>() {
-                        @Override
-                        public String apply(Collection<String> input) {
-                            if (input==null || input.isEmpty()) return null;
-                            // when authentication is used all members have the same value
-                            return Iterables.getFirst(input, null);
-                        }})
+                .computing(new RootUsernameReducer())
                 .build());
-    };
+    }
+
+    public static class RootUsernameReducer implements Function<Collection<String>, String>{
+        @Override
+        public String apply(Collection<String> input) {
+            // when authentication is used all members have the same value
+            return (input == null || input.isEmpty()) ? null : Iterables.getFirst(input, null);
+        };
+    }
 
     /** @return {@link #NON_PRIMARY_REMOVAL_STRATEGY} */
     @Override
@@ -397,26 +399,9 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
                 .build());
         
         addEnricher(Enrichers.builder()
-                .aggregating(MongoDBServer.MONGO_SERVER_ENDPOINT)
+                .transforming(REPLICA_SET_ENDPOINTS)
                 .publishing(DATASTORE_URL)
-                .fromMembers()
-                .valueToReportIfNoSensors(null)
-                .computing(new Function<Collection<String>, String>() {
-                        @Override
-                        public String apply(Collection<String> input) {
-                            Set<String> endpoints = Sets.newHashSet();
-                            for (String endpoint: input) {
-                                if (!Strings.isBlank(endpoint)) {
-                                    
-                                    endpoints.add(endpoint);
-                                }
-                            }
-                            String credentials = MongoDBAuthenticationUtils.usesAuthentication(MongoDBReplicaSetImpl.this) ? 
-                                    String.format("%s:%s@", 
-                                            config().get(MongoDBAuthenticationMixins.ROOT_USERNAME), 
-                                            config().get(MongoDBAuthenticationMixins.ROOT_PASSWORD)) : "";
-                            return String.format("mongodb://%s%s", credentials, Strings.join(endpoints, ","));
-                        }})
+                .computing(new EndpointsToDatastoreUrlMapper(this))
                 .build());
 
         subscribeToMembers(this, MongoDBServer.IS_PRIMARY_FOR_REPLICA_SET, new SensorEventListener<Boolean>() {
@@ -427,6 +412,25 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
         });
 
     }
+    
+    public static class EndpointsToDatastoreUrlMapper implements Function<Collection<String>, String> {
+        
+        private Entity entity;
+
+        public EndpointsToDatastoreUrlMapper(Entity entity) {
+            this.entity = entity;
+        }
+        
+        @Override
+        public String apply(Collection<String> input) {
+            String credentials = MongoDBAuthenticationUtils.usesAuthentication(entity) 
+                    ? String.format("%s:%s@", 
+                            entity.config().get(MongoDBAuthenticationMixins.ROOT_USERNAME), 
+                            entity.config().get(MongoDBAuthenticationMixins.ROOT_PASSWORD)) 
+                    : "";
+            return String.format("mongodb://%s%s", credentials, Strings.join(input, ","));
+        }
+    }
 
     @Override
     public void stop() {


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

Posted by sj...@apache.org.
This closes #903


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

Branch: refs/heads/master
Commit: 324bb38be6db5b093638e08a033a526830c6e7e0
Parents: f8a07ab ed4a76e
Author: Sam Corbett <sa...@cloudsoftcorp.com>
Authored: Thu Sep 17 14:27:30 2015 +0100
Committer: Sam Corbett <sa...@cloudsoftcorp.com>
Committed: Thu Sep 17 14:27:30 2015 +0100

----------------------------------------------------------------------
 .../entity/database/mysql/MySqlCluster.java     | 19 ++++++---
 .../entity/database/mysql/MySqlClusterImpl.java | 43 ++++++++++++--------
 .../mysql/MySqlClusterIntegrationTest.java      | 24 ++++++++++-
 .../database/mysql/MySqlClusterTestHelper.java  | 16 ++++++--
 4 files changed, 74 insertions(+), 28 deletions(-)
----------------------------------------------------------------------



[3/7] incubator-brooklyn git commit: makes connection string available on MongoDbReplicaSet

Posted by sj...@apache.org.
makes connection string available on MongoDbReplicaSet


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

Branch: refs/heads/master
Commit: 16b8b1e393d525c88ba5b540ad880dab899888d6
Parents: b3ce256
Author: Robert Moss <ro...@gmail.com>
Authored: Tue Sep 15 15:06:43 2015 +0100
Committer: Robert Moss <ro...@gmail.com>
Committed: Wed Sep 16 15:35:28 2015 +0100

----------------------------------------------------------------------
 .../nosql/mongodb/AbstractMongoDBSshDriver.java |  8 ++--
 .../entity/nosql/mongodb/MongoDBReplicaSet.java |  4 +-
 .../nosql/mongodb/MongoDBReplicaSetImpl.java    | 48 ++++++++++++++-----
 .../entity/nosql/mongodb/MongoDBServerImpl.java | 50 ++++++++++----------
 4 files changed, 68 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/16b8b1e3/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/AbstractMongoDBSshDriver.java
----------------------------------------------------------------------
diff --git a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/AbstractMongoDBSshDriver.java b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/AbstractMongoDBSshDriver.java
index c182355..14c495e 100644
--- a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/AbstractMongoDBSshDriver.java
+++ b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/AbstractMongoDBSshDriver.java
@@ -27,15 +27,15 @@ import org.apache.brooklyn.api.location.OsDetails;
 import org.apache.brooklyn.core.entity.Entities;
 import org.apache.brooklyn.entity.software.base.AbstractSoftwareProcessSshDriver;
 import org.apache.brooklyn.entity.software.base.lifecycle.ScriptHelper;
-import org.apache.brooklyn.util.core.internal.ssh.SshTool;
-import org.apache.brooklyn.util.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.brooklyn.location.ssh.SshMachineLocation;
+import org.apache.brooklyn.util.core.internal.ssh.SshTool;
 import org.apache.brooklyn.util.exceptions.Exceptions;
 import org.apache.brooklyn.util.net.Networking;
 import org.apache.brooklyn.util.os.Os;
 import org.apache.brooklyn.util.ssh.BashCommands;
+import org.apache.brooklyn.util.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Strings;

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/16b8b1e3/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSet.java
----------------------------------------------------------------------
diff --git a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSet.java b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSet.java
index 12bbe87..6ebc17e 100644
--- a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSet.java
+++ b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSet.java
@@ -26,6 +26,7 @@ import org.apache.brooklyn.api.sensor.AttributeSensor;
 import org.apache.brooklyn.config.ConfigKey;
 import org.apache.brooklyn.core.config.ConfigKeys;
 import org.apache.brooklyn.core.sensor.Sensors;
+import org.apache.brooklyn.entity.database.DatastoreMixins.HasDatastoreUrl;
 import org.apache.brooklyn.entity.group.Cluster;
 import org.apache.brooklyn.entity.group.DynamicCluster;
 import org.apache.brooklyn.util.core.flags.SetFromFlag;
@@ -46,7 +47,7 @@ import com.google.common.reflect.TypeToken;
  * @see <a href="http://docs.mongodb.org/manual/replication/">http://docs.mongodb.org/manual/replication/</a>
  */
 @ImplementedBy(MongoDBReplicaSetImpl.class)
-public interface MongoDBReplicaSet extends DynamicCluster, MongoDBAuthenticationMixins {
+public interface MongoDBReplicaSet extends DynamicCluster, MongoDBAuthenticationMixins, HasDatastoreUrl {
 
     @SetFromFlag("replicaSetName")
     ConfigKey<String> REPLICA_SET_NAME = ConfigKeys.newStringConfigKey(
@@ -60,6 +61,7 @@ public interface MongoDBReplicaSet extends DynamicCluster, MongoDBAuthentication
     @SuppressWarnings("serial")
     AttributeSensor<List<String>> REPLICA_SET_ENDPOINTS = Sensors.newSensor(new TypeToken<List<String>>() {}, 
         "mongodb.replicaSet.endpoints", "Endpoints active for this replica set");
+    
 
     /**
      * The name of the replica set.

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/16b8b1e3/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
----------------------------------------------------------------------
diff --git a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
index f96a56a..c4d675d 100644
--- a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
+++ b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBReplicaSetImpl.java
@@ -54,6 +54,7 @@ import org.apache.brooklyn.util.text.Strings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.api.client.util.Sets;
 import com.google.common.base.Function;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
@@ -108,7 +109,7 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
         @Override public boolean apply(@Nullable Entity input) {
             return input != null
                     && input instanceof MongoDBServer
-                    && ReplicaSetMemberStatus.PRIMARY.equals(input.getAttribute(MongoDBServer.REPLICA_SET_MEMBER_STATUS));
+                    && ReplicaSetMemberStatus.PRIMARY.equals(input.sensors().get(MongoDBServer.REPLICA_SET_MEMBER_STATUS));
         }
     };
 
@@ -118,7 +119,7 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
             // getSecondaries relies on instanceof check
             return input != null
                     && input instanceof MongoDBServer
-                    && ReplicaSetMemberStatus.SECONDARY.equals(input.getAttribute(MongoDBServer.REPLICA_SET_MEMBER_STATUS));
+                    && ReplicaSetMemberStatus.SECONDARY.equals(input.sensors().get(MongoDBServer.REPLICA_SET_MEMBER_STATUS));
         }
     };
 
@@ -165,7 +166,7 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
     @Override
     public String getName() {
         // FIXME: Names must be unique if the replica sets are used in a sharded cluster
-        return getConfig(REPLICA_SET_NAME) + this.getId();
+        return config().get(REPLICA_SET_NAME) + this.getId();
     }
 
     @Override
@@ -197,7 +198,7 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
      */
     private void serverAdded(MongoDBServer server) {
         try {
-            LOG.debug("Server added: {}. SERVICE_UP: {}", server, server.getAttribute(MongoDBServer.SERVICE_UP));
+            LOG.debug("Server added: {}. SERVICE_UP: {}", server, server.sensors().get(MongoDBServer.SERVICE_UP));
 
             // Set the primary if the replica set hasn't been initialised.
             if (mustInitialise.compareAndSet(true, false)) {
@@ -205,8 +206,8 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
                     LOG.info("First server up in {} is: {}", getName(), server);
                 boolean replicaSetInitialised = server.initializeReplicaSet(getName(), nextMemberId.getAndIncrement());
                 if (replicaSetInitialised) {
-                    setAttribute(PRIMARY_ENTITY, server);
-                    setAttribute(Startable.SERVICE_UP, true);
+                    sensors().set(PRIMARY_ENTITY, server);
+                    sensors().set(Startable.SERVICE_UP, true);
                 } else {
                     ServiceStateLogic.ServiceNotUpLogic.updateNotUpIndicator(this, "initialization", "replicaset failed to initialize");
                     ServiceStateLogic.setExpectedState(this, Lifecycle.ON_FIRE);
@@ -234,7 +235,7 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
             @Override
             public void run() {
                 // SERVICE_UP is not guaranteed when additional members are added to the set.
-                Boolean isAvailable = secondary.getAttribute(MongoDBServer.SERVICE_UP);
+                Boolean isAvailable = secondary.sensors().get(MongoDBServer.SERVICE_UP);
                 MongoDBServer primary = getPrimary();
                 boolean reschedule;
                 if (Boolean.TRUE.equals(isAvailable) && primary != null) {
@@ -278,14 +279,14 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
             if (LOG.isDebugEnabled())
                 LOG.debug("Scheduling removal of member from {}: {}", getName(), member);
             // FIXME is there a chance of race here?
-            if (member.equals(getAttribute(PRIMARY_ENTITY)))
-                setAttribute(PRIMARY_ENTITY, null);
+            if (member.equals(sensors().get(PRIMARY_ENTITY)))
+                sensors().set(PRIMARY_ENTITY, null);
             executor.submit(new Runnable() {
                 @Override
                 public void run() {
                     // Wait until the server has been stopped before reconfiguring the set. Quoth the MongoDB doc:
                     // for best results always shut down the mongod instance before removing it from a replica set.
-                    Boolean isAvailable = member.getAttribute(MongoDBServer.SERVICE_UP);
+                    Boolean isAvailable = member.sensors().get(MongoDBServer.SERVICE_UP);
                     // Wait for the replica set to elect a new primary if the set is reconfiguring itself.
                     MongoDBServer primary = getPrimary();
                     boolean reschedule;
@@ -377,11 +378,34 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
                             return MutableList.copyOf(endpoints);
                         }})
                 .build());
+        
+        addEnricher(Enrichers.builder()
+                .aggregating(MongoDBServer.MONGO_SERVER_ENDPOINT)
+                .publishing(DATASTORE_URL)
+                .fromMembers()
+                .valueToReportIfNoSensors(null)
+                .computing(new Function<Collection<String>, String>() {
+                        @Override
+                        public String apply(Collection<String> input) {
+                            Set<String> endpoints = Sets.newHashSet();
+                            for (String endpoint: input) {
+                                if (!Strings.isBlank(endpoint)) {
+                                    
+                                    endpoints.add(endpoint);
+                                }
+                            }
+                            String credentials = MongoDBAuthenticationUtils.usesAuthentication(MongoDBReplicaSetImpl.this) ? 
+                                    String.format("%s:%s@", 
+                                            config().get(MongoDBAuthenticationMixins.ROOT_USERNAME), 
+                                            config().get(MongoDBAuthenticationMixins.ROOT_PASSWORD)) : "";
+                            return String.format("mongodb://%s%s", credentials, Strings.join(endpoints, ","));
+                        }})
+                .build());
 
         subscribeToMembers(this, MongoDBServer.IS_PRIMARY_FOR_REPLICA_SET, new SensorEventListener<Boolean>() {
             @Override public void onEvent(SensorEvent<Boolean> event) {
                 if (Boolean.TRUE == event.getValue())
-                    setAttribute(PRIMARY_ENTITY, (MongoDBServer)event.getSource());
+                    sensors().set(PRIMARY_ENTITY, (MongoDBServer)event.getSource());
             }
         });
 
@@ -396,7 +420,7 @@ public class MongoDBReplicaSetImpl extends DynamicClusterImpl implements MongoDB
         // TODO Note that after this the executor will not run if the set is restarted.
         executor.shutdownNow();
         super.stop();
-        setAttribute(Startable.SERVICE_UP, false);
+        sensors().set(Startable.SERVICE_UP, false);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-brooklyn/blob/16b8b1e3/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBServerImpl.java
----------------------------------------------------------------------
diff --git a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBServerImpl.java b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBServerImpl.java
index 941dd8e..040199b 100644
--- a/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBServerImpl.java
+++ b/software/nosql/src/main/java/org/apache/brooklyn/entity/nosql/mongodb/MongoDBServerImpl.java
@@ -62,13 +62,13 @@ public class MongoDBServerImpl extends SoftwareProcessImpl implements MongoDBSer
         super.connectSensors();
         connectServiceUpIsRunning();
 
-        int port = getAttribute(MongoDBServer.PORT);
+        int port = sensors().get(MongoDBServer.PORT);
         HostAndPort accessibleAddress = BrooklynAccessUtils.getBrooklynAccessibleAddress(this, port);
-        setAttribute(MONGO_SERVER_ENDPOINT, String.format("http://%s:%d",
+        sensors().set(MONGO_SERVER_ENDPOINT, String.format("%s:%d",
                 accessibleAddress.getHostText(), accessibleAddress.getPort()));
 
-        int httpConsolePort = BrooklynAccessUtils.getBrooklynAccessibleAddress(this, getAttribute(HTTP_PORT)).getPort();
-        setAttribute(HTTP_INTERFACE_URL, String.format("http://%s:%d",
+        int httpConsolePort = BrooklynAccessUtils.getBrooklynAccessibleAddress(this, sensors().get(HTTP_PORT)).getPort();
+        sensors().set(HTTP_INTERFACE_URL, String.format("http://%s:%d",
                 accessibleAddress.getHostText(), httpConsolePort));
 
         try {
@@ -85,7 +85,7 @@ public class MongoDBServerImpl extends SoftwareProcessImpl implements MongoDBSer
                         .callable(new Callable<BasicBSONObject>() {
                             @Override
                             public BasicBSONObject call() throws Exception {
-                                return MongoDBServerImpl.this.getAttribute(SERVICE_UP)
+                                return MongoDBServerImpl.this.sensors().get(SERVICE_UP)
                                     ? client.getServerStatus()
                                     : null;
                             }
@@ -117,8 +117,8 @@ public class MongoDBServerImpl extends SoftwareProcessImpl implements MongoDBSer
                             .suppressDuplicates(true))
                     .build();
         } else {
-            setAttribute(IS_PRIMARY_FOR_REPLICA_SET, false);
-            setAttribute(IS_SECONDARY_FOR_REPLICA_SET, false);
+            sensors().set(IS_PRIMARY_FOR_REPLICA_SET, false);
+            sensors().set(IS_SECONDARY_FOR_REPLICA_SET, false);
         }
 
         // Take interesting details from STATUS.
@@ -126,29 +126,29 @@ public class MongoDBServerImpl extends SoftwareProcessImpl implements MongoDBSer
                 @Override public void onEvent(SensorEvent<BasicBSONObject> event) {
                     BasicBSONObject map = event.getValue();
                     if (map != null && !map.isEmpty()) {
-                        setAttribute(UPTIME_SECONDS, map.getDouble("uptime", 0));
+                        sensors().set(UPTIME_SECONDS, map.getDouble("uptime", 0));
 
                         // Operations
                         BasicBSONObject opcounters = (BasicBSONObject) map.get("opcounters");
-                        setAttribute(OPCOUNTERS_INSERTS, opcounters.getLong("insert", 0));
-                        setAttribute(OPCOUNTERS_QUERIES, opcounters.getLong("query", 0));
-                        setAttribute(OPCOUNTERS_UPDATES, opcounters.getLong("update", 0));
-                        setAttribute(OPCOUNTERS_DELETES, opcounters.getLong("delete", 0));
-                        setAttribute(OPCOUNTERS_GETMORE, opcounters.getLong("getmore", 0));
-                        setAttribute(OPCOUNTERS_COMMAND, opcounters.getLong("command", 0));
+                        sensors().set(OPCOUNTERS_INSERTS, opcounters.getLong("insert", 0));
+                        sensors().set(OPCOUNTERS_QUERIES, opcounters.getLong("query", 0));
+                        sensors().set(OPCOUNTERS_UPDATES, opcounters.getLong("update", 0));
+                        sensors().set(OPCOUNTERS_DELETES, opcounters.getLong("delete", 0));
+                        sensors().set(OPCOUNTERS_GETMORE, opcounters.getLong("getmore", 0));
+                        sensors().set(OPCOUNTERS_COMMAND, opcounters.getLong("command", 0));
 
                         // Network stats
                         BasicBSONObject network = (BasicBSONObject) map.get("network");
-                        setAttribute(NETWORK_BYTES_IN, network.getLong("bytesIn", 0));
-                        setAttribute(NETWORK_BYTES_OUT, network.getLong("bytesOut", 0));
-                        setAttribute(NETWORK_NUM_REQUESTS, network.getLong("numRequests", 0));
+                        sensors().set(NETWORK_BYTES_IN, network.getLong("bytesIn", 0));
+                        sensors().set(NETWORK_BYTES_OUT, network.getLong("bytesOut", 0));
+                        sensors().set(NETWORK_NUM_REQUESTS, network.getLong("numRequests", 0));
 
                         // Replica set stats
                         BasicBSONObject repl = (BasicBSONObject) map.get("repl");
                         if (isReplicaSetMember() && repl != null) {
-                            setAttribute(IS_PRIMARY_FOR_REPLICA_SET, repl.getBoolean("ismaster"));
-                            setAttribute(IS_SECONDARY_FOR_REPLICA_SET, repl.getBoolean("secondary"));
-                            setAttribute(REPLICA_SET_PRIMARY_ENDPOINT, repl.getString("primary"));
+                            sensors().set(IS_PRIMARY_FOR_REPLICA_SET, repl.getBoolean("ismaster"));
+                            sensors().set(IS_SECONDARY_FOR_REPLICA_SET, repl.getBoolean("secondary"));
+                            sensors().set(REPLICA_SET_PRIMARY_ENDPOINT, repl.getString("primary"));
                         }
                     }
                 }
@@ -165,7 +165,7 @@ public class MongoDBServerImpl extends SoftwareProcessImpl implements MongoDBSer
 
     @Override
     public MongoDBReplicaSet getReplicaSet() {
-        return getConfig(MongoDBServer.REPLICA_SET);
+        return config().get(MongoDBServer.REPLICA_SET);
     }
 
     @Override
@@ -186,7 +186,7 @@ public class MongoDBServerImpl extends SoftwareProcessImpl implements MongoDBSer
         // The ReplicaSet uses REPLICA_SET_MEMBER_STATUS to determine which node to call.
         // 
         // Relying on caller to respect the `false` result, to retry.
-        if (!getAttribute(IS_PRIMARY_FOR_REPLICA_SET)) {
+        if (!sensors().get(IS_PRIMARY_FOR_REPLICA_SET)) {
             LOG.warn("Attempted to add {} to replica set at server that is not primary: {}", secondary, this);
             return false;
         }
@@ -195,7 +195,7 @@ public class MongoDBServerImpl extends SoftwareProcessImpl implements MongoDBSer
 
     @Override
     public boolean removeMemberFromReplicaSet(MongoDBServer server) {
-        if (!getAttribute(IS_PRIMARY_FOR_REPLICA_SET)) {
+        if (!sensors().get(IS_PRIMARY_FOR_REPLICA_SET)) {
             LOG.warn("Attempted to remove {} from replica set at server that is not primary: {}", server, this);
             return false;
         }
@@ -206,8 +206,8 @@ public class MongoDBServerImpl extends SoftwareProcessImpl implements MongoDBSer
     public String toString() {
         return Objects.toStringHelper(this)
                 .add("id", getId())
-                .add("hostname", getAttribute(HOSTNAME))
-                .add("port", getAttribute(PORT))
+                .add("hostname", sensors().get(HOSTNAME))
+                .add("port", sensors().get(PORT))
                 .toString();
     }
 }