You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/04/28 08:05:55 UTC

svn commit: r769257 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/filter/ src/java/org/apache/hadoop/hbase/mapred/ src/java/org/apache/hadoop/hbase/util/ src/test/org/apache/hadoop/hbase/client/ src/tes...

Author: stack
Date: Tue Apr 28 06:05:53 2009
New Revision: 769257

URL: http://svn.apache.org/viewvc?rev=769257&view=rev
Log:
HBASE-1320  hbase-1234 broke filter tests

Added:
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java
Removed:
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestColumnValueFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestStopRowFilter.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestWhileMatchRowFilter.java
Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Tue Apr 28 06:05:53 2009
@@ -89,6 +89,7 @@
    HBASE-1340  Fix new javadoc warnings (Evgeny Ryabitskiy via Stack)
    HBASE-1287  Partitioner class not used in TableMapReduceUtil.initTableReduceJob()
                (Lars George and Billy Pearson via Stack)
+   HBASE-1320  hbase-1234 broke filter tests
 
   IMPROVEMENTS
    HBASE-1089  Add count of regions on filesystem to master UI; add percentage

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/KeyValue.java Tue Apr 28 06:05:53 2009
@@ -512,7 +512,7 @@
    * @return Clone of this key with type set to <code>newtype</code>
    */
   private KeyValue createKey(final Type newtype) {
-    int keylength= getKeyLength();
+    int keylength = getKeyLength();
     int l = keylength + ROW_OFFSET;
     byte [] other = new byte[l];
     System.arraycopy(getBuffer(), getOffset(), other, 0, l);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/ColumnValueFilter.java Tue Apr 28 06:05:53 2009
@@ -37,11 +37,10 @@
  * an operator (equal, greater, not equal, etc) and either a byte [] value or a
  * byte [] comparator. If we have a byte [] value then we just do a
  * lexicographic compare. If this is not sufficient (eg you want to deserialize
- * a long and then compare it to a fixed long value, then you can pass in your
+ * a long and then compare it to a fixed long value), then you can pass in your
  * own comparator instead.
  */
 public class ColumnValueFilter implements RowFilterInterface {
-
   /** Comparison operators. */
   public enum CompareOp {
     /** less than */
@@ -132,6 +131,7 @@
     return false;
   }
 
+  
   public boolean filterColumn(final byte[] rowKey,
       final byte[] colKey, final byte[] data) {
     if (!filterIfColumnMissing) {
@@ -140,18 +140,25 @@
     if (!Arrays.equals(colKey, columnName)) {
       return false;
     }
-    return filterColumnValue(data); 
+    return filterColumnValue(data, 0, data.length); 
   }
 
 
   public boolean filterColumn(byte[] rowKey, int roffset, int rlength,
-      byte[] colunmName, int coffset, int clength, byte[] columnValue,
+      byte[] cn, int coffset, int clength, byte[] columnValue,
       int voffset, int vlength) {
-    if (true) throw new RuntimeException("Not yet implemented");
-    return false;
+    if (!filterIfColumnMissing) {
+      return false; // Must filter on the whole row
+    }
+    if (Bytes.compareTo(cn, coffset, clength,
+        this.columnName, 0, this.columnName.length) != 0) {
+      return false;
+    }
+    return filterColumnValue(columnValue, voffset, vlength);
   }
 
-  private boolean filterColumnValue(final byte [] data) {
+  private boolean filterColumnValue(final byte [] data, final int offset,
+      final int length) {
     int compareResult;
     if (comparator != null) {
       compareResult = comparator.compareTo(data);
@@ -192,13 +199,37 @@
       if (colCell == null) {
         return false;
       }
-      return this.filterColumnValue(colCell.getValue());
+      byte [] v = colCell.getValue();
+      return this.filterColumnValue(v, 0, v.length);
   }
 
-
   public boolean filterRow(List<KeyValue> results) {
-    if (true) throw new RuntimeException("Not yet implemented");
-    return false;
+    if (results == null) return false;
+    KeyValue found = null;
+    if (filterIfColumnMissing) {
+      boolean doesntHaveIt = true;
+      for (KeyValue kv: results) {
+        if (kv.matchingColumn(columnName)) {
+          doesntHaveIt = false;
+          found = kv;
+          break;
+        }
+      }
+      if (doesntHaveIt) return doesntHaveIt;
+    }
+    if (found == null) {
+      for (KeyValue kv: results) {
+        if (kv.matchingColumn(columnName)) {
+          found = kv;
+          break;
+        }
+      }
+    }
+    if (found == null) {
+      return false;
+    }
+    return this.filterColumnValue(found.getValue(), found.getValueOffset(),
+      found.getValueLength());
   }
 
   private int compare(final byte[] b1, final byte[] b2) {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java Tue Apr 28 06:05:53 2009
@@ -44,16 +44,16 @@
  * 
  * Note that column value filtering in this interface has been replaced by
  * {@link ColumnValueFilter}.
+ * @deprecated This interface doesn't really work well in new KeyValue world.
+ * Needs to be refactored/removed.  Marking it as deprecated till it gets
+ * cleaned up.  Its also inefficient as written.
  */
 public class RegExpRowFilter implements RowFilterInterface {
 
   private Pattern rowKeyPattern = null;
   private String rowKeyRegExp = null;
-
-  @Deprecated
   private Map<byte [], byte[]> equalsMap =
     new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
-  @Deprecated
   private Set<byte []> nullColumns =
     new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
 
@@ -177,12 +177,42 @@
 
 
   public boolean filterColumn(byte[] rowKey, int roffset, int rlength,
-      byte[] colunmName, int coffset, int clength, byte[] columnValue,
+      byte [] colunmName, int coffset, int clength, byte[] columnValue,
       int voffset, int vlength) {
-    if (true) throw new RuntimeException("Not implemented yet");
+    if (filterRowKey(rowKey, roffset, rlength)) {
+      return true;
+    }
+    byte [] colkey = null;
+    if (filtersByColumnValue()) {
+      colkey = getColKey(colunmName, coffset, clength);
+      byte [] filterValue = equalsMap.get(colkey);
+      if (null != filterValue) {
+        return Bytes.compareTo(filterValue, 0, filterValue.length, columnValue,
+          voffset, vlength) != 0;
+      }
+    }
+    if (colkey == null) {
+      colkey = getColKey(colunmName, coffset, clength);
+    }
+    if (nullColumns.contains(colkey)) {
+      if (columnValue != null /* TODO: FIX!!! && !HLogEdit.isDeleted(data)*/) {
+        return true;
+      }
+    }
     return false;
   }
 
+  private byte [] getColKey(final byte [] c, final int offset, final int length) {
+    byte [] colkey = null;
+    if (offset == 0) {
+      colkey = c;
+    } else {
+      colkey = new byte [length];
+      System.arraycopy(c, offset, colkey, 0, length);
+    }
+    return colkey;
+  }
+
   public boolean filterRow(final SortedMap<byte [], Cell> columns) {
     for (Entry<byte [], Cell> col : columns.entrySet()) {
       if (nullColumns.contains(col.getKey())
@@ -198,12 +228,20 @@
     return false;
   }
 
-  public boolean filterRow(List<KeyValue> results) {
-    if (true) throw new RuntimeException("NOT YET IMPLEMENTED");
+  // THIS METHOD IS HORRIDLY EXPENSIVE TO RUN.  NEEDS FIXUP.
+  public boolean filterRow(List<KeyValue> kvs) {
+    for (KeyValue kv: kvs) {
+      byte [] column = kv.getColumn();
+      if (nullColumns.contains(column) && !kv.isDeleteType()) {
+        return true;
+      }
+      if (!equalsMap.containsKey(column)) {
+        return true;
+      }
+    }
     return false;
   }
 
-  @Deprecated
   private boolean filtersByColumnValue() {
     return equalsMap != null && equalsMap.size() > 0;
   }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java Tue Apr 28 06:05:53 2009
@@ -243,8 +243,26 @@
   }
 
   public boolean filterRow(List<KeyValue> results) {
-    if (true) throw new RuntimeException("Not Yet Implemented");
-    return false;
+    boolean resultFound = false;
+    boolean result = operator == Operator.MUST_PASS_ONE;
+    for (RowFilterInterface filter : filters) {
+      if (!resultFound) {
+        if (operator == Operator.MUST_PASS_ALL) {
+          if (filter.filterAllRemaining() || filter.filterRow(results)) {
+            result = true;
+            resultFound = true;
+          }
+        } else if (operator == Operator.MUST_PASS_ONE) {
+          if (!filter.filterAllRemaining() && !filter.filterRow(results)) {
+            result = false;
+            resultFound = true;
+          }
+        }
+      } else if (filter.processAlways()) {
+        filter.filterRow(results);
+      }
+    }
+    return result;
   }
 
   public void readFields(final DataInput in) throws IOException {

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java Tue Apr 28 06:05:53 2009
@@ -97,7 +97,8 @@
       }
       return false;
     }
-    return Bytes.compareTo(stopRowKey, rowKey) <= 0;
+    return Bytes.compareTo(stopRowKey, 0, stopRowKey.length, rowKey, offset,
+      length) <= 0;
   }
 
   /**

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormat.java Tue Apr 28 06:05:53 2009
@@ -36,7 +36,6 @@
  */
 public class TableInputFormat extends TableInputFormatBase implements
     JobConfigurable {
-  @SuppressWarnings("hiding")
   private final Log LOG = LogFactory.getLog(TableInputFormat.class);
 
   /**

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/util/Keying.java Tue Apr 28 06:05:53 2009
@@ -28,6 +28,7 @@
  * Use fabricating row names or column qualifiers.
  * <p>TODO: Add createSchemeless key, a key that doesn't care if scheme is
  * http or https.
+ * @see Bytes#split(byte[], byte[], int)
  */
 public class Keying {
   private static final String SCHEME = "r:";

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/client/TestGetRowVersions.java Tue Apr 28 06:05:53 2009
@@ -101,4 +101,4 @@
       }
     }
   }
-}
+}
\ No newline at end of file

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/DisabledTestRowFilterAfterWrite.java Tue Apr 28 06:05:53 2009
@@ -1,201 +0,0 @@
-/**
- * Copyright 2008 The Apache Software Foundation
- *
- * 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.hadoop.hbase.filter;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
-
-import junit.framework.Assert;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HBaseClusterTestCase;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Scanner;
-import org.apache.hadoop.hbase.io.BatchUpdate;
-import org.apache.hadoop.hbase.io.Cell;
-import org.apache.hadoop.hbase.io.RowResult;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/** Test regexp filters HBASE-476 */
-public class DisabledTestRowFilterAfterWrite extends HBaseClusterTestCase {
-
-  private static final Log LOG = LogFactory.getLog(DisabledTestRowFilterAfterWrite.class.getName());
-
-  static final String TABLE_NAME = "TestTable";
-  static final String FAMILY = "C:";
-  static final String COLUMN1 = FAMILY + "col1";
-  static final byte [] TEXT_COLUMN1 = Bytes.toBytes(COLUMN1);
-  static final String COLUMN2 = FAMILY + "col2";
-  static final byte [] TEXT_COLUMN2 = Bytes.toBytes(COLUMN2);
-
-  private static final byte [][] columns = {
-    TEXT_COLUMN1, TEXT_COLUMN2
-  };
-
-  private static final int NUM_ROWS = 10;
-  private static final int VALUE_SIZE = 1000;
-  private static final byte[] VALUE = new byte[VALUE_SIZE];
-  private static final int COL_2_SIZE = 5;
-  private static final int KEY_SIZE = 9;
-  private static final int NUM_REWRITES = 10;
-  static {
-    Arrays.fill(VALUE, (byte) 'a');
-  }
-
-  /** constructor */
-  public DisabledTestRowFilterAfterWrite() {
-    super();
-
-    // Make sure the cache gets flushed so we get multiple stores
-    conf.setInt("hbase.hregion.memcache.flush.size", (NUM_ROWS * (VALUE_SIZE + COL_2_SIZE + KEY_SIZE)));
-    LOG.info("memcach flush : " + conf.get("hbase.hregion.memcache.flush.size"));
-    conf.setInt("hbase.regionserver.optionalcacheflushinterval", 100000000);
-    // Avoid compaction to keep multiple stores
-    conf.setInt("hbase.hstore.compactionThreshold", 10000);
-
-    // Make lease timeout longer, lease checks less frequent
-    conf.setInt("hbase.master.lease.period", 10 * 1000);
-
-    // For debugging
-    conf.setInt("hbase.regionserver.lease.period", 20 * 60 * 1000);
-    conf.setInt("ipc.client.timeout", 20 * 60 * 1000);
-  }
-
-  @Override
-  public void tearDown() throws Exception {
-    super.tearDown();
-  }
-
-  /**
-   * Test hbase mapreduce jobs against single region and multi-region tables.
-   * 
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void testAfterWrite() throws IOException, InterruptedException {
-    singleTableTest();
-  }
-
-  /*
-   * Test against a single region. @throws IOException
-   */
-  private void singleTableTest() throws IOException, InterruptedException {
-    HTableDescriptor desc = new HTableDescriptor(TABLE_NAME);
-    desc.addFamily(new HColumnDescriptor(FAMILY));
-
-    // Create a table.
-    HBaseAdmin admin = new HBaseAdmin(this.conf);
-    admin.createTable(desc);
-
-    // insert some data into the test table
-    HTable table = new HTable(conf, TABLE_NAME);
-
-    for (int i = 0; i < NUM_ROWS; i++) {
-      BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
-
-      b.put(TEXT_COLUMN1, VALUE);
-      b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
-      table.commit(b);
-    }
-
-    // LOG.info("Print table contents using scanner before map/reduce for " + TABLE_NAME);
-    // scanTable(TABLE_NAME, false);
-    // LOG.info("Print table contents using scanner+filter before map/reduce for " + TABLE_NAME);
-    // scanTableWithRowFilter(TABLE_NAME, false);
-
-    // Do some identity write operations on one column of the data.
-    for (int n = 0; n < NUM_REWRITES; n++) {
-      for (int i = 0; i < NUM_ROWS; i++) {
-        BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
-
-        b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
-        table.commit(b);
-      }
-    }
-
-    // Wait for the flush to happen
-    LOG.info("Waiting, for flushes to complete");
-    Thread.sleep(5 * 1000);
-    // Wait for the flush to happen
-    LOG.info("Done. No flush should happen after this");
-
-    // Do another round so to populate the mem cache
-    for (int i = 0; i < NUM_ROWS; i++) {
-      BatchUpdate b = new BatchUpdate("row_" + String.format("%1$05d", i));
-      b.put(TEXT_COLUMN2, String.format("%1$05d", i).getBytes());
-      table.commit(b);
-    }
-
-    LOG.info("Print table contents using scanner after map/reduce for " + TABLE_NAME);
-    scanTable(TABLE_NAME, true);
-    LOG.info("Print table contents using scanner+filter after map/reduce for " + TABLE_NAME);
-    scanTableWithRowFilter(TABLE_NAME, true);
-  }
-
-  private void scanTable(final String tableName, final boolean printValues) throws IOException {
-    HTable table = new HTable(conf, tableName);
-
-    Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW);
-    int numFound = doScan(scanner, printValues);
-    Assert.assertEquals(NUM_ROWS, numFound);
-  }
-
-  private void scanTableWithRowFilter(final String tableName, final boolean printValues) throws IOException {
-    HTable table = new HTable(conf, tableName);
-    Map<byte [], Cell> columnMap = new HashMap<byte [], Cell>();
-    columnMap.put(TEXT_COLUMN1,
-        new Cell(VALUE, HConstants.LATEST_TIMESTAMP));
-    RegExpRowFilter filter = new RegExpRowFilter(null, columnMap);
-    Scanner scanner = table.getScanner(columns, HConstants.EMPTY_START_ROW, filter);
-    int numFound = doScan(scanner, printValues);
-    Assert.assertEquals(NUM_ROWS, numFound);
-  }
-
-  private int doScan(final Scanner scanner, final boolean printValues) throws IOException {
-    {
-      int count = 0;
-
-      try {
-        for (RowResult result : scanner) {
-          if (printValues) {
-            LOG.info("row: " + Bytes.toString(result.getRow()));
-            for (Map.Entry<byte [], Cell> e : result.entrySet()) {
-              LOG.info(" column: " + e.getKey() + " value: "
-                  + new String(e.getValue().getValue(), HConstants.UTF8_ENCODING));
-            }
-          }
-          count++;
-        }
-
-      } finally {
-        scanner.close();
-      }
-      return count;
-    }
-  }
-}

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java?rev=769257&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestColumnValueFilter.java Tue Apr 28 06:05:53 2009
@@ -0,0 +1,153 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.filter;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests the stop row filter
+ */
+public class TestColumnValueFilter extends TestCase {
+
+  private static final byte[] ROW = Bytes.toBytes("test");
+  private static final byte[] COLUMN = Bytes.toBytes("test:foo");
+  private static final byte[] VAL_1 = Bytes.toBytes("a");
+  private static final byte[] VAL_2 = Bytes.toBytes("ab");
+  private static final byte[] VAL_3 = Bytes.toBytes("abc");
+  private static final byte[] VAL_4 = Bytes.toBytes("abcd");
+  private static final byte[] FULLSTRING_1 = 
+    Bytes.toBytes("The quick brown fox jumps over the lazy dog.");
+  private static final byte[] FULLSTRING_2 = 
+    Bytes.toBytes("The slow grey fox trips over the lazy dog.");
+  private static final String QUICK_SUBSTR = "quick";
+  private static final String QUICK_REGEX = "[q][u][i][c][k]";
+
+  private RowFilterInterface basicFilterNew() {
+    return new ColumnValueFilter(COLUMN,
+        ColumnValueFilter.CompareOp.GREATER_OR_EQUAL, VAL_2);
+  }
+
+  private RowFilterInterface substrFilterNew() {
+    return new ColumnValueFilter(COLUMN, ColumnValueFilter.CompareOp.EQUAL,
+      new SubstringComparator(QUICK_SUBSTR));
+  }
+
+  private RowFilterInterface regexFilterNew() {
+    return new ColumnValueFilter(COLUMN, ColumnValueFilter.CompareOp.EQUAL,
+      new RegexStringComparator(QUICK_REGEX));
+  }
+
+  private void basicFilterTests(RowFilterInterface filter)
+      throws Exception {
+    assertTrue("basicFilter1", filter.filterColumn(ROW, 0, ROW.length,
+      COLUMN, 0, COLUMN.length, VAL_1, 0, VAL_1.length));
+    assertFalse("basicFilter2", filter.filterColumn(ROW, 0, ROW.length,
+      COLUMN, 0, COLUMN.length, VAL_2, 0, VAL_2.length));
+    assertFalse("basicFilter3", filter.filterColumn(ROW, 0, ROW.length,
+      COLUMN, 0, COLUMN.length, VAL_3, 0, VAL_3.length));
+    assertFalse("basicFilter4", filter.filterColumn(ROW, 0, ROW.length,
+      COLUMN, 0, COLUMN.length, VAL_4, 0, VAL_4.length));
+    assertFalse("basicFilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("basicFilterNotNull", filter.filterRow((List<KeyValue>)null));
+  }
+
+  private void substrFilterTests(RowFilterInterface filter) 
+      throws Exception {
+    assertTrue("substrTrue", filter.filterColumn(ROW, 0, ROW.length,
+        COLUMN, 0, COLUMN.length, FULLSTRING_1, 0, FULLSTRING_1.length));
+    assertFalse("substrFalse", filter.filterColumn(ROW, 0, ROW.length,
+        COLUMN, 0, COLUMN.length, FULLSTRING_2, 0, FULLSTRING_2.length));
+    assertFalse("substrFilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("substrFilterNotNull", filter.filterRow((List<KeyValue>)null));
+  }
+
+  private void regexFilterTests(RowFilterInterface filter) 
+      throws Exception {
+    assertTrue("regexTrue", filter.filterColumn(ROW, 0, ROW.length,
+        COLUMN, 0, COLUMN.length, FULLSTRING_1, 0, FULLSTRING_1.length));
+    assertFalse("regexFalse", filter.filterColumn(ROW, 0, ROW.length,
+        COLUMN, 0, COLUMN.length, FULLSTRING_2, 0, FULLSTRING_2.length));
+    assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("regexFilterNotNull", filter.filterRow((List<KeyValue>)null));
+  }
+
+  private RowFilterInterface serializationTest(RowFilterInterface filter)
+      throws Exception {
+    // Decompose filter to bytes.
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(stream);
+    filter.write(out);
+    out.close();
+    byte[] buffer = stream.toByteArray();
+  
+    // Recompose filter.
+    DataInputStream in =
+      new DataInputStream(new ByteArrayInputStream(buffer));
+    RowFilterInterface newFilter = new ColumnValueFilter();
+    newFilter.readFields(in);
+  
+    return newFilter;
+  }
+
+  RowFilterInterface basicFilter;
+  RowFilterInterface substrFilter;
+  RowFilterInterface regexFilter;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    basicFilter = basicFilterNew();
+    substrFilter = substrFilterNew();
+    regexFilter = regexFilterNew();
+  }
+
+  /**
+   * Tests identification of the stop row
+   * @throws Exception
+   */
+  public void testStop() throws Exception {
+    basicFilterTests(basicFilter);
+    substrFilterTests(substrFilter);
+    regexFilterTests(regexFilter);
+  }
+
+  /**
+   * Tests serialization
+   * @throws Exception
+   */
+  public void testSerialization() throws Exception {
+    RowFilterInterface newFilter = serializationTest(basicFilter);
+    basicFilterTests(newFilter);
+    newFilter = serializationTest(substrFilter);
+    substrFilterTests(newFilter);
+    newFilter = serializationTest(regexFilter);
+    regexFilterTests(newFilter);
+  }
+
+}

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java?rev=769257&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java Tue Apr 28 06:05:53 2009
@@ -0,0 +1,98 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.filter;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests the stop row filter
+ */
+public class TestStopRowFilter extends TestCase {
+  private final byte [] STOP_ROW = Bytes.toBytes("stop_row");
+  private final byte [] GOOD_ROW = Bytes.toBytes("good_row");
+  private final byte [] PAST_STOP_ROW = Bytes.toBytes("zzzzzz");
+  
+  RowFilterInterface mainFilter;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    mainFilter = new StopRowFilter(STOP_ROW);
+  }
+  
+  /**
+   * Tests identification of the stop row
+   * @throws Exception
+   */
+  public void testStopRowIdentification() throws Exception {
+    stopRowTests(mainFilter);
+  }
+  
+  /**
+   * Tests serialization
+   * @throws Exception
+   */
+  public void testSerialization() throws Exception {
+    // Decompose mainFilter to bytes.
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(stream);
+    mainFilter.write(out);
+    out.close();
+    byte[] buffer = stream.toByteArray();
+    
+    // Recompose mainFilter.
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+    RowFilterInterface newFilter = new StopRowFilter();
+    newFilter.readFields(in);
+    
+    // Ensure the serialization preserved the filter by running a full test.
+    stopRowTests(newFilter);
+  }
+
+  private void stopRowTests(RowFilterInterface filter) throws Exception {
+    assertFalse("Filtering on " + Bytes.toString(GOOD_ROW),
+      filter.filterRowKey(GOOD_ROW, 0, GOOD_ROW.length));
+    assertTrue("Filtering on " + Bytes.toString(STOP_ROW),
+      filter.filterRowKey(STOP_ROW, 0, STOP_ROW.length));
+    assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW),
+      filter.filterRowKey(PAST_STOP_ROW, 0, PAST_STOP_ROW.length));
+    assertFalse("Filtering on " + Bytes.toString(GOOD_ROW),
+      filter.filterColumn(GOOD_ROW, 0, GOOD_ROW.length, null, 0, 0,
+      null, 0, 0));
+    assertTrue("Filtering on " + Bytes.toString(STOP_ROW),
+      filter.filterColumn(STOP_ROW, 0, STOP_ROW.length, null, 0, 0, null, 0, 0));
+    assertTrue("Filtering on " + Bytes.toString(PAST_STOP_ROW),
+      filter.filterColumn(PAST_STOP_ROW, 0, PAST_STOP_ROW.length, null, 0, 0,
+        null, 0, 0));
+    assertFalse("FilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("FilterNotNull", filter.filterRow((List<KeyValue>)null));
+    
+    assertFalse("Filter a null", filter.filterRowKey(null, 0, 0));
+  }
+}
\ No newline at end of file

Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java?rev=769257&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java Tue Apr 28 06:05:53 2009
@@ -0,0 +1,157 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.filter;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.util.List;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+
+/**
+ * Tests for the while-match filter
+ */
+public class TestWhileMatchRowFilter extends TestCase {
+
+  WhileMatchRowFilter wmStopRowFilter;
+  WhileMatchRowFilter wmRegExpRowFilter;
+
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter(
+        Bytes.toBytes("s")));
+    wmRegExpRowFilter = new WhileMatchRowFilter(new RegExpRowFilter(
+    ".*regex.*"));
+  }
+  
+  /**
+   * Tests while match stop row
+   * @throws Exception
+   */
+  public void testWhileMatchStopRow() throws Exception {
+    whileMatchStopRowTests(wmStopRowFilter);
+  }
+  
+  /**
+   * Tests while match regex
+   * @throws Exception
+   */
+  public void testWhileMatchRegExp() throws Exception {
+    whileMatchRegExpTests(wmRegExpRowFilter);
+  }
+  
+  /**
+   * Tests serialization
+   * @throws Exception
+   */
+  public void testSerialization() throws Exception {
+    // Decompose wmRegExpRowFilter to bytes.
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(stream);
+    wmRegExpRowFilter.write(out);
+    out.close();
+    byte[] buffer = stream.toByteArray();
+    
+    // Recompose wmRegExpRowFilter.
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+    WhileMatchRowFilter newFilter = new WhileMatchRowFilter();
+    newFilter.readFields(in);
+
+    // Ensure the serialization preserved the filter by running a full test.
+    whileMatchRegExpTests(newFilter);
+  }
+  
+  private void whileMatchStopRowTests(WhileMatchRowFilter filter) throws 
+    Exception {
+    RowFilterInterface innerFilter = filter.getInternalFilter();
+    String toTest;
+    
+    // Test cases that should pass the row
+    toTest = "apples";
+    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    byte [] toTestBytes = Bytes.toBytes(toTest);
+    assertFalse("innerFilter: '" + toTest + "'",
+      innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
+
+    // Test cases that should fail the row
+    toTest = "tuna";
+    toTestBytes = Bytes.toBytes(toTest);
+    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(toTestBytes));
+    assertTrue("innerFilter: '" + toTest + "'",
+      innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
+    
+    // The difference in switch
+    assertTrue("filter: filterAllRemaining", filter.filterAllRemaining());
+    assertFalse("innerFilter: filterAllRemaining pre-reset", 
+      innerFilter.filterAllRemaining());
+    
+    // Test resetting
+    filter.reset();
+    assertFalse("filter: filterAllRemaining post-reset", 
+        filter.filterAllRemaining());
+    
+    // Test filterNotNull for functionality only (no switch-cases)
+    assertFalse("filter: filterNotNull", filter.filterRow((List<KeyValue>)null));
+  }
+  
+  private void whileMatchRegExpTests(WhileMatchRowFilter filter) throws 
+    Exception {
+    RowFilterInterface innerFilter = filter.getInternalFilter();
+    String toTest;
+    
+    // Test cases that should pass the row
+    toTest = "regex_match";
+    byte [] toTestBytes = Bytes.toBytes(toTest);
+    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertFalse("innerFilter: '" + toTest + "'",
+      innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
+    
+    // Test cases that should fail the row
+    toTest = "not_a_match";
+    toTestBytes = Bytes.toBytes(toTest);
+    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertTrue("innerFilter: '" + toTest + "'",
+      innerFilter.filterRowKey(toTestBytes, 0, toTestBytes.length));
+    
+    // The difference in switch
+    assertTrue("filter: filterAllRemaining", filter.filterAllRemaining());
+    assertFalse("innerFilter: filterAllRemaining pre-reset", 
+      innerFilter.filterAllRemaining());
+    
+    // Test resetting
+    filter.reset();
+    assertFalse("filter: filterAllRemaining post-reset", 
+        filter.filterAllRemaining());
+    
+    // Test filter(Text, Text, byte[]) for functionality only (no switch-cases)
+    toTest = "asdf_regex_hjkl";
+    toTestBytes = Bytes.toBytes(toTest);
+    assertFalse("filter: '" + toTest + "'",
+      filter.filterColumn(toTestBytes, 0, toTestBytes.length, 
+      null, 0, 0, null, 0, 0));
+  }
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java Tue Apr 28 06:05:53 2009
@@ -90,6 +90,53 @@
     
   }
 
+  /**
+   * Test basic stop row filter works.
+   * @throws Exception 
+   */
+  public void testStopRow() throws Exception {
+    byte [] startrow = Bytes.toBytes("bbb");
+    byte [] stoprow = Bytes.toBytes("ccc");
+    try {
+      this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
+      addContent(this.r, HConstants.COLUMN_FAMILY);
+      List<KeyValue> results = new ArrayList<KeyValue>();
+      // Do simple test of getting one row only first.
+      InternalScanner s = r.getScanner(HConstants.COLUMN_FAMILY_ARRAY,
+          Bytes.toBytes("abc"), HConstants.LATEST_TIMESTAMP,
+          new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("abd"))));
+      int count = 0;
+      while (s.next(results)) {
+        count++;
+      }
+      s.close();
+      assertEquals(1, count);
+      // Now do something a bit more imvolved.
+      s = r.getScanner(HConstants.COLUMN_FAMILY_ARRAY,
+        startrow, HConstants.LATEST_TIMESTAMP,
+        new WhileMatchRowFilter(new StopRowFilter(stoprow)));
+      count = 0;
+      KeyValue kv = null;
+      results = new ArrayList<KeyValue>();
+      for (boolean first = true; s.next(results);) {
+        kv = results.get(0);
+        if (first) {
+          assertTrue(Bytes.BYTES_COMPARATOR.compare(startrow, kv.getRow()) == 0);
+          first = false;
+        }
+        count++;
+      }
+      assertTrue(Bytes.BYTES_COMPARATOR.compare(stoprow, kv.getRow()) > 0);
+      // We got something back.
+      assertTrue(count > 10);
+      s.close();
+    } finally {
+      this.r.close();
+      this.r.getLog().closeAndDelete();
+      shutdownDfs(this.cluster);
+    }
+  }
+
   /** The test!
    * @throws IOException
    */
@@ -227,7 +274,6 @@
   throws IOException {  
     InternalScanner scanner = null;
     List<KeyValue> results = new ArrayList<KeyValue>();
-
     byte [][][] scanColumns = {
         COLS,
         EXPLICIT_COLS
@@ -238,27 +284,26 @@
         scanner = r.getScanner(scanColumns[i], FIRST_ROW,
             System.currentTimeMillis(), null);
         while (scanner.next(results)) {
-          // FIX!!!
-//          assertTrue(results.containsKey(HConstants.COL_REGIONINFO));
-//          byte [] val = results.get(HConstants.COL_REGIONINFO).getValue(); 
-//          validateRegionInfo(val);
-//          if(validateStartcode) {
-//            assertTrue(results.containsKey(HConstants.COL_STARTCODE));
-//            val = results.get(HConstants.COL_STARTCODE).getValue();
-//            assertNotNull(val);
-//            assertFalse(val.length == 0);
-//            long startCode = Bytes.toLong(val);
-//            assertEquals(START_CODE, startCode);
-//          }
-//          
-//          if(serverName != null) {
-//            assertTrue(results.containsKey(HConstants.COL_SERVER));
-//            val = results.get(HConstants.COL_SERVER).getValue();
-//            assertNotNull(val);
-//            assertFalse(val.length == 0);
-//            String server = Bytes.toString(val);
-//            assertEquals(0, server.compareTo(serverName));
-//          }
+          assertTrue(hasColumn(results, HConstants.COL_REGIONINFO));
+          byte [] val = getColumn(results, HConstants.COL_REGIONINFO).getValue(); 
+          validateRegionInfo(val);
+          if(validateStartcode) {
+            assertTrue(hasColumn(results, HConstants.COL_STARTCODE));
+            val = getColumn(results, HConstants.COL_STARTCODE).getValue();
+            assertNotNull(val);
+            assertFalse(val.length == 0);
+            long startCode = Bytes.toLong(val);
+            assertEquals(START_CODE, startCode);
+          }
+          
+          if(serverName != null) {
+            assertTrue(hasColumn(results, HConstants.COL_SERVER));
+            val = getColumn(results, HConstants.COL_SERVER).getValue();
+            assertNotNull(val);
+            assertFalse(val.length == 0);
+            String server = Bytes.toString(val);
+            assertEquals(0, server.compareTo(serverName));
+          }
           results.clear();
         }
 
@@ -272,45 +317,28 @@
     }
   }
 
-  /** Use get to retrieve the HRegionInfo and validate it */
-  private void getRegionInfo() throws IOException {
-    byte [] bytes = region.get(ROW_KEY, HConstants.COL_REGIONINFO).getValue();
-    validateRegionInfo(bytes);  
+  private boolean hasColumn(final List<KeyValue> kvs, final byte [] column) {
+    for (KeyValue kv: kvs) {
+      if (kv.matchingColumn(column)) {
+        return true;
+      }
+    }
+    return false;
   }
 
-  /**
-   * Test basic stop row filter works.
-   * @throws Exception 
-   */
-  public void testStopRow() throws Exception {
-    byte [] startrow = Bytes.toBytes("bbb");
-    byte [] stoprow = Bytes.toBytes("ccc");
-    try {
-      this.r = createNewHRegion(REGION_INFO.getTableDesc(), null, null);
-      addContent(this.r, HConstants.COLUMN_FAMILY);
-      InternalScanner s = r.getScanner(HConstants.COLUMN_FAMILY_ARRAY,
-        startrow, HConstants.LATEST_TIMESTAMP,
-        new WhileMatchRowFilter(new StopRowFilter(stoprow)));
-      List<KeyValue> results = new ArrayList<KeyValue>();
-      int count = 0;
-      KeyValue kv = null;
-      for (boolean first = true; s.next(results);) {
-        kv = results.get(0);
-        if (first) {
-          assertTrue(Bytes.BYTES_COMPARATOR.compare(startrow, kv.getRow()) == 0);
-          first = false;
-        }
-        count++;
+  private KeyValue getColumn(final List<KeyValue> kvs, final byte [] column) {
+    for (KeyValue kv: kvs) {
+      if (kv.matchingColumn(column)) {
+        return kv;
       }
-      assertTrue(Bytes.BYTES_COMPARATOR.compare(stoprow, kv.getRow()) > 0);
-      // We got something back.
-      assertTrue(count > 10);
-      s.close();
-    } finally {
-      this.r.close();
-      this.r.getLog().closeAndDelete();
-      shutdownDfs(this.cluster);
     }
+    return null;
+  }
+
+  /** Use get to retrieve the HRegionInfo and validate it */
+  private void getRegionInfo() throws IOException {
+    byte [] bytes = region.get(ROW_KEY, HConstants.COL_REGIONINFO).getValue();
+    validateRegionInfo(bytes);  
   }
 
   /**

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java?rev=769257&r1=769256&r2=769257&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/util/TestBytes.java Tue Apr 28 06:05:53 2009
@@ -54,7 +54,7 @@
     for (int i = 0; i < parts.length; i++) {
       System.out.println(Bytes.toString(parts[i]));
     }
-    assertEquals(2, parts.length);
+    assertEquals(3, parts.length);
     assertTrue(Bytes.equals(parts[1], middle));
   }