HBASE-1906 fixup for broken build

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@825334 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-10-14 23:13:18 +00:00
parent cb4c765635
commit 27667c858b
4 changed files with 12 additions and 12 deletions

View File

@ -1751,8 +1751,7 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
boolean samerow = Bytes.equals(currentRow, row);
if (samerow && filterCurrentRow) {
// Filter all columns until row changes
this.storeHeap.next(this.results);
this.results.clear();
readAndDumpCurrentResult();
continue;
}
if (!samerow) {
@ -1769,14 +1768,21 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
}
// See if current row should be filtered based on row key
if (this.filter != null && this.filter.filterRowKey(row, 0, row.length)) {
readAndDumpCurrentResult();
resetFilters();
filterCurrentRow = true;
currentRow = row;
continue;
}
this.storeHeap.next(results);
}
}
private void readAndDumpCurrentResult() throws IOException {
this.storeHeap.next(this.results);
this.results.clear();
}
/*
* Do we have results to return or should we continue. Call when we get to
* the end of a row. Does house cleaning -- clearing results and resetting
@ -1787,7 +1793,7 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
if (this.results.isEmpty() ||
this.filter != null && this.filter.filterRow()) {
// Make sure results is empty, reset filters
results.clear();
this.results.clear();
resetFilters();
return false;
}

View File

@ -1883,10 +1883,10 @@ public class HRegionServer implements HConstants, HRegionInterface,
// Collect values to be returned here
List<KeyValue> values = new ArrayList<KeyValue>();
boolean moreRows = s.next(values);
if(!values.isEmpty()) {
if (!values.isEmpty()) {
results.add(new Result(values));
}
if(!moreRows) {
if (!moreRows) {
break;
}
}

View File

@ -145,7 +145,6 @@ public class TestFilter extends HBaseTestCase {
}
public void testNoFilter() throws Exception {
// No filter
long expectedRows = this.numRows;
long expectedKeys = this.colsPerRow;
@ -161,17 +160,12 @@ public class TestFilter extends HBaseTestCase {
}
public void testPrefixFilter() throws Exception {
// Grab rows from group one (half of total)
long expectedRows = this.numRows / 2;
long expectedKeys = this.colsPerRow;
Scan s = new Scan();
s.setFilter(new PrefixFilter(Bytes.toBytes("testRowOne")));
verifyScan(s, expectedRows, expectedKeys);
}
public void testPageFilter() throws Exception {
@ -931,6 +925,7 @@ public class TestFilter extends HBaseTestCase {
Arrays.sort(results.toArray(new KeyValue[results.size()]),
KeyValue.COMPARATOR);
LOG.info("counter=" + i + ", " + results);
if (results.isEmpty()) break;
assertTrue("Scanned too many rows! Only expected " + expectedRows +
" total but already scanned " + (i+1), expectedRows > i);
assertEquals("Expected " + expectedKeys + " keys per row but " +

View File

@ -180,7 +180,6 @@ public class TestScanner extends HBaseTestCase {
Filter newFilter = new PrefixFilter(prefix);
Scan scan = new Scan();
scan.setFilter(newFilter);
scan.setStartRow(prefix);
rowPrefixFilter(scan);
byte[] stopRow = Bytes.toBytes("bbc");