HBASE-1599 Fix TestFilterSet, broken up on hudson

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@790717 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-07-02 19:31:10 +00:00
parent ed63b575a0
commit ad8968141d
5 changed files with 111 additions and 43 deletions

View File

@ -25,6 +25,7 @@ Release 0.20.0 - Unreleased
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)

View File

@ -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
@ -22,8 +22,8 @@ package org.apache.hadoop.hbase.filter;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.HashSet;
import java.util.Set;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
@ -31,13 +31,13 @@ import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
/**
* Implementation of {@link Filter} that represents a Set of Filters
* 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 Sets as children
* of Filter Sets, you can create a hierarchy of filters to be evaluated.
* (!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 FilterSet implements Filter {
public class FilterList implements Filter {
/** set operator */
public static enum Operator {
@ -48,13 +48,13 @@ public class FilterSet implements Filter {
}
private Operator operator = Operator.MUST_PASS_ALL;
private Set<Filter> filters = new HashSet<Filter>();
private List<Filter> filters = new ArrayList<Filter>();
/**
* Default constructor, filters nothing. Required though for RPC
* deserialization.
*/
public FilterSet() {
public FilterList() {
super();
}
@ -64,7 +64,7 @@ public class FilterSet implements Filter {
*
* @param rowFilters
*/
public FilterSet(final Set<Filter> rowFilters) {
public FilterList(final List<Filter> rowFilters) {
this.filters = rowFilters;
}
@ -74,7 +74,7 @@ public class FilterSet implements Filter {
* @param operator Operator to process filter set with.
* @param rowFilters Set of row filters.
*/
public FilterSet(final Operator operator, final Set<Filter> rowFilters) {
public FilterList(final Operator operator, final List<Filter> rowFilters) {
this.filters = rowFilters;
this.operator = operator;
}
@ -93,7 +93,7 @@ public class FilterSet implements Filter {
*
* @return filters
*/
public Set<Filter> getFilters() {
public List<Filter> getFilters() {
return filters;
}
@ -198,7 +198,7 @@ public class FilterSet implements Filter {
operator = Operator.values()[opByte];
int size = in.readInt();
if (size > 0) {
filters = new HashSet<Filter>();
filters = new ArrayList<Filter>(size);
for (int i = 0; i < size; i++) {
Filter filter = (Filter)HbaseObjectWritable.readObject(in, conf);
filters.add(filter);

View File

@ -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 @@ import org.apache.hadoop.hbase.KeyValue;
/**
* 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 class PageFilter implements Filter {
}
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 class PageFilter implements Filter {
}
public ReturnCode filterKeyValue(KeyValue v) {
this.rowsAccepted++;
return filterAllRemaining()? ReturnCode.NEXT_ROW: ReturnCode.INCLUDE;
return filterAllRemaining() ? ReturnCode.NEXT_ROW : ReturnCode.INCLUDE;
}
public boolean filterRow() {

View File

@ -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
@ -23,8 +23,8 @@ 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.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.KeyValue;
@ -36,8 +36,8 @@ import junit.framework.TestCase;
* Tests filter sets
*
*/
public class TestFilterSet extends TestCase {
static final int MAX_PAGES = 5;
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");
@ -48,11 +48,11 @@ public class TestFilterSet extends TestCase {
* @throws Exception
*/
public void testMPONE() throws Exception {
Set<Filter> filters = new HashSet<Filter>();
List<Filter> filters = new ArrayList<Filter>();
filters.add(new PageFilter(MAX_PAGES));
filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
Filter filterMPONE =
new FilterSet(FilterSet.Operator.MUST_PASS_ONE, filters);
new FilterList(FilterList.Operator.MUST_PASS_ONE, filters);
/* Filter must do all below steps:
* <ul>
* <li>{@link #reset()}</li>
@ -67,31 +67,40 @@ public class TestFilterSet extends TestCase {
*/
filterMPONE.reset();
assertFalse(filterMPONE.filterAllRemaining());
/* Will pass both */
byte [] rowkey = Bytes.toBytes("yyyyyyyyy");
for (int i = 0; i < MAX_PAGES; i++) {
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));
}
/* Weird -- below passes in IDE but not command-line
/* Only pass PageFilter */
rowkey = Bytes.toBytes("z");
for (int i = 0; i < MAX_PAGES - 1; i++) {
assertFalse("i=" + i, 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));
}
assertFalse(filterMPONE.filterRowKey(rowkey, 0, rowkey.length));
// Should fail here
KeyValue kv = new KeyValue(rowkey, rowkey, rowkey, rowkey);
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());
*/
}
/**
@ -99,11 +108,11 @@ public class TestFilterSet extends TestCase {
* @throws Exception
*/
public void testMPALL() throws Exception {
Set<Filter> filters = new HashSet<Filter>();
List<Filter> filters = new ArrayList<Filter>();
filters.add(new PageFilter(MAX_PAGES));
filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
Filter filterMPALL =
new FilterSet(FilterSet.Operator.MUST_PASS_ALL, filters);
new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
/* Filter must do all below steps:
* <ul>
* <li>{@link #reset()}</li>
@ -136,16 +145,74 @@ public class TestFilterSet extends TestCase {
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 {
Set<Filter> filters = new HashSet<Filter>();
List<Filter> filters = new ArrayList<Filter>();
filters.add(new PageFilter(MAX_PAGES));
filters.add(new WhileMatchFilter(new PrefixFilter(Bytes.toBytes("yyy"))));
Filter filterMPALL =
new FilterSet(FilterSet.Operator.MUST_PASS_ALL, filters);
new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
// Decompose filterMPALL to bytes.
ByteArrayOutputStream stream = new ByteArrayOutputStream();
@ -156,7 +223,7 @@ public class TestFilterSet extends TestCase {
// Recompose filterMPALL.
DataInputStream in = new DataInputStream(new ByteArrayInputStream(buffer));
FilterSet newFilter = new FilterSet();
FilterList newFilter = new FilterList();
newFilter.readFields(in);
// TODO: Run TESTS!!!

View File

@ -88,10 +88,10 @@ public class TestPageFilter extends TestCase {
assertTrue("Disagrees with 'filter'", f.filterAllRemaining());
assertTrue(i >= pageSize);
}
count++;
if (Filter.ReturnCode.NEXT_ROW == f.filterKeyValue(kv)) {
break;
}
count++;
}
assertEquals(pageSize, count);
}