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 2008/05/22 22:49:27 UTC

svn commit: r659249 [3/3] - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/filter/ src/java/org/apache/hadoop/hbase/mapred/ src/java/org/apache/hadoop/hbase/regionserver/ src/test/org/apache/hadoop/...

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/HStoreScanner.java Thu May 22 13:49:25 2008
@@ -1,273 +1,273 @@
-/**
- * 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.regionserver;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HStoreKey;
-import org.apache.hadoop.hbase.filter.RowFilterInterface;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Scanner scans both the memcache and the HStore
- */
-class HStoreScanner implements InternalScanner {
-  static final Log LOG = LogFactory.getLog(HStoreScanner.class);
-
-  private InternalScanner[] scanners;
-  private TreeMap<byte [], byte []>[] resultSets;
-  private HStoreKey[] keys;
-  private boolean wildcardMatch = false;
-  private boolean multipleMatchers = false;
-  private RowFilterInterface dataFilter;
-  private HStore store;
-  
-  /** Create an Scanner with a handle on the memcache and HStore files. */
-  @SuppressWarnings("unchecked")
-  HStoreScanner(HStore store, byte [][] targetCols, byte [] firstRow,
-    long timestamp, RowFilterInterface filter) 
-  throws IOException {
-    this.store = store;
-    this.dataFilter = filter;
-    if (null != dataFilter) {
-      dataFilter.reset();
-    }
-    this.scanners = new InternalScanner[2];
-    this.resultSets = new TreeMap[scanners.length];
-    this.keys = new HStoreKey[scanners.length];
-
-    try {
-      scanners[0] = store.memcache.getScanner(timestamp, targetCols, firstRow);
-      scanners[1] = new StoreFileScanner(store, timestamp, targetCols, firstRow);
-      for (int i = 0; i < scanners.length; i++) {
-        if (scanners[i].isWildcardScanner()) {
-          this.wildcardMatch = true;
-        }
-        if (scanners[i].isMultipleMatchScanner()) {
-          this.multipleMatchers = true;
-        }
-      }
-    } catch(IOException e) {
-      for (int i = 0; i < this.scanners.length; i++) {
-        if(scanners[i] != null) {
-          closeScanner(i);
-        }
-      }
-      throw e;
-    }
-    
-    // Advance to the first key in each scanner.
-    // All results will match the required column-set and scanTime.
-    for (int i = 0; i < scanners.length; i++) {
-      keys[i] = new HStoreKey();
-      resultSets[i] = new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
-      if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) {
-        closeScanner(i);
-      }
-    }
-  }
-
-  /** @return true if the scanner is a wild card scanner */
-  public boolean isWildcardScanner() {
-    return wildcardMatch;
-  }
-
-  /** @return true if the scanner is a multiple match scanner */
-  public boolean isMultipleMatchScanner() {
-    return multipleMatchers;
-  }
-
-  /** {@inheritDoc} */
-  public boolean next(HStoreKey key, SortedMap<byte [], byte[]> results)
-    throws IOException {
-
-    // Filtered flag is set by filters.  If a cell has been 'filtered out'
-    // -- i.e. it is not to be returned to the caller -- the flag is 'true'.
-    boolean filtered = true;
-    boolean moreToFollow = true;
-    while (filtered && moreToFollow) {
-      // Find the lowest-possible key.
-      byte [] chosenRow = null;
-      long chosenTimestamp = -1;
-      for (int i = 0; i < this.keys.length; i++) {
-        if (scanners[i] != null &&
-            (chosenRow == null ||
-            (Bytes.compareTo(keys[i].getRow(), chosenRow) < 0) ||
-            ((Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) &&
-            (keys[i].getTimestamp() > chosenTimestamp)))) {
-          chosenRow = keys[i].getRow();
-          chosenTimestamp = keys[i].getTimestamp();
-        }
-      }
-      
-      // Filter whole row by row key?
-      filtered = dataFilter != null? dataFilter.filterRowKey(chosenRow) : false;
-
-      // Store the key and results for each sub-scanner. Merge them as
-      // appropriate.
-      if (chosenTimestamp >= 0 && !filtered) {
-        // Here we are setting the passed in key with current row+timestamp
-        key.setRow(chosenRow);
-        key.setVersion(chosenTimestamp);
-        key.setColumn(HConstants.EMPTY_BYTE_ARRAY);
-        // Keep list of deleted cell keys within this row.  We need this
-        // because as we go through scanners, the delete record may be in an
-        // early scanner and then the same record with a non-delete, non-null
-        // value in a later. Without history of what we've seen, we'll return
-        // deleted values. This List should not ever grow too large since we
-        // are only keeping rows and columns that match those set on the
-        // scanner and which have delete values.  If memory usage becomes a
-        // problem, could redo as bloom filter.
-        List<HStoreKey> deletes = new ArrayList<HStoreKey>();
-        for (int i = 0; i < scanners.length && !filtered; i++) {
-          while ((scanners[i] != null
-              && !filtered
-              && moreToFollow)
-              && (Bytes.compareTo(keys[i].getRow(), 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;
-            }
-
-            // 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.
-            HStoreKey hsk = new HStoreKey(key.getRow(), HConstants.EMPTY_BYTE_ARRAY,
-              key.getTimestamp());
-            for (Map.Entry<byte [], byte[]> e : resultSets[i].entrySet()) {
-              hsk.setColumn(e.getKey());
-              if (HLogEdit.isDeleted(e.getValue())) {
-                if (!deletes.contains(hsk)) {
-                  // Key changes as we cycle the for loop so add a copy to
-                  // the set of deletes.
-                  deletes.add(new HStoreKey(hsk));
-                }
-              } else if (!deletes.contains(hsk) &&
-                  !filtered &&
-                  moreToFollow &&
-                  !results.containsKey(e.getKey())) {
-                if (dataFilter != null) {
-                  // Filter whole row by column data?
-                  filtered =
-                      dataFilter.filterColumn(chosenRow, e.getKey(), e.getValue());
-                  if (filtered) {
-                    results.clear();
-                    break;
-                  }
-                }
-                results.put(e.getKey(), e.getValue());
-              }
-            }
-            resultSets[i].clear();
-            if (!scanners[i].next(keys[i], resultSets[i])) {
-              closeScanner(i);
-            }
-          }
-        }          
-      }
-      
-      for (int i = 0; i < scanners.length; 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) &&
-            (Bytes.compareTo(keys[i].getRow(), chosenRow) <= 0)) {
-          resultSets[i].clear();
-          if (!scanners[i].next(keys[i], resultSets[i])) {
-            closeScanner(i);
-          }
-        }
-      }
-
-      moreToFollow = chosenTimestamp >= 0;
-      
-      if (dataFilter != null) {
-        if (dataFilter.filterAllRemaining()) {
-          moreToFollow = false;
-        }
-      }
-      
-      if (results.size() <= 0 && !filtered) {
-        // There were no results found for this row.  Marked it as 
-        // 'filtered'-out otherwise we will not move on to the next row.
-        filtered = true;
-      }
-    }
-    
-    // If we got no results, then there is no more to follow.
-    if (results == null || results.size() <= 0) {
-      moreToFollow = false;
-    }
-    
-    // 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 moreToFollow;
-  }
-  
-  /** Shut down a single scanner */
-  void closeScanner(int i) {
-    try {
-      try {
-        scanners[i].close();
-      } catch (IOException e) {
-        LOG.warn(store.storeName + " failed closing scanner " + i, e);
-      }
-    } finally {
-      scanners[i] = null;
-      keys[i] = null;
-      resultSets[i] = null;
-    }
-  }
-
-  /** {@inheritDoc} */
-  public void close() {
-    for(int i = 0; i < scanners.length; i++) {
-      if(scanners[i] != null) {
-        closeScanner(i);
-      }
-    }
-  }
-
-  public Iterator<Map.Entry<HStoreKey, SortedMap<byte [], byte[]>>> iterator() {
-    throw new UnsupportedOperationException("Unimplemented serverside. " +
-      "next(HStoreKey, StortedMap(...) is more efficient");
-  }
-}
+/**
+ * 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.regionserver;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HStoreKey;
+import org.apache.hadoop.hbase.filter.RowFilterInterface;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Scanner scans both the memcache and the HStore
+ */
+class HStoreScanner implements InternalScanner {
+  static final Log LOG = LogFactory.getLog(HStoreScanner.class);
+
+  private InternalScanner[] scanners;
+  private TreeMap<byte [], byte []>[] resultSets;
+  private HStoreKey[] keys;
+  private boolean wildcardMatch = false;
+  private boolean multipleMatchers = false;
+  private RowFilterInterface dataFilter;
+  private HStore store;
+  
+  /** Create an Scanner with a handle on the memcache and HStore files. */
+  @SuppressWarnings("unchecked")
+  HStoreScanner(HStore store, byte [][] targetCols, byte [] firstRow,
+    long timestamp, RowFilterInterface filter) 
+  throws IOException {
+    this.store = store;
+    this.dataFilter = filter;
+    if (null != dataFilter) {
+      dataFilter.reset();
+    }
+    this.scanners = new InternalScanner[2];
+    this.resultSets = new TreeMap[scanners.length];
+    this.keys = new HStoreKey[scanners.length];
+
+    try {
+      scanners[0] = store.memcache.getScanner(timestamp, targetCols, firstRow);
+      scanners[1] = new StoreFileScanner(store, timestamp, targetCols, firstRow);
+      for (int i = 0; i < scanners.length; i++) {
+        if (scanners[i].isWildcardScanner()) {
+          this.wildcardMatch = true;
+        }
+        if (scanners[i].isMultipleMatchScanner()) {
+          this.multipleMatchers = true;
+        }
+      }
+    } catch(IOException e) {
+      for (int i = 0; i < this.scanners.length; i++) {
+        if(scanners[i] != null) {
+          closeScanner(i);
+        }
+      }
+      throw e;
+    }
+    
+    // Advance to the first key in each scanner.
+    // All results will match the required column-set and scanTime.
+    for (int i = 0; i < scanners.length; i++) {
+      keys[i] = new HStoreKey();
+      resultSets[i] = new TreeMap<byte [], byte []>(Bytes.BYTES_COMPARATOR);
+      if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) {
+        closeScanner(i);
+      }
+    }
+  }
+
+  /** @return true if the scanner is a wild card scanner */
+  public boolean isWildcardScanner() {
+    return wildcardMatch;
+  }
+
+  /** @return true if the scanner is a multiple match scanner */
+  public boolean isMultipleMatchScanner() {
+    return multipleMatchers;
+  }
+
+  /** {@inheritDoc} */
+  public boolean next(HStoreKey key, SortedMap<byte [], byte[]> results)
+    throws IOException {
+
+    // Filtered flag is set by filters.  If a cell has been 'filtered out'
+    // -- i.e. it is not to be returned to the caller -- the flag is 'true'.
+    boolean filtered = true;
+    boolean moreToFollow = true;
+    while (filtered && moreToFollow) {
+      // Find the lowest-possible key.
+      byte [] chosenRow = null;
+      long chosenTimestamp = -1;
+      for (int i = 0; i < this.keys.length; i++) {
+        if (scanners[i] != null &&
+            (chosenRow == null ||
+            (Bytes.compareTo(keys[i].getRow(), chosenRow) < 0) ||
+            ((Bytes.compareTo(keys[i].getRow(), chosenRow) == 0) &&
+            (keys[i].getTimestamp() > chosenTimestamp)))) {
+          chosenRow = keys[i].getRow();
+          chosenTimestamp = keys[i].getTimestamp();
+        }
+      }
+      
+      // Filter whole row by row key?
+      filtered = dataFilter != null? dataFilter.filterRowKey(chosenRow) : false;
+
+      // Store the key and results for each sub-scanner. Merge them as
+      // appropriate.
+      if (chosenTimestamp >= 0 && !filtered) {
+        // Here we are setting the passed in key with current row+timestamp
+        key.setRow(chosenRow);
+        key.setVersion(chosenTimestamp);
+        key.setColumn(HConstants.EMPTY_BYTE_ARRAY);
+        // Keep list of deleted cell keys within this row.  We need this
+        // because as we go through scanners, the delete record may be in an
+        // early scanner and then the same record with a non-delete, non-null
+        // value in a later. Without history of what we've seen, we'll return
+        // deleted values. This List should not ever grow too large since we
+        // are only keeping rows and columns that match those set on the
+        // scanner and which have delete values.  If memory usage becomes a
+        // problem, could redo as bloom filter.
+        List<HStoreKey> deletes = new ArrayList<HStoreKey>();
+        for (int i = 0; i < scanners.length && !filtered; i++) {
+          while ((scanners[i] != null
+              && !filtered
+              && moreToFollow)
+              && (Bytes.compareTo(keys[i].getRow(), 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;
+            }
+
+            // 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.
+            HStoreKey hsk = new HStoreKey(key.getRow(), HConstants.EMPTY_BYTE_ARRAY,
+              key.getTimestamp());
+            for (Map.Entry<byte [], byte[]> e : resultSets[i].entrySet()) {
+              hsk.setColumn(e.getKey());
+              if (HLogEdit.isDeleted(e.getValue())) {
+                if (!deletes.contains(hsk)) {
+                  // Key changes as we cycle the for loop so add a copy to
+                  // the set of deletes.
+                  deletes.add(new HStoreKey(hsk));
+                }
+              } else if (!deletes.contains(hsk) &&
+                  !filtered &&
+                  moreToFollow &&
+                  !results.containsKey(e.getKey())) {
+                if (dataFilter != null) {
+                  // Filter whole row by column data?
+                  filtered =
+                      dataFilter.filterColumn(chosenRow, e.getKey(), e.getValue());
+                  if (filtered) {
+                    results.clear();
+                    break;
+                  }
+                }
+                results.put(e.getKey(), e.getValue());
+              }
+            }
+            resultSets[i].clear();
+            if (!scanners[i].next(keys[i], resultSets[i])) {
+              closeScanner(i);
+            }
+          }
+        }          
+      }
+      
+      for (int i = 0; i < scanners.length; 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) &&
+            (Bytes.compareTo(keys[i].getRow(), chosenRow) <= 0)) {
+          resultSets[i].clear();
+          if (!scanners[i].next(keys[i], resultSets[i])) {
+            closeScanner(i);
+          }
+        }
+      }
+
+      moreToFollow = chosenTimestamp >= 0;
+      
+      if (dataFilter != null) {
+        if (dataFilter.filterAllRemaining()) {
+          moreToFollow = false;
+        }
+      }
+      
+      if (results.size() <= 0 && !filtered) {
+        // There were no results found for this row.  Marked it as 
+        // 'filtered'-out otherwise we will not move on to the next row.
+        filtered = true;
+      }
+    }
+    
+    // If we got no results, then there is no more to follow.
+    if (results == null || results.size() <= 0) {
+      moreToFollow = false;
+    }
+    
+    // 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 moreToFollow;
+  }
+  
+  /** Shut down a single scanner */
+  void closeScanner(int i) {
+    try {
+      try {
+        scanners[i].close();
+      } catch (IOException e) {
+        LOG.warn(store.storeName + " failed closing scanner " + i, e);
+      }
+    } finally {
+      scanners[i] = null;
+      keys[i] = null;
+      resultSets[i] = null;
+    }
+  }
+
+  /** {@inheritDoc} */
+  public void close() {
+    for(int i = 0; i < scanners.length; i++) {
+      if(scanners[i] != null) {
+        closeScanner(i);
+      }
+    }
+  }
+
+  public Iterator<Map.Entry<HStoreKey, SortedMap<byte [], byte[]>>> iterator() {
+    throw new UnsupportedOperationException("Unimplemented serverside. " +
+      "next(HStoreKey, StortedMap(...) is more efficient");
+  }
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestInclusiveStopRowFilter.java Thu May 22 13:49:25 2008
@@ -1,93 +1,93 @@
-/**
- * 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 org.apache.hadoop.hbase.util.Bytes;
-
-import junit.framework.TestCase;
-
-/**
- * Tests the inclusive stop row filter
- */
-public class TestInclusiveStopRowFilter 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;
-
-  /** {@inheritDoc} */
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    mainFilter = new InclusiveStopRowFilter(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 InclusiveStopRowFilter();
-    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 " + GOOD_ROW, filter.filterRowKey(GOOD_ROW));
-    assertFalse("Filtering on " + STOP_ROW, filter.filterRowKey(STOP_ROW));
-    assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterRowKey(PAST_STOP_ROW));
-    
-    assertFalse("Filtering on " + GOOD_ROW, filter.filterColumn(GOOD_ROW, null, 
-      null));
-    assertFalse("Filtering on " + STOP_ROW, filter.filterColumn(STOP_ROW, null, null));
-    assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterColumn(PAST_STOP_ROW, 
-      null, null));
-
-    assertFalse("FilterAllRemaining", filter.filterAllRemaining());
-    assertFalse("FilterNotNull", filter.filterRow(null));
-    
-    assertFalse("Filter a null", filter.filterRowKey(null));
-  }
-}
+/**
+ * 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 org.apache.hadoop.hbase.util.Bytes;
+
+import junit.framework.TestCase;
+
+/**
+ * Tests the inclusive stop row filter
+ */
+public class TestInclusiveStopRowFilter 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;
+
+  /** {@inheritDoc} */
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    mainFilter = new InclusiveStopRowFilter(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 InclusiveStopRowFilter();
+    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 " + GOOD_ROW, filter.filterRowKey(GOOD_ROW));
+    assertFalse("Filtering on " + STOP_ROW, filter.filterRowKey(STOP_ROW));
+    assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterRowKey(PAST_STOP_ROW));
+    
+    assertFalse("Filtering on " + GOOD_ROW, filter.filterColumn(GOOD_ROW, null, 
+      null));
+    assertFalse("Filtering on " + STOP_ROW, filter.filterColumn(STOP_ROW, null, null));
+    assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterColumn(PAST_STOP_ROW, 
+      null, null));
+
+    assertFalse("FilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("FilterNotNull", filter.filterRow(null));
+    
+    assertFalse("Filter a null", filter.filterRowKey(null));
+  }
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java Thu May 22 13:49:25 2008
@@ -1,99 +1,99 @@
-/**
- * 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 org.apache.hadoop.hbase.util.Bytes;
-
-
-import junit.framework.TestCase;
-
-/**
- * Tests for the page row filter
- */
-public class TestPageRowFilter extends TestCase {
-  
-  RowFilterInterface mainFilter;
-  final int ROW_LIMIT = 3;
-  
-  /** {@inheritDoc} */
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    mainFilter = new PageRowFilter(ROW_LIMIT);
-  }
-  
-  /**
-   * test page size filter
-   * @throws Exception
-   */
-  public void testPageSize() throws Exception {
-    pageSizeTests(mainFilter);
-  }
-  
-  /**
-   * Test filter 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 PageRowFilter();
-    newFilter.readFields(in);
-    
-    // Ensure the serialization preserved the filter by running a full test.
-    pageSizeTests(newFilter);
-  }
-  
-  private void pageSizeTests(RowFilterInterface filter) throws Exception {
-    testFiltersBeyondPageSize(filter, ROW_LIMIT);
-    // Test reset works by going in again.
-    filter.reset();
-    testFiltersBeyondPageSize(filter, ROW_LIMIT);
-  }
-  
-  private void testFiltersBeyondPageSize(final RowFilterInterface filter,
-    final int pageSize) {
-    for (int i = 0; i < (pageSize * 2); i++) {
-      byte [] row = Bytes.toBytes(Integer.toString(i));
-      boolean filterOut = filter.filterRowKey(row);
-      if (!filterOut) {
-        assertFalse("Disagrees with 'filter'", filter.filterAllRemaining());
-      } else {
-        // Once we have all for a page, calls to filterAllRemaining should
-        // stay true.
-        assertTrue("Disagrees with 'filter'", filter.filterAllRemaining());
-        assertTrue(i >= pageSize);
-      }
-      filter.rowProcessed(filterOut, row);
-    }
-  }
-}
+/**
+ * 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 org.apache.hadoop.hbase.util.Bytes;
+
+
+import junit.framework.TestCase;
+
+/**
+ * Tests for the page row filter
+ */
+public class TestPageRowFilter extends TestCase {
+  
+  RowFilterInterface mainFilter;
+  final int ROW_LIMIT = 3;
+  
+  /** {@inheritDoc} */
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    mainFilter = new PageRowFilter(ROW_LIMIT);
+  }
+  
+  /**
+   * test page size filter
+   * @throws Exception
+   */
+  public void testPageSize() throws Exception {
+    pageSizeTests(mainFilter);
+  }
+  
+  /**
+   * Test filter 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 PageRowFilter();
+    newFilter.readFields(in);
+    
+    // Ensure the serialization preserved the filter by running a full test.
+    pageSizeTests(newFilter);
+  }
+  
+  private void pageSizeTests(RowFilterInterface filter) throws Exception {
+    testFiltersBeyondPageSize(filter, ROW_LIMIT);
+    // Test reset works by going in again.
+    filter.reset();
+    testFiltersBeyondPageSize(filter, ROW_LIMIT);
+  }
+  
+  private void testFiltersBeyondPageSize(final RowFilterInterface filter,
+    final int pageSize) {
+    for (int i = 0; i < (pageSize * 2); i++) {
+      byte [] row = Bytes.toBytes(Integer.toString(i));
+      boolean filterOut = filter.filterRowKey(row);
+      if (!filterOut) {
+        assertFalse("Disagrees with 'filter'", filter.filterAllRemaining());
+      } else {
+        // Once we have all for a page, calls to filterAllRemaining should
+        // stay true.
+        assertTrue("Disagrees with 'filter'", filter.filterAllRemaining());
+        assertTrue(i >= pageSize);
+      }
+      filter.rowProcessed(filterOut, row);
+    }
+  }
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java Thu May 22 13:49:25 2008
@@ -1,196 +1,196 @@
-/**
- * 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.io.UnsupportedEncodingException;
-import java.util.Map;
-import java.util.TreeMap;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.regionserver.HLogEdit;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Tests for regular expression row filter
- */
-public class TestRegExpRowFilter extends TestCase {
-  TreeMap<byte [], byte []> colvalues;
-  RowFilterInterface mainFilter;
-  final char FIRST_CHAR = 'a';
-  final char LAST_CHAR = 'e';
-  final String HOST_PREFIX = "org.apache.site-";
-  static byte [] GOOD_BYTES = null;
-
-  static {
-    try {
-      GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
-    } catch (UnsupportedEncodingException e) {
-      fail();
-    }
-  }
-  /** {@inheritDoc} */
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    this.colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
-    for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
-      colvalues.put(Bytes.toBytes(new String(new char [] {c})), GOOD_BYTES);
-    }
-    this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues);
-  }
-  
-  /**
-   * Tests filtering using a regex on the row key
-   * @throws Exception
-   */
-  public void testRegexOnRow() throws Exception {
-    regexRowTests(mainFilter);
-  }
-
-  /**
-   * Tests filtering using a regex on row and colum
-   * @throws Exception
-   */
-  public void testRegexOnRowAndColumn() throws Exception {
-    regexRowColumnTests(mainFilter);
-  }
-  
-  /**
-   * Only return values that are not null
-   * @throws Exception
-   */
-  public void testFilterNotNull() throws Exception {
-    filterNotNullTests(mainFilter);
-  }
-  
-  /**
-   * Test 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 filter.
-    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
-    RowFilterInterface newFilter = new RegExpRowFilter();
-    newFilter.readFields(in);
-    
-    // Ensure the serialization preserved the filter by running all test.
-    regexRowTests(newFilter);
-    newFilter.reset();
-    regexRowColumnTests(newFilter);
-    newFilter.reset();
-    filterNotNullTests(newFilter);
-  }
- 
-  private void regexRowTests(RowFilterInterface filter) throws Exception {
-    for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
-      byte [] t = createRow(c);
-      assertFalse("Failed with characer " + c, filter.filterRowKey(t));
-    }
-    String yahooSite = "com.yahoo.www";
-    assertTrue("Failed with character " +
-      yahooSite, filter.filterRowKey(Bytes.toBytes(yahooSite)));
-  }
-  
-  private void regexRowColumnTests(RowFilterInterface filter)
-    throws UnsupportedEncodingException {
-    
-    for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
-      byte [] t = createRow(c);
-      for (Map.Entry<byte [], byte []> e: this.colvalues.entrySet()) {
-        assertFalse("Failed on " + c,
-          filter.filterColumn(t, e.getKey(), e.getValue()));
-      }
-    }
-    // Try a row and column I know will pass.
-    char c = 'c';
-    byte [] r = createRow(c);
-    byte [] col = Bytes.toBytes(Character.toString(c));
-    assertFalse("Failed with character " + c,
-      filter.filterColumn(r, col, GOOD_BYTES));
-    
-    // Do same but with bad bytes.
-    assertTrue("Failed with character " + c,
-      filter.filterColumn(r, col, "badbytes".getBytes(HConstants.UTF8_ENCODING)));
-    
-    // Do with good bytes but bad column name.  Should not filter out.
-    assertFalse("Failed with character " + c,
-      filter.filterColumn(r, Bytes.toBytes("badcolumn"), GOOD_BYTES));
-    
-    // Good column, good bytes but bad row.
-    assertTrue("Failed with character " + c,
-      filter.filterColumn(Bytes.toBytes("bad row"),
-        Bytes.toBytes("badcolumn"), GOOD_BYTES));
-  }
- 
-  private void filterNotNullTests(RowFilterInterface filter) throws Exception {
-    // Modify the filter to expect certain columns to be null:
-    // Expecting a row WITH columnKeys: a-d, WITHOUT columnKey: e
-    ((RegExpRowFilter)filter).setColumnFilter(new byte [] {LAST_CHAR}, null);
-    
-    char secondToLast = (char)(LAST_CHAR - 1);
-    char thirdToLast = (char)(LAST_CHAR - 2);
-    
-    // Modify the row to be missing an expected columnKey (d)
-    colvalues.remove(new byte [] {(byte)secondToLast});
-
-    // Try a row that is missing an expected columnKey.
-    // Testing row with columnKeys: a-c
-    assertTrue("Failed with last columnKey " + thirdToLast, filter.
-      filterRow(colvalues));
-
-    // Try a row that has all expected columnKeys, and NO null-expected
-    // columnKeys.
-    // Testing row with columnKeys: a-d
-    colvalues.put(new byte [] {(byte)secondToLast}, GOOD_BYTES);
-    assertFalse("Failed with last columnKey " + secondToLast, filter.
-      filterRow(colvalues));
-
-    // Try a row that has all expected columnKeys AND a null-expected columnKey.
-    // Testing row with columnKeys: a-e
-    colvalues.put(new byte [] {LAST_CHAR}, GOOD_BYTES);
-    assertTrue("Failed with last columnKey " + LAST_CHAR, filter.
-      filterRow(colvalues));
-    
-    // Try a row that has all expected columnKeys and a null-expected columnKey 
-    // that maps to a null value.
-    // Testing row with columnKeys: a-e, e maps to null
-    colvalues.put(new byte [] {LAST_CHAR}, 
-      HLogEdit.deleteBytes.get());
-    assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.", 
-      filter.filterRow(colvalues));
-  }
-
-  private byte [] createRow(final char c) {
-    return Bytes.toBytes(HOST_PREFIX + Character.toString(c));
-  }
-}
+/**
+ * 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.io.UnsupportedEncodingException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.regionserver.HLogEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Tests for regular expression row filter
+ */
+public class TestRegExpRowFilter extends TestCase {
+  TreeMap<byte [], byte []> colvalues;
+  RowFilterInterface mainFilter;
+  final char FIRST_CHAR = 'a';
+  final char LAST_CHAR = 'e';
+  final String HOST_PREFIX = "org.apache.site-";
+  static byte [] GOOD_BYTES = null;
+
+  static {
+    try {
+      GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
+    } catch (UnsupportedEncodingException e) {
+      fail();
+    }
+  }
+  /** {@inheritDoc} */
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    this.colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
+    for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
+      colvalues.put(Bytes.toBytes(new String(new char [] {c})), GOOD_BYTES);
+    }
+    this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues);
+  }
+  
+  /**
+   * Tests filtering using a regex on the row key
+   * @throws Exception
+   */
+  public void testRegexOnRow() throws Exception {
+    regexRowTests(mainFilter);
+  }
+
+  /**
+   * Tests filtering using a regex on row and colum
+   * @throws Exception
+   */
+  public void testRegexOnRowAndColumn() throws Exception {
+    regexRowColumnTests(mainFilter);
+  }
+  
+  /**
+   * Only return values that are not null
+   * @throws Exception
+   */
+  public void testFilterNotNull() throws Exception {
+    filterNotNullTests(mainFilter);
+  }
+  
+  /**
+   * Test 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 filter.
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+    RowFilterInterface newFilter = new RegExpRowFilter();
+    newFilter.readFields(in);
+    
+    // Ensure the serialization preserved the filter by running all test.
+    regexRowTests(newFilter);
+    newFilter.reset();
+    regexRowColumnTests(newFilter);
+    newFilter.reset();
+    filterNotNullTests(newFilter);
+  }
+ 
+  private void regexRowTests(RowFilterInterface filter) throws Exception {
+    for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
+      byte [] t = createRow(c);
+      assertFalse("Failed with characer " + c, filter.filterRowKey(t));
+    }
+    String yahooSite = "com.yahoo.www";
+    assertTrue("Failed with character " +
+      yahooSite, filter.filterRowKey(Bytes.toBytes(yahooSite)));
+  }
+  
+  private void regexRowColumnTests(RowFilterInterface filter)
+    throws UnsupportedEncodingException {
+    
+    for (char c = FIRST_CHAR; c <= LAST_CHAR; c++) {
+      byte [] t = createRow(c);
+      for (Map.Entry<byte [], byte []> e: this.colvalues.entrySet()) {
+        assertFalse("Failed on " + c,
+          filter.filterColumn(t, e.getKey(), e.getValue()));
+      }
+    }
+    // Try a row and column I know will pass.
+    char c = 'c';
+    byte [] r = createRow(c);
+    byte [] col = Bytes.toBytes(Character.toString(c));
+    assertFalse("Failed with character " + c,
+      filter.filterColumn(r, col, GOOD_BYTES));
+    
+    // Do same but with bad bytes.
+    assertTrue("Failed with character " + c,
+      filter.filterColumn(r, col, "badbytes".getBytes(HConstants.UTF8_ENCODING)));
+    
+    // Do with good bytes but bad column name.  Should not filter out.
+    assertFalse("Failed with character " + c,
+      filter.filterColumn(r, Bytes.toBytes("badcolumn"), GOOD_BYTES));
+    
+    // Good column, good bytes but bad row.
+    assertTrue("Failed with character " + c,
+      filter.filterColumn(Bytes.toBytes("bad row"),
+        Bytes.toBytes("badcolumn"), GOOD_BYTES));
+  }
+ 
+  private void filterNotNullTests(RowFilterInterface filter) throws Exception {
+    // Modify the filter to expect certain columns to be null:
+    // Expecting a row WITH columnKeys: a-d, WITHOUT columnKey: e
+    ((RegExpRowFilter)filter).setColumnFilter(new byte [] {LAST_CHAR}, null);
+    
+    char secondToLast = (char)(LAST_CHAR - 1);
+    char thirdToLast = (char)(LAST_CHAR - 2);
+    
+    // Modify the row to be missing an expected columnKey (d)
+    colvalues.remove(new byte [] {(byte)secondToLast});
+
+    // Try a row that is missing an expected columnKey.
+    // Testing row with columnKeys: a-c
+    assertTrue("Failed with last columnKey " + thirdToLast, filter.
+      filterRow(colvalues));
+
+    // Try a row that has all expected columnKeys, and NO null-expected
+    // columnKeys.
+    // Testing row with columnKeys: a-d
+    colvalues.put(new byte [] {(byte)secondToLast}, GOOD_BYTES);
+    assertFalse("Failed with last columnKey " + secondToLast, filter.
+      filterRow(colvalues));
+
+    // Try a row that has all expected columnKeys AND a null-expected columnKey.
+    // Testing row with columnKeys: a-e
+    colvalues.put(new byte [] {LAST_CHAR}, GOOD_BYTES);
+    assertTrue("Failed with last columnKey " + LAST_CHAR, filter.
+      filterRow(colvalues));
+    
+    // Try a row that has all expected columnKeys and a null-expected columnKey 
+    // that maps to a null value.
+    // Testing row with columnKeys: a-e, e maps to null
+    colvalues.put(new byte [] {LAST_CHAR}, 
+      HLogEdit.deleteBytes.get());
+    assertFalse("Failed with last columnKey " + LAST_CHAR + " mapping to null.", 
+      filter.filterRow(colvalues));
+  }
+
+  private byte [] createRow(final char c) {
+    return Bytes.toBytes(HOST_PREFIX + Character.toString(c));
+  }
+}

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java?rev=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java Thu May 22 13:49:25 2008
@@ -1,187 +1,187 @@
-/**
- * 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.io.UnsupportedEncodingException;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.TreeMap;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.util.Bytes;
-
-
-import junit.framework.TestCase;
-
-/**
- * Tests filter sets
- */
-public class TestRowFilterSet extends TestCase {
-
-  RowFilterInterface filterMPALL;
-  RowFilterInterface filterMPONE;
-  static final int MAX_PAGES = 5;
-  final char FIRST_CHAR = 'a';
-  final char LAST_CHAR = 'e';
-  TreeMap<byte [], byte[]> colvalues;
-  static byte[] GOOD_BYTES = null;
-  static byte[] BAD_BYTES = null;
-
-  static {
-    try {
-      GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
-      BAD_BYTES = "def".getBytes(HConstants.UTF8_ENCODING);
-    } catch (UnsupportedEncodingException e) {
-      fail();
-    }
-  }
-  
-  /** {@inheritDoc} */
-  @Override
-  protected void setUp() throws Exception {
-    super.setUp();
-    
-    colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
-    for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
-      colvalues.put(new byte [] {(byte)c}, GOOD_BYTES);
-    }
-    
-    Set<RowFilterInterface> filters = new HashSet<RowFilterInterface>();
-    filters.add(new PageRowFilter(MAX_PAGES));
-    filters.add(new RegExpRowFilter(".*regex.*", colvalues));
-    filters.add(new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("yyy"))));
-    filters.add(new WhileMatchRowFilter(new RegExpRowFilter(".*match.*")));
-    filterMPALL = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL, 
-      filters);
-    filterMPONE = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ONE, 
-      filters);
-  }
-  
-  /**
-   * Test "must pass one"
-   * @throws Exception
-   */
-  public void testMPONE() throws Exception {
-    MPONETests(filterMPONE);
-  }
-
-  /**
-   * Test "must pass all"
-   * @throws Exception
-   */
-  public void testMPALL() throws Exception {
-    MPALLTests(filterMPALL);
-  }
-  
-  /**
-   * Test serialization
-   * @throws Exception
-   */
-  public void testSerialization() throws Exception {
-    // Decompose filterMPALL to bytes.
-    ByteArrayOutputStream stream = new ByteArrayOutputStream();
-    DataOutputStream out = new DataOutputStream(stream);
-    filterMPALL.write(out);
-    out.close();
-    byte[] buffer = stream.toByteArray();
-    
-    // Recompose filterMPALL.
-    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
-    RowFilterInterface newFilter = new RowFilterSet();
-    newFilter.readFields(in);
-    
-    // Ensure the serialization preserved the filter by running a full test.
-    MPALLTests(newFilter);
-  }
-  
-  private void MPONETests(RowFilterInterface filter) throws Exception {
-    // A row that shouldn't cause any filters to return true.
-    RFSAssertion(filter, "regex_match", false);
-    
-    // A row that should cause the WhileMatchRowFilter to filter all remaining.
-    RFSAssertion(filter, "regex_only", false);
-    
-    // Make sure the overall filterAllRemaining is unchanged (correct for 
-    // MUST_PASS_ONE).
-    assertFalse(filter.filterAllRemaining());
-    
-    // A row that should cause the RegExpRowFilter to fail and the 
-    // StopRowFilter to filter all remaining.
-    RFSAssertion(filter, "yyy_match", false);
-    
-    // Accept several more rows such that PageRowFilter will exceed its limit.
-    for (int i=0; i<=MAX_PAGES-3; i++)
-      filter.rowProcessed(false, Bytes.toBytes("unimportant_key"));
-    
-    // A row that should cause the RegExpRowFilter to filter this row, making 
-    // all the filters return true and thus the RowFilterSet as well.
-    RFSAssertion(filter, "bad_column", true);
-    
-    // Make sure the overall filterAllRemaining is unchanged (correct for 
-    // MUST_PASS_ONE).
-    assertFalse(filter.filterAllRemaining());
-  }
-  
-  private void MPALLTests(RowFilterInterface filter) throws Exception {
-    // A row that shouldn't cause any filters to return true.
-    RFSAssertion(filter, "regex_match", false);
-    
-    // A row that should cause WhileMatchRowFilter to filter all remaining.
-    RFSAssertion(filter, "regex_only", true);
-
-    // Make sure the overall filterAllRemaining is changed (correct for 
-    // MUST_PASS_ALL).
-    RFSAssertReset(filter);
-    
-    // A row that should cause the RegExpRowFilter to fail and the 
-    // StopRowFilter to filter all remaining.
-    RFSAssertion(filter, "yyy_match", true);
-
-    // Make sure the overall filterAllRemaining is changed (correct for 
-    // MUST_PASS_ALL).
-    RFSAssertReset(filter);
-    
-    // A row that should cause the RegExpRowFilter to fail.
-    boolean filtered = filter.filterColumn(Bytes.toBytes("regex_match"), 
-      new byte [] { FIRST_CHAR }, BAD_BYTES);
-    assertTrue("Filtering on 'regex_match' and bad column data.", filtered);
-    filterMPALL.rowProcessed(filtered, Bytes.toBytes("regex_match"));
-  }
-  
-  private void RFSAssertion(RowFilterInterface filter, String toTest, 
-    boolean assertTrue) throws Exception {
-    byte [] testText = Bytes.toBytes(toTest);
-    boolean filtered = filter.filterRowKey(testText);
-    assertTrue("Filtering on '" + toTest + "'", 
-      assertTrue? filtered : !filtered);
-    filter.rowProcessed(filtered, testText);
-  }
-  
-  private void RFSAssertReset(RowFilterInterface filter) throws Exception{
-    assertTrue(filter.filterAllRemaining());
-    // Reset for continued testing
-    filter.reset();
-    assertFalse(filter.filterAllRemaining());
-  }
-}
+/**
+ * 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.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.TreeMap;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.util.Bytes;
+
+
+import junit.framework.TestCase;
+
+/**
+ * Tests filter sets
+ */
+public class TestRowFilterSet extends TestCase {
+
+  RowFilterInterface filterMPALL;
+  RowFilterInterface filterMPONE;
+  static final int MAX_PAGES = 5;
+  final char FIRST_CHAR = 'a';
+  final char LAST_CHAR = 'e';
+  TreeMap<byte [], byte[]> colvalues;
+  static byte[] GOOD_BYTES = null;
+  static byte[] BAD_BYTES = null;
+
+  static {
+    try {
+      GOOD_BYTES = "abc".getBytes(HConstants.UTF8_ENCODING);
+      BAD_BYTES = "def".getBytes(HConstants.UTF8_ENCODING);
+    } catch (UnsupportedEncodingException e) {
+      fail();
+    }
+  }
+  
+  /** {@inheritDoc} */
+  @Override
+  protected void setUp() throws Exception {
+    super.setUp();
+    
+    colvalues = new TreeMap<byte [], byte[]>(Bytes.BYTES_COMPARATOR);
+    for (char c = FIRST_CHAR; c < LAST_CHAR; c++) {
+      colvalues.put(new byte [] {(byte)c}, GOOD_BYTES);
+    }
+    
+    Set<RowFilterInterface> filters = new HashSet<RowFilterInterface>();
+    filters.add(new PageRowFilter(MAX_PAGES));
+    filters.add(new RegExpRowFilter(".*regex.*", colvalues));
+    filters.add(new WhileMatchRowFilter(new StopRowFilter(Bytes.toBytes("yyy"))));
+    filters.add(new WhileMatchRowFilter(new RegExpRowFilter(".*match.*")));
+    filterMPALL = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL, 
+      filters);
+    filterMPONE = new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ONE, 
+      filters);
+  }
+  
+  /**
+   * Test "must pass one"
+   * @throws Exception
+   */
+  public void testMPONE() throws Exception {
+    MPONETests(filterMPONE);
+  }
+
+  /**
+   * Test "must pass all"
+   * @throws Exception
+   */
+  public void testMPALL() throws Exception {
+    MPALLTests(filterMPALL);
+  }
+  
+  /**
+   * Test serialization
+   * @throws Exception
+   */
+  public void testSerialization() throws Exception {
+    // Decompose filterMPALL to bytes.
+    ByteArrayOutputStream stream = new ByteArrayOutputStream();
+    DataOutputStream out = new DataOutputStream(stream);
+    filterMPALL.write(out);
+    out.close();
+    byte[] buffer = stream.toByteArray();
+    
+    // Recompose filterMPALL.
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
+    RowFilterInterface newFilter = new RowFilterSet();
+    newFilter.readFields(in);
+    
+    // Ensure the serialization preserved the filter by running a full test.
+    MPALLTests(newFilter);
+  }
+  
+  private void MPONETests(RowFilterInterface filter) throws Exception {
+    // A row that shouldn't cause any filters to return true.
+    RFSAssertion(filter, "regex_match", false);
+    
+    // A row that should cause the WhileMatchRowFilter to filter all remaining.
+    RFSAssertion(filter, "regex_only", false);
+    
+    // Make sure the overall filterAllRemaining is unchanged (correct for 
+    // MUST_PASS_ONE).
+    assertFalse(filter.filterAllRemaining());
+    
+    // A row that should cause the RegExpRowFilter to fail and the 
+    // StopRowFilter to filter all remaining.
+    RFSAssertion(filter, "yyy_match", false);
+    
+    // Accept several more rows such that PageRowFilter will exceed its limit.
+    for (int i=0; i<=MAX_PAGES-3; i++)
+      filter.rowProcessed(false, Bytes.toBytes("unimportant_key"));
+    
+    // A row that should cause the RegExpRowFilter to filter this row, making 
+    // all the filters return true and thus the RowFilterSet as well.
+    RFSAssertion(filter, "bad_column", true);
+    
+    // Make sure the overall filterAllRemaining is unchanged (correct for 
+    // MUST_PASS_ONE).
+    assertFalse(filter.filterAllRemaining());
+  }
+  
+  private void MPALLTests(RowFilterInterface filter) throws Exception {
+    // A row that shouldn't cause any filters to return true.
+    RFSAssertion(filter, "regex_match", false);
+    
+    // A row that should cause WhileMatchRowFilter to filter all remaining.
+    RFSAssertion(filter, "regex_only", true);
+
+    // Make sure the overall filterAllRemaining is changed (correct for 
+    // MUST_PASS_ALL).
+    RFSAssertReset(filter);
+    
+    // A row that should cause the RegExpRowFilter to fail and the 
+    // StopRowFilter to filter all remaining.
+    RFSAssertion(filter, "yyy_match", true);
+
+    // Make sure the overall filterAllRemaining is changed (correct for 
+    // MUST_PASS_ALL).
+    RFSAssertReset(filter);
+    
+    // A row that should cause the RegExpRowFilter to fail.
+    boolean filtered = filter.filterColumn(Bytes.toBytes("regex_match"), 
+      new byte [] { FIRST_CHAR }, BAD_BYTES);
+    assertTrue("Filtering on 'regex_match' and bad column data.", filtered);
+    filterMPALL.rowProcessed(filtered, Bytes.toBytes("regex_match"));
+  }
+  
+  private void RFSAssertion(RowFilterInterface filter, String toTest, 
+    boolean assertTrue) throws Exception {
+    byte [] testText = Bytes.toBytes(toTest);
+    boolean filtered = filter.filterRowKey(testText);
+    assertTrue("Filtering on '" + toTest + "'", 
+      assertTrue? filtered : !filtered);
+    filter.rowProcessed(filtered, testText);
+  }
+  
+  private void RFSAssertReset(RowFilterInterface filter) throws Exception{
+    assertTrue(filter.filterAllRemaining());
+    // Reset for continued testing
+    filter.reset();
+    assertFalse(filter.filterAllRemaining());
+  }
+}

