You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2009/07/02 21:31:10 UTC
svn commit: r790717 - in /hadoop/hbase/trunk: ./
src/java/org/apache/hadoop/hbase/filter/
src/test/org/apache/hadoop/hbase/filter/
Author: stack
Date: Thu Jul 2 19:31:10 2009
New Revision: 790717
URL: http://svn.apache.org/viewvc?rev=790717&view=rev
Log:
HBASE-1599 Fix TestFilterSet, broken up on hudson
Added:
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/FilterList.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterList.java
Removed:
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/FilterSet.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterSet.java
Modified:
hadoop/hbase/trunk/CHANGES.txt
hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageFilter.java
hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageFilter.java
Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=790717&r1=790716&r2=790717&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Thu Jul 2 19:31:10 2009
@@ -25,6 +25,7 @@
handled (holstad, jgray, rawson, stack)
HBASE-1582 Translate ColumnValueFilter and RowFilterSet to the new
Filter interface (Clint Morgan and Stack)
+ HBASE-1599 Fix TestFilterSet, broken up on hudson (Jon Gray via Stack)
BUG FIXES
HBASE-1140 "ant clean test" fails (Nitay Joffe via Stack)
Added: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/FilterList.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/FilterList.java?rev=790717&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/FilterList.java (added)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/FilterList.java Thu Jul 2 19:31:10 2009
@@ -0,0 +1,217 @@
+/**
+ * Copyright 2009 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.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.HbaseObjectWritable;
+
+/**
+ * Implementation of {@link Filter} that represents an ordered List of Filters
+ * which will be evaluated with a specified boolean operator MUST_PASS_ALL
+ * (!AND) or MUST_PASS_ONE (!OR). Since you can use Filter Lists as children
+ * of Filter Lists, you can create a hierarchy of filters to be evaluated.
+ * <p>TODO: Fix creation of Configuration on serialization and deserialization.
+ */
+public class FilterList implements Filter {
+
+ /** set operator */
+ public static enum Operator {
+ /** !AND */
+ MUST_PASS_ALL,
+ /** !OR */
+ MUST_PASS_ONE
+ }
+
+ private Operator operator = Operator.MUST_PASS_ALL;
+ private List<Filter> filters = new ArrayList<Filter>();
+
+ /**
+ * Default constructor, filters nothing. Required though for RPC
+ * deserialization.
+ */
+ public FilterList() {
+ super();
+ }
+
+ /**
+ * Constructor that takes a set of {@link Filter}s. The default operator
+ * MUST_PASS_ALL is assumed.
+ *
+ * @param rowFilters
+ */
+ public FilterList(final List<Filter> rowFilters) {
+ this.filters = rowFilters;
+ }
+
+ /**
+ * Constructor that takes a set of {@link Filter}s and an operator.
+ *
+ * @param operator Operator to process filter set with.
+ * @param rowFilters Set of row filters.
+ */
+ public FilterList(final Operator operator, final List<Filter> rowFilters) {
+ this.filters = rowFilters;
+ this.operator = operator;
+ }
+
+ /**
+ * Get the operator.
+ *
+ * @return operator
+ */
+ public Operator getOperator() {
+ return operator;
+ }
+
+ /**
+ * Get the filters.
+ *
+ * @return filters
+ */
+ public List<Filter> getFilters() {
+ return filters;
+ }
+
+ /**
+ * Add a filter.
+ *
+ * @param filter
+ */
+ public void addFilter(Filter filter) {
+ this.filters.add(filter);
+ }
+
+ public void reset() {
+ for (Filter filter : filters) {
+ filter.reset();
+ }
+ }
+
+ public boolean filterRowKey(byte[] rowKey, int offset, int length) {
+ for (Filter filter : filters) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining()
+ || filter.filterRowKey(rowKey, offset, length)) {
+ return true;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterAllRemaining()
+ && !filter.filterRowKey(rowKey, offset, length)) {
+ return false;
+ }
+ }
+ }
+ return operator == Operator.MUST_PASS_ONE;
+ }
+
+ public boolean filterAllRemaining() {
+ for (Filter filter : filters) {
+ if (filter.filterAllRemaining()) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ return true;
+ }
+ } else {
+ if (operator == Operator.MUST_PASS_ONE) {
+ return false;
+ }
+ }
+ }
+ return operator == Operator.MUST_PASS_ONE;
+ }
+
+ public ReturnCode filterKeyValue(KeyValue v) {
+ for (Filter filter : filters) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining()) {
+ return ReturnCode.NEXT_ROW;
+ }
+ switch (filter.filterKeyValue(v)) {
+ case INCLUDE:
+ continue;
+ case NEXT_ROW:
+ case SKIP:
+ return ReturnCode.SKIP;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (filter.filterAllRemaining()) {
+ continue;
+ }
+
+ switch (filter.filterKeyValue(v)) {
+ case INCLUDE:
+ return ReturnCode.INCLUDE;
+ case NEXT_ROW:
+ case SKIP:
+ continue;
+ }
+ }
+ }
+ return operator == Operator.MUST_PASS_ONE?
+ ReturnCode.SKIP: ReturnCode.INCLUDE;
+ }
+
+ public boolean filterRow() {
+ for (Filter filter : filters) {
+ if (operator == Operator.MUST_PASS_ALL) {
+ if (filter.filterAllRemaining()
+ || filter.filterRow()) {
+ return true;
+ }
+ } else if (operator == Operator.MUST_PASS_ONE) {
+ if (!filter.filterAllRemaining()
+ && !filter.filterRow()) {
+ return false;
+ }
+ }
+ }
+ return operator == Operator.MUST_PASS_ONE;
+ }
+
+ public void readFields(final DataInput in) throws IOException {
+ Configuration conf = new HBaseConfiguration();
+ byte opByte = in.readByte();
+ operator = Operator.values()[opByte];
+ int size = in.readInt();
+ if (size > 0) {
+ filters = new ArrayList<Filter>(size);
+ for (int i = 0; i < size; i++) {
+ Filter filter = (Filter)HbaseObjectWritable.readObject(in, conf);
+ filters.add(filter);
+ }
+ }
+ }
+
+ public void write(final DataOutput out) throws IOException {
+ Configuration conf = new HBaseConfiguration();
+ out.writeByte(operator.ordinal());
+ out.writeInt(filters.size());
+ for (Filter filter : filters) {
+ HbaseObjectWritable.writeObject(out, filter, filter.getClass(), conf);
+ }
+ }
+}
\ No newline at end of file
Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageFilter.java?rev=790717&r1=790716&r2=790717&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageFilter.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/filter/PageFilter.java Thu Jul 2 19:31:10 2009
@@ -1,5 +1,5 @@
/**
- * Copyright 2007 The Apache Software Foundation
+ * Copyright 2009 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
@@ -27,7 +27,7 @@
/**
* Implementation of Filter interface that limits results to a specific page
- * size. It terminates scanning once the number of filter-passed results is >=
+ * size. It terminates scanning once the number of filter-passed rows is >
* the given page size.
*
* <p>
@@ -64,10 +64,11 @@
}
public boolean filterAllRemaining() {
- return this.rowsAccepted >= this.pageSize;
+ return this.rowsAccepted > this.pageSize;
}
public boolean filterRowKey(byte[] rowKey, int offset, int length) {
+ this.rowsAccepted++;
return filterAllRemaining();
}
@@ -80,8 +81,7 @@
}
public ReturnCode filterKeyValue(KeyValue v) {
- this.rowsAccepted++;
- return filterAllRemaining()? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
+ return filterAllRemaining() ? ReturnCode.NEXT_ROW : ReturnCode.INCLUDE;
}
public boolean filterRow() {
Added: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterList.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterList.java?rev=790717&view=auto
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterList.java (added)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestFilterList.java Thu Jul 2 19:31:10 2009
@@ -0,0 +1,231 @@
+/**
+ * Copyright 2009 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.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.KeyValue;
+
+
+import junit.framework.TestCase;
+
+/**
+ * Tests filter sets
+ *
+ */
+public class TestFilterList extends TestCase {
+ static final int MAX_PAGES = 2;
+ static final char FIRST_CHAR = 'a';
+ static final char LAST_CHAR = 'e';
+ static byte[] GOOD_BYTES = Bytes.toBytes("abc");
+ static byte[] BAD_BYTES = Bytes.toBytes("def");
+
+ /**
+ * Test "must pass one"
+ * @throws Exception
+ */
+ public void testMPONE() throws Exception {
+ List<Filter> filters = new ArrayList<Filter>();
+ filters.add(new PageFilter(MAX_PAGES));
+ filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
+ Filter filterMPONE =
+ new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
+ /* Filter must do all below steps:
+ * <ul>
+ * <li>{@link #reset()}</li>
+ * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
+ * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
+ * if false, we will also call</li>
+ * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
+ * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
+ * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
+ * </li>
+ * </ul>
+ */
+ filterMPONE.reset();
+ assertFalse(filterMPONE.filterAllRemaining());
+
+ /* Will pass both */
+ byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
+ for (int i = 0; i < MAX_PAGES - 1; i++) {
+ assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
+ Bytes.toBytes(i));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+ }
+
+ /* Only pass PageFilter */
+ rowkey = Bytes.toBytes("z");
+ assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
+ Bytes.toBytes(0));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+
+ /* PageFilter will fail now, but should pass because we match yyy */
+ rowkey = Bytes.toBytes("yyy");
+ assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
+ Bytes.toBytes(0));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+
+ /* We should filter the row key now if we match neither */
+ rowkey = Bytes.toBytes("x");
+ assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(0),
+ Bytes.toBytes(0));
+ assertTrue(Filter.ReturnCode.SKIP == filterMPONE.filterKeyValue(kv));
+
+ // Both filters in Set should be satisfied by now
+ assertTrue(filterMPONE.filterRow());
+
+ }
+
+ /**
+ * Test "must pass all"
+ * @throws Exception
+ */
+ public void testMPALL() throws Exception {
+ List<Filter> filters = new ArrayList<Filter>();
+ filters.add(new PageFilter(MAX_PAGES));
+ filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
+ Filter filterMPALL =
+ new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+ /* Filter must do all below steps:
+ * <ul>
+ * <li>{@link #reset()}</li>
+ * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
+ * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
+ * if false, we will also call</li>
+ * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
+ * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
+ * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
+ * </li>
+ * </ul>
+ */
+ filterMPALL.reset();
+ assertFalse(filterMPALL.filterAllRemaining());
+ byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
+ for (int i = 0; i < MAX_PAGES - 1; i++) {
+ assertFalse(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
+ Bytes.toBytes(i));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPALL.filterKeyValue(kv));
+ }
+ filterMPALL.reset();
+ rowkey = Bytes.toBytes("z");
+ assertTrue(filterMPALL.filterRowKey(rowkey, 0, rowkey.length));
+ // Should fail here; row should be filtered out.
+ KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
+ assertTrue(Filter.ReturnCode.NEXT_ROW == filterMPALL.filterKeyValue(kv));
+
+ // Both filters in Set should be satisfied by now
+ assertTrue(filterMPALL.filterRow());
+ }
+
+ /**
+ * Test list ordering
+ * @throws Exception
+ */
+ public void testOrdering() throws Exception {
+ List<Filter> filters = new ArrayList<Filter>();
+ filters.add(new PrefixFilter(Bytes.toBytes("yyy")));
+ filters.add(new PageFilter(MAX_PAGES));
+ Filter filterMPONE =
+ new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
+ /* Filter must do all below steps:
+ * <ul>
+ * <li>{@link #reset()}</li>
+ * <li>{@link #filterAllRemaining()} -> true indicates scan is over, false, keep going on.</li>
+ * <li>{@link #filterRowKey(byte[],int,int)} -> true to drop this row,
+ * if false, we will also call</li>
+ * <li>{@link #filterKeyValue(org.apache.hadoop.hbase.KeyValue)} -> true to drop this key/value</li>
+ * <li>{@link #filterRow()} -> last chance to drop entire row based on the sequence of
+ * filterValue() calls. Eg: filter a row if it doesn't contain a specified column.
+ * </li>
+ * </ul>
+ */
+ filterMPONE.reset();
+ assertFalse(filterMPONE.filterAllRemaining());
+
+ /* We should be able to fill MAX_PAGES without incrementing page counter */
+ byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
+ for (int i = 0; i < MAX_PAGES; i++) {
+ assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
+ Bytes.toBytes(i));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+ }
+
+ /* Now let's fill the page filter */
+ rowkey = Bytes.toBytes("zzzzzzzz");
+ for (int i = 0; i < MAX_PAGES; i++) {
+ assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
+ Bytes.toBytes(i));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+ }
+
+ /* We should still be able to include even though page filter is at max */
+ rowkey = Bytes.toBytes("yyy");
+ for (int i = 0; i < MAX_PAGES; i++) {
+ assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+ KeyValue kv = new KeyValue(rowkey, rowkey, Bytes.toBytes(i),
+ Bytes.toBytes(i));
+ assertTrue(Filter.ReturnCode.INCLUDE == filterMPONE.filterKeyValue(kv));
+ }
+
+ /* We should filter the row key now if we don't match neither */
+ rowkey = Bytes.toBytes("x");
+ assertTrue(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
+
+ }
+
+ /**
+ * Test serialization
+ * @throws Exception
+ */
+ public void testSerialization() throws Exception {
+ List<Filter> filters = new ArrayList<Filter>();
+ filters.add(new PageFilter(MAX_PAGES));
+ filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
+ Filter filterMPALL =
+ new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+
+ // 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));
+ FilterList newFilter = new FilterList();
+ newFilter.readFields(in);
+
+ // TODO: Run TESTS!!!
+ }
+}
Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageFilter.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageFilter.java?rev=790717&r1=790716&r2=790717&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageFilter.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/filter/TestPageFilter.java Thu Jul 2 19:31:10 2009
@@ -88,10 +88,10 @@
assertTrue("Disagrees with 'filter'", f.filterAllRemaining());
assertTrue(i >= pageSize);
}
- count++;
if (Filter.ReturnCode.NEXT_ROW == f.filterKeyValue(kv)) {
break;
}
+ count++;
}
assertEquals(pageSize, count);
}