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/04/28 23:31:08 UTC

svn commit: r1097617 - in /incubator/whirr/trunk: ./ cli/src/main/java/org/apache/whirr/cli/command/ cli/src/test/java/org/apache/whirr/cli/command/ core/src/main/java/org/apache/whirr/

Author: asavu
Date: Thu Apr 28 21:31:08 2011
New Revision: 1097617

URL: http://svn.apache.org/viewvc?rev=1097617&view=rev
Log:
WHIRR-289. Display role names in list-cluster command (asavu)

Modified:
    incubator/whirr/trunk/CHANGES.txt
    incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ListClusterCommand.java
    incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ListClusterCommandTest.java
    incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/RunScriptCommandTest.java
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/ClusterController.java

Modified: incubator/whirr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/CHANGES.txt?rev=1097617&r1=1097616&r2=1097617&view=diff
==============================================================================
--- incubator/whirr/trunk/CHANGES.txt (original)
+++ incubator/whirr/trunk/CHANGES.txt Thu Apr 28 21:31:08 2011
@@ -50,6 +50,8 @@ Trunk (unreleased changes)
 
     WHIRR-246. Single place to store/load cluster state (David Alves and asavu)
 
+    WHIRR-289. Display role names in list-cluster command (asavu)
+
   BUG FIXES
 
     WHIRR-253. ZooKeeper service should only authorize ingress to ZooKeeper 

Modified: incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ListClusterCommand.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ListClusterCommand.java?rev=1097617&r1=1097616&r2=1097617&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ListClusterCommand.java (original)
+++ incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ListClusterCommand.java Thu Apr 28 21:31:08 2011
@@ -30,11 +30,16 @@ import java.util.Set;
 import joptsimple.OptionParser;
 import joptsimple.OptionSet;
 
+import org.apache.whirr.Cluster;
 import org.apache.whirr.ClusterController;
 import org.apache.whirr.ClusterControllerFactory;
 import org.apache.whirr.ClusterSpec;
+import org.apache.whirr.service.ClusterStateStore;
+import org.apache.whirr.service.ClusterStateStoreFactory;
 import org.jclouds.compute.domain.NodeMetadata;
 
+import static org.apache.whirr.RolePredicates.withIds;
+
 /**
  * A command to list the nodes in a cluster.
  */
@@ -45,7 +50,12 @@ public class ListClusterCommand extends 
   }
 
   public ListClusterCommand(ClusterControllerFactory factory) {
-    super("list-cluster", "List the nodes in a cluster.", factory);
+    this(factory, new ClusterStateStoreFactory());
+  }
+
+  public ListClusterCommand(ClusterControllerFactory factory,
+                            ClusterStateStoreFactory stateStoreFactory) {
+    super("list-cluster", "List the nodes in a cluster.", factory, stateStoreFactory);
   }
   
   @Override
@@ -60,14 +70,20 @@ public class ListClusterCommand extends 
     }
     try {
       ClusterSpec clusterSpec = getClusterSpec(optionSet);
-
+      ClusterStateStore stateStore = createClusterStateStore(clusterSpec);
       ClusterController controller = createClusterController(clusterSpec.getServiceName());
-      Set<? extends NodeMetadata> nodes = controller.getNodes(clusterSpec);
-      for (NodeMetadata node : nodes) {
-        out.println(Joiner.on('\t').join(node.getId(), node.getImageId(),
-            getFirstAddress(node.getPublicAddresses()),
-            getFirstAddress(node.getPrivateAddresses()),
-            node.getState(), node.getLocation().getId()));
+
+      for (Cluster.Instance instance : controller.getInstances(clusterSpec, stateStore)) {
+        out.println(Joiner.on('\t').join(
+            instance.getId(),
+            instance.getNodeMetadata().getImageId(),
+            instance.getPublicIp(),
+            instance.getPrivateIp(),
+            instance.getNodeMetadata().getState(),
+            instance.getNodeMetadata().getLocation().getId(),
+            Joiner.on(",").join(instance.getRoles())
+          )
+        );
       }
       return 0;
     } catch (IllegalArgumentException e) {
@@ -76,10 +92,6 @@ public class ListClusterCommand extends 
       return -1;
     }
   }
-  
-  private String getFirstAddress(Set<String> addresses) {
-    return addresses.isEmpty() ? "" : Iterables.get(addresses, 0);
-  }
 
   private void printUsage(OptionParser parser, PrintStream stream) throws IOException {
     stream.println("Usage: whirr list-cluster [OPTIONS]");

Modified: incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ListClusterCommandTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ListClusterCommandTest.java?rev=1097617&r1=1097616&r2=1097617&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ListClusterCommandTest.java (original)
+++ incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ListClusterCommandTest.java Thu Apr 28 21:31:08 2011
@@ -18,6 +18,7 @@
 
 package org.apache.whirr.cli.command;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Matchers.any;
@@ -35,14 +36,19 @@ import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.whirr.Cluster;
 import org.apache.whirr.ClusterController;
 import org.apache.whirr.ClusterControllerFactory;
 import org.apache.whirr.ClusterSpec;
+import org.apache.whirr.cli.MemoryClusterStateStore;
+import org.apache.whirr.service.ClusterStateStore;
+import org.apache.whirr.service.ClusterStateStoreFactory;
 import org.apache.whirr.util.KeyPair;
 import org.hamcrest.Matcher;
 import org.jclouds.compute.domain.NodeMetadata;
 import org.jclouds.compute.domain.NodeMetadataBuilder;
 import org.jclouds.compute.domain.NodeState;
+import org.jclouds.domain.Credentials;
 import org.jclouds.domain.LocationBuilder;
 import org.jclouds.domain.LocationScope;
 import org.junit.Before;
@@ -63,7 +69,7 @@ public class ListClusterCommandTest {
     errBytes = new ByteArrayOutputStream();
     err = new PrintStream(errBytes);
   }
-  
+
   @Test
   public void testInsufficientOptions() throws Exception {
     ListClusterCommand command = new ListClusterCommand();
@@ -71,33 +77,45 @@ public class ListClusterCommandTest {
     assertThat(rc, is(-1));
     assertThat(errBytes.toString(), containsUsageString());
   }
-  
+
   private Matcher<String> containsUsageString() {
     return StringContains.containsString("Usage: whirr list-cluster [OPTIONS]");
   }
-  
+
   @Test
   public void testAllOptions() throws Exception {
-    
+
     ClusterControllerFactory factory = mock(ClusterControllerFactory.class);
     ClusterController controller = mock(ClusterController.class);
     when(factory.create((String) any())).thenReturn(controller);
+
     NodeMetadata node1 = new NodeMetadataBuilder().name("name1").ids("id1")
         .location(new LocationBuilder().scope(LocationScope.PROVIDER)
           .id("location-id1").description("location-desc1").build())
         .imageId("image-id").state(NodeState.RUNNING)
-        .publicAddresses(Lists.newArrayList("100.0.0.1"))
-        .privateAddresses(Lists.newArrayList("10.0.0.1")).build();
+        .publicAddresses(Lists.newArrayList("127.0.0.1"))
+        .privateAddresses(Lists.newArrayList("127.0.0.1")).build();
+
     NodeMetadata node2 = new NodeMetadataBuilder().name("name2").ids("id2")
         .location(new LocationBuilder().scope(LocationScope.PROVIDER)
           .id("location-id2").description("location-desc2").build())
         .imageId("image-id").state(NodeState.RUNNING)
-        .publicAddresses(Lists.newArrayList("100.0.0.2"))
-        .privateAddresses(Lists.newArrayList("10.0.0.2")).build();
+        .publicAddresses(Lists.newArrayList("127.0.0.2"))
+        .privateAddresses(Lists.newArrayList("127.0.0.2")).build();
+
     when(controller.getNodes((ClusterSpec) any())).thenReturn(
         (Set) Sets.newLinkedHashSet(Lists.newArrayList(node1, node2)));
+    when(controller.getInstances((ClusterSpec)any(), (ClusterStateStore)any()))
+        .thenCallRealMethod();
+
+    ClusterStateStore memStore = new MemoryClusterStateStore();
+    memStore.save(createTestCluster(
+      new String[]{"id1", "id2"}, new String[]{"role1", "role2"}));
 
-    ListClusterCommand command = new ListClusterCommand(factory);
+    ClusterStateStoreFactory stateStoreFactory = mock(ClusterStateStoreFactory.class);
+    when(stateStoreFactory.create((ClusterSpec) any())).thenReturn(memStore);
+
+    ListClusterCommand command = new ListClusterCommand(factory, stateStoreFactory);
 
     Map<String, File> keys = KeyPair.generateTemporaryFiles();
     int rc = command.run(null, out, null, Lists.newArrayList(
@@ -110,10 +128,27 @@ public class ListClusterCommandTest {
     assertThat(rc, is(0));
     
     assertThat(outBytes.toString(), is(
-      "id1\timage-id\t100.0.0.1\t10.0.0.1\tRUNNING\tlocation-id1\n" +
-      "id2\timage-id\t100.0.0.2\t10.0.0.2\tRUNNING\tlocation-id2\n"));
+      "id1\timage-id\t127.0.0.1\t127.0.0.1\tRUNNING\tlocation-id1\trole1\n" +
+      "id2\timage-id\t127.0.0.2\t127.0.0.2\tRUNNING\tlocation-id2\trole2\n"));
     
     verify(factory).create("test-service");
     
   }
+
+  private Cluster createTestCluster(String[] ids, String[] roles) {
+    checkArgument(ids.length == roles.length, "each ID should have a role");
+
+    Credentials credentials = new Credentials("dummy", "dummy");
+    Set<Cluster.Instance> instances = Sets.newHashSet();
+
+    for(int i = 0; i < ids.length; i++) {
+      String ip = "127.0.0." + (i + 1);
+      instances.add(new Cluster.Instance(credentials,
+        Sets.newHashSet(roles[i]), ip, ip, ids[i], null));
+    }
+
+    return new Cluster(instances);
+  }
+
+
 }

Modified: incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/RunScriptCommandTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/RunScriptCommandTest.java?rev=1097617&r1=1097616&r2=1097617&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/RunScriptCommandTest.java (original)
+++ incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/RunScriptCommandTest.java Thu Apr 28 21:31:08 2011
@@ -22,7 +22,6 @@ import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import com.jcraft.jsch.JSchException;
 import org.apache.whirr.Cluster;
 import org.apache.whirr.ClusterController;
 import org.apache.whirr.ClusterControllerFactory;
@@ -51,7 +50,6 @@ import static org.hamcrest.Matchers.is;
 import static org.jclouds.compute.predicates.NodePredicates.withIds;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 

Modified: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/ClusterController.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/main/java/org/apache/whirr/ClusterController.java?rev=1097617&r1=1097616&r2=1097617&view=diff
==============================================================================
--- incubator/whirr/trunk/core/src/main/java/org/apache/whirr/ClusterController.java (original)
+++ incubator/whirr/trunk/core/src/main/java/org/apache/whirr/ClusterController.java Thu Apr 28 21:31:08 2011
@@ -25,6 +25,8 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
 import org.apache.whirr.actions.BootstrapClusterAction;
 import org.apache.whirr.actions.ConfigureClusterAction;
 import org.apache.whirr.actions.DestroyClusterAction;
@@ -144,7 +146,8 @@ public class ClusterController {
       context.close();
     }
   }
-  
+
+  @Deprecated
   public Set<? extends NodeMetadata> getNodes(ClusterSpec clusterSpec)
     throws IOException, InterruptedException {
     ComputeService computeService =
@@ -152,6 +155,38 @@ public class ClusterController {
     return computeService.listNodesDetailsMatching(
         runningInGroup(clusterSpec.getClusterName()));
   }
+
+  public Set<Cluster.Instance> getInstances(ClusterSpec spec)
+      throws IOException, InterruptedException {
+    return getInstances(spec, null);
+  }
+
+  public Set<Cluster.Instance> getInstances(ClusterSpec spec, ClusterStateStore stateStore)
+      throws IOException, InterruptedException {
+    Set<Cluster.Instance> instances = Sets.newLinkedHashSet();
+    if (stateStore != null) {
+      /* enrich the instance information with node metadata */
+      Cluster cluster = stateStore.load();
+
+      for(NodeMetadata node : getNodes(spec)) {
+        Cluster.Instance instance = cluster.getInstanceMatching(withIds(node.getId()));
+        instances.add(new Cluster.Instance(instance.getLoginCredentials(), instance.getRoles(),
+          instance.getPublicIp(), instance.getPrivateIp(), node.getId(), node)
+        );
+      }
+    } else {
+      /* return a list of instances with no roles attached */
+      Credentials credentials = new Credentials(spec.getClusterUser(), spec.getPrivateKey());
+      for(NodeMetadata node : getNodes(spec)) {
+        instances.add(new Cluster.Instance(credentials, Sets.<String>newHashSet(),
+            Iterables.getFirst(node.getPublicAddresses(), null),
+            Iterables.getFirst(node.getPrivateAddresses(), null),
+            node.getId(), node));
+      }
+
+    }
+    return instances;
+  }
   
   public static Predicate<ComputeMetadata> runningInGroup(final String group) {
     return new Predicate<ComputeMetadata>() {