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 2011/01/28 20:42:59 UTC

svn commit: r1064840 - in /incubator/whirr/trunk: ./ services/hbase/src/main/java/org/apache/whirr/service/hbase/ services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/ services/hbase/src/test/resources/

Author: tomwhite
Date: Fri Jan 28 19:42:58 2011
New Revision: 1064840

URL: http://svn.apache.org/viewvc?rev=1064840&view=rev
Log:
WHIRR-201. [HBase] Integration test fails. Contributed by largsgeorge.

Modified:
    incubator/whirr/trunk/CHANGES.txt
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java
    incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java
    incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java
    incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties

Modified: incubator/whirr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/CHANGES.txt?rev=1064840&r1=1064839&r2=1064840&view=diff
==============================================================================
--- incubator/whirr/trunk/CHANGES.txt (original)
+++ incubator/whirr/trunk/CHANGES.txt Fri Jan 28 19:42:58 2011
@@ -17,6 +17,9 @@ Trunk (unreleased changes)
     the template (tomwhite via asavu)
 
     WHIRR-186. [HBase] Add version support configurable in properties file.
+    (largsgeorge)
+
+    WHIRR-201. [HBase] Integration test fails. (largsgeorge via tomwhite)
 
 Release 0.3.0 - 2011-01-15
 

Modified: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java?rev=1064840&r1=1064839&r2=1064840&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java Fri Jan 28 19:42:58 2011
@@ -27,8 +27,6 @@ import com.google.common.io.Files;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Map.Entry;
 import java.util.Properties;
 
@@ -36,7 +34,6 @@ 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;
-import org.apache.whirr.service.ClusterActionHandlerSupport;
 import org.apache.whirr.service.ClusterSpec;
 import org.apache.whirr.service.ComputeServiceContextBuilder;
 import org.apache.whirr.service.hadoop.HadoopProxy;
@@ -130,6 +127,8 @@ public class HBaseMasterClusterActionHan
     Properties config = new Properties();
     config.setProperty(HBaseConstants.PROP_HBASE_ZOOKEEPER_QUORUM, quorum);
     config.setProperty(HBaseConstants.PROP_HBASE_ZOOKEEPER_CLIENTPORT, "2181");
+    config.setProperty("hadoop.socks.server", "localhost:6666");
+    config.setProperty("hadoop.rpc.socket.factory.class.default", "org.apache.hadoop.net.SocksSocketFactory");
     return config;
   }
 

Modified: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java?rev=1064840&r1=1064839&r2=1064840&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java Fri Jan 28 19:42:58 2011
@@ -27,7 +27,6 @@ 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;
-import org.apache.whirr.service.ClusterActionHandlerSupport;
 import org.apache.whirr.service.ClusterSpec;
 import org.apache.whirr.service.ComputeServiceContextBuilder;
 import org.apache.whirr.service.jclouds.FirewallSettings;

Modified: incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java?rev=1064840&r1=1064839&r2=1064840&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java (original)
+++ incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java Fri Jan 28 19:42:58 2011
@@ -21,18 +21,24 @@ package org.apache.whirr.service.hbase.i
 import org.apache.commons.configuration.CompositeConfiguration;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.rest.client.Client;
+import org.apache.hadoop.hbase.rest.client.RemoteAdmin;
+import org.apache.hadoop.hbase.rest.client.RemoteHTable;
 import org.apache.whirr.service.Cluster;
 import org.apache.whirr.service.ClusterSpec;
+import org.apache.whirr.service.RolePredicates;
 import org.apache.whirr.service.Service;
 import org.apache.whirr.service.hadoop.HadoopProxy;
