You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@whirr.apache.org by as...@apache.org on 2011/08/23 09:55:38 UTC

svn commit: r1160573 - in /incubator/whirr/trunk: ./ bin/ cli/ recipes/ services/ganglia/ services/ganglia/src/ services/ganglia/src/main/ services/ganglia/src/main/java/ services/ganglia/src/main/java/org/ services/ganglia/src/main/java/org/apache/ se...

Author: asavu
Date: Tue Aug 23 07:55:36 2011
New Revision: 1160573

URL: http://svn.apache.org/viewvc?rev=1160573&view=rev
Log:
WHIRR-258. Add Ganglia as a service (Karel Vervaeke via asavu)

Added:
    incubator/whirr/trunk/recipes/ganglia-byon.properties
    incubator/whirr/trunk/recipes/ganglia-ec2.properties
    incubator/whirr/trunk/services/ganglia/   (with props)
    incubator/whirr/trunk/services/ganglia/pom.xml
    incubator/whirr/trunk/services/ganglia/src/
    incubator/whirr/trunk/services/ganglia/src/main/
    incubator/whirr/trunk/services/ganglia/src/main/java/
    incubator/whirr/trunk/services/ganglia/src/main/java/org/
    incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/
    incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/
    incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/
    incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/
    incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaCluster.java
    incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMetadClusterActionHandler.java
    incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMonitorClusterActionHandler.java
    incubator/whirr/trunk/services/ganglia/src/main/resources/
    incubator/whirr/trunk/services/ganglia/src/main/resources/META-INF/
    incubator/whirr/trunk/services/ganglia/src/main/resources/META-INF/services/
    incubator/whirr/trunk/services/ganglia/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler
    incubator/whirr/trunk/services/ganglia/src/main/resources/functions/
    incubator/whirr/trunk/services/ganglia/src/main/resources/functions/configure_ganglia.sh
    incubator/whirr/trunk/services/ganglia/src/main/resources/functions/install_ganglia.sh
    incubator/whirr/trunk/services/ganglia/src/main/resources/whirr-ganglia-default.properties
    incubator/whirr/trunk/services/ganglia/src/test/
    incubator/whirr/trunk/services/ganglia/src/test/java/
    incubator/whirr/trunk/services/ganglia/src/test/java/org/
    incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/
    incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/whirr/
    incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/whirr/service/
    incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/whirr/service/ganglia/
    incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/whirr/service/ganglia/integration/
    incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/whirr/service/ganglia/integration/GangliaServiceTest.java
    incubator/whirr/trunk/services/ganglia/src/test/resources/
    incubator/whirr/trunk/services/ganglia/src/test/resources/log4j.xml
    incubator/whirr/trunk/services/ganglia/src/test/resources/whirr-ganglia-test.properties
Modified:
    incubator/whirr/trunk/CHANGES.txt
    incubator/whirr/trunk/bin/whirr
    incubator/whirr/trunk/cli/pom.xml
    incubator/whirr/trunk/pom.xml

Modified: incubator/whirr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/CHANGES.txt?rev=1160573&r1=1160572&r2=1160573&view=diff
==============================================================================
--- incubator/whirr/trunk/CHANGES.txt (original)
+++ incubator/whirr/trunk/CHANGES.txt Tue Aug 23 07:55:36 2011
@@ -2,6 +2,10 @@ Apache Whirr Change Log
 
 Trunk (unreleased changes)
 
+  NEW FEATURES
+
+    WHIRR-258. Add Ganglia as a service (Karel Vervaeke via asavu)
+
 Release 0.6.0 - 2011-08-17
 
   NEW FEATURES

Modified: incubator/whirr/trunk/bin/whirr
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/bin/whirr?rev=1160573&r1=1160572&r2=1160573&view=diff
==============================================================================
--- incubator/whirr/trunk/bin/whirr (original)
+++ incubator/whirr/trunk/bin/whirr Tue Aug 23 07:55:36 2011
@@ -25,5 +25,5 @@ else
   classpath="$bin/../conf/:$bin/..:$bin/../lib/*"
 fi
 
-java -cp "$classpath" $opts org.apache.whirr.cli.Main "$@"
-
+java $WHIRR_CLI_OPTS -cp "$classpath" org.apache.whirr.cli.Main "$@"
+ 

Modified: incubator/whirr/trunk/cli/pom.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/pom.xml?rev=1160573&r1=1160572&r2=1160573&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/pom.xml (original)
+++ incubator/whirr/trunk/cli/pom.xml Tue Aug 23 07:55:36 2011
@@ -51,6 +51,11 @@
     </dependency>
     <dependency>
       <groupId>${project.groupId}</groupId>
+      <artifactId>whirr-ganglia</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>${project.groupId}</groupId>
       <artifactId>whirr-hadoop</artifactId>
       <version>${project.version}</version>
     </dependency>

Modified: incubator/whirr/trunk/pom.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/pom.xml?rev=1160573&r1=1160572&r2=1160573&view=diff
==============================================================================
--- incubator/whirr/trunk/pom.xml (original)
+++ incubator/whirr/trunk/pom.xml Tue Aug 23 07:55:36 2011
@@ -42,6 +42,7 @@
     <module>examples</module>
     <module>services/cassandra</module>
     <module>services/cdh</module>
+    <module>services/ganglia</module>
     <module>services/hadoop</module>
     <module>services/zookeeper</module>
     <module>services/hbase</module>
@@ -125,6 +126,11 @@
         <scope>test</scope>
       </dependency>
       <dependency>
+        <groupId>commons-httpclient</groupId>
+        <artifactId>commons-httpclient</artifactId>
+        <version>3.1</version>
+      </dependency>
+      <dependency>
         <groupId>commons-configuration</groupId>
         <artifactId>commons-configuration</artifactId>
         <version>1.6</version>
@@ -356,6 +362,8 @@
             <exclude>docs/**</exclude>
             <exclude>**/*.log*</exclude>
             <exclude>.idea/**</exclude>
+            <exclude>**/.project</exclude>
+            <exclude>**/.classpath</exclude>
             <exclude>src/site/resources/images/whirr-logo.ai</exclude> <!-- binary -->
           </excludes>
         </configuration>

Added: incubator/whirr/trunk/recipes/ganglia-byon.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/recipes/ganglia-byon.properties?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/recipes/ganglia-byon.properties (added)
+++ incubator/whirr/trunk/recipes/ganglia-byon.properties Tue Aug 23 07:55:36 2011
@@ -0,0 +1,43 @@
+#
+# 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.
+#
+
+#
+# ZooKeeper Cluster on BYON provider
+# 
+
+# Read the Configuration Guide for more info:
+# http://incubator.apache.org/whirr/configuration-guide.html
+
+# Change the cluster name here
+whirr.cluster-name=ganglia
+
+# Change the number of machines in the cluster here
+whirr.instance-templates=1 ganglia-metad, 1 ganglia-monitor
+
+# Node connection details
+whirr.service-name=byon
+whirr.provider=byon
+
+whirr.identity=notused
+whirr.credential=notused
+
+jclouds.byon.endpoint=file://${sys:user.dir}/recipes/nodes-byon.yaml
+
+# By default use the user system SSH keys. Override them here.
+# whirr.private-key-file=${sys:user.home}/.ssh/id_rsa
+# whirr.public-key-file=${whirr.private-key-file}.pub
+

Added: incubator/whirr/trunk/recipes/ganglia-ec2.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/recipes/ganglia-ec2.properties?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/recipes/ganglia-ec2.properties (added)
+++ incubator/whirr/trunk/recipes/ganglia-ec2.properties Tue Aug 23 07:55:36 2011
@@ -0,0 +1,40 @@
+#
+# 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.
+#
+
+#
+# ZooKeeper Cluster on AWS EC2
+# 
+
+# Read the Configuration Guide for more info:
+# http://incubator.apache.org/whirr/configuration-guide.html
+
+# Change the cluster name here
+whirr.cluster-name=ganglia
+
+# Change the number of machines in the cluster here
+# Also, setting up a cluster just for the sake of monitoring doesn't make much sense :)
+whirr.instance-templates=1 ganglia-metad, 2 ganglia-monitor
+
+# For EC2 set AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY environment variables.
+whirr.provider=aws-ec2
+whirr.identity=${env:AWS_ACCESS_KEY_ID}
+whirr.credential=${env:AWS_SECRET_ACCESS_KEY}
+
+# By default use the user system SSH keys. Override them here.
+# whirr.private-key-file=${sys:user.home}/.ssh/id_rsa
+# whirr.public-key-file=${whirr.private-key-file}.pub
+

Propchange: incubator/whirr/trunk/services/ganglia/
------------------------------------------------------------------------------
--- svn:ignore (added)
+++ svn:ignore Tue Aug 23 07:55:36 2011
@@ -0,0 +1 @@
+target

Added: incubator/whirr/trunk/services/ganglia/pom.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/pom.xml?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/pom.xml (added)
+++ incubator/whirr/trunk/services/ganglia/pom.xml Tue Aug 23 07:55:36 2011
@@ -0,0 +1,75 @@
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.whirr</groupId>
+    <artifactId>whirr</artifactId>
+    <version>0.7.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+  <groupId>org.apache.whirr</groupId>
+  <artifactId>whirr-ganglia</artifactId>
+  <packaging>jar</packaging>
+  <name>Apache Whirr Ganglia</name>
+  <dependencies>
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>whirr-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>whirr-core</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-httpclient</groupId>
+      <artifactId>commons-httpclient</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-configuration</groupId>
+      <artifactId>commons-configuration</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

Added: incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaCluster.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaCluster.java?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaCluster.java (added)
+++ incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaCluster.java Tue Aug 23 07:55:36 2011
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.whirr.service.ganglia;
+
+import org.apache.whirr.Cluster;
+import org.apache.whirr.RolePredicates;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableSet;
+
+public class GangliaCluster {
+
+  public static final String INSTALL_FUNCTION = "install_ganglia";
+  public static final String CONFIGURE_FUNCTION = "configure_ganglia";
+
+  public static String getHosts(Cluster cluster) {
+    return Joiner.on(',').join(
+      GangliaMonitorClusterActionHandler.getHosts(cluster.getInstancesMatching(
+        RolePredicates.anyRoleIn(ImmutableSet.<String>of(GangliaMonitorClusterActionHandler.GANGLIA_MONITOR_ROLE,
+                                                         GangliaMetadClusterActionHandler.GANGLIA_METAD_ROLE)))));
+  }
+  
+}

Added: incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMetadClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMetadClusterActionHandler.java?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMetadClusterActionHandler.java (added)
+++ incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMetadClusterActionHandler.java Tue Aug 23 07:55:36 2011
@@ -0,0 +1,126 @@
+package org.apache.whirr.service.ganglia;
+/**
+ * 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.
+ */
+
+import static org.apache.whirr.RolePredicates.role;
+import static org.jclouds.scriptbuilder.domain.Statements.call;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.whirr.Cluster;
+import org.apache.whirr.Cluster.Instance;
+import org.apache.whirr.ClusterSpec;
+import org.apache.whirr.RolePredicates;
+import org.apache.whirr.service.ClusterActionEvent;
+import org.apache.whirr.service.ClusterActionHandlerSupport;
+import org.apache.whirr.service.FirewallManager.Rule;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+
+public class GangliaMetadClusterActionHandler extends ClusterActionHandlerSupport {
+  
+  private static final Logger LOG =
+    LoggerFactory.getLogger(GangliaMetadClusterActionHandler.class);
+    
+  public static final String GANGLIA_METAD_ROLE = "ganglia-metad";
+  public static final int GANGLIA_META_PORT = 8651;
+  public static final int HTTP_PORT = 80;
+
+  @Override
+  public String getRole() {
+    return GANGLIA_METAD_ROLE;
+  }
+
+  protected Configuration getConfiguration(ClusterSpec spec)
+    throws IOException {
+    return getConfiguration(spec, "whirr-ganglia-default.properties");
+  }
+
+  protected String getInstallFunction(Configuration config) {
+    return getInstallFunction(config, getRole(), GangliaCluster.INSTALL_FUNCTION);
+  }
+
+  protected String getConfigureFunction(Configuration config) {
+    return getConfigureFunction(config, getRole(), GangliaCluster.CONFIGURE_FUNCTION);
+  }
+
+  @Override
+  protected void beforeBootstrap(ClusterActionEvent event) throws IOException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Configuration config = getConfiguration(clusterSpec);
+
+    addStatement(event, call(getInstallFunction(config),
+      "-c", clusterSpec.getProvider(),
+      "-r", GANGLIA_METAD_ROLE)
+    );
+  }
+
+  @Override
+  protected void beforeConfigure(ClusterActionEvent event) throws IOException, InterruptedException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Cluster cluster = event.getCluster();
+
+    // FIXME: the ganglia port is only opened so the ganglia xml dump can be checked in GangliaServiceTest
+    event.getFirewallManager().addRule(
+        Rule.create().destination(role(GANGLIA_METAD_ROLE)).ports(HTTP_PORT, GANGLIA_META_PORT)
+    );
+
+    Configuration config = getConfiguration(clusterSpec);
+    String configureFunction = getConfigureFunction(config);
+
+    // Call the configure function.
+    addStatement(event, call(configureFunction,
+            "-c", clusterSpec.getProvider(),
+            "-m", cluster.getInstanceMatching(RolePredicates.role(GangliaMetadClusterActionHandler.GANGLIA_METAD_ROLE)).getPrivateIp(),
+            "-n", clusterSpec.getClusterName()));
+
+  }
+  
+  @Override
+  protected void afterConfigure(ClusterActionEvent event) {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Cluster cluster = event.getCluster();
+
+    LOG.info("Completed configuration of {}", clusterSpec.getClusterName());
+    String hosts = Joiner.on(',').join(getHosts(cluster.getInstancesMatching(
+      role(GANGLIA_METAD_ROLE))));
+    LOG.info("Meta host: {}. You should be able to connect on http://{}/ganglia", hosts, hosts);
+  }
+
+  static List<String> getHosts(Set<Instance> instances) {
+      return Lists.transform(Lists.newArrayList(instances),
+          new Function<Instance, String>() {
+        @Override
+        public String apply(Instance instance) {
+          try {
+            return instance.getPublicHostName();
+          } catch (IOException e) {
+            throw new IllegalArgumentException(e);
+          }
+        }
+      });
+    }
+
+}

