Forráskód Böngészése

HADOOP-1913 Build a Lucene index on an HBase table

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@578987 13f79535-47bb-0310-9956-ffa450edef68
Michael Stack 18 éve
szülő
commit
7d35c9f1e4

+ 2 - 0
src/contrib/hbase/CHANGES.txt

@@ -20,6 +20,8 @@ Trunk (unreleased changes)
                 set of operators, for creating, altering, dropping, inserting,
                 deleting, and selecting, etc., data in hbase.
                 (Inchul Song and Edward Yoon via Stack)
+    HADOOP-1913 Build a Lucene index on an HBase table
+                (Ning Li via Stack)
 
   OPTIMIZATIONS
 

+ 2 - 1
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java

@@ -92,7 +92,8 @@ public class HConnectionManager implements HConstants {
   
   /* encapsulates finding the servers for an HBase instance */
   private static class TableServers implements HConnection, HConstants {
-    private final Log LOG = LogFactory.getLog(this.getClass().getName());
+    private static final Log LOG = LogFactory.getLog(TableServers.class.
+      getName());
     private final Class<? extends HRegionInterface> serverInterfaceClass;
     private final long threadWakeFrequency;
     private final long pause;

+ 9 - 3
src/contrib/hbase/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java

@@ -27,9 +27,12 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
 
 /** Abstract base class for merge tests */
 public abstract class AbstractMergeTestBase extends HBaseTestCase {
+  static final Logger LOG =
+    Logger.getLogger(AbstractMergeTestBase.class.getName());
   protected static final Text COLUMN_NAME = new Text("contents:");
   protected final Random rand = new Random();
   protected HTableDescriptor desc;
@@ -119,8 +122,12 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
     if (dfsCluster != null) {
       dfsCluster.shutdown();
     }
-    if (fs != null) {
-      fs.close();
+    if (this.fs != null) {
+      try {
+        this.fs.close();
+      } catch (IOException e) {
+        LOG.info("During tear down got a " + e.getMessage());
+      }
     }
   }
 
@@ -151,5 +158,4 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase {
     region.getRegionInfo().offLine = true;
     return region;
   }
-
 }

+ 2 - 2
src/contrib/hbase/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java

@@ -97,7 +97,7 @@ public class MiniHBaseCluster implements HConstants {
 
     this.conf = conf;
     this.fs = dfsCluster.getFileSystem();
-    this.cluster = null;
+    this.cluster = dfsCluster;
     init(nRegionNodes);
   }
 
@@ -390,7 +390,7 @@ public class MiniHBaseCluster implements HConstants {
       regionServerThreads.size() + " region server(s)");
   }
   
-  void shutdown() {
+  public void shutdown() {
     MiniHBaseCluster.shutdown(this.masterThread, this.regionThreads);
     
     try {

+ 17 - 11
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestRegionServerAbort.java

@@ -22,6 +22,8 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import java.util.TreeMap;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -30,6 +32,7 @@ import org.apache.log4j.Logger;
  * Tests region server failover when a region server exits.
  */
 public class TestRegionServerAbort extends HBaseClusterTestCase {
+  private final Log LOG = LogFactory.getLog(this.getClass().getName());
   private HTable table;
 
   /** constructor */
@@ -39,7 +42,8 @@ public class TestRegionServerAbort extends HBaseClusterTestCase {
     conf.setInt("ipc.client.connect.max.retries", 5);   // and number of retries
     conf.setInt("hbase.client.retries.number", 5);      // reduce HBase retries
     Logger.getRootLogger().setLevel(Level.WARN);
-    Logger.getLogger(this.getClass().getPackage().getName()).setLevel(Level.DEBUG);
+    Logger.getLogger(this.getClass().getPackage().getName()).
+      setLevel(Level.DEBUG);
   }
   
   /**
@@ -68,14 +72,14 @@ public class TestRegionServerAbort extends HBaseClusterTestCase {
     this.cluster.startRegionServer();
     // Now shutdown the region server and wait for it to go down.
     this.cluster.abortRegionServer(0);
-    this.cluster.waitOnRegionServer(0);
-    
-    // Verify that the client can find the data after the region has been moved
-    // to a different server
-    HScannerInterface scanner =
-      table.obtainScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text());
-
+    LOG.info(this.cluster.waitOnRegionServer(0) + " has been shutdown");
+    HScannerInterface scanner = null;
     try {
+      // Verify that the client can find the data after the region has moved
+      // to a different server
+      scanner =
+        table.obtainScanner(HConstants.COLUMN_FAMILY_ARRAY, new Text());
+      LOG.info("Obtained scanner " + scanner);
       HStoreKey key = new HStoreKey();
       TreeMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
       while (scanner.next(key, results)) {
@@ -83,11 +87,13 @@ public class TestRegionServerAbort extends HBaseClusterTestCase {
         assertEquals(1, results.size());
         byte[] bytes = results.get(HConstants.COLUMN_FAMILY);
         assertNotNull(bytes);
-        assertTrue(tableName.equals(new String(bytes, HConstants.UTF8_ENCODING)));
+        assertTrue(tableName.equals(new String(bytes,
+            HConstants.UTF8_ENCODING)));
       }
-      System.out.println("Success!");
+      LOG.info("Success!");
     } finally {
+      LOG.info("Closing scanner " + scanner);
       scanner.close();
     }
   }
-}
+}

+ 5 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestTableMapReduce.java

@@ -118,7 +118,11 @@ public class TestTableMapReduce extends MultiRegionTable {
     }
     
     if (fs != null) {
-      fs.close();
+      try {
+        fs.close();
+      } catch (IOException e) {
+        LOG.info("During tear down got a " + e.getMessage());
+      }
     }
   }