+import org.apache.whirr.service.hbase.HBaseRestServerClusterActionHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.util.Map.Entry;
 
 public class HBaseServiceController {
@@ -52,6 +58,9 @@ public class HBaseServiceController {
   private Service service;
   private HadoopProxy proxy;
   private Cluster cluster;
+  private RemoteHTable remoteMetaTable;
+  private RemoteAdmin remoteAdmin;
+  private Client restClient;
 
   private HBaseServiceController() {
   }
@@ -81,29 +90,34 @@ public class HBaseServiceController {
     proxy.start();
 
     Configuration conf = getConfiguration();
-    waitForMaster(conf);
-    running = true;
-  }
 
-  public Cluster getCluster() {
-    return cluster;
+    InetAddress restAddress = cluster.getInstanceMatching(RolePredicates.role(
+      HBaseRestServerClusterActionHandler.ROLE)).getPublicAddress();
+    restClient = new Client(new org.apache.hadoop.hbase.rest.client.Cluster()
+      .add(restAddress.getHostName(), HBaseRestServerClusterActionHandler.
+        PORT));
+    remoteAdmin = new RemoteAdmin(restClient, conf);
+    remoteMetaTable = new RemoteHTable(restClient, conf,
+      HConstants.META_TABLE_NAME, null);
+    waitForMaster();
+    running = true;
   }
 
   public Configuration getConfiguration() {
-    Configuration conf = new Configuration();
+    Configuration conf = HBaseConfiguration.create();
     for (Entry<Object, Object> entry : cluster.getConfiguration().entrySet()) {
       conf.set(entry.getKey().toString(), entry.getValue().toString());
     }
     return conf;
   }
 
-  private static void waitForMaster(Configuration conf) throws IOException {
+  private void waitForMaster() throws IOException {
     LOG.info("Waiting for master...");
-    HTable t = new HTable(conf, HConstants.META_TABLE_NAME);
-    ResultScanner s = t.getScanner(new Scan());
+    ResultScanner s = remoteMetaTable.getScanner(new Scan());
     while (s.next() != null) {
       continue;
     }
+    s.close();
     LOG.info("Master reported in. Continuing.");
   }
 
@@ -112,8 +126,17 @@ public class HBaseServiceController {
     if (proxy != null) {
       proxy.stop();
     }
-    service.destroyCluster(clusterSpec);
+    if (service != null) {
+      service.destroyCluster(clusterSpec);
+    }
     running = false;
   }
 
+  public RemoteAdmin getRemoteAdmin() {
+    return remoteAdmin;
+  }
+
+  public RemoteHTable getRemoteHTable(String tableName) {
+    return new RemoteHTable(restClient, getConfiguration(), tableName, null);
+  }
 }

Modified: incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java?rev=1064840&r1=1064839&r2=1064840&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java (original)
+++ incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java Fri Jan 28 19:42:58 2011
@@ -20,13 +20,14 @@ package org.apache.whirr.service.hbase.i
 
 import static org.junit.Assert.assertTrue;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.rest.client.RemoteAdmin;
+import org.apache.hadoop.hbase.rest.client.RemoteHTable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -34,17 +35,22 @@ import org.junit.Test;
 
 public class HBaseServiceTest {
 
+  private static final String TABLE = "testtable";
   private static final byte [] ROW = Bytes.toBytes("testRow");
-  private static final byte [] FAMILY = Bytes.toBytes("testFamily");
+  private static final byte [] FAMILY1 = Bytes.toBytes("testFamily1");
+  private static final byte [] FAMILY2 = Bytes.toBytes("testFamily2");
   private static final byte [] QUALIFIER = Bytes.toBytes("testQualifier");
   private static final byte [] VALUE = Bytes.toBytes("testValue");
 
+  private static RemoteAdmin remoteAdmin = null;
+
   private static HBaseServiceController controller =
     HBaseServiceController.getInstance();
 
   @BeforeClass
   public static void setUp() throws Exception {
     controller.ensureClusterRunning();
+    remoteAdmin = controller.getRemoteAdmin();
   }
 
   @AfterClass
@@ -54,19 +60,26 @@ public class HBaseServiceTest {
 
   @Test
   public void test() throws Exception {
-    Configuration conf = controller.getConfiguration();
-    HBaseTestingUtility testUtil = new HBaseTestingUtility(conf);
-    byte [] table = Bytes.toBytes("testtable");
-    HTable ht = testUtil.createTable(table, FAMILY);
+    HTableDescriptor td = new HTableDescriptor(TABLE);
+    HColumnDescriptor cd = new HColumnDescriptor(FAMILY1);
+    td.addFamily(cd);
+    cd = new HColumnDescriptor(FAMILY2);
+    td.addFamily(cd);
+    remoteAdmin.createTable(td);
+
+    RemoteHTable table = controller.getRemoteHTable(TABLE);
+
     Put put = new Put(ROW);
-    put.add(FAMILY, QUALIFIER, VALUE);
-    ht.put(put);
+    put.add(FAMILY1, QUALIFIER, VALUE);
+    table.put(put);
+
     Scan scan = new Scan();
-    scan.addColumn(FAMILY, table);
-    ResultScanner scanner = ht.getScanner(scan);
+    scan.addColumn(FAMILY2, Bytes.toBytes("bogus"));
+    ResultScanner scanner = table.getScanner(scan);
     Result result = scanner.next();
     assertTrue("Expected null result", result == null);
     scanner.close();
+
     System.out.println("Done.");
   }
 

Modified: incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties?rev=1064840&r1=1064839&r2=1064840&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties (original)
+++ incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties Fri Jan 28 19:42:58 2011
@@ -16,7 +16,14 @@
 # limitations under the License.
 #
 whirr.cluster-name=hbaseclustertest
-whirr.instance-templates=1 zk,1 nn+jt+hbase-master,1 dn+tt+hbase-regionserver
+whirr.instance-templates=1 zk,1 nn+jt+hbase-master,1 hbase-restserver,1 dn+tt+hbase-regionserver
 whirr.provider=${sys:whirr.test.provider}
 whirr.identity=${sys:whirr.test.identity}
 whirr.credential=${sys:whirr.test.credential}
+whirr.private-key-file=${sys:user.home}/.ssh/id_rsa
+# enforce large instance during the test
+whirr.hardware-id=m1.large
+# Ubuntu 10.04 LTS Lucid instance-store - see http://alestic.com/
+whirr.image-id=us-east-1/ami-da0cf8b3
+whirr.location-id=us-east-1
+whirr.hbase.tarball.url=http://archive.apache.org/dist/hbase/hbase-0.89.20100924/hbase-0.89.20100924-bin.tar.gz
\ No newline at end of file