HBASE-544 filters generate StackOverflowException
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@646031 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
49e0eaf8d9
commit
6410feb071
|
@ -14,6 +14,7 @@ Hbase Change Log
|
|||
0.0.0.0:60100 rather than relying on conf
|
||||
HBASE-507 Use Callable pattern to sleep between retries
|
||||
HBASE-564 Don't do a cache flush if there are zero entries in the cache.
|
||||
HBASE-544 filters generate StackOverflowException
|
||||
|
||||
NEW FEATURES
|
||||
HBASE-548 Tool to online single region
|
||||
|
|
|
@ -1739,69 +1739,73 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
|
||||
/** {@inheritDoc} */
|
||||
@SuppressWarnings("null")
|
||||
public boolean next(HStoreKey key, SortedMap<Text, byte[]> results)
|
||||
throws IOException {
|
||||
boolean moreToFollow = false;
|
||||
|
||||
// Find the lowest-possible key.
|
||||
do {
|
||||
// Find the lowest-possible key.
|
||||
|
||||
Text chosenRow = null;
|
||||
long chosenTimestamp = -1;
|
||||
for (int i = 0; i < this.keys.length; i++) {
|
||||
if (scanners[i] != null &&
|
||||
(chosenRow == null ||
|
||||
(keys[i].getRow().compareTo(chosenRow) < 0) ||
|
||||
((keys[i].getRow().compareTo(chosenRow) == 0) &&
|
||||
(keys[i].getTimestamp() > chosenTimestamp)))) {
|
||||
chosenRow = new Text(keys[i].getRow());
|
||||
chosenTimestamp = keys[i].getTimestamp();
|
||||
Text chosenRow = null;
|
||||
long chosenTimestamp = -1;
|
||||
for (int i = 0; i < this.keys.length; i++) {
|
||||
if (scanners[i] != null &&
|
||||
(chosenRow == null ||
|
||||
(keys[i].getRow().compareTo(chosenRow) < 0) ||
|
||||
((keys[i].getRow().compareTo(chosenRow) == 0) &&
|
||||
(keys[i].getTimestamp() > chosenTimestamp)))) {
|
||||
chosenRow = new Text(keys[i].getRow());
|
||||
chosenTimestamp = keys[i].getTimestamp();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Store the key and results for each sub-scanner. Merge them as
|
||||
// appropriate.
|
||||
if (chosenTimestamp >= 0) {
|
||||
// Here we are setting the passed in key with current row+timestamp
|
||||
key.setRow(chosenRow);
|
||||
key.setVersion(chosenTimestamp);
|
||||
key.setColumn(HConstants.EMPTY_TEXT);
|
||||
// Store the key and results for each sub-scanner. Merge them as
|
||||
// appropriate.
|
||||
if (chosenTimestamp >= 0) {
|
||||
// Here we are setting the passed in key with current row+timestamp
|
||||
key.setRow(chosenRow);
|
||||
key.setVersion(chosenTimestamp);
|
||||
key.setColumn(HConstants.EMPTY_TEXT);
|
||||
|
||||
for (int i = 0; i < scanners.length; i++) {
|
||||
if (scanners[i] != null && keys[i].getRow().compareTo(chosenRow) == 0) {
|
||||
// NOTE: We used to do results.putAll(resultSets[i]);
|
||||
// but this had the effect of overwriting newer
|
||||
// values with older ones. So now we only insert
|
||||
// a result if the map does not contain the key.
|
||||
for (Map.Entry<Text, byte[]> e : resultSets[i].entrySet()) {
|
||||
if (!results.containsKey(e.getKey())) {
|
||||
results.put(e.getKey(), e.getValue());
|
||||
for (int i = 0; i < scanners.length; i++) {
|
||||
if (scanners[i] != null &&
|
||||
keys[i].getRow().compareTo(chosenRow) == 0) {
|
||||
// NOTE: We used to do results.putAll(resultSets[i]);
|
||||
// but this had the effect of overwriting newer
|
||||
// values with older ones. So now we only insert
|
||||
// a result if the map does not contain the key.
|
||||
for (Map.Entry<Text, byte[]> e : resultSets[i].entrySet()) {
|
||||
if (!results.containsKey(e.getKey())) {
|
||||
results.put(e.getKey(), e.getValue());
|
||||
}
|
||||
}
|
||||
resultSets[i].clear();
|
||||
if (!scanners[i].next(keys[i], resultSets[i])) {
|
||||
closeScanner(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (int i = 0; i < scanners.length; i++) {
|
||||
// If the current scanner is non-null AND has a lower-or-equal
|
||||
// row label, then its timestamp is bad. We need to advance it.
|
||||
while ((scanners[i] != null) &&
|
||||
(keys[i].getRow().compareTo(chosenRow) <= 0)) {
|
||||
resultSets[i].clear();
|
||||
if (!scanners[i].next(keys[i], resultSets[i])) {
|
||||
closeScanner(i);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (int i = 0; i < scanners.length; i++) {
|
||||
// If the current scanner is non-null AND has a lower-or-equal
|
||||
// row label, then its timestamp is bad. We need to advance it.
|
||||
while ((scanners[i] != null) &&
|
||||
(keys[i].getRow().compareTo(chosenRow) <= 0)) {
|
||||
resultSets[i].clear();
|
||||
if (!scanners[i].next(keys[i], resultSets[i])) {
|
||||
closeScanner(i);
|
||||
}
|
||||
moreToFollow = chosenTimestamp >= 0;
|
||||
if (results == null || results.size() <= 0) {
|
||||
// If we got no results, then there is no more to follow.
|
||||
moreToFollow = false;
|
||||
}
|
||||
}
|
||||
|
||||
moreToFollow = chosenTimestamp >= 0;
|
||||
if (results == null || results.size() <= 0) {
|
||||
// If we got no results, then there is no more to follow.
|
||||
moreToFollow = false;
|
||||
}
|
||||
} while(filter != null && filter.filterNotNull(results) && moreToFollow);
|
||||
|
||||
// Make sure scanners closed if no more results
|
||||
if (!moreToFollow) {
|
||||
|
@ -1812,11 +1816,6 @@ public class HRegion implements HConstants {
|
|||
}
|
||||
}
|
||||
|
||||
if (filter != null && filter.filterNotNull(results)) {
|
||||
LOG.warn("Filter return true on assembled Results in hstore");
|
||||
return moreToFollow == true && this.next(key, results);
|
||||
}
|
||||
|
||||
return moreToFollow;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue