Author: stack
Date: Thu Jul  5 12:50:04 2007
New Revision: 553620

URL: http://svn.apache.org/viewvc?view=rev&rev=553620
Log:
HADOOP-1531 Add RowFilter to HRegion.HScanner.
Adds a row/column filter interface and two implementations: A pager and a
row/column-value regex filter.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
    (openScanner): Add override that specifies a row fliter.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
    (obtainScanner): Add override that specifies a row fliter.
    (ColumnScanner): Add filter parameter to constructor.
M src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
    (getScanner): Add override with filter parameter.
    (next): Add handling of filtering.
A 
src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
A src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java
A 
src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java
    Row-filter interface, exception and implementations.
A 
src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
A 
src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
    Simple pager and regex filter tests.

Added:
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
    
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/
    
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
    
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
Modified:
    lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
    
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java

Modified: lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt?view=diff&rev=553620&r1=553619&r2=553620
==============================================================================
--- lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt (original)
+++ lucene/hadoop/trunk/src/contrib/hbase/CHANGES.txt Thu Jul  5 12:50:04 2007
@@ -38,11 +38,15 @@
  23. HADOOP-1509.  Made methods/inner classes in HRegionServer and HClient 
protected
      instead of private for easier extension. Also made HRegion and 
HRegionInfo public too.
      Added an hbase-default.xml property for specifying what HRegionInterface 
extension to use
-     for proxy server connection. 
+     for proxy server connection. (James Kennedy via Jim Kellerman)
  24. HADOOP-1534. [hbase] Memcache scanner fails if start key not present
  25. HADOOP-1537. Catch exceptions in testCleanRegionServerExit so we can see
      what is failing.
  26. HADOOP-1543 [hbase] Add HClient.tableExists
- 27. HADOOP-1519 [hbase] map/reduce interface for HBase
+ 27. HADOOP-1519 [hbase] map/reduce interface for HBase.  (Vuk Ercegovac and
+     Jim Kellerman)
  28. HADOOP-1523 Hung region server waiting on write locks 
  29. HADOOP-1560 NPE in MiniHBaseCluster on Windows
+ 30. HADOOP-1531 Add RowFilter to HRegion.HScanner
+     Adds a row filtering interface and two implemenentations: A page scanner,
+     and a regex row/column-data matcher. (James Kennedy via Stack)

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java?view=diff&rev=553620&r1=553619&r2=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
 Thu Jul  5 12:50:04 2007
@@ -30,6 +30,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
 import org.apache.hadoop.hbase.io.KeyedData;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.Text;
@@ -1173,18 +1174,34 @@
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
-   * @param columns     - array of columns to return
-   * @param startRow    - starting row in table to scan
-   * @return            - scanner
+   * @param columns array of columns to return
+   * @param startRow starting row in table to scan
+   * @return scanner
    * @throws IOException
    */
   public synchronized HScannerInterface obtainScanner(Text[] columns,
-      Text startRow) throws IOException {
-    
+      Text startRow)
+  throws IOException {
+    return obtainScanner(columns, startRow, null);
+  }
+  
+  /** 
+   * Get a scanner on the current table starting at the specified row.
+   * Return the specified columns.
+   *
+   * @param columns array of columns to return
+   * @param startRow starting row in table to scan
+   * @param filter a row filter using row-key regexp and/or column data filter.
+   * @return scanner
+   * @throws IOException
+   */
+  public synchronized HScannerInterface obtainScanner(Text[] columns,
+      Text startRow, RowFilterInterface filter)
+  throws IOException { 
     if(this.tableServers == null) {
       throw new IllegalStateException("Must open table first");
     }
-    return new ClientScanner(columns, startRow);
+    return new ClientScanner(columns, startRow, filter);
   }
   
   /*
@@ -1388,6 +1405,7 @@
     private int currentRegion;
     private HRegionInterface server;
     private long scannerId;
+    private RowFilterInterface filter;
     
     private void loadRegions() {
       Text firstServer = null;
@@ -1404,11 +1422,15 @@
       this.regions = info.toArray(new RegionLocation[info.size()]);
     }
     
-    ClientScanner(Text[] columns, Text startRow) throws IOException {
+    ClientScanner(Text[] columns, Text startRow, RowFilterInterface filter)
+    throws IOException {
       this.columns = columns;
       this.startRow = startRow;
       this.closed = false;
-      
+      this.filter = filter;
+      if (filter != null) {
+        filter.validate(columns);
+      }
       loadRegions();
       this.currentRegion = -1;
       this.server = null;
@@ -1437,9 +1459,16 @@
           RegionLocation info = this.regions[currentRegion];
           
           try {
-            this.scannerId = 
this.server.openScanner(info.regionInfo.regionName,
-                this.columns, currentRegion == 0 ? this.startRow : 
EMPTY_START_ROW);
-            
+            if (this.filter == null) {
+              this.scannerId = 
this.server.openScanner(info.regionInfo.regionName,
+                      this.columns, currentRegion == 0 ? this.startRow
+                          : EMPTY_START_ROW);
+            } else {
+              this.scannerId = 
this.server.openScanner(info.regionInfo.regionName,
+                      this.columns, currentRegion == 0 ? this.startRow
+                          : EMPTY_START_ROW, filter);
+            }
+
             break;
         
           } catch(NotServingRegionException e) {

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java?view=diff&rev=553620&r1=553619&r2=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
 Thu Jul  5 12:50:04 2007
@@ -15,14 +15,26 @@
  */
 package org.apache.hadoop.hbase;
 
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Random;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.Vector;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.conf.*;
-
-import java.io.*;
-import java.util.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.Text;
 
 /**
  * HRegion stores data for a certain region of a table.  It stores all columns
@@ -974,6 +986,15 @@
    */
   public HInternalScannerInterface getScanner(Text[] cols, Text firstRow)
   throws IOException {
+    return getScanner(cols, firstRow, null);
+  }
+
+  /**
+   * Return an iterator that scans over the HRegion, returning the indicated 
+   * columns for only the rows that match the data filter.  This Iterator must 
be closed by the caller.
+   */
+  public HInternalScannerInterface getScanner(Text[] cols, Text firstRow, 
RowFilterInterface filter) 
+       throws IOException {
     lock.obtainReadLock();
     try {
       TreeSet<Text> families = new TreeSet<Text>();
@@ -986,7 +1007,7 @@
       for (Text family: families) {
         storelist[i++] = stores.get(family);
       }
-      return new HScanner(cols, firstRow, memcache, storelist);
+      return new HScanner(cols, firstRow, memcache, storelist, filter);
     } finally {
       lock.releaseReadLock();
     }
@@ -1262,12 +1283,17 @@
     private HStoreKey[] keys;
     private boolean wildcardMatch;
     private boolean multipleMatchers;
+    private RowFilterInterface dataFilter;
 
     /** Create an HScanner with a handle on many HStores. */
     @SuppressWarnings("unchecked")
-    HScanner(Text[] cols, Text firstRow, HMemcache memcache, HStore[] stores)
+    HScanner(Text[] cols, Text firstRow, HMemcache memcache, HStore[] stores, 
RowFilterInterface filter)
     throws IOException {  
       long scanTime = System.currentTimeMillis();
+      this.dataFilter = filter;
+      if (null != dataFilter) {
+        dataFilter.reset();
+      }
       this.scanners = new HInternalScannerInterface[stores.length + 1];
       for(int i = 0; i < this.scanners.length; i++) {
         this.scanners[i] = null;
@@ -1335,83 +1361,134 @@
       return multipleMatchers;
     }
     
-    /* (non-Javadoc)
+    /*
+     * (non-Javadoc)
      * 
-     * Grab the next row's worth of values.  The HScanner will return the most 
+     * Grab the next row's worth of values. The HScanner will return the most
      * recent data value for each row that is not newer than the target time.
-     *
-     * @see 
org.apache.hadoop.hbase.HInternalScannerInterface#next(org.apache.hadoop.hbase.HStoreKey,
 java.util.TreeMap)
+     * 
+     * If a dataFilter is defined, it will be used to skip rows that do not
+     * match its criteria. It may cause the scanner to stop prematurely if it
+     * knows that it will no longer accept the remaining results.
+     * 
+     * @see 
org.apache.hadoop.hbase.HInternalScannerInterface#next(org.apache.hadoop.hbase.HStoreKey,
+     *      java.util.TreeMap)
      */
-    public boolean next(HStoreKey key, TreeMap<Text, byte []> results)
+    public boolean next(HStoreKey key, TreeMap<Text, byte[]> results)
     throws IOException {
-      // Find the lowest-possible key.
-      Text chosenRow = null;
-      long chosenTimestamp = -1;
-      for(int i = 0; i < this.keys.length; i++) {
-        if(scanners[i] != null
-            && (chosenRow == null
-                || (keys[i].getRow().compareTo(chosenRow) < 0)
-                || ((keys[i].getRow().compareTo(chosenRow) == 0)
-                    && (keys[i].getTimestamp() > chosenTimestamp)))) {
-          
-          chosenRow = new Text(keys[i].getRow());
-          chosenTimestamp = keys[i].getTimestamp();
+      boolean filtered = true;
+      boolean moreToFollow = true;
+      while (filtered && moreToFollow) {
+        // Find the lowest-possible key.
+        Text chosenRow = null;
+        long chosenTimestamp = -1;
+        for (int i = 0; i < this.keys.length; i++) {
+          if (scanners[i] != null &&
+              (chosenRow == null ||
+              (keys[i].getRow().compareTo(chosenRow) < 0) ||
+              ((keys[i].getRow().compareTo(chosenRow) == 0) &&
+              (keys[i].getTimestamp() > chosenTimestamp)))) {
+            chosenRow = new Text(keys[i].getRow());
+            chosenTimestamp = keys[i].getTimestamp();
+          }
         }
-      }
 
-      // Store the key and results for each sub-scanner. Merge them as 
appropriate.
-      boolean insertedItem = false;
-      if(chosenTimestamp > 0) {
-        key.setRow(chosenRow);
-        key.setVersion(chosenTimestamp);
-        key.setColumn(new Text(""));
-
-        for(int i = 0; i < scanners.length; i++) {
-          while((scanners[i] != null)
-              && (keys[i].getRow().compareTo(chosenRow) == 0)) {
-            // If we are doing a wild card match or there are multiple matchers
-            // per column, we need to scan all the older versions of this row
-            // to pick up the rest of the family members
-            
-            if(!wildcardMatch
-                && !multipleMatchers
-                && (keys[i].getTimestamp() != chosenTimestamp)) {
-              break;
-            }
+        // Filter whole row by row key?
+        filtered = dataFilter != null? dataFilter.filter(chosenRow) : false;
 
-            // NOTE: We used to do results.putAll(resultSets[i]);
-            //       but this had the effect of overwriting newer
-            //       values with older ones. So now we only insert
-            //       a result if the map does not contain the key.
-            
-            for(Map.Entry<Text, byte []> e: resultSets[i].entrySet()) {
-              if(!results.containsKey(e.getKey())) {
-                results.put(e.getKey(), e.getValue());
-                insertedItem = true;
+        // Store the key and results for each sub-scanner. Merge them as
+        // appropriate.
+        if (chosenTimestamp > 0 && !filtered) {
+          key.setRow(chosenRow);
+          key.setVersion(chosenTimestamp);
+          key.setColumn(new Text(""));
+
+          for (int i = 0; i < scanners.length && !filtered; i++) {
+
+            while ((scanners[i] != null
+                && !filtered
+                && moreToFollow)
+                && (keys[i].getRow().compareTo(chosenRow) == 0)) {
+              // If we are doing a wild card match or there are multiple
+              // matchers
+              // per column, we need to scan all the older versions of this row
+              // to pick up the rest of the family members
+
+              if (!wildcardMatch
+                  && !multipleMatchers
+                  && (keys[i].getTimestamp() != chosenTimestamp)) {
+                break;
               }
-            }
 
-            resultSets[i].clear();
-            if(! scanners[i].next(keys[i], resultSets[i])) {
-              closeScanner(i);
-            }
-          }
+              // Filter out null criteria columns that are not null
+              if (dataFilter != null) {
+                filtered = dataFilter.filterNotNull(resultSets[i]);
+              }
+
+              // NOTE: We used to do results.putAll(resultSets[i]);
+              // but this had the effect of overwriting newer
+              // values with older ones. So now we only insert
+              // a result if the map does not contain the key.
+
+              for (Map.Entry<Text, byte[]> e : resultSets[i].entrySet()) {
+                if (!filtered && moreToFollow &&
+                    !results.containsKey(e.getKey())) {
+                  if (dataFilter != null) {
+                    // Filter whole row by column data?
+                    filtered =
+                        dataFilter.filter(chosenRow, e.getKey(), e.getValue());
+                    if (filtered) {
+                      results.clear();
+                      break;
+                    }
+                  }
+                  results.put(e.getKey(), e.getValue());
+                }
+              }
 
-          // If the current scanner is non-null AND has a lower-or-equal
-          // row label, then its timestamp is bad.  We need to advance it.
+              resultSets[i].clear();
+              if (!scanners[i].next(keys[i], resultSets[i])) {
+                closeScanner(i);
+              }
+            }
 
-          while((scanners[i] != null)
-              && (keys[i].getRow().compareTo(chosenRow) <= 0)) {
-            
-            resultSets[i].clear();
-            if(! scanners[i].next(keys[i], resultSets[i])) {
-              closeScanner(i);
+            // If the current scanner is non-null AND has a lower-or-equal
+            // row label, then its timestamp is bad. We need to advance it.
+            while ((scanners[i] != null) &&
+                (keys[i].getRow().compareTo(chosenRow) <= 0)) {
+              resultSets[i].clear();
+              if (!scanners[i].next(keys[i], resultSets[i])) {
+                closeScanner(i);
+              }
             }
+          }          
+        }
+        
+        moreToFollow = chosenTimestamp > 0;
+        
+        if (dataFilter != null) {
+          if (moreToFollow && !filtered) {
+            dataFilter.acceptedRow(chosenRow);
+          }
+          if (dataFilter.filterAllRemaining()) {
+            moreToFollow = false;
+            LOG.debug("page limit");
+          }
+        }        
+      }
+      
+      // Make sure scanners closed if no more results
+      if (!moreToFollow) {
+        for (int i = 0; i < scanners.length; i++) {
+          if (null != scanners[i]) {
+            closeScanner(i);
           }
         }
       }
-      return insertedItem;
+      
+      return moreToFollow;
     }
+
 
     /** Shut down a single scanner */
     void closeScanner(int i) {

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java?view=diff&rev=553620&r1=553619&r2=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInterface.java
 Thu Jul  5 12:50:04 2007
@@ -17,6 +17,7 @@
 
 import java.io.IOException;
 
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
 import org.apache.hadoop.hbase.io.KeyedData;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.VersionedProtocol;
@@ -200,6 +201,20 @@
    * @throws IOException
    */
   public long openScanner(Text regionName, Text[] columns, Text startRow)
+  throws IOException;
+  
+  /**
+   * Opens a remote scanner with a RowFilter.
+   * 
+   * @param regionName name of region to scan
+   * @param columns columns to scan
+   * @param startRow starting row to scan
+   * @param filter RowFilter for filtering results at the row-level.
+   *
+   * @return scannerId scanner identifier used in other calls
+   * @throws IOException
+   */
+  public long openScanner(Text regionName, Text[] columns, Text startRow, 
RowFilterInterface filter)
   throws IOException;
 
   /**

Modified: 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java?view=diff&rev=553620&r1=553619&r2=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
 (original)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
 Thu Jul  5 12:50:04 2007
@@ -33,6 +33,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
 import org.apache.hadoop.hbase.io.KeyedData;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.RetryProxy;
@@ -1150,15 +1151,25 @@
     }
   }
   
-  /* (non-Javadoc)
-   * @see 
org.apache.hadoop.hbase.HRegionInterface#openScanner(org.apache.hadoop.io.Text, 
org.apache.hadoop.io.Text[], org.apache.hadoop.io.Text)
+  /**
+   * [EMAIL PROTECTED]
+   */
+  public long openScanner(final Text regionName, final Text[] cols,
+      final Text firstRow)
+  throws IOException{
+    return openScanner(regionName, cols, firstRow, null);
+  }
+
+  /**
+   * [EMAIL PROTECTED]
    */
-  public long openScanner(Text regionName, Text[] cols, Text firstRow)
+  public long openScanner(Text regionName, Text[] cols, Text firstRow,
+      final RowFilterInterface filter)
   throws IOException {
     HRegion r = getRegion(regionName);
     long scannerId = -1L;
     try {
-      HInternalScannerInterface s = r.getScanner(cols, firstRow);
+      HInternalScannerInterface s = r.getScanner(cols, firstRow, filter);
       scannerId = rand.nextLong();
       String scannerName = String.valueOf(scannerId);
       synchronized(scanners) {

Added: 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java?view=auto&rev=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
 (added)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
 Thu Jul  5 12:50:04 2007
@@ -0,0 +1,31 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.filter;
+
+/**
+ * Used to indicate an invalid RowFilter.
+ */
+public class InvalidRowFilterException extends RuntimeException {
+  private static final long serialVersionUID = 2667894046345657865L;
+
+  public InvalidRowFilterException() {
+    super();
+  }
+
+  public InvalidRowFilterException(String s) {
+    super(s);
+  }
+}
\ No newline at end of file

Added: 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java?view=auto&rev=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java
 (added)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java
 Thu Jul  5 12:50:04 2007
@@ -0,0 +1,137 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.Text;
+
+/**
+ * Implementation of RowFilterInterface that limits results to a specific page
+ * size. It terminates scanning once the number of filter-passed results is >=
+ * the given page size.
+ * 
+ * <p>
+ * Note that this filter cannot guarantee that the number of results returned
+ * to a client are <= page size. This is because the filter is applied
+ * separately on different region servers. It does however optimize the scan of
+ * individual HRegions by making sure that the page size is never exceeded
+ * locally.
+ * </p>
+ */
+public class PageRowFilter implements RowFilterInterface {
+
+  private long pageSize = Long.MAX_VALUE;
+  private int rowsAccepted = 0;
+
+  /**
+   * Default constructor, filters nothing. Required though for RPC
+   * deserialization.
+   */
+  public PageRowFilter() {
+    super();
+  }
+
+  /**
+   * Constructor that takes a maximum page size.
+   * 
+   * @param pageSize Maximum result size.
+   */
+  public PageRowFilter(final long pageSize) {
+    this.pageSize = pageSize;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void validate(@SuppressWarnings("unused") final Text[] columns) {
+    // Doesn't filter columns
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void reset() {
+    rowsAccepted = 0;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void acceptedRow(@SuppressWarnings("unused") final Text key) {
+    rowsAccepted++;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filterAllRemaining() {
+    if (this.rowsAccepted > this.pageSize) {
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filter(@SuppressWarnings("unused") final Text rowKey) {
+    return filterAllRemaining();
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filter(@SuppressWarnings("unused") final Text rowKey,
+    @SuppressWarnings("unused") final Text colKey,
+    @SuppressWarnings("unused") final byte[] data) {
+    return filterAllRemaining();
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filterNotNull(@SuppressWarnings("unused")
+      final TreeMap<Text, byte[]> columns) {
+    return filterAllRemaining();
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void readFields(final DataInput in) throws IOException {
+    this.pageSize = in.readLong();
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void write(final DataOutput out) throws IOException {
+    out.writeLong(pageSize);
+  }
+}
\ No newline at end of file

Added: 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java?view=auto&rev=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
 (added)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
 Thu Jul  5 12:50:04 2007
@@ -0,0 +1,300 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.Map.Entry;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.io.Text;
+
+/**
+ * Implementation of RowFilterInterface that can filter by rowkey regular
+ * expression and/or individual column values (equals comparison only).
+ * Multiple column filters imply an implicit conjunction of filter criteria.
+ */
+public class RegExpRowFilter implements RowFilterInterface {
+
+  private Pattern rowKeyPattern = null;
+  private String rowKeyRegExp = null;
+
+  private Map<Text, byte[]> equalsMap = new HashMap<Text, byte[]>();
+  private Set<Text> nullColumns = new HashSet<Text>();
+
+  /**
+   * Default constructor, filters nothing. Required though for RPC
+   * deserialization.
+   */
+  public RegExpRowFilter() {
+    super();
+  }
+
+  /**
+   * Constructor that takes a row key regular expression to filter on.
+   * 
+   * @param rowKeyRegExp
+   */
+  public RegExpRowFilter(final String rowKeyRegExp) {
+    this.rowKeyRegExp = rowKeyRegExp;
+  }
+
+  /**
+   * Constructor that takes a row key regular expression to filter on.
+   * 
+   * @param rowKeyRegExp
+   * @param columnFilter
+   */
+  public RegExpRowFilter(final String rowKeyRegExp,
+      final Map<Text, byte[]> columnFilter) {
+    this.rowKeyRegExp = rowKeyRegExp;
+    this.setColumnFilters(columnFilter);
+  }
+  
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void acceptedRow(@SuppressWarnings("unused") final Text key) {
+    //doesn't care
+  }
+
+  /**
+   * Specify a value that must be matched for the given column.
+   * 
+   * @param colKey
+   *          the column to match on
+   * @param value
+   *          the value that must equal the stored value.
+   */
+  public void setColumnFilter(final Text colKey, final byte[] value) {
+    if (null == value) {
+      nullColumns.add(colKey);
+    } else {
+      equalsMap.put(colKey, value);
+    }
+  }
+
+  /**
+   * Set column filters for a number of columns.
+   * 
+   * @param columnFilter
+   *          Map of columns with value criteria.
+   */
+  public void setColumnFilters(final Map<Text, byte[]> columnFilter) {
+    if (null == columnFilter) {
+      nullColumns.clear();
+      equalsMap.clear();
+    } else {
+      for (Entry<Text, byte[]> entry : columnFilter.entrySet()) {
+        setColumnFilter(entry.getKey(), entry.getValue());
+      }
+    }
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void reset() {
+    // Nothing to reset
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filterAllRemaining() {
+    return false;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filter(final Text rowKey) {
+    if (filtersByRowKey() && rowKey != null) {
+      return !getRowKeyPattern().matcher(rowKey.toString()).matches();
+    }
+    return false;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filter(final Text rowKey, final Text colKey,
+      final byte[] data) {
+    if (filter(rowKey)) {
+      return true;
+    }
+    if (filtersByColumnValue()) {
+      byte[] filterValue = equalsMap.get(colKey);
+      if (null != filterValue) {
+        return !Arrays.equals(filterValue, data);
+      }
+    }
+    if (nullColumns.contains(colKey)) {
+      if (data != null && !Arrays.equals(HConstants.DELETE_BYTES.get(), data)) 
{
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filterNotNull(final TreeMap<Text, byte[]> columns) {
+    for (Entry<Text, byte[]> col : columns.entrySet()) {
+      if (nullColumns.contains(col.getKey())
+          && !Arrays.equals(HConstants.DELETE_BYTES.get(), col.getValue())) {
+        return true;
+      }
+    }
+    for (Text col : equalsMap.keySet()) {
+      if (!columns.containsKey(col)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  private boolean filtersByColumnValue() {
+    return equalsMap != null && equalsMap.size() > 0;
+  }
+
+  private boolean filtersByRowKey() {
+    return null != rowKeyPattern || null != rowKeyRegExp;
+  }
+
+  private String getRowKeyRegExp() {
+    if (null == rowKeyRegExp && rowKeyPattern != null) {
+      rowKeyRegExp = rowKeyPattern.toString();
+    }
+    return rowKeyRegExp;
+  }
+
+  private Pattern getRowKeyPattern() {
+    if (rowKeyPattern == null && rowKeyRegExp != null) {
+      rowKeyPattern = Pattern.compile(rowKeyRegExp);
+    }
+    return rowKeyPattern;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void readFields(final DataInput in) throws IOException {
+    boolean hasRowKeyPattern = in.readBoolean();
+    if (hasRowKeyPattern) {
+      rowKeyRegExp = in.readLine();
+    }
+    // equals map
+    equalsMap.clear();
+    int size = in.readInt();
+    for (int i = 0; i < size; i++) {
+      Text key = new Text();
+      key.readFields(in);
+      int len = in.readInt();
+      byte[] value = null;
+      if (len >= 0) {
+        value = new byte[len];
+        in.readFully(value);
+      }
+      setColumnFilter(key, value);
+    }
+    // nullColumns
+    nullColumns.clear();
+    size = in.readInt();
+    for (int i = 0; i < size; i++) {
+      Text key = new Text();
+      key.readFields(in);
+      setColumnFilter(key, null);
+    }
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void validate(final Text[] columns) {
+    Set<Text> invalids = new HashSet<Text>();
+    for (Text colKey : getFilterColumns()) {
+      boolean found = false;
+      for (Text col : columns) {
+        if (col.equals(colKey)) {
+          found = true;
+          break;
+        }
+      }
+      if (!found) {
+        invalids.add(colKey);
+      }
+    }
+
+    if (invalids.size() > 0) {
+      throw new InvalidRowFilterException(String.format(
+          "RowFilter contains criteria on columns %s not in %s", invalids,
+          Arrays.toString(columns)));
+    }
+  }
+
+  private Set<Text> getFilterColumns() {
+    Set<Text> cols = new HashSet<Text>();
+    cols.addAll(equalsMap.keySet());
+    cols.addAll(nullColumns);
+    return cols;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void write(final DataOutput out) throws IOException {
+    if (!filtersByRowKey()) {
+      out.writeBoolean(false);
+    } else {
+      out.writeBoolean(true);
+      out.writeChars(getRowKeyRegExp());
+    }
+
+    // equalsMap
+    out.writeInt(equalsMap.size());
+    for (Entry<Text, byte[]> entry : equalsMap.entrySet()) {
+      entry.getKey().write(out);
+      byte[] value = entry.getValue();
+      out.writeInt(value.length);
+      out.write(value);
+    }
+
+    // null columns
+    out.writeInt(nullColumns.size());
+    for (Text col : nullColumns) {
+      col.write(out);
+    }
+  }
+}
\ No newline at end of file

Added: 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java?view=auto&rev=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java
 (added)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java
 Thu Jul  5 12:50:04 2007
@@ -0,0 +1,106 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.util.TreeMap;
+
+import org.apache.hadoop.hbase.HRegion;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * 
+ * Interface used for row-level filters applied to HRegion.HScanner scan
+ * results during calls to next().
+ */
+public interface RowFilterInterface extends Writable {
+
+  /**
+   * Resets the state of the filter. Used prior to the start of a Region scan.
+   * 
+   */
+  void reset();
+
+  /**
+   * Called to let filter know that the specified row has been included in the
+   * results (passed all filtering). With out HScanner calling this, the filter
+   * does not know if a row passed filtering even if it passed the row itself
+   * because other filters may have failed the row. E.g. when this filter is a
+   * member of a RowFilterSet with an OR operator.
+   * 
+   * @see RowFilterSet
+   * @param key
+   */
+  void acceptedRow(final Text key);
+
+  /**
+   * Determines if the filter has decided that all remaining results should be
+   * filtered (skipped). This is used to prevent the scanner from scanning a
+   * the rest of the HRegion when for sure the filter will exclude all
+   * remaining rows.
+   * 
+   * @return true if the filter intends to filter all remaining rows.
+   */
+  boolean filterAllRemaining();
+
+  /**
+   * Filters on just a row key.
+   * 
+   * @param rowKey
+   * @return true if given row key is filtered and row should not be processed.
+   */
+  boolean filter(final Text rowKey);
+
+  /**
+   * Filters on row key and/or a column key.
+   * 
+   * @param rowKey
+   *          row key to filter on. May be null for no filtering of row key.
+   * @param colKey
+   *          column whose data will be filtered
+   * @param data
+   *          column value
+   * @return true if row filtered and should not be processed.
+   */
+  boolean filter(final Text rowKey, final Text colKey, final byte[] data);
+
+  /**
+   * Filters row if given columns are non-null and have null criteria or if
+   * there exists criteria on columns not included in the column set. A column
+   * is considered null if it:
+   * <ul>
+   * <li>Is not included in the given columns.</li>
+   * <li>Has a value of HConstants.DELETE_BYTES</li>
+   * </ul>
+   * 
+   * @param columns
+   * @return true if null/non-null criteria not met.
+   */
+  boolean filterNotNull(final TreeMap<Text, byte[]> columns);
+
+  /**
+   * Validates that this filter applies only to a subset of the given columns.
+   * This check is done prior to opening of scanner due to the limitation that
+   * filtering of columns is dependent on the retrieval of those columns within
+   * the HRegion. Criteria on columns that are not part of a scanner's column
+   * list will be ignored. In the case of null value filters, all rows will 
pass
+   * the filter. This behavior should be 'undefined' for the user and therefore
+   * not permitted.
+   * 
+   * @param columns
+   */
+  void validate(final Text[] columns);
+}
\ No newline at end of file

Added: 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java?view=auto&rev=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
 (added)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
 Thu Jul  5 12:50:04 2007
@@ -0,0 +1,230 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.Text;
+
+/**
+ * Implementation of RowFilterInterface that represents a set of RowFilters
+ * which will be evaluated with a specified boolean operator AND/OR. Since you
+ * can use RowFilterSets as children of RowFilterSet, you can create a
+ * hierarchy of filters to be evaluated.
+ */
+public class RowFilterSet implements RowFilterInterface {
+
+  enum Operator {
+    AND, OR
+  }
+
+  private Operator operator = Operator.AND;
+  private Set<RowFilterInterface> filters = new HashSet<RowFilterInterface>();
+
+  /**
+   * Default constructor, filters nothing. Required though for RPC
+   * deserialization.
+   */
+  public RowFilterSet() {
+    super();
+  }
+
+  /**
+   * Constructor that takes a set of RowFilters. The default operator AND is
+   * assumed.
+   * 
+   * @param rowFilters
+   */
+  public RowFilterSet(final Set<RowFilterInterface> rowFilters) {
+    this.filters = rowFilters;
+  }
+
+  /**
+   * Constructor that takes a set of RowFilters and an operator.
+   * 
+   * @param operator Operator to process filter set with.
+   * @param rowFilters Set of row filters.
+   */
+  public RowFilterSet(final Operator operator,
+      final Set<RowFilterInterface> rowFilters) {
+    this.filters = rowFilters;
+    this.operator = operator;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void validate(final Text[] columns) {
+    for (RowFilterInterface filter : filters) {
+      filter.validate(columns);
+    }
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void reset() {
+    for (RowFilterInterface filter : filters) {
+      filter.reset();
+    }
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void acceptedRow(final Text key) {
+    for (RowFilterInterface filter : filters) {
+      filter.acceptedRow(key);
+    }
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filterAllRemaining() {
+    boolean result = operator == Operator.OR;
+    for (RowFilterInterface filter : filters) {
+      if (operator == Operator.AND) {
+        if (filter.filterAllRemaining()) {
+          return true;
+        }
+      } else if (operator == Operator.OR) {
+        if (!filter.filterAllRemaining()) {
+          return false;
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filter(final Text rowKey) {
+    boolean result = operator == Operator.OR;
+    for (RowFilterInterface filter : filters) {
+      if (operator == Operator.AND) {
+        if (filter.filterAllRemaining() || filter.filter(rowKey)) {
+          return true;
+        }
+      } else if (operator == Operator.OR) {
+        if (!filter.filterAllRemaining() && !filter.filter(rowKey)) {
+          return false;
+        }
+      }
+    }
+    return result;
+
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filter(final Text rowKey, final Text colKey, final byte[] 
data) {
+    boolean result = operator == Operator.OR;
+    for (RowFilterInterface filter : filters) {
+      if (operator == Operator.AND) {
+        if (filter.filterAllRemaining() || filter.filter(rowKey, colKey, 
data)) {
+          return true;
+        }
+      } else if (operator == Operator.OR) {
+        if (!filter.filterAllRemaining()
+            && !filter.filter(rowKey, colKey, data)) {
+          return false;
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public boolean filterNotNull(final TreeMap<Text, byte[]> columns) {
+    boolean result = operator == Operator.OR;
+    for (RowFilterInterface filter : filters) {
+      if (operator == Operator.AND) {
+        if (filter.filterAllRemaining() || filter.filterNotNull(columns)) {
+          return true;
+        }
+      } else if (operator == Operator.OR) {
+        if (!filter.filterAllRemaining() && !filter.filterNotNull(columns)) {
+          return false;
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void readFields(final DataInput in) throws IOException {
+    byte opByte = in.readByte();
+    operator = Operator.values()[opByte];
+    int size = in.readInt();
+    if (size > 0) {
+      filters = new HashSet<RowFilterInterface>();
+      try {
+        for (int i = 0; i < size; i++) {
+          String className = in.readUTF();
+          Class<?> clazz = Class.forName(className);
+          RowFilterInterface filter;
+          filter = (RowFilterInterface) clazz.newInstance();
+          filter.readFields(in);
+          filters.add(filter);
+        }
+      } catch (InstantiationException e) {
+        throw new RuntimeException("Failed to deserialize RowFilterInterface.",
+            e);
+      } catch (IllegalAccessException e) {
+        throw new RuntimeException("Failed to deserialize RowFilterInterface.",
+            e);
+      } catch (ClassNotFoundException e) {
+        throw new RuntimeException("Failed to deserialize RowFilterInterface.",
+            e);
+      }
+    }
+
+  }
+
+  /**
+   * 
+   * [EMAIL PROTECTED]
+   */
+  public void write(final DataOutput out) throws IOException {
+    out.writeByte(operator.ordinal());
+    out.writeInt(filters.size());
+    for (RowFilterInterface filter : filters) {
+      out.writeUTF(filter.getClass().getName());
+      filter.write(out);
+    }
+  }
+
+}

Added: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java?view=auto&rev=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
 (added)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
 Thu Jul  5 12:50:04 2007
@@ -0,0 +1,48 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import org.apache.hadoop.io.Text;
+
+import junit.framework.TestCase;
+
+public class TestPageRowFilter extends TestCase {
+  public void testPageSize() throws Exception {
+    final int pageSize = 3;
+    RowFilterInterface filter = new PageRowFilter(pageSize);
+    testFiltersBeyondPageSize(filter, pageSize);
+    // Test reset works by going in again.
+    filter.reset();
+    testFiltersBeyondPageSize(filter, pageSize);
+  }
+  
+  private void testFiltersBeyondPageSize(final RowFilterInterface filter,
+      final int pageSize) {
+    for (int i = 0; i < (pageSize * 2); i++) {
+      Text row = new Text(Integer.toString(i));
+      boolean filterOut = filter.filter(row);
+      if (!filterOut) {
+        assertFalse("Disagrees with 'filter'", filter.filterAllRemaining());
+        filter.acceptedRow(row);
+      } else {
+        // Once we have all for a page, calls to filterAllRemaining should
+        // stay true.
+        assertTrue("Disagrees with 'filter'", filter.filterAllRemaining());
+        assertTrue(i >= pageSize);
+      }
+    }
+  }
+}

Added: 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
URL: 
http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java?view=auto&rev=553620
==============================================================================
--- 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
 (added)
+++ 
lucene/hadoop/trunk/src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
 Thu Jul  5 12:50:04 2007
@@ -0,0 +1,81 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.
+ */
+package org.apache.hadoop.hbase.filter;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.io.Text;
+
+public class TestRegExpRowFilter extends TestCase {
+  TreeMap<Text, byte []> colvalues;
+  RowFilterInterface filter;
+  final char FIRST_CHAR = 'a';
+  final char LAST_CHAR = 'e';
+  byte [] GOOD_BYTES = "abc".getBytes();
+  final String HOST_PREFIX = "org.apache.site-";
+  
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    this.colvalues = new TreeMap<Text, byte[]>();
+    for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
+      colvalues.put(new Text(new String(new char [] {c})), GOOD_BYTES);
+    }
+    this.filter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues);
+  }
+  
+  public void testRegexOnRow() throws Exception {
+    for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
+      Text t = createRow(c);
+      assertFalse("Failed with characer " + c, filter.filter(t));
+    }
+    String yahooSite = "com.yahoo.www";
+    assertTrue("Failed with character " +
+      yahooSite, filter.filter(new Text(yahooSite)));
+  }
+  
+  public void testRegexOnRowAndColumn() throws Exception {
+    for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
+      Text t = createRow(c);
+      for (Map.Entry<Text, byte []> e: this.colvalues.entrySet()) {
+        assertFalse("Failed on " + c,
+          this.filter.filter(t, e.getKey(), e.getValue()));
+      }
+    }
+    // Try a row and column I know will pass.
+    char c = 'c';
+    Text r = createRow(c);
+    Text col = new Text(Character.toString(c));
+    assertFalse("Failed with character " + c,
+      filter.filter(r, col, GOOD_BYTES));
+    // Do same but with bad bytes.
+    assertTrue("Failed with character " + c,
+      filter.filter(r, col, "badbytes".getBytes()));
+    // Do with good bytes but bad column name.  Should not filter out.
+    assertFalse("Failed with character " + c,
+      filter.filter(r, new Text("badcolumn"), GOOD_BYTES));
+    // Good column, good bytes but bad row.
+    assertTrue("Failed with character " + c,
+      filter.filter(new Text("bad row"), new Text("badcolumn"), GOOD_BYTES));
+  }
+  
+  private Text createRow(final char c) {
+    return new Text(HOST_PREFIX + Character.toString(c));
+  }
+}


Reply via email to