From 377bf72458b3708f4078b043dce9425ebb807a46 Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Mon, 23 Jul 2007 21:33:47 +0000 Subject: [PATCH] HADOOP-1579 Add new WhileMatchRowFilter and StopRowFilter filters git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk/src/contrib/hbase@558867 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 2 + .../hadoop/hbase/filter/StopRowFilter.java | 156 ++++++++++++++ .../hbase/filter/WhileMatchRowFilter.java | 204 ++++++++++++++++++ .../hadoop/hbase/filter/TestRowFilterSet.java | 159 ++++++++++++++ .../hbase/filter/TestStopRowFilter.java | 78 +++++++ .../hbase/filter/TestWhileMatchRowFilter.java | 132 ++++++++++++ 6 files changed, 731 insertions(+) create mode 100644 src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java create mode 100644 src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java create mode 100644 src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java create mode 100644 src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java create mode 100644 src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java diff --git a/CHANGES.txt b/CHANGES.txt index fa0a076ffe9..407bb15afc4 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -69,3 +69,5 @@ Trunk (unreleased changes) java.util.concurrent.BlockingQueue in HMaster, HRegionServer 45. HADOOP-1606 Updated implementation of RowFilterSet, RowFilterInterface (Izaak Rubin via Stack) + 46. HADOOP-1579 Add new WhileMatchRowFilter and StopRowFilter filters + (Izaak Rubin via Stack) diff --git a/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java new file mode 100644 index 00000000000..1dca4c70f0f --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java @@ -0,0 +1,156 @@ +/** + * 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.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.Text; + +/** + * Implementation of RowFilterInterface that filters out rows greater than or + * equal to a specified rowKey. + */ +public class StopRowFilter implements RowFilterInterface { + + private Text stopRowKey; + + static final Log LOG = LogFactory.getLog(StopRowFilter.class); + + /** + * Default constructor, filters nothing. Required though for RPC + * deserialization. + */ + public StopRowFilter() { + super(); + } + + /** + * Constructor that takes a stopRowKey on which to filter + * + * @param stopRowKey rowKey to filter on. + */ + public StopRowFilter(final Text stopRowKey) { + this.stopRowKey = stopRowKey; + } + + /** + * An accessor for the stopRowKey + * + * @return the filter's stopRowKey + */ + public Text getStopRowKey() { + return this.stopRowKey; + } + + /** + * + * {@inheritDoc} + */ + public void validate(@SuppressWarnings("unused") final Text[] columns) { + // Doesn't filter columns + } + + /** + * + * {@inheritDoc} + */ + public void reset() { + // Nothing to reset + } + + /** + * + * {@inheritDoc} + */ + public void rowProcessed(boolean filtered, Text rowKey) { + // Doesn't care + } + + /** + * + * {@inheritDoc} + */ + public boolean processAlways() { + return false; + } + + /** + * + * {@inheritDoc} + */ + public boolean filterAllRemaining() { + return false; + } + + /** + * + * {@inheritDoc} + */ + public boolean filter(final Text rowKey) { + boolean result = this.stopRowKey.compareTo(rowKey) <= 0; + if (LOG.isDebugEnabled()) { + LOG.debug("Filter result for rowKey: " + rowKey + ". Result: " + + result); + } + return result; + } + + /** + * Because StopRowFilter does not examine column information, this method + * defaults to calling the rowKey-only version of filter. + */ + public boolean filter(@SuppressWarnings("unused") final Text rowKey, + @SuppressWarnings("unused") final Text colKey, + @SuppressWarnings("unused") final byte[] data) { + return filter(rowKey); + } + + /** + * Because StopRowFilter does not examine column information, this method + * defaults to calling filterAllRemaining(). + * + * @param columns + */ + public boolean filterNotNull(@SuppressWarnings("unused") + final TreeMap columns) { + return filterAllRemaining(); + } + + /** + * + * {@inheritDoc} + */ + public void readFields(DataInput in) throws IOException { + stopRowKey = new Text(in.readUTF()); + } + + /** + * + * {@inheritDoc} + */ + public void write(DataOutput out) throws IOException { + out.writeUTF(stopRowKey.toString()); + } +} diff --git a/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java b/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java new file mode 100644 index 00000000000..278a4990b0f --- /dev/null +++ b/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java @@ -0,0 +1,204 @@ +/** + * 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.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.Text; + +/** + * WhileMatchRowFilter is a wrapper filter that filters everything after the + * first filtered row. Once the nested filter returns true for either of it's + * filter(..) methods or filterNotNull(TreeMap), this wrapper's + * filterAllRemaining() will return true. All filtering methods will + * thereafter defer to the result of filterAllRemaining(). + */ +public class WhileMatchRowFilter implements RowFilterInterface { + + private boolean filterAllRemaining = false; + private RowFilterInterface filter; + + static final Log LOG = LogFactory.getLog(WhileMatchRowFilter.class); + + /** + * Default constructor, filters nothing. Required though for RPC + * deserialization. + */ + public WhileMatchRowFilter() { + super(); + } + + public WhileMatchRowFilter(RowFilterInterface filter) { + this.filter = filter; + } + + /** + * Returns the internal filter being wrapped + * + * @return the internal filter + */ + public RowFilterInterface getInternalFilter() { + return this.filter; + } + + /** + * + * {@inheritDoc} + */ + public void reset() { + if (LOG.isDebugEnabled()) { + LOG.debug("Resetting."); + } + this.filterAllRemaining = false; + this.filter.reset(); + } + + /** + * + * {@inheritDoc} + */ + public boolean processAlways() { + return true; + } + + /** + * Returns true once the nested filter has filtered out a row (returned true + * on a call to one of it's filtering methods). Until then it returns false. + * + * @return true/false whether the nested filter has returned true on a filter + * call. + */ + public boolean filterAllRemaining() { + return this.filterAllRemaining || this.filter.filterAllRemaining(); + } + + /** + * + * {@inheritDoc} + */ + public boolean filter(final Text rowKey) { + changeFAR(this.filter.filter(rowKey)); + boolean result = filterAllRemaining(); + if (LOG.isDebugEnabled()) { + LOG.debug("Filter on rowKey:" + rowKey + ". Result = " + result); + } + return result; + } + + /** + * + * {@inheritDoc} + */ + public boolean filter(final Text rowKey, final Text colKey, + final byte[] data) { + changeFAR(this.filter.filter(rowKey, colKey, data)); + boolean result = filterAllRemaining(); + if (LOG.isDebugEnabled()) { + LOG.debug("Filter on rowKey:" + rowKey + ", colKey: " + colKey + + ", data: " + data + ". Result = " + result); + } + return result; + } + + /** + * + * {@inheritDoc} + */ + public boolean filterNotNull(final TreeMap columns) { + changeFAR(this.filter.filterNotNull(columns)); + boolean result = filterAllRemaining(); + if (LOG.isDebugEnabled()) { + LOG.debug("FilterNotNull on cols:" + columns + ". Result = " + + result); + } + return result; + } + + /** + * Change filterAllRemaining from false to true if value is true, otherwise + * leave as is. + * + * @param value + */ + private void changeFAR(boolean value) { + this.filterAllRemaining = this.filterAllRemaining || value; + if (LOG.isDebugEnabled()) { + LOG.debug("this.filterAllRemaining is now: " + + this.filterAllRemaining); + } + } + + /** + * + * {@inheritDoc} + */ + public void rowProcessed(boolean filtered, Text rowKey) { + this.filter.rowProcessed(filtered, rowKey); + } + + /** + * + * {@inheritDoc} + */ + public void validate(Text[] columns) { + this.filter.validate(columns); + } + + /** + * + * {@inheritDoc} + */ + public void readFields(DataInput in) throws IOException { + String className = in.readUTF(); + + try { + this.filter = (RowFilterInterface)(Class.forName(className). + newInstance()); + this.filter.readFields(in); + if (LOG.isDebugEnabled()) { + LOG.debug("Successfully read a sub-filter of type: " + + className); + } + } catch (InstantiationException e) { + throw new RuntimeException("Failed to deserialize WhileMatchRowFilter.", + e); + } catch (IllegalAccessException e) { + throw new RuntimeException("Failed to deserialize WhileMatchRowFilter.", + e); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Failed to deserialize WhileMatchRowFilter.", + e); + } + } + + /** + * + * {@inheritDoc} + */ + public void write(DataOutput out) throws IOException { + out.writeUTF(this.filter.getClass().getName()); + this.filter.write(out); + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java new file mode 100644 index 00000000000..3ea7e1a4b4d --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java @@ -0,0 +1,159 @@ +/** + * 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.HashSet; +import java.util.Set; +import java.util.TreeMap; + +import org.apache.hadoop.io.Text; + + +import junit.framework.TestCase; + +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'; + final byte[] GOOD_BYTES = "abc".getBytes(); + final byte[] BAD_BYTES = "def".getBytes(); + TreeMap colvalues; + + protected void setUp() throws Exception { + super.setUp(); + + colvalues = new TreeMap(); + for (char c = FIRST_CHAR; c < LAST_CHAR; c++) { + colvalues.put(new Text(new String(new char [] {c})), GOOD_BYTES); + } + + Set filters = new HashSet(); + filters.add(new PageRowFilter(MAX_PAGES)); + filters.add(new RegExpRowFilter(".*regex.*", colvalues)); + filters.add(new WhileMatchRowFilter(new StopRowFilter(new Text("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); + } + + public void testMPONE() throws Exception { + MPONETests(filterMPONE); + } + + public void testMPALL() throws Exception { + MPALLTests(filterMPALL); + } + + 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, new Text("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.filter(new Text("regex_match"), + new Text(new String(new char[] { FIRST_CHAR })), BAD_BYTES); + assertTrue("Filtering on 'regex_match' and bad column data.", filtered); + filterMPALL.rowProcessed(filtered, new Text("regex_match")); + } + + private void RFSAssertion(RowFilterInterface filter, String toTest, + boolean assertTrue) throws Exception { + Text testText = new Text(toTest); + boolean filtered = filter.filter(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()); + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java new file mode 100644 index 00000000000..8d6e994cdd4 --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java @@ -0,0 +1,78 @@ +/** + * 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.io.Text; + +import junit.framework.TestCase; + +public class TestStopRowFilter extends TestCase { + private final Text STOP_ROW = new Text("stop_row"); + private final Text GOOD_ROW = new Text("good_row"); + private final Text PAST_STOP_ROW = new Text("zzzzzz"); + + RowFilterInterface mainFilter; + + protected void setUp() throws Exception { + super.setUp(); + mainFilter = new StopRowFilter(STOP_ROW); + } + + public void testStopRowIdentification() throws Exception { + stopRowTests(mainFilter); + } + + 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.filter(GOOD_ROW)); + assertTrue("Filtering on " + STOP_ROW, filter.filter(STOP_ROW)); + assertTrue("Filtering on " + PAST_STOP_ROW, filter.filter(PAST_STOP_ROW)); + + assertFalse("Filtering on " + GOOD_ROW, filter.filter(GOOD_ROW, null, + null)); + assertTrue("Filtering on " + STOP_ROW, filter.filter(STOP_ROW, null, null)); + assertTrue("Filtering on " + PAST_STOP_ROW, filter.filter(PAST_STOP_ROW, + null, null)); + + assertFalse("FilterAllRemaining", filter.filterAllRemaining()); + assertFalse("FilterNotNull", filter.filterNotNull(null)); + } +} diff --git a/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java b/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java new file mode 100644 index 00000000000..7be2f1ff11a --- /dev/null +++ b/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java @@ -0,0 +1,132 @@ +/** + * 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 junit.framework.TestCase; + +import org.apache.hadoop.io.Text; + +public class TestWhileMatchRowFilter extends TestCase { + + WhileMatchRowFilter wmStopRowFilter; + WhileMatchRowFilter wmRegExpRowFilter; + + protected void setUp() throws Exception { + super.setUp(); + wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter( + new Text("s"))); + wmRegExpRowFilter = new WhileMatchRowFilter(new RegExpRowFilter( + ".*regex.*")); + } + + public void testWhileMatchStopRow() throws Exception { + whileMatchStopRowTests(wmStopRowFilter); + } + + public void testWhileMatchRegExp() throws Exception { + whileMatchRegExpTests(wmRegExpRowFilter); + } + + 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.filter(new Text(toTest))); + assertFalse("innerFilter: '" + toTest + "'", innerFilter.filter(new Text( + toTest))); + + // Test cases that should fail the row + toTest = "tuna"; + assertTrue("filter: '" + toTest + "'", filter.filter(new Text(toTest))); + assertTrue("innerFilter: '" + toTest + "'", innerFilter.filter(new Text( + 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.filterNotNull(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.filter(new Text(toTest))); + assertFalse("innerFilter: '" + toTest + "'", innerFilter.filter(new Text( + toTest))); + + // Test cases that should fail the row + toTest = "not_a_match"; + assertTrue("filter: '" + toTest + "'", filter.filter(new Text(toTest))); + assertTrue("innerFilter: '" + toTest + "'", innerFilter.filter(new Text( + 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.filter(new Text(toTest), + null, null)); + } +}