HBASE-751 dfs exception and regionserver stuck during heavy write load

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@681612 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2008-08-01 06:40:18 +00:00
parent 5710476a73
commit 70e8c417d0
9 changed files with 529 additions and 229 deletions

View File

@ -221,6 +221,7 @@ Release 0.2.0
HBASE-771 Names legal in 0.1 are not in 0.2; breaks migration
HBASE-788 Div by zero in Master.jsp (Clint Morgan via Jim Kellerman)
HBASE-791 RowCount doesn't work (Jean-Daniel Cryans via Stack)
HBASE-751 dfs exception and regionserver stuck during heavy write load
IMPROVEMENTS
HBASE-559 MR example job to count table rows

View File

@ -1182,7 +1182,7 @@ public class HTable {
*/
public synchronized void commit(final List<BatchUpdate> batchUpdates)
throws IOException {
for(BatchUpdate batchUpdate : batchUpdates)
for (BatchUpdate batchUpdate : batchUpdates)
commit(batchUpdate);
}

View File

@ -49,7 +49,7 @@ public class BatchUpdate implements Writable, Iterable<BatchOperation> {
private long timestamp = HConstants.LATEST_TIMESTAMP;
/**
* Default constructor used serializing.
* Default constructor used serializing. Do not use directly.
*/
public BatchUpdate() {
this ((byte [])null);

View File

@ -0,0 +1,119 @@
/**
* Copyright 2008 The Apache Software Foundation
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.HStoreKey;
/**
* Pass this class into {@link org.apache.hadoop.io.MapFile}.getClosest when
* searching for the key that comes BEFORE this one but NOT this one. THis
* class will return > 0 when asked to compare against itself rather than 0.
* This is a hack for case where getClosest returns a deleted key and we want
* to get the previous. Can't unless use use this class; it'll just keep
* returning us the deleted key (getClosest gets exact or nearest before when
* you pass true argument). TODO: Throw this class away when MapFile has
* a real 'previous' method. See HBASE-751.
*/
public class BeforeThisStoreKey extends HStoreKey {
private final HStoreKey beforeThisKey;
/**
* @param beforeThisKey
*/
public BeforeThisStoreKey(final HStoreKey beforeThisKey) {
super();
this.beforeThisKey = beforeThisKey;
}
@Override
public int compareTo(Object o) {
int result = this.beforeThisKey.compareTo(o);
return result == 0? -1: result;
}
@Override
public boolean equals(@SuppressWarnings("unused") Object obj) {
return false;
}
public byte[] getColumn() {
return this.beforeThisKey.getColumn();
}
public byte[] getRow() {
return this.beforeThisKey.getRow();
}
public long getSize() {
return this.beforeThisKey.getSize();
}
public long getTimestamp() {
return this.beforeThisKey.getTimestamp();
}
public int hashCode() {
return this.beforeThisKey.hashCode();
}
public boolean matchesRowCol(HStoreKey other) {
return this.beforeThisKey.matchesRowCol(other);
}
public boolean matchesRowFamily(HStoreKey that) {
return this.beforeThisKey.matchesRowFamily(that);
}
public boolean matchesWithoutColumn(HStoreKey other) {
return this.beforeThisKey.matchesWithoutColumn(other);
}
public void readFields(DataInput in) throws IOException {
this.beforeThisKey.readFields(in);
}
public void set(HStoreKey k) {
this.beforeThisKey.set(k);
}
public void setColumn(byte[] c) {
this.beforeThisKey.setColumn(c);
}
public void setRow(byte[] newrow) {
this.beforeThisKey.setRow(newrow);
}
public void setVersion(long timestamp) {
this.beforeThisKey.setVersion(timestamp);
}
public String toString() {
return this.beforeThisKey.toString();
}
public void write(DataOutput out) throws IOException {
this.beforeThisKey.write(out);
}
}

View File

@ -92,7 +92,7 @@ class CompactSplitThread extends Thread implements HConstants {
} catch (InterruptedException ex) {
continue;
} catch (IOException ex) {
LOG.error("Compaction failed" +
LOG.error("Compaction/Split failed" +
(r != null ? (" for region " + Bytes.toString(r.getRegionName())) : ""),
RemoteExceptionHandler.checkIOException(ex));
if (!server.checkFileSystem()) {

View File

@ -105,8 +105,7 @@ public class HStore implements HConstants {
Collections.synchronizedSortedMap(new TreeMap<Long, HStoreFile>());
/*
* Sorted Map of readers keyed by sequence id (Most recent should be last in
* in list).
* Sorted Map of readers keyed by sequence id (Most recent is last in list).
*/
private final SortedMap<Long, MapFile.Reader> readers =
new TreeMap<Long, MapFile.Reader>();
@ -761,7 +760,8 @@ public class HStore implements HConstants {
return checkSplit();
}
// HBASE-745, preparing all store file size for incremental compacting selection.
// HBASE-745, preparing all store file size for incremental compacting
// selection.
int countOfFiles = filesToCompact.size();
long totalSize = 0;
long[] fileSizes = new long[countOfFiles];
@ -780,7 +780,7 @@ public class HStore implements HConstants {
if (!force && !hasReferences(filesToCompact)) {
// Here we select files for incremental compaction.
// The rule is: if the largest(oldest) one is more than twice the
// size of the second, skip the largest, and continue to next...,
// size of the second, skip the largest, and continue to next...,
// until we meet the compactionThreshold limit.
for (point = 0; point < compactionThreshold - 1; point++) {
if (fileSizes[point] < fileSizes[point + 1] * 2) {
@ -791,7 +791,8 @@ public class HStore implements HConstants {
filesToCompact = new ArrayList<HStoreFile>(filesToCompact.subList(point,
countOfFiles));
if (LOG.isDebugEnabled()) {
LOG.debug("Compaction size " + totalSize + ", skipped " + point +
LOG.debug("Compaction size of " + this.storeNameStr + ": " +
StringUtils.humanReadableInt(totalSize) + ", skipped " + point +
", " + skipped);
}
}
@ -855,7 +856,7 @@ public class HStore implements HConstants {
}
return checkSplit();
}
/*
* Compact a list of MapFile.Readers into MapFile.Writer.
*
@ -1219,7 +1220,10 @@ public class HStore implements HConstants {
} while(map.next(readkey, readval));
}
}
/**
* @return Array of readers ordered oldest to newest.
*/
MapFile.Reader [] getReaders() {
return this.readers.values().
toArray(new MapFile.Reader[this.readers.size()]);
@ -1427,7 +1431,10 @@ public class HStore implements HConstants {
* Find the key that matches <i>row</i> exactly, or the one that immediately
* preceeds it. WARNING: Only use this method on a table where writes occur
* with stricly increasing timestamps. This method assumes this pattern of
* writes in order to make it reasonably performant.
* writes in order to make it reasonably performant.
* @param row
* @return Found row
* @throws IOException
*/
byte [] getRowKeyAtOrBefore(final byte [] row)
throws IOException{
@ -1436,14 +1443,14 @@ public class HStore implements HConstants {
// deletes found all over the place as we go along before finally reading
// the best key out of it at the end.
SortedMap<HStoreKey, Long> candidateKeys = new TreeMap<HStoreKey, Long>();
// Obtain read lock
this.lock.readLock().lock();
try {
// Process each store file
// Process each store file. Run through from oldest to newest so deletes
// have chance to overshadow deleted cells
MapFile.Reader[] maparray = getReaders();
for (int i = maparray.length - 1; i >= 0; i--) {
// update the candidate keys from the current map file
for (int i = 0; i < maparray.length; i++) {
// Update the candidate keys from the current map file
rowAtOrBeforeFromMapFile(maparray[i], row, candidateKeys);
}
@ -1457,70 +1464,100 @@ public class HStore implements HConstants {
}
}
/**
/*
* Check an individual MapFile for the row at or before a given key
* and timestamp
* @param map
* @param row
* @param candidateKeys
* @throws IOException
*/
private void rowAtOrBeforeFromMapFile(MapFile.Reader map, final byte [] row,
SortedMap<HStoreKey, Long> candidateKeys)
throws IOException {
ImmutableBytesWritable readval = new ImmutableBytesWritable();
HStoreKey readkey = new HStoreKey();
HStoreKey startKey = new HStoreKey();
ImmutableBytesWritable startValue = new ImmutableBytesWritable();
synchronized(map) {
// don't bother with the rest of this if the file is empty
// Don't bother with the rest of this if the file is empty
map.reset();
if (!map.next(readkey, readval)) {
if (!map.next(startKey, startValue)) {
return;
}
// If start row for this file is beyond passed in row, return; nothing
// in here is of use to us.
if (Bytes.compareTo(startKey.getRow(), row) > 0) {
return;
}
long now = System.currentTimeMillis();
// if there aren't any candidate keys yet, we'll do some things slightly
// different
// if there aren't any candidate keys yet, we'll do some things different
if (candidateKeys.isEmpty()) {
rowKeyFromMapFileEmptyKeys(map, row, candidateKeys, now);
rowAtOrBeforeCandidate(startKey, map, row, candidateKeys, now);
} else {
rowKeyAtOrBeforeExistingCandKeys(map, row, candidateKeys, now);
rowAtOrBeforeWithCandidates(startKey, map, row, candidateKeys,
now);
}
}
}
private void rowKeyFromMapFileEmptyKeys(MapFile.Reader map, byte[] row,
SortedMap<HStoreKey, Long> candidateKeys, long now)
/* Find a candidate for row that is at or before passed row in passed
* mapfile.
* @param startKey First key in the mapfile.
* @param map
* @param row
* @param candidateKeys
* @param now
* @throws IOException
*/
private void rowAtOrBeforeCandidate(final HStoreKey startKey,
final MapFile.Reader map, final byte[] row,
final SortedMap<HStoreKey, Long> candidateKeys, final long now)
throws IOException {
HStoreKey searchKey = new HStoreKey(row);
ImmutableBytesWritable readval = new ImmutableBytesWritable();
HStoreKey readkey = new HStoreKey();
// if the row we're looking for is past the end of this mapfile, just
// save time and add the last key to the candidates.
HStoreKey finalKey = new HStoreKey();
map.finalKey(finalKey);
// if the row we're looking for is past the end of this mapfile, set the
// search key to be the last key. If its a deleted key, then we'll back
// up to the row before and return that.
HStoreKey finalKey = getFinalKey(map);
HStoreKey searchKey = null;
if (Bytes.compareTo(finalKey.getRow(), row) < 0) {
candidateKeys.put(stripTimestamp(finalKey),
new Long(finalKey.getTimestamp()));
return;
}
HStoreKey deletedOrExpiredRow = null;
boolean foundCandidate = false;
while (!foundCandidate) {
// seek to the exact row, or the one that would be immediately before it
readkey = (HStoreKey)map.getClosest(searchKey, readval, true);
if (readkey == null) {
// didn't find anything that would match, so return
return;
searchKey = finalKey;
} else {
searchKey = new HStoreKey(row);
if (searchKey.compareTo(startKey) < 0) {
searchKey = startKey;
}
}
rowAtOrBeforeCandidate(map, searchKey, candidateKeys, now);
}
/* Find a candidate for row that is at or before passed key, sk, in mapfile.
* @param map
* @param sk Key to go search the mapfile with.
* @param candidateKeys
* @param now
* @throws IOException
* @see {@link #rowAtOrBeforeCandidate(HStoreKey, org.apache.hadoop.io.MapFile.Reader, byte[], SortedMap, long)}
*/
private void rowAtOrBeforeCandidate(final MapFile.Reader map,
final HStoreKey sk, final SortedMap<HStoreKey, Long> candidateKeys,
final long now)
throws IOException {
HStoreKey searchKey = sk;
HStoreKey readkey = new HStoreKey();
ImmutableBytesWritable readval = new ImmutableBytesWritable();
HStoreKey knownNoGoodKey = null;
for (boolean foundCandidate = false; !foundCandidate;) {
// Seek to the exact row, or the one that would be immediately before it
readkey = (HStoreKey)map.getClosest(searchKey, readval, true);
if (readkey == null) {
// If null, we are at the start or end of the file.
break;
}
HStoreKey deletedOrExpiredRow = null;
do {
// if we have an exact match on row, and it's not a delete, save this
// If we have an exact match on row, and it's not a delete, save this
// as a candidate key
if (Bytes.equals(readkey.getRow(), row)) {
if (Bytes.equals(readkey.getRow(), searchKey.getRow())) {
if (!HLogEdit.isDeleted(readval.get())) {
if (ttl == HConstants.FOREVER ||
if (ttl == HConstants.FOREVER ||
now < readkey.getTimestamp() + ttl) {
candidateKeys.put(stripTimestamp(readkey),
new Long(readkey.getTimestamp()));
@ -1528,17 +1565,20 @@ public class HStore implements HConstants {
continue;
}
if (LOG.isDebugEnabled()) {
LOG.debug("rowAtOrBeforeFromMapFile:" + readkey +
": expired, skipped");
LOG.debug("rowAtOrBeforeCandidate 1:" + readkey +
": expired, skipped");
}
}
deletedOrExpiredRow = stripTimestamp(readkey);
} else if (Bytes.compareTo(readkey.getRow(), row) > 0 ) {
// Deleted value.
if (deletedOrExpiredRow == null) {
deletedOrExpiredRow = new HStoreKey(readkey);
}
} else if (Bytes.compareTo(readkey.getRow(), searchKey.getRow()) > 0) {
// if the row key we just read is beyond the key we're searching for,
// then we're done. return.
// then we're done.
break;
} else {
// so, the row key doesn't match, but we haven't gone past the row
// So, the row key doesn't match, but we haven't gone past the row
// we're seeking yet, so this row is a candidate for closest
// (assuming that it isn't a delete).
if (!HLogEdit.isDeleted(readval.get())) {
@ -1550,78 +1590,61 @@ public class HStore implements HConstants {
continue;
}
if (LOG.isDebugEnabled()) {
LOG.debug("rowAtOrBeforeFromMapFile:" + readkey +
": expired, skipped");
LOG.debug("rowAtOrBeforeCandidate 2:" + readkey +
": expired, skipped");
}
}
deletedOrExpiredRow = stripTimestamp(readkey);
if (deletedOrExpiredRow == null) {
deletedOrExpiredRow = new HStoreKey(readkey);
}
}
} while(map.next(readkey, readval));
} while(map.next(readkey, readval) && (knownNoGoodKey == null ||
readkey.compareTo(knownNoGoodKey) < 0));
// If we get here and have no candidates but we did find a deleted or
// expired candidate, we need to look at the key before that
if (!foundCandidate && deletedOrExpiredRow != null) {
searchKey = deletedOrExpiredRow;
deletedOrExpiredRow = null;
knownNoGoodKey = deletedOrExpiredRow;
searchKey = new BeforeThisStoreKey(deletedOrExpiredRow);
} else {
// No candidates and no deleted or expired candidates. Give up.
break;
}
}
// arriving here just means that we consumed the whole rest of the map
// Arriving here just means that we consumed the whole rest of the map
// without going "past" the key we're searching for. we can just fall
// through here.
}
private void rowKeyAtOrBeforeExistingCandKeys(MapFile.Reader map, byte[] row,
SortedMap<HStoreKey, Long> candidateKeys, long now)
private void rowAtOrBeforeWithCandidates(final HStoreKey startKey,
final MapFile.Reader map, final byte[] row,
final SortedMap<HStoreKey, Long> candidateKeys, final long now)
throws IOException {
HStoreKey strippedKey = null;
ImmutableBytesWritable readval = new ImmutableBytesWritable();
HStoreKey readkey = new HStoreKey();
ImmutableBytesWritable readval = new ImmutableBytesWritable();
// if there are already candidate keys, we need to start our search
// at the earliest possible key so that we can discover any possible
// deletes for keys between the start and the search key.
// deletes for keys between the start and the search key. Back up to start
// of the row in case there are deletes for this candidate in this mapfile
// BUT do not backup before the first key in the mapfile else getClosest
// will return null
HStoreKey searchKey = new HStoreKey(candidateKeys.firstKey().getRow());
// if the row we're looking for is past the end of this mapfile, just
// save time and add the last key to the candidates.
HStoreKey finalKey = new HStoreKey();
map.finalKey(finalKey);
if (Bytes.compareTo(finalKey.getRow(), searchKey.getRow()) < 0) {
strippedKey = stripTimestamp(finalKey);
// if the candidate keys has a cell like this one already,
// then we might want to update the timestamp we're using on it
if (candidateKeys.containsKey(strippedKey)) {
long bestCandidateTs =
candidateKeys.get(strippedKey).longValue();
if (bestCandidateTs < finalKey.getTimestamp()) {
candidateKeys.put(strippedKey, new Long(finalKey.getTimestamp()));
}
} else {
// otherwise, this is a new key, so put it up as a candidate
candidateKeys.put(strippedKey, new Long(finalKey.getTimestamp()));
}
return;
if (searchKey.compareTo(startKey) < 0) {
searchKey = startKey;
}
// seek to the exact row, or the one that would be immediately before it
// Seek to the exact row, or the one that would be immediately before it
readkey = (HStoreKey)map.getClosest(searchKey, readval, true);
if (readkey == null) {
// didn't find anything that would match, so return
// If null, we are at the start or end of the file.
// Didn't find anything that would match, so return
return;
}
do {
HStoreKey strippedKey = null;
// if we have an exact match on row, and it's not a delete, save this
// as a candidate key
if (Bytes.equals(readkey.getRow(), row)) {
@ -1633,12 +1656,12 @@ public class HStore implements HConstants {
new Long(readkey.getTimestamp()));
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("rowAtOrBeforeFromMapFile: " + readkey +
LOG.debug("rowAtOrBeforeWithCandidates 1: " + readkey +
": expired, skipped");
}
}
} else {
// if the candidate keys contain any that might match by timestamp,
// If the candidate keys contain any that might match by timestamp,
// then check for a match and remove it if it's too young to
// survive the delete
if (candidateKeys.containsKey(strippedKey)) {
@ -1651,26 +1674,25 @@ public class HStore implements HConstants {
}
} else if (Bytes.compareTo(readkey.getRow(), row) > 0 ) {
// if the row key we just read is beyond the key we're searching for,
// then we're done. return.
return;
// then we're done.
break;
} else {
strippedKey = stripTimestamp(readkey);
// so, the row key doesn't match, but we haven't gone past the row
// So, the row key doesn't match, but we haven't gone past the row
// we're seeking yet, so this row is a candidate for closest
// (assuming that it isn't a delete).
if (!HLogEdit.isDeleted(readval.get())) {
if (ttl == HConstants.FOREVER ||
now < readkey.getTimestamp() + ttl) {
candidateKeys.put(strippedKey, readkey.getTimestamp());
candidateKeys.put(strippedKey, Long.valueOf(readkey.getTimestamp()));
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("rowAtOrBeforeFromMapFile: " + readkey +
LOG.debug("rowAtOrBeforeWithCandidates 2: " + readkey +
": expired, skipped");
}
}
} else {
// if the candidate keys contain any that might match by timestamp,
// If the candidate keys contain any that might match by timestamp,
// then check for a match and remove it if it's too young to
// survive the delete
if (candidateKeys.containsKey(strippedKey)) {
@ -1685,6 +1707,17 @@ public class HStore implements HConstants {
} while(map.next(readkey, readval));
}
/*
* @param mf MapFile to dig in.
* @return Final key from passed <code>mf</code>
* @throws IOException
*/
private HStoreKey getFinalKey(final MapFile.Reader mf) throws IOException {
HStoreKey finalKey = new HStoreKey();
mf.finalKey(finalKey);
return finalKey;
}
static HStoreKey stripTimestamp(HStoreKey key) {
return new HStoreKey(key.getRow(), key.getColumn());
}

View File

@ -344,141 +344,161 @@ class Memcache {
}
private void internalGetRowKeyAtOrBefore(SortedMap<HStoreKey, byte []> map,
byte [] key, SortedMap<HStoreKey, Long> candidateKeys) {
HStoreKey strippedKey = null;
// we want the earliest possible to start searching from
HStoreKey search_key = candidateKeys.isEmpty() ?
new HStoreKey(key) : new HStoreKey(candidateKeys.firstKey().getRow());
Iterator<HStoreKey> key_iterator = null;
HStoreKey found_key = null;
ArrayList<HStoreKey> victims = new ArrayList<HStoreKey>();
byte [] row, SortedMap<HStoreKey, Long> candidateKeys) {
// We want the earliest possible to start searching from. Start before
// the candidate key in case it turns out a delete came in later.
HStoreKey search_key = candidateKeys.isEmpty()? new HStoreKey(row):
new HStoreKey(candidateKeys.firstKey().getRow());
List<HStoreKey> victims = new ArrayList<HStoreKey>();
long now = System.currentTimeMillis();
// get all the entries that come equal or after our search key
// Get all the entries that come equal or after our search key
SortedMap<HStoreKey, byte []> tailMap = map.tailMap(search_key);
// if there are items in the tail map, there's either a direct match to
// the search key, or a range of values between the first candidate key
// and the ultimate search key (or the end of the cache)
if (!tailMap.isEmpty() &&
Bytes.compareTo(tailMap.firstKey().getRow(), key) <= 0) {
key_iterator = tailMap.keySet().iterator();
Bytes.compareTo(tailMap.firstKey().getRow(), search_key.getRow()) <= 0) {
Iterator<HStoreKey> key_iterator = tailMap.keySet().iterator();
// keep looking at cells as long as they are no greater than the
// Keep looking at cells as long as they are no greater than the
// ultimate search key and there's still records left in the map.
do {
HStoreKey deletedOrExpiredRow = null;
for (HStoreKey found_key = null; key_iterator.hasNext() &&
(found_key == null ||
Bytes.compareTo(found_key.getRow(), row) <= 0);) {
found_key = key_iterator.next();
if (Bytes.compareTo(found_key.getRow(), key) <= 0) {
strippedKey = stripTimestamp(found_key);
if (Bytes.compareTo(found_key.getRow(), row) <= 0) {
if (HLogEdit.isDeleted(tailMap.get(found_key))) {
if (candidateKeys.containsKey(strippedKey)) {
long bestCandidateTs =
candidateKeys.get(strippedKey).longValue();
if (bestCandidateTs <= found_key.getTimestamp()) {
candidateKeys.remove(strippedKey);
}
handleDeleted(found_key, candidateKeys);
if (deletedOrExpiredRow == null) {
deletedOrExpiredRow = found_key;
}
} else {
if (ttl == HConstants.FOREVER ||
now < found_key.getTimestamp() + ttl) {
candidateKeys.put(strippedKey,
now < found_key.getTimestamp() + ttl) {
HStoreKey strippedKey = stripTimestamp(found_key);
candidateKeys.put(strippedKey,
new Long(found_key.getTimestamp()));
} else {
if (deletedOrExpiredRow == null) {
deletedOrExpiredRow = new HStoreKey(found_key);
}
victims.add(found_key);
if (LOG.isDebugEnabled()) {
LOG.debug(":" + found_key + ": expired, skipped");
LOG.debug("internalGetRowKeyAtOrBefore:" + found_key +
" expired, skipped");
}
}
}
}
} while (Bytes.compareTo(found_key.getRow(), key) <= 0
&& key_iterator.hasNext());
} else {
// the tail didn't contain any keys that matched our criteria, or was
// empty. examine all the keys that preceed our splitting point.
SortedMap<HStoreKey, byte []> headMap = map.headMap(search_key);
// if we tried to create a headMap and got an empty map, then there are
// no keys at or before the search key, so we're done.
if (headMap.isEmpty()) {
return;
}
// if there aren't any candidate keys at this point, we need to search
// backwards until we find at least one candidate or run out of headMap.
if (candidateKeys.isEmpty()) {
HStoreKey[] cells =
headMap.keySet().toArray(new HStoreKey[headMap.keySet().size()]);
byte [] lastRowFound = null;
for(int i = cells.length - 1; i >= 0; i--) {
HStoreKey thisKey = cells[i];
// if the last row we found a candidate key for is different than
// the row of the current candidate, we can stop looking.
if (lastRowFound != null &&
!Bytes.equals(lastRowFound, thisKey.getRow())) {
break;
}
// if this isn't a delete, record it as a candidate key. also
// take note of the row of this candidate so that we'll know when
// we cross the row boundary into the previous row.
if (!HLogEdit.isDeleted(headMap.get(thisKey))) {
if (ttl == HConstants.FOREVER) {
lastRowFound = thisKey.getRow();
candidateKeys.put(stripTimestamp(thisKey),
new Long(thisKey.getTimestamp()));
} else {
victims.add(found_key);
if (LOG.isDebugEnabled()) {
LOG.debug("internalGetRowKeyAtOrBefore: " + found_key +
": expired, skipped");
}
}
}
}
} else {
// if there are already some candidate keys, we only need to consider
// the very last row's worth of keys in the headMap, because any
// smaller acceptable candidate keys would have caused us to start
// our search earlier in the list, and we wouldn't be searching here.
SortedMap<HStoreKey, byte[]> thisRowTailMap =
headMap.tailMap(new HStoreKey(headMap.lastKey().getRow()));
key_iterator = thisRowTailMap.keySet().iterator();
do {
found_key = key_iterator.next();
if (HLogEdit.isDeleted(thisRowTailMap.get(found_key))) {
strippedKey = stripTimestamp(found_key);
if (candidateKeys.containsKey(strippedKey)) {
long bestCandidateTs =
candidateKeys.get(strippedKey).longValue();
if (bestCandidateTs <= found_key.getTimestamp()) {
candidateKeys.remove(strippedKey);
}
}
} else {
if (ttl == HConstants.FOREVER ||
now < found_key.getTimestamp() + ttl) {
candidateKeys.put(stripTimestamp(found_key),
Long.valueOf(found_key.getTimestamp()));
} else {
victims.add(found_key);
if (LOG.isDebugEnabled()) {
LOG.debug("internalGetRowKeyAtOrBefore: " + found_key +
": expired, skipped");
}
}
}
} while (key_iterator.hasNext());
}
if (candidateKeys.isEmpty() && deletedOrExpiredRow != null) {
getRowKeyBefore(map, deletedOrExpiredRow, candidateKeys, victims, now);
}
} else {
// The tail didn't contain any keys that matched our criteria, or was
// empty. Examine all the keys that proceed our splitting point.
getRowKeyBefore(map, search_key, candidateKeys, victims, now);
}
// Remove expired victims from the map.
for (HStoreKey victim: victims)
for (HStoreKey victim: victims) {
map.remove(victim);
}
}
/*
* Get row key that comes before passed <code>search_key</code>
* Use when we know search_key is not in the map and we need to search
* earlier in the cache.
* @param map
* @param search_key
* @param candidateKeys
* @param victims
*/
private void getRowKeyBefore(SortedMap<HStoreKey, byte []> map,
HStoreKey search_key, SortedMap<HStoreKey, Long> candidateKeys,
List<HStoreKey> victims, final long now) {
SortedMap<HStoreKey, byte []> headMap = map.headMap(search_key);
// If we tried to create a headMap and got an empty map, then there are
// no keys at or before the search key, so we're done.
if (headMap.isEmpty()) {
return;
}
// If there aren't any candidate keys at this point, we need to search
// backwards until we find at least one candidate or run out of headMap.
HStoreKey found_key = null;
if (candidateKeys.isEmpty()) {
Set<HStoreKey> keys = headMap.keySet();
HStoreKey [] cells = keys.toArray(new HStoreKey[keys.size()]);
byte [] lastRowFound = null;
for (int i = cells.length - 1; i >= 0; i--) {
HStoreKey thisKey = cells[i];
// if the last row we found a candidate key for is different than
// the row of the current candidate, we can stop looking -- if its
// not a delete record.
boolean deleted = HLogEdit.isDeleted(headMap.get(thisKey));
if (lastRowFound != null &&
!Bytes.equals(lastRowFound, thisKey.getRow()) && !deleted) {
break;
}
// If this isn't a delete, record it as a candidate key. Also
// take note of the row of this candidate so that we'll know when
// we cross the row boundary into the previous row.
if (!deleted) {
if (ttl == HConstants.FOREVER || now < thisKey.getTimestamp() + ttl) {
lastRowFound = thisKey.getRow();
candidateKeys.put(stripTimestamp(thisKey),
new Long(thisKey.getTimestamp()));
} else {
victims.add(found_key);
if (LOG.isDebugEnabled()) {
LOG.debug("getRowKeyBefore: " + found_key + ": expired, skipped");
}
}
}
}
} else {
// If there are already some candidate keys, we only need to consider
// the very last row's worth of keys in the headMap, because any
// smaller acceptable candidate keys would have caused us to start
// our search earlier in the list, and we wouldn't be searching here.
SortedMap<HStoreKey, byte[]> thisRowTailMap =
headMap.tailMap(new HStoreKey(headMap.lastKey().getRow()));
Iterator<HStoreKey> key_iterator = thisRowTailMap.keySet().iterator();
do {
found_key = key_iterator.next();
if (HLogEdit.isDeleted(thisRowTailMap.get(found_key))) {
handleDeleted(found_key, candidateKeys);
} else {
if (ttl == HConstants.FOREVER ||
now < found_key.getTimestamp() + ttl) {
candidateKeys.put(stripTimestamp(found_key),
Long.valueOf(found_key.getTimestamp()));
} else {
victims.add(found_key);
if (LOG.isDebugEnabled()) {
LOG.debug("internalGetRowKeyAtOrBefore: " + found_key +
": expired, skipped");
}
}
}
} while (key_iterator.hasNext());
}
}
private void handleDeleted(final HStoreKey k,
final SortedMap<HStoreKey, Long> candidateKeys) {
HStoreKey strippedKey = stripTimestamp(k);
if (candidateKeys.containsKey(strippedKey)) {
long bestCandidateTs =
candidateKeys.get(strippedKey).longValue();
if (bestCandidateTs <= k.getTimestamp()) {
candidateKeys.remove(strippedKey);
}
}
}
static HStoreKey stripTimestamp(HStoreKey key) {

View File

@ -42,6 +42,15 @@ public class TestCompare extends TestCase {
assertTrue(past.compareTo(now) > 0);
assertTrue(now.compareTo(now) == 0);
assertTrue(future.compareTo(now) < 0);
// Check that empty column comes before one with a column
HStoreKey nocolumn = new HStoreKey(a, timestamp);
HStoreKey withcolumn = new HStoreKey(a, a, timestamp);
assertTrue(nocolumn.compareTo(withcolumn) < 0);
// Check that empty column comes and LATEST comes before one with a column
// and old timestamp.
nocolumn = new HStoreKey(a, HConstants.LATEST_TIMESTAMP);
withcolumn = new HStoreKey(a, a, timestamp);
assertTrue(nocolumn.compareTo(withcolumn) < 0);
}
/**

View File

@ -21,20 +21,19 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Map;
import java.util.HashSet;
import java.util.TreeMap;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.hadoop.dfs.MiniDFSCluster;
import org.apache.hadoop.hbase.filter.StopRowFilter;
import org.apache.hadoop.hbase.filter.WhileMatchRowFilter;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HStoreKey;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.filter.StopRowFilter;
import org.apache.hadoop.hbase.filter.WhileMatchRowFilter;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -43,6 +42,14 @@ import org.apache.hadoop.hbase.util.Bytes;
*/
public class TestGet2 extends HBaseTestCase implements HConstants {
private MiniDFSCluster miniHdfs;
private static final String T00 = "000";
private static final String T10 = "010";
private static final String T11 = "011";
private static final String T12 = "012";
private static final String T20 = "020";
private static final String T30 = "030";
private static final String T31 = "031";
@Override
protected void setUp() throws Exception {
@ -52,7 +59,118 @@ public class TestGet2 extends HBaseTestCase implements HConstants {
this.conf.set(HConstants.HBASE_DIR,
this.miniHdfs.getFileSystem().getHomeDirectory().toString());
}
/**
* Test file of multiple deletes and with deletes as final key.
* @see <a href="https://issues.apache.org/jira/browse/HBASE-751">HBASE-751</a>
*/
public void testGetClosestRowBefore3() throws IOException{
HRegion region = null;
BatchUpdate batchUpdate = null;
try {
HTableDescriptor htd = createTableDescriptor(getName());
region = createNewHRegion(htd, null, null);
batchUpdate = new BatchUpdate(T00);
batchUpdate.put(COLUMNS[0], T00.getBytes());
region.batchUpdate(batchUpdate);
batchUpdate = new BatchUpdate(T10);
batchUpdate.put(COLUMNS[0], T10.getBytes());
region.batchUpdate(batchUpdate);
batchUpdate = new BatchUpdate(T20);
batchUpdate.put(COLUMNS[0], T20.getBytes());
region.batchUpdate(batchUpdate);
Map<byte [], Cell> results =
region.getClosestRowBefore(Bytes.toBytes(T20));
assertEquals(T20, new String(results.get(COLUMNS[0]).getValue()));
batchUpdate = new BatchUpdate(T20);
batchUpdate.delete(COLUMNS[0]);
region.batchUpdate(batchUpdate);
results = region.getClosestRowBefore(Bytes.toBytes(T10));
assertEquals(T10, new String(results.get(COLUMNS[0]).getValue()));
batchUpdate = new BatchUpdate(T30);
batchUpdate.put(COLUMNS[0], T30.getBytes());
region.batchUpdate(batchUpdate);
results = region.getClosestRowBefore(Bytes.toBytes(T30));
assertEquals(T30, new String(results.get(COLUMNS[0]).getValue()));
batchUpdate = new BatchUpdate(T30);
batchUpdate.delete(COLUMNS[0]);
region.batchUpdate(batchUpdate);
results = region.getClosestRowBefore(Bytes.toBytes(T30));
assertEquals(T10, new String(results.get(COLUMNS[0]).getValue()));
results = region.getClosestRowBefore(Bytes.toBytes(T31));
assertEquals(T10, new String(results.get(COLUMNS[0]).getValue()));
region.flushcache();
// try finding "010" after flush
results = region.getClosestRowBefore(Bytes.toBytes(T30));
assertEquals(T10, new String(results.get(COLUMNS[0]).getValue()));
results = region.getClosestRowBefore(Bytes.toBytes(T31));
assertEquals(T10, new String(results.get(COLUMNS[0]).getValue()));
// Put into a different column family. Should make it so I get
// and answer of t20.
batchUpdate = new BatchUpdate(T20);
batchUpdate.put(COLUMNS[1], T20.getBytes());
region.batchUpdate(batchUpdate);
results = region.getClosestRowBefore(Bytes.toBytes(T30));
assertEquals(T20, new String(results.get(COLUMNS[1]).getValue()));
results = region.getClosestRowBefore(Bytes.toBytes(T31));
assertEquals(T20, new String(results.get(COLUMNS[1]).getValue()));
region.flushcache();
results = region.getClosestRowBefore(Bytes.toBytes(T30));
assertEquals(T20, new String(results.get(COLUMNS[1]).getValue()));
results = region.getClosestRowBefore(Bytes.toBytes(T31));
assertEquals(T20, new String(results.get(COLUMNS[1]).getValue()));
// Now try combo of memcache and mapfiles. Delete the t20 COLUMS[1]
// in memory; make sure we get back t10 again.
batchUpdate = new BatchUpdate(T20);
batchUpdate.delete(COLUMNS[1]);
region.batchUpdate(batchUpdate);
results = region.getClosestRowBefore(Bytes.toBytes(T30));
assertEquals(T10, new String(results.get(COLUMNS[0]).getValue()));
// Ask for a value off the end of the file. Should return t10.
results = region.getClosestRowBefore(Bytes.toBytes(T31));
assertEquals(T10, new String(results.get(COLUMNS[0]).getValue()));
region.flushcache();
results = region.getClosestRowBefore(Bytes.toBytes(T31));
assertEquals(T10, new String(results.get(COLUMNS[0]).getValue()));
// Ok. Let the candidate come out of mapfiles but have delete of
// the candidate be in memory.
batchUpdate = new BatchUpdate(T11);
batchUpdate.put(COLUMNS[0], T11.getBytes());
region.batchUpdate(batchUpdate);
batchUpdate = new BatchUpdate(T10);
batchUpdate.delete(COLUMNS[0]);
region.batchUpdate(batchUpdate);
results = region.getClosestRowBefore(Bytes.toBytes(T12));
assertEquals(T11, new String(results.get(COLUMNS[0]).getValue()));
} finally {
if (region != null) {
try {
region.close();
} catch (Exception e) {
e.printStackTrace();
}
region.getLog().closeAndDelete();
}
}
}
/**
* Tests for HADOOP-2161.
* @throws Exception