Author: tedyu
Date: Fri Mar 30 19:33:27 2012
New Revision: 1307580
URL: http://svn.apache.org/viewvc?rev=1307580&view=rev
Log:
HBASE-5667 RegexStringComparator supports java.util.regex.Pattern flags (David
Arthur)
Modified:
hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
Modified:
hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
URL:
http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java?rev=1307580&r1=1307579&r2=1307580&view=diff
==============================================================================
---
hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
(original)
+++
hbase/trunk/src/main/java/org/apache/hadoop/hbase/filter/RegexStringComparator.java
Fri Mar 30 19:33:27 2012
@@ -55,6 +55,14 @@ import java.util.regex.Pattern;
* "((([\\dA-Fa-f]{1,4}:){7}[\\dA-Fa-f]{1,4})(:([\\d]{1,3}.)" +
* "{3}[\\d]{1,3})?)(\\/[0-9]+)?"));
* </pre>
+ * <p>
+ * Supports {@link java.util.regex.Pattern} flags as well:
+ * <p>
+ * <pre>
+ * ValueFilter vf = new ValueFilter(CompareOp.EQUAL,
+ * new RegexStringComparator("regex", Pattern.CASE_INSENSITIVE |
Pattern.DOTALL));
+ * </pre>
+ * @see java.util.regex.Pattern;
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
@@ -71,11 +79,21 @@ public class RegexStringComparator exten
/**
* Constructor
+ * Adds Pattern.DOTALL to the underlying Pattern
* @param expr a valid regular expression
*/
public RegexStringComparator(String expr) {
+ this(expr, Pattern.DOTALL);
+ }
+
+ /**
+ * Constructor
+ * @param expr a valid regular expression
+ * @param flags java.util.regex.Pattern flags
+ */
+ public RegexStringComparator(String expr, int flags) {
super(Bytes.toBytes(expr));
- this.pattern = Pattern.compile(expr, Pattern.DOTALL);
+ this.pattern = Pattern.compile(expr, flags);
}
/**
@@ -105,7 +123,8 @@ public class RegexStringComparator exten
public void readFields(DataInput in) throws IOException {
final String expr = in.readUTF();
this.value = Bytes.toBytes(expr);
- this.pattern = Pattern.compile(expr);
+ int flags = in.readInt();
+ this.pattern = Pattern.compile(expr, flags);
final String charset = in.readUTF();
if (charset.length() > 0) {
try {
@@ -119,6 +138,7 @@ public class RegexStringComparator exten
@Override
public void write(DataOutput out) throws IOException {
out.writeUTF(pattern.toString());
+ out.writeInt(pattern.flags());
out.writeUTF(charset.name());
}
Modified:
hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
URL:
http://svn.apache.org/viewvc/hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java?rev=1307580&r1=1307579&r2=1307580&view=diff
==============================================================================
---
hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
(original)
+++
hbase/trunk/src/test/java/org/apache/hadoop/hbase/filter/TestSingleColumnValueFilter.java
Fri Mar 30 19:33:27 2012
@@ -23,6 +23,7 @@ import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
+import java.util.regex.Pattern;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.SmallTests;
@@ -50,10 +51,12 @@ public class TestSingleColumnValueFilter
Bytes.toBytes("The slow grey fox trips over the lazy dog.");
private static final String QUICK_SUBSTR = "quick";
private static final String QUICK_REGEX = ".+quick.+";
+ private static final Pattern QUICK_PATTERN = Pattern.compile("QuIcK",
Pattern.CASE_INSENSITIVE | Pattern.DOTALL);
Filter basicFilter;
Filter substrFilter;
Filter regexFilter;
+ Filter regexPatternFilter;
@Override
protected void setUp() throws Exception {
@@ -61,6 +64,7 @@ public class TestSingleColumnValueFilter
basicFilter = basicFilterNew();
substrFilter = substrFilterNew();
regexFilter = regexFilterNew();
+ regexPatternFilter = regexFilterNew(QUICK_PATTERN);
}
private Filter basicFilterNew() {
@@ -80,6 +84,12 @@ public class TestSingleColumnValueFilter
new RegexStringComparator(QUICK_REGEX));
}
+ private Filter regexFilterNew(Pattern pattern) {
+ return new SingleColumnValueFilter(COLUMN_FAMILY, COLUMN_QUALIFIER,
+ CompareOp.EQUAL,
+ new RegexStringComparator(pattern.pattern(), pattern.flags()));
+ }
+
private void basicFilterTests(SingleColumnValueFilter filter)
throws Exception {
KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER, VAL_2);
@@ -131,6 +141,16 @@ public class TestSingleColumnValueFilter
assertFalse("regexFilterNotNull", filter.filterRow());
}
+ private void regexPatternFilterTests(Filter filter)
+ throws Exception {
+ KeyValue kv = new KeyValue(ROW, COLUMN_FAMILY, COLUMN_QUALIFIER,
+ FULLSTRING_1);
+ assertTrue("regexTrue",
+ filter.filterKeyValue(kv) == Filter.ReturnCode.INCLUDE);
+ assertFalse("regexFilterAllRemaining", filter.filterAllRemaining());
+ assertFalse("regexFilterNotNull", filter.filterRow());
+ }
+
private Filter serializationTest(Filter filter)
throws Exception {
// Decompose filter to bytes.
@@ -145,7 +165,6 @@ public class TestSingleColumnValueFilter
new DataInputStream(new ByteArrayInputStream(buffer));
Filter newFilter = new SingleColumnValueFilter();
newFilter.readFields(in);
-
return newFilter;
}
@@ -157,6 +176,7 @@ public class TestSingleColumnValueFilter
basicFilterTests((SingleColumnValueFilter)basicFilter);
substrFilterTests(substrFilter);
regexFilterTests(regexFilter);
+ regexPatternFilterTests(regexPatternFilter);
}
/**
@@ -170,6 +190,8 @@ public class TestSingleColumnValueFilter
substrFilterTests(newFilter);
newFilter = serializationTest(regexFilter);
regexFilterTests(newFilter);
+ newFilter = serializationTest(regexPatternFilter);
+ regexPatternFilterTests(newFilter);
}
@org.junit.Rule