Added: incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMonitorClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMonitorClusterActionHandler.java?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMonitorClusterActionHandler.java (added)
+++ incubator/whirr/trunk/services/ganglia/src/main/java/org/apache/whirr/service/ganglia/GangliaMonitorClusterActionHandler.java Tue Aug 23 07:55:36 2011
@@ -0,0 +1,123 @@
+package org.apache.whirr.service.ganglia;
+/**
+ * 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.
+ */
+
+import static org.apache.whirr.RolePredicates.role;
+import static org.jclouds.scriptbuilder.domain.Statements.call;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.configuration.Configuration;
+import org.apache.whirr.Cluster;
+import org.apache.whirr.Cluster.Instance;
+import org.apache.whirr.ClusterSpec;
+import org.apache.whirr.RolePredicates;
+import org.apache.whirr.service.ClusterActionEvent;
+import org.apache.whirr.service.ClusterActionHandlerSupport;
+import org.apache.whirr.service.FirewallManager.Rule;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+
+public class GangliaMonitorClusterActionHandler extends ClusterActionHandlerSupport {
+  
+  private static final Logger LOG =
+    LoggerFactory.getLogger(GangliaMonitorClusterActionHandler.class);
+    
+  public static final String GANGLIA_MONITOR_ROLE = "ganglia-monitor";
+  public static final int GANGLIA_MONITOR_PORT = 8649;
+
+  @Override
+  public String getRole() {
+    return GANGLIA_MONITOR_ROLE;
+  }
+
+  protected Configuration getConfiguration(ClusterSpec spec)
+    throws IOException {
+    return getConfiguration(spec, "whirr-ganglia-default.properties");
+  }
+
+  protected String getInstallFunction(Configuration config) {
+    return getInstallFunction(config, getRole(), GangliaCluster.INSTALL_FUNCTION);
+  }
+
+  protected String getConfigureFunction(Configuration config) {
+    return getConfigureFunction(config, getRole(), GangliaCluster.CONFIGURE_FUNCTION);
+  }
+
+  @Override
+  protected void beforeBootstrap(ClusterActionEvent event) throws IOException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Configuration config = getConfiguration(clusterSpec);
+
+    addStatement(event, call(getInstallFunction(config),
+      "-c", clusterSpec.getProvider(),
+      "-r", GANGLIA_MONITOR_ROLE)
+    );
+  }
+
+  @Override
+  protected void beforeConfigure(ClusterActionEvent event) throws IOException, InterruptedException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Cluster cluster = event.getCluster();
+
+    event.getFirewallManager().addRule(
+        Rule.create().destination(role(GANGLIA_MONITOR_ROLE)).port(GANGLIA_MONITOR_PORT)
+    );
+
+    Configuration config = getConfiguration(clusterSpec);
+    String configureFunction = getConfigureFunction(config);
+
+    // Call the configure function.
+    addStatement(event, call(configureFunction,
+            "-c", clusterSpec.getProvider(),
+            "-m", cluster.getInstanceMatching(RolePredicates.role(GangliaMetadClusterActionHandler.GANGLIA_METAD_ROLE)).getPrivateIp()));
+
+  }
+  
+  @Override
+  protected void afterConfigure(ClusterActionEvent event) {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Cluster cluster = event.getCluster();
+
+    LOG.info("Completed configuration of {}", clusterSpec.getClusterName());
+    String hosts = Joiner.on(',').join(getHosts(cluster.getInstancesMatching(
+      role(GANGLIA_MONITOR_ROLE))));
+    LOG.info("Monitors: {}", hosts);
+  }
+
+  static List<String> getHosts(Set<Instance> instances) {
+    return Lists.transform(Lists.newArrayList(instances),
+        new Function<Instance, String>() {
+      @Override
+      public String apply(Instance instance) {
+        try {
+          return instance.getPublicHostName();
+        } catch (IOException e) {
+          throw new IllegalArgumentException(e);
+        }
+      }
+    });
+  }
+
+}

Added: incubator/whirr/trunk/services/ganglia/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler (added)
+++ incubator/whirr/trunk/services/ganglia/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler Tue Aug 23 07:55:36 2011
@@ -0,0 +1,13 @@
+#   Licensed 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.
+org.apache.whirr.service.ganglia.GangliaMonitorClusterActionHandler
+org.apache.whirr.service.ganglia.GangliaMetadClusterActionHandler

Added: incubator/whirr/trunk/services/ganglia/src/main/resources/functions/configure_ganglia.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/main/resources/functions/configure_ganglia.sh?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/main/resources/functions/configure_ganglia.sh (added)
+++ incubator/whirr/trunk/services/ganglia/src/main/resources/functions/configure_ganglia.sh Tue Aug 23 07:55:36 2011
@@ -0,0 +1,178 @@
+#
+# 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.
+#
+function configure_ganglia() {
+  local OPTIND
+  local OPTARG
+
+  GRID_NAME=WhirrGrid  
+  CLOUD_PROVIDER=
+  METAD_HOST=localhost
+  CLUSTER_NAME=whirrcluster
+  while getopts "c:m:n:" OPTION; do
+    case $OPTION in
+    c)
+      CLOUD_PROVIDER="$OPTARG"
+      shift $((OPTIND-1)); OPTIND=1
+      ;;
+    m)
+      METAD_HOST="$OPTARG"
+      ;;
+    n)
+      CLUSTER_NAME="$OPTARG"
+      ;;
+    esac
+  done
+  
+  # Use private IP for SELF_HOST
+  case $CLOUD_PROVIDER in
+    ec2 | aws-ec2 )
+      SELF_HOST=`wget -q -O - http://169.254.169.254/latest/meta-data/local-ipv4`
+      ;;
+    cloudservers-uk | cloudservers-us)
+      SELF_HOST=`/sbin/ifconfig eth1 | grep 'inet addr:' | cut -d: -f2 | awk '{ print $1}'`
+      ;;
+    *)
+      SELF_HOST=`/sbin/ifconfig eth0 | grep 'inet addr:' | cut -d: -f2 | awk '{ print $1}'`
+      ;;
+  esac
+  
+  # The service scripts have different names on different distros
+  SVC_APACHE=apache2
+  SVC_GMOND=ganglia-monitor
+  SVC_GMETAD=gmetad
+  
+  if which rpm &> /dev/null; then 
+      SVC_GMOND=gmond
+      SVC_APACHE=httpd
+  fi
+    
+  function remove_ganglia_conf_section() {
+    section_name=$1
+    file=$2
+
+    sed -i -n '1h;1!H;${;g;s/'$section_name' {[^}]*}//g;p;}' "$2"
+  }
+  
+  # create the conf.d directory included in gmond.conf
+  mkdir -p /etc/ganglia/conf.d
+    
+  # On the master, update gmond and gmetad
+  echo "Comparing self with metad_host: $SELF_HOST == $METAD_HOST"
+  if [ "$SELF_HOST" == "$METAD_HOST" ]; then
+    
+    ### Configure the gmetad instance
+    
+    remove_ganglia_conf_section cluster /etc/ganglia/gmond.conf
+    remove_ganglia_conf_section host /etc/ganglia/gmond.conf
+    remove_ganglia_conf_section udp_send_channel /etc/ganglia/gmond.conf
+    remove_ganglia_conf_section udp_recv_channel /etc/ganglia/gmond.conf
+    remove_ganglia_conf_section tcp_accept_channel /etc/ganglia/gmond.conf
+    
+    cat > /etc/ganglia/conf.d/cluster.conf <<EOF
+cluster {
+  name = "$CLUSTER_NAME"
+  owner = "unspecified"
+  latlong = "unspecified"
+  url = "unspecified"
+}
+EOF
+
+    cat > /etc/ganglia/conf.d/host.conf <<EOF
+host {
+  location = "unspecified"
+}
+EOF
+
+    cat > /etc/ganglia/conf.d/channels.conf <<EOF
+udp_send_channel {
+  host = $METAD_HOST
+  port = 8649
+  ttl = 1
+}
+
+udp_recv_channel {
+  bind = $METAD_HOST
+  port = 8649
+}
+
+tcp_accept_channel {
+  port = 8649
+}
+EOF
+
+    # create the the gmetad.conf file
+    cat > /etc/ganglia/gmetad.conf <<EOF
+    gridname "$GRID_NAME"
+    data_source "$CLUSTER_NAME" $METAD_HOST
+EOF
+
+    if which dpkg &> /dev/null; then
+      # For debian-based systems we need to copy the apache configuration in place
+      cp /etc/ganglia-webfrontend/apache.conf /etc/apache2/sites-enabled
+      service $SVC_GMOND restart || true
+      service $SVC_GMETAD restart || true
+      service $SVC_APACHE restart || true
+    elif which rpm &> /dev/null; then
+      # For rpm based systems the apache conf file is automatically copied to /etc/apache/conf.d
+      service $SVC_GMOND restart || true
+      service $SVC_GMETAD restart || true
+      service $SVC_APACHE restart || true # not sure if this is needed
+    fi
+    
+  else
+    
+    ### Configure the monitor instances
+
+    remove_ganglia_conf_section cluster /etc/ganglia/gmond.conf
+    remove_ganglia_conf_section host /etc/ganglia/gmond.conf
+    remove_ganglia_conf_section udp_send_channel /etc/ganglia/gmond.conf
+    remove_ganglia_conf_section udp_recv_channel /etc/ganglia/gmond.conf
+    remove_ganglia_conf_section tcp_accept_channel /etc/ganglia/gmond.conf
+    
+    cat > /etc/ganglia/conf.d/cluster.conf <<EOF
+cluster {
+  name = "$CLUSTER_NAME"
+  owner = "unspecified"
+  latlong = "unspecified"
+  url = "unspecified"
+}
+EOF
+    
+    cat > /etc/ganglia/conf.d/channels.conf <<EOF
+udp_send_channel {
+  host = $METAD_HOST
+  port = 8649
+  ttl = 1
+}
+
+# This section should be commented out, but must be present because of a bug in gmond
+# TODO: find out if that bug is already fixed 
+udp_recv_channel {
+  bind = localhost
+  port = 8649
+}
+
+tcp_accept_channel {
+  port = 8649  
+}
+EOF
+
+    service $SVC_GMOND restart
+
+  fi
+  
+}
\ No newline at end of file

Added: incubator/whirr/trunk/services/ganglia/src/main/resources/functions/install_ganglia.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/main/resources/functions/install_ganglia.sh?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/main/resources/functions/install_ganglia.sh (added)
+++ incubator/whirr/trunk/services/ganglia/src/main/resources/functions/install_ganglia.sh Tue Aug 23 07:55:36 2011
@@ -0,0 +1,90 @@
+#
+# 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.
+#
+
+function update_repo_ganglia() {
+  if which dpkg &> /dev/null; then
+    sudo apt-get update
+  elif which rpm &> /dev/null; then
+    #Registers the EPEL repository (this contains the ganglia rpms) 
+    rpm -Uvh http://download.fedora.redhat.com/pub/epel/6/$(uname -i)/epel-release-6-5.noarch.rpm
+    yum update -y yum
+  fi
+}
+
+function install_ganglia() {
+  local OPTIND
+  local OPTARG
+  
+  CLOUD_PROVIDER=
+  ROLE=
+  while getopts "c:r:" OPTION; do
+    case $OPTION in
+    c)
+      CLOUD_PROVIDER="$OPTARG"
+      shift $((OPTIND-1)); OPTIND=1
+      ;;
+    r)
+      ROLE="$OPTARG"
+      ;;
+    esac
+  done
+
+  update_repo_ganglia
+    
+  # set SELF_HOST to private ip address
+  case $CLOUD_PROVIDER in
+    ec2 | aws-ec2 )
+      SELF_HOST=`wget -q -O - http://169.254.169.254/latest/meta-data/local-ipv4`
+      ;;
+    cloudservers-uk | cloudservers-us)
+      SELF_HOST=`/sbin/ifconfig eth1 | grep 'inet addr:' | cut -d: -f2 | awk '{ print $1}'`
+      ;;
+    *)
+      SELF_HOST=`/sbin/ifconfig eth0 | grep 'inet addr:' | cut -d: -f2 | awk '{ print $1}'`
+      ;;
+  esac
+
+  PACKAGES=()
+
+  # add ganglia-monitor to the list of packages to install.
+  if which dpkg &> /dev/null; then
+    PACKAGES[${#PACKAGES[@]}]=ganglia-monitor
+  elif which rpm &> /dev/null; then
+    #TODO: check package name for ganglia-monitor. requires epel repo?
+    PACKAGES[${#PACKAGES[@]}]=ganglia-gmond
+  fi
+  
+  # For the metad instance, add gmetad to the list of packages to install
+  if [ "$ROLE" = "ganglia-metad" ]; then
+    if which dpkg &> /dev/null; then
+      PACKAGES[${#PACKAGES[@]}]=gmetad
+      PACKAGES[${#PACKAGES[@]}]=ganglia-webfrontend
+    elif which rpm &> /dev/null; then
+      #TODO: check package name for ganglia-monitor. requires epel repo?
+      PACKAGES[${#PACKAGES[@]}]=ganglia-gmetad
+      PACKAGES[${#PACKAGES[@]}]=ganglia-web
+    fi
+  fi;
+  
+  if which dpkg &> /dev/null; then
+    #'noninteractive' -> avoids blue configuration screens during installation
+    sudo sh -c "DEBIAN_FRONTEND=noninteractive apt-get -y install ${PACKAGES[*]}"
+  elif which rpm &> /dev/null; then
+    sudo sh -c "yum install -y ${PACKAGES[*]}"
+  fi
+
+}
\ No newline at end of file

Added: incubator/whirr/trunk/services/ganglia/src/main/resources/whirr-ganglia-default.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/main/resources/whirr-ganglia-default.properties?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/main/resources/whirr-ganglia-default.properties (added)
+++ incubator/whirr/trunk/services/ganglia/src/main/resources/whirr-ganglia-default.properties Tue Aug 23 07:55:36 2011
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+

Added: incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/whirr/service/ganglia/integration/GangliaServiceTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/whirr/service/ganglia/integration/GangliaServiceTest.java?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/whirr/service/ganglia/integration/GangliaServiceTest.java (added)
+++ incubator/whirr/trunk/services/ganglia/src/test/java/org/apache/whirr/service/ganglia/integration/GangliaServiceTest.java Tue Aug 23 07:55:36 2011
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.whirr.service.ganglia.integration;
+
+import static org.apache.whirr.RolePredicates.anyRoleIn;
+import static org.apache.whirr.service.ganglia.GangliaMetadClusterActionHandler.GANGLIA_METAD_ROLE;
+import static org.apache.whirr.service.ganglia.GangliaMonitorClusterActionHandler.GANGLIA_MONITOR_ROLE;
+
+import java.io.IOException;
+import java.net.Socket;
+
+import junit.framework.TestCase;
+
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.HttpStatus;
+import org.apache.commons.httpclient.methods.GetMethod;
+import org.apache.commons.io.IOUtils;
+import org.apache.whirr.Cluster;
+import org.apache.whirr.Cluster.Instance;
+import org.apache.whirr.ClusterController;
+import org.apache.whirr.ClusterControllerFactory;
+import org.apache.whirr.ClusterSpec;
+import org.apache.whirr.RolePredicates;
+import org.apache.whirr.service.ganglia.GangliaMetadClusterActionHandler;
+import org.apache.whirr.service.ganglia.GangliaMonitorClusterActionHandler;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Sets;
+
+public class GangliaServiceTest extends TestCase {
+  
+  private ClusterSpec clusterSpec;
+  private Cluster cluster;
+  private ClusterController controller;
+  
+  @Before
+  public void setUp() throws Exception {
+    CompositeConfiguration config = new CompositeConfiguration();
+    if (System.getProperty("config") != null) {
+      config.addConfiguration(new PropertiesConfiguration(System.getProperty("config")));
+    }
+    config.addConfiguration(new PropertiesConfiguration("whirr-ganglia-test.properties"));
+    clusterSpec = ClusterSpec.withTemporaryKeys(config);
+    controller = new ClusterControllerFactory().create(clusterSpec.getServiceName());
+    
+    cluster = controller.launchCluster(clusterSpec);
+  }
+  
+  @Test
+  public void test() throws Exception {
+    Instance metad = cluster.getInstanceMatching(RolePredicates.role(GangliaMetadClusterActionHandler.GANGLIA_METAD_ROLE));
+    String metadHostname = metad.getPublicHostName();
+
+    assertNotNull(metadHostname);
+    HttpClient client = new HttpClient();
+    GetMethod getIndex = new GetMethod(String.format("http://%s/ganglia/", metadHostname));
+    int statusCode = client.executeMethod(getIndex);
+
+    assertEquals("Status code should be 200", HttpStatus.SC_OK, statusCode);
+    String indexPageHTML = getIndex.getResponseBodyAsString();
+    assertTrue("The string 'Ganglia' should appear on the index page", indexPageHTML.contains("Ganglia"));    
+    assertTrue("The string 'WhirrGrid' should appear on the index page", indexPageHTML.contains("WhirrGrid"));    
+
+    // Now check the xml produced when connecting to the ganglia monitor port on all instances.
+    for (Instance instance: cluster.getInstancesMatching(anyRoleIn(Sets.<String>newHashSet(GANGLIA_METAD_ROLE, GANGLIA_MONITOR_ROLE)))) {
+      testMonitorResponse(instance);
+    }
+  }
+  
+  private void testMonitorResponse(Instance instance) throws IOException {
+    Socket s = null;
+    try {
+      s = new Socket(instance.getPublicAddress(), GangliaMonitorClusterActionHandler.GANGLIA_MONITOR_PORT);
+      String gangliaXml = IOUtils.toString(s.getInputStream());
+      assertTrue(String.format("The ganglia monitor output on instance %s with roles %s did not contain 'HOST NAME'", 
+              instance.getPublicHostName(), instance.getRoles()), gangliaXml.contains("HOST NAME"));
+    } finally {
+      if (s != null) s.close();
+    }    
+  }
+
+  @After
+  public void tearDown() throws IOException, InterruptedException {
+    controller.destroyCluster(clusterSpec);
+  }
+  
+}

Added: incubator/whirr/trunk/services/ganglia/src/test/resources/log4j.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/test/resources/log4j.xml?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/test/resources/log4j.xml (added)
+++ incubator/whirr/trunk/services/ganglia/src/test/resources/log4j.xml Tue Aug 23 07:55:36 2011
@@ -0,0 +1,196 @@
+<?xml version="1.0" encoding="UTF-8"?>
+    <!--
+   Licensed to the Apache Software Foundation (ASF) under one
+   or more contributor license agreements.  See the NOTICE file
+   distributed with this work for additional information
+   regarding copyright ownership.  The ASF licenses this file
+   to you under the Apache License, Version 2.0 (the
+   "License"); you may not use this file except in compliance
+   with the License.  You may obtain a copy of the License at
+  
+       http://www.apache.org/licenses/LICENSE-2.0
+  
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+    -->
+<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
+
+    <!--
+        For more configuration infromation and examples see the Apache
+        Log4j website: http://logging.apache.org/log4j/
+    -->
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/"
+    debug="false">
+    
+    <appender name="CONSOLE" class="org.apache.log4j.ConsoleAppender">
+        <param name="Threshold" value="INFO" />
+        <layout class="org.apache.log4j.PatternLayout"> 
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n"/> 
+        </layout> 
+    </appender>
+
+    <!-- A time/date based rolling appender -->
+    <appender name="WIREFILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/jclouds-wire.log" />
+        <param name="Append" value="true" />
+
+        <!-- Rollover at midnight each day -->
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+
+        <param name="Threshold" value="TRACE" />
+
+        <layout class="org.apache.log4j.PatternLayout">
+            <!-- The default pattern: Date Priority [Category] Message\n -->
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+
+            <!--
+                The full pattern: Date MS Priority [Category]
+                (Thread:NDC) Message\n <param name="ConversionPattern"
+                value="%d %-5r %-5p [%c] (%t:%x) %m%n"/>
+            -->
+        </layout>
+    </appender>
+
+    <!-- A time/date based rolling appender -->
+    <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/jclouds.log" />
+        <param name="Append" value="true" />
+
+        <!-- Rollover at midnight each day -->
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+
+        <param name="Threshold" value="TRACE" />
+
+        <layout class="org.apache.log4j.PatternLayout">
+            <!-- The default pattern: Date Priority [Category] Message\n -->
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+
+            <!--
+                The full pattern: Date MS Priority [Category]
+                (Thread:NDC) Message\n <param name="ConversionPattern"
+                value="%d %-5r %-5p [%c] (%t:%x) %m%n"/>
+            -->
+        </layout>
+    </appender>
+
+    <!-- A time/date based rolling appender -->
+    <appender name="BLOBSTOREFILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/jclouds-blobstore.log" />
+        <param name="Append" value="true" />
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+        <param name="Threshold" value="TRACE" />
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+        </layout>
+    </appender>
+    
+
+    <!-- A time/date based rolling appender -->
+    <appender name="COMPUTEFILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/jclouds-compute.log" />
+        <param name="Append" value="true" />
+
+        <!-- Rollover at midnight each day -->
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+
+        <param name="Threshold" value="TRACE" />
+
+        <layout class="org.apache.log4j.PatternLayout">
+            <!-- The default pattern: Date Priority [Category] Message\n -->
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+
+            <!--
+                The full pattern: Date MS Priority [Category]
+                (Thread:NDC) Message\n <param name="ConversionPattern"
+                value="%d %-5r %-5p [%c] (%t:%x) %m%n"/>
+            -->
+        </layout>
+    </appender>
+    
+    <!-- A time/date based rolling appender -->
+    <appender name="WHIRRFILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/whirr.log" />
+        <param name="Append" value="true" />
+
+        <!-- Rollover at midnight each day -->
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+
+        <param name="Threshold" value="TRACE" />
+
+        <layout class="org.apache.log4j.PatternLayout">
+            <!-- The default pattern: Date Priority [Category] Message\n -->
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+
+            <!--
+                The full pattern: Date MS Priority [Category]
+                (Thread:NDC) Message\n <param name="ConversionPattern"
+                value="%d %-5r %-5p [%c] (%t:%x) %m%n"/>
+            -->
+        </layout>
+    </appender>
+
+    <appender name="ASYNCCOMPUTE" class="org.apache.log4j.AsyncAppender">
+        <appender-ref ref="COMPUTEFILE" />
+    </appender>
+    <appender name="ASYNC" class="org.apache.log4j.AsyncAppender">
+        <appender-ref ref="FILE" />
+    </appender>
+
+    <appender name="ASYNCWIRE" class="org.apache.log4j.AsyncAppender">
+        <appender-ref ref="WIREFILE" />
+    </appender>
+
+    <appender name="ASYNCBLOBSTORE" class="org.apache.log4j.AsyncAppender">
+        <appender-ref ref="BLOBSTOREFILE" />
+    </appender>
+    <!-- ================ -->
+    <!-- Limit categories -->
+    <!-- ================ -->
+    <category name="jclouds.blobstore">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCBLOBSTORE" />
+    </category>
+
+    <category name="org.jclouds">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNC" />
+    </category>
+    
+    <category name="org.apache.whirr">
+        <priority value="DEBUG" />
+        <appender-ref ref="WHIRRFILE" />
+    </category>
+
+    <category name="jclouds.headers">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCWIRE" />
+    </category>
+    <category name="jclouds.compute">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCCOMPUTE" />
+    </category>
+
+    <category name="jclouds.wire">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCWIRE" />
+    </category><!--
+    
+       <category name="jclouds.signature">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCWIRE" />
+    </category>
+    
+    
+    --><!--  ======================= -->
+    <!-- Setup the Root category -->
+    <!-- ======================= -->
+
+    <root>
+        <priority value="WARN" />
+        <appender-ref ref="CONSOLE" />
+    </root>
+
+</log4j:configuration>
\ No newline at end of file

Added: incubator/whirr/trunk/services/ganglia/src/test/resources/whirr-ganglia-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/ganglia/src/test/resources/whirr-ganglia-test.properties?rev=1160573&view=auto
==============================================================================
--- incubator/whirr/trunk/services/ganglia/src/test/resources/whirr-ganglia-test.properties (added)
+++ incubator/whirr/trunk/services/ganglia/src/test/resources/whirr-ganglia-test.properties Tue Aug 23 07:55:36 2011
@@ -0,0 +1,23 @@
+#
+# 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.
+#
+whirr.cluster-name=gangliaclustertest
+whirr.instance-templates=1 ganglia-metad, 1 ganglia-monitor 
+whirr.provider=${sys:whirr.test.provider}
+whirr.identity=${sys:whirr.test.identity}
+whirr.credential=${sys:whirr.test.credential}
+whirr.hardware-min-ram=512