Modified: 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=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java Thu May 22 13:49:25 2008
@@ -1,93 +1,93 @@
-/**
- * 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 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;
-
-  /** {@inheritDoc} */
-  @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 " + GOOD_ROW, filter.filterRowKey(GOOD_ROW));
-    assertTrue("Filtering on " + STOP_ROW, filter.filterRowKey(STOP_ROW));
-    assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterRowKey(PAST_STOP_ROW));
-    
-    assertFalse("Filtering on " + GOOD_ROW, filter.filterColumn(GOOD_ROW, null, 
-      null));
-    assertTrue("Filtering on " + STOP_ROW, filter.filterColumn(STOP_ROW, null, null));
-    assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterColumn(PAST_STOP_ROW, 
-      null, null));
-
-    assertFalse("FilterAllRemaining", filter.filterAllRemaining());
-    assertFalse("FilterNotNull", filter.filterRow(null));
-    
-    assertFalse("Filter a null", filter.filterRowKey(null));
-  }
-}
+/**
+ * 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 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;
+
+  /** {@inheritDoc} */
+  @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 " + GOOD_ROW, filter.filterRowKey(GOOD_ROW));
+    assertTrue("Filtering on " + STOP_ROW, filter.filterRowKey(STOP_ROW));
+    assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterRowKey(PAST_STOP_ROW));
+    
+    assertFalse("Filtering on " + GOOD_ROW, filter.filterColumn(GOOD_ROW, null, 
+      null));
+    assertTrue("Filtering on " + STOP_ROW, filter.filterColumn(STOP_ROW, null, null));
+    assertTrue("Filtering on " + PAST_STOP_ROW, filter.filterColumn(PAST_STOP_ROW, 
+      null, null));
+
+    assertFalse("FilterAllRemaining", filter.filterAllRemaining());
+    assertFalse("FilterNotNull", filter.filterRow(null));
+    
+    assertFalse("Filter a null", filter.filterRowKey(null));
+  }
+}

