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:
parent
cb4c765635
commit
27667c858b
|
@ -1751,8 +1751,7 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
|
||||||
boolean samerow = Bytes.equals(currentRow, row);
|
boolean samerow = Bytes.equals(currentRow, row);
|
||||||
if (samerow && filterCurrentRow) {
|
if (samerow && filterCurrentRow) {
|
||||||
// Filter all columns until row changes
|
// Filter all columns until row changes
|
||||||
this.storeHeap.next(this.results);
|
readAndDumpCurrentResult();
|
||||||
this.results.clear();
|
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
if (!samerow) {
|
if (!samerow) {
|
||||||
|
@ -1769,14 +1768,21 @@ public class HRegion implements HConstants, HeapSize { // , Writable{
|
||||||
}
|
}
|
||||||
// See if current row should be filtered based on row key
|
// See if current row should be filtered based on row key
|
||||||
if (this.filter != null && this.filter.filterRowKey(row, 0, row.length)) {
|
if (this.filter != null && this.filter.filterRowKey(row, 0, row.length)) {
|
||||||
|
readAndDumpCurrentResult();
|
||||||
resetFilters();
|
resetFilters();
|
||||||
filterCurrentRow = true;
|
filterCurrentRow = true;
|
||||||
currentRow = row;
|
currentRow = row;
|
||||||
|
continue;
|
||||||
}
|
}
|
||||||
this.storeHeap.next(results);
|
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
|
* 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
|
* 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() ||
|
if (this.results.isEmpty() ||
|
||||||
this.filter != null && this.filter.filterRow()) {
|
this.filter != null && this.filter.filterRow()) {
|
||||||
// Make sure results is empty, reset filters
|
// Make sure results is empty, reset filters
|
||||||
results.clear();
|
this.results.clear();
|
||||||
resetFilters();
|
resetFilters();
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
|
@ -1883,10 +1883,10 @@ public class HRegionServer implements HConstants, HRegionInterface,
|
||||||
// Collect values to be returned here
|
// Collect values to be returned here
|
||||||
List<KeyValue> values = new ArrayList<KeyValue>();
|
List<KeyValue> values = new ArrayList<KeyValue>();
|
||||||
boolean moreRows = s.next(values);
|
boolean moreRows = s.next(values);
|
||||||
if(!values.isEmpty()) {
|
if (!values.isEmpty()) {
|
||||||
results.add(new Result(values));
|
results.add(new Result(values));
|
||||||
}
|
}
|
||||||
if(!moreRows) {
|
if (!moreRows) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -145,7 +145,6 @@ public class TestFilter extends HBaseTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testNoFilter() throws Exception {
|
public void testNoFilter() throws Exception {
|
||||||
|
|
||||||
// No filter
|
// No filter
|
||||||
long expectedRows = this.numRows;
|
long expectedRows = this.numRows;
|
||||||
long expectedKeys = this.colsPerRow;
|
long expectedKeys = this.colsPerRow;
|
||||||
|
@ -161,17 +160,12 @@ public class TestFilter extends HBaseTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testPrefixFilter() throws Exception {
|
public void testPrefixFilter() throws Exception {
|
||||||
|
|
||||||
// Grab rows from group one (half of total)
|
// Grab rows from group one (half of total)
|
||||||
|
|
||||||
long expectedRows = this.numRows / 2;
|
long expectedRows = this.numRows / 2;
|
||||||
long expectedKeys = this.colsPerRow;
|
long expectedKeys = this.colsPerRow;
|
||||||
|
|
||||||
Scan s = new Scan();
|
Scan s = new Scan();
|
||||||
s.setFilter(new PrefixFilter(Bytes.toBytes("testRowOne")));
|
s.setFilter(new PrefixFilter(Bytes.toBytes("testRowOne")));
|
||||||
|
|
||||||
verifyScan(s, expectedRows, expectedKeys);
|
verifyScan(s, expectedRows, expectedKeys);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testPageFilter() throws Exception {
|
public void testPageFilter() throws Exception {
|
||||||
|
@ -931,6 +925,7 @@ public class TestFilter extends HBaseTestCase {
|
||||||
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
Arrays.sort(results.toArray(new KeyValue[results.size()]),
|
||||||
KeyValue.COMPARATOR);
|
KeyValue.COMPARATOR);
|
||||||
LOG.info("counter=" + i + ", " + results);
|
LOG.info("counter=" + i + ", " + results);
|
||||||
|
if (results.isEmpty()) break;
|
||||||
assertTrue("Scanned too many rows! Only expected " + expectedRows +
|
assertTrue("Scanned too many rows! Only expected " + expectedRows +
|
||||||
" total but already scanned " + (i+1), expectedRows > i);
|
" total but already scanned " + (i+1), expectedRows > i);
|
||||||
assertEquals("Expected " + expectedKeys + " keys per row but " +
|
assertEquals("Expected " + expectedKeys + " keys per row but " +
|
||||||
|
|
|
@ -180,7 +180,6 @@ public class TestScanner extends HBaseTestCase {
|
||||||
Filter newFilter = new PrefixFilter(prefix);
|
Filter newFilter = new PrefixFilter(prefix);
|
||||||
Scan scan = new Scan();
|
Scan scan = new Scan();
|
||||||
scan.setFilter(newFilter);
|
scan.setFilter(newFilter);
|
||||||
scan.setStartRow(prefix);
|
|
||||||
rowPrefixFilter(scan);
|
rowPrefixFilter(scan);
|
||||||
|
|
||||||
byte[] stopRow = Bytes.toBytes("bbc");
|
byte[] stopRow = Bytes.toBytes("bbc");
|
||||||
|
|
Loading…
Reference in New Issue