Author: kturner Date: Fri Mar 9 22:42:29 2012 New Revision: 1299085 URL: http://svn.apache.org/viewvc?rev=1299085&view=rev Log: ACCUMULO-403 confine seeks in RowFilter accept function to row
Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java Modified: incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java?rev=1299085&r1=1299084&r2=1299085&view=diff ============================================================================== --- incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java (original) +++ incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/RowFilter.java Fri Mar 9 22:42:29 2012 @@ -49,13 +49,43 @@ import org.apache.hadoop.io.Text; */ public abstract class RowFilter extends WrappingIterator { - private SortedKeyValueIterator<Key,Value> decisionIterator; + private RowIterator decisionIterator; private Collection<ByteSequence> columnFamilies; Text currentRow; private boolean inclusive; private Range range; private boolean hasTop; + private static class RowIterator extends WrappingIterator { + private Range rowRange; + private boolean hasTop; + + RowIterator(SortedKeyValueIterator<Key,Value> source) { + super.setSource(source); + } + + void setRow(Range row) { + this.rowRange = row; + } + + @Override + public boolean hasTop() { + return hasTop && super.hasTop(); + } + + @Override + public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException { + + range = rowRange.clip(range, true); + if (range == null) { + hasTop = false; + } else { + hasTop = true; + super.seek(range, columnFamilies, inclusive); + } + } + } + private void skipRows() throws IOException { SortedKeyValueIterator<Key,Value> source = getSource(); while (source.hasTop()) { @@ -64,7 +94,9 @@ public abstract class RowFilter extends if (currentRow != null && currentRow.equals(row)) break; - decisionIterator.seek(new Range(row), columnFamilies, inclusive); + Range rowRange = new Range(row); + decisionIterator.setRow(rowRange); + decisionIterator.seek(rowRange, columnFamilies, inclusive); if (acceptRow(decisionIterator)) { currentRow = row; @@ -103,7 +135,7 @@ public abstract class RowFilter extends @Override public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException { super.init(source, options, env); - this.decisionIterator = source.deepCopy(env); + this.decisionIterator = new RowIterator(source.deepCopy(env)); } @Override Modified: incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java URL: http://svn.apache.org/viewvc/incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java?rev=1299085&r1=1299084&r2=1299085&view=diff ============================================================================== --- incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java (original) +++ incubator/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java Fri Mar 9 22:42:29 2012 @@ -29,6 +29,7 @@ import org.apache.accumulo.core.client.C import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.mock.MockInstance; +import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; @@ -47,13 +48,33 @@ public class RowFilterTest extends TestC @Override public boolean acceptRow(SortedKeyValueIterator<Key,Value> rowIterator) throws IOException { int sum = 0; + int sum2 = 0; + Key firstKey = null; + + if (rowIterator.hasTop()) { + firstKey = new Key(rowIterator.getTopKey()); + } + while (rowIterator.hasTop()) { sum += Integer.parseInt(rowIterator.getTopValue().toString()); rowIterator.next(); } - return sum == 2; + // ensure that seeks are confined to the row + rowIterator.seek(new Range(), new HashSet<ByteSequence>(), false); + while (rowIterator.hasTop()) { + sum2 += Integer.parseInt(rowIterator.getTopValue().toString()); + rowIterator.next(); + } + + rowIterator.seek(new Range(firstKey.getRow(), false, null, true), new HashSet<ByteSequence>(), false); + while (rowIterator.hasTop()) { + sum2 += Integer.parseInt(rowIterator.getTopValue().toString()); + rowIterator.next(); + } + + return sum == 2 && sum2 == 2; } } @@ -110,7 +131,7 @@ public class RowFilterTest extends TestC IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class); conn.tableOperations().attachIterator("table1", is); - + Scanner scanner = conn.createScanner("table1", Constants.NO_AUTHS); assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));