Modified: 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=659249&r1=659248&r2=659249&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java Thu May 22 13:49:25 2008
@@ -1,150 +1,150 @@
-/**
- * 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 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;
-
-  /** {@inheritDoc} */
-  @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)));
-    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
-        toTest)));
-    
-    // Test cases that should fail the row
-    toTest = "tuna";
-    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
-    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
-        toTest)));
-    
-    // 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(null));
-  }
-  
-  private void whileMatchRegExpTests(WhileMatchRowFilter filter) throws 
-    Exception {
-    RowFilterInterface innerFilter = filter.getInternalFilter();
-    String toTest;
-    
-    // Test cases that should pass the row
-    toTest = "regex_match";
-    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
-    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
-        toTest)));
-    
-    // Test cases that should fail the row
-    toTest = "not_a_match";
-    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
-    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
-        toTest)));
-    
-    // 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";
-    assertFalse("filter: '" + toTest + "'", filter.filterColumn(Bytes.toBytes(toTest), 
-      null, null));
-  }
-}
+/**
+ * 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 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;
+
+  /** {@inheritDoc} */
+  @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)));
+    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+        toTest)));
+    
+    // Test cases that should fail the row
+    toTest = "tuna";
+    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+        toTest)));
+    
+    // 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(null));
+  }
+  
+  private void whileMatchRegExpTests(WhileMatchRowFilter filter) throws 
+    Exception {
+    RowFilterInterface innerFilter = filter.getInternalFilter();
+    String toTest;
+    
+    // Test cases that should pass the row
+    toTest = "regex_match";
+    assertFalse("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertFalse("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+        toTest)));
+    
+    // Test cases that should fail the row
+    toTest = "not_a_match";
+    assertTrue("filter: '" + toTest + "'", filter.filterRowKey(Bytes.toBytes(toTest)));
+    assertTrue("innerFilter: '" + toTest + "'", innerFilter.filterRowKey(Bytes.toBytes(
+        toTest)));
+    
+    // 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";
+    assertFalse("filter: '" + toTest + "'", filter.filterColumn(Bytes.toBytes(toTest), 
+      null, null));
+  }
+}