You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@whirr.apache.org by to...@apache.org on 2010/12/16 22:24:47 UTC

svn commit: r1050166 - in /incubator/whirr/trunk: ./ core/ core/src/main/java/org/apache/whirr/net/ core/src/main/java/org/apache/whirr/service/ core/src/test/java/org/apache/whirr/net/ core/src/test/java/org/apache/whirr/net/integration/ services/hado...

Author: tomwhite
Date: Thu Dec 16 21:24:46 2010
New Revision: 1050166

URL: http://svn.apache.org/viewvc?rev=1050166&view=rev
Log:
WHIRR-150. Allow retrieval of instance roles.

Added:
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/net/
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/net/DnsUtil.java   (with props)
    incubator/whirr/trunk/core/src/test/java/org/apache/whirr/net/
    incubator/whirr/trunk/core/src/test/java/org/apache/whirr/net/integration/
    incubator/whirr/trunk/core/src/test/java/org/apache/whirr/net/integration/DnsUtilTest.java   (with props)
Removed:
    incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/DnsUtil.java
    incubator/whirr/trunk/services/hadoop/src/test/java/org/apache/whirr/service/hadoop/DnsUtilTest.java
Modified:
    incubator/whirr/trunk/CHANGES.txt
    incubator/whirr/trunk/core/pom.xml
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java
    incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java
    incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java
    incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopProxy.java

Modified: incubator/whirr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/CHANGES.txt?rev=1050166&r1=1050165&r2=1050166&view=diff
==============================================================================
--- incubator/whirr/trunk/CHANGES.txt (original)
+++ incubator/whirr/trunk/CHANGES.txt Thu Dec 16 21:24:46 2010
@@ -21,6 +21,8 @@ Trunk (unreleased changes)
 
     WHIRR-160. Improve SSH key diagnostics. (Andrei Savu via tomwhite)
 
+    WHIRR-150. Allow retrieval of instance roles. (tomwhite)
+
   BUG FIXES
 
     WHIRR-128. Fix DNS resolution for clients running within EC2.

Modified: incubator/whirr/trunk/core/pom.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/pom.xml?rev=1050166&r1=1050165&r2=1050166&view=diff
==============================================================================
--- incubator/whirr/trunk/core/pom.xml (original)
+++ incubator/whirr/trunk/core/pom.xml Thu Dec 16 21:24:46 2010
@@ -89,6 +89,10 @@
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
     </dependency>
+    <dependency>
+      <groupId>dnsjava</groupId>
+      <artifactId>dnsjava</artifactId>
+    </dependency>
   </dependencies>
   <build>
     <resources>

Added: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/net/DnsUtil.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/main/java/org/apache/whirr/net/DnsUtil.java?rev=1050166&view=auto
==============================================================================
--- incubator/whirr/trunk/core/src/main/java/org/apache/whirr/net/DnsUtil.java (added)
+++ incubator/whirr/trunk/core/src/main/java/org/apache/whirr/net/DnsUtil.java Thu Dec 16 21:24:46 2010
@@ -0,0 +1,60 @@
+/**
+ * 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.net;
+
+import java.io.IOException;
+
+import org.xbill.DNS.DClass;
+import org.xbill.DNS.ExtendedResolver;
+import org.xbill.DNS.Message;
+import org.xbill.DNS.Name;
+import org.xbill.DNS.Record;
+import org.xbill.DNS.Resolver;
+import org.xbill.DNS.ReverseMap;
+import org.xbill.DNS.Section;
+import org.xbill.DNS.Type;
+
+public class DnsUtil {
+
+  /**
+   * resolve the reverse dns name for the given IP address
+   * 
+   * @param hostIp
+   * @return
+   * @throws IOException
+   */
+  public static String resolveAddress(String hostIp) throws IOException {
+    Resolver res = new ExtendedResolver();
+
+    Name name = ReverseMap.fromAddress(hostIp);
+    int type = Type.PTR;
+    int dclass = DClass.IN;
+    Record rec = Record.newRecord(name, type, dclass);
+    Message query = Message.newQuery(rec);
+    Message response = res.send(query);
+
+    Record[] answers = response.getSectionArray(Section.ANSWER);
+    if (answers.length == 0)
+      return hostIp;
+    else {
+      String revaddr = answers[0].rdataToString();
+      return revaddr.endsWith(".") ? revaddr.substring(0, revaddr.length() - 1) : revaddr;
+    }
+  }
+}

Propchange: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/net/DnsUtil.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java?rev=1050166&r1=1050165&r2=1050166&view=diff
==============================================================================
--- incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java (original)
+++ incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java Thu Dec 16 21:24:46 2010
@@ -419,6 +419,17 @@ public class ClusterSpec {
     }
     return c;
   }
+  
+  /**
+   * @return the directory for storing cluster-related files
+   */
+  public File getClusterDirectory() {
+    File clusterDir = new File(new File(System.getProperty("user.home")),
+        ".whirr");
+    clusterDir = new File(clusterDir, getClusterName());
+    clusterDir.mkdirs();
+    return clusterDir;
+  }
     
   public boolean equals(Object o) {
     if (o instanceof ClusterSpec) {

Modified: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java?rev=1050166&r1=1050165&r2=1050166&view=diff
==============================================================================
--- incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java (original)
+++ incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java Thu Dec 16 21:24:46 2010
@@ -18,25 +18,35 @@
 
 package org.apache.whirr.service;
 
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
 import com.google.common.base.Predicate;
+import com.google.common.io.Files;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.Set;
 
 import org.apache.whirr.cluster.actions.BootstrapClusterAction;
 import org.apache.whirr.cluster.actions.ConfigureClusterAction;
 import org.apache.whirr.cluster.actions.DestroyClusterAction;
+import org.apache.whirr.net.DnsUtil;
+import org.apache.whirr.service.Cluster.Instance;
 import org.jclouds.compute.ComputeService;
 import org.jclouds.compute.domain.ComputeMetadata;
 import org.jclouds.compute.domain.NodeMetadata;
 import org.jclouds.compute.domain.NodeState;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class represents a service that a client wants to use. This class is
  * used to start and stop clusters that provide the service.
  */
 public class Service {
-  
+
+  private static final Logger LOG = LoggerFactory.getLogger(Service.class);
+
   /**
    * @return the unique name of the service.
    */
@@ -63,9 +73,36 @@ public class Service {
     ConfigureClusterAction configurer = new ConfigureClusterAction();
     cluster = configurer.execute(clusterSpec, cluster);
     
+    createInstancesFile(clusterSpec, cluster);
+    
     return cluster;
   }
   
+  private void createInstancesFile(ClusterSpec clusterSpec, Cluster cluster)
+      throws IOException {
+    
+    File clusterDir = clusterSpec.getClusterDirectory();
+    File instancesFile = new File(clusterDir, "instances");
+    StringBuilder sb = new StringBuilder();
+    for (Instance instance : cluster.getInstances()) {
+      String id = instance.getId();
+      String roles = Joiner.on(',').join(instance.getRoles());
+      String publicAddress = DnsUtil.resolveAddress(instance.getPublicAddress()
+          .getHostAddress());
+      String privateAddress = instance.getPrivateAddress().getHostAddress();
+      sb.append(id).append("\t");
+      sb.append(roles).append("\t");
+      sb.append(publicAddress).append("\t");
+      sb.append(privateAddress).append("\n");
+    }
+    try {
+      Files.write(sb.toString(), instancesFile, Charsets.UTF_8);
+      LOG.info("Wrote instances file {}", instancesFile);
+    } catch (IOException e) {
+      LOG.error("Problem writing instances file {}", instancesFile, e);
+    }
+  }
+  
   /**
    * Stop the cluster and destroy all resources associated with it.
    * @throws IOException if there is a problem while stopping the cluster. The
@@ -76,6 +113,7 @@ public class Service {
       InterruptedException {
     DestroyClusterAction destroyer = new DestroyClusterAction();
     destroyer.execute(clusterSpec, null);
+    Files.deleteRecursively(clusterSpec.getClusterDirectory());
   }
   
   public Set<? extends NodeMetadata> getNodes(ClusterSpec clusterSpec)

Added: incubator/whirr/trunk/core/src/test/java/org/apache/whirr/net/integration/DnsUtilTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/test/java/org/apache/whirr/net/integration/DnsUtilTest.java?rev=1050166&view=auto
==============================================================================
--- incubator/whirr/trunk/core/src/test/java/org/apache/whirr/net/integration/DnsUtilTest.java (added)
+++ incubator/whirr/trunk/core/src/test/java/org/apache/whirr/net/integration/DnsUtilTest.java Thu Dec 16 21:24:46 2010
@@ -0,0 +1,84 @@
+/**
+ * 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.net.integration;
+
+import static org.junit.Assert.*;
+import static java.lang.System.out;
+
+import java.io.IOException;
+import java.net.Inet4Address;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.Enumeration;
+
+import org.apache.whirr.net.DnsUtil;
+import org.junit.Test;
+import org.xbill.DNS.Address;
+
+public class DnsUtilTest {
+
+  @Test
+  public void testResolveAddress() throws IOException {
+    // test it with all interfaces
+    Enumeration<NetworkInterface> en = NetworkInterface.getNetworkInterfaces();
+    while (en.hasMoreElements()) {
+      NetworkInterface netint = (NetworkInterface) en.nextElement();
+      Enumeration<InetAddress> inetAddresses = netint.getInetAddresses();
+      for (InetAddress inetAddress : Collections.list(inetAddresses)) {
+        if (inetAddress instanceof Inet4Address) {
+          long start = System.currentTimeMillis();
+          String reverse = DnsUtil.resolveAddress(inetAddress.getHostAddress());
+          long end = System.currentTimeMillis();
+          // we know that java.net.InetAddress's getHostName takes > 4.5s if
+          // there is no reverse address assigned to it
+          // but DnsUtil can resolve any address without this delaying problem.
+          assertTrue("DnsUtil.resolveAddress takes " + (end - start)
+              + " millis, it should be shorter than a second",
+              end - start < 1000);
+          if (inetAddress.toString().substring(1).equals(reverse)) {
+            out.printf(
+                "InetAddress %s on interface %s does not have reverse dns name, so their reverse remains: %s\n",
+                inetAddress, netint.getDisplayName(), reverse);
+          } else {
+            if (inetAddress.isLoopbackAddress()) {
+              out.printf(
+                  "InetAddress %s on loopback interface %s obtained reverse name as %s\n",
+                  inetAddress, netint.getDisplayName(), reverse);
+            } else {
+              out.printf(
+                  "InetAddress %s on interface %s has reverse dns name: %s\n",
+                  inetAddress, netint.getDisplayName(), reverse);
+              try {
+                InetAddress checkedAddr = Address.getByName(reverse);
+                assertEquals(inetAddress, checkedAddr);
+              } catch (UnknownHostException uhex) {
+                fail("InetAddress " + inetAddress + " on interface "
+                    + netint.getDisplayName() + " got " + reverse
+                    + " reverse dns name which in return is an unknown host!");
+              }
+            }
+          }
+        }
+      }
+    }
+  }
+
+}

Propchange: incubator/whirr/trunk/core/src/test/java/org/apache/whirr/net/integration/DnsUtilTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java?rev=1050166&r1=1050165&r2=1050166&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java Thu Dec 16 21:24:46 2010
@@ -23,6 +23,7 @@ import static org.apache.whirr.service.R
 import java.io.IOException;
 import java.net.InetAddress;
 
+import org.apache.whirr.net.DnsUtil;
 import org.apache.whirr.service.Cluster;
 import org.apache.whirr.service.Cluster.Instance;
 import org.apache.whirr.service.ClusterActionEvent;

Modified: incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java?rev=1050166&r1=1050165&r2=1050166&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java Thu Dec 16 21:24:46 2010
@@ -30,6 +30,7 @@ import java.net.InetAddress;
 import java.util.Map.Entry;
 import java.util.Properties;
 
+import org.apache.whirr.net.DnsUtil;
 import org.apache.whirr.service.Cluster;
 import org.apache.whirr.service.Cluster.Instance;
 import org.apache.whirr.service.ClusterActionEvent;

Modified: incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopProxy.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopProxy.java?rev=1050166&r1=1050165&r2=1050166&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopProxy.java (original)
+++ incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopProxy.java Thu Dec 16 21:24:46 2010
@@ -30,6 +30,7 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.net.InetAddress;
 
+import org.apache.whirr.net.DnsUtil;
 import org.apache.whirr.service.Cluster;
 import org.apache.whirr.service.ClusterSpec;