HBASE-5010 Filter HFiles based on TTL (Mikhail)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1224792 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2011-12-26 19:33:51 +00:00
parent 300f900354
commit 2fe6ab663c
21 changed files with 230 additions and 122 deletions

View File

@ -27,6 +27,8 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
@ -722,4 +724,22 @@ public class LruBlockCache implements BlockCache, HeapSize {
public void shutdown() {
this.scheduleThreadPool.shutdown();
}
/** Clears the cache. Used in tests. */
public void clearCache() {
map.clear();
}
/**
* Used in testing. May be very inefficient.
* @return the set of cached file names
*/
SortedSet<String> getCachedFileNamesForTest() {
SortedSet<String> fileNames = new TreeSet<String>();
for (BlockCacheKey cacheKey : map.keySet()) {
fileNames.add(cacheKey.getHfileName());
}
return fileNames;
}
}

View File

@ -71,13 +71,15 @@ public class ExplicitColumnTracker implements ColumnTracker {
* @param columns columns specified user in query
* @param minVersions minimum number of versions to keep
* @param maxVersions maximum versions to return per column
* @param oldestUnexpiredTS the oldest timestamp we are interested in,
* based on TTL
* @param ttl The timeToLive to enforce
*/
public ExplicitColumnTracker(NavigableSet<byte[]> columns, int minVersions,
int maxVersions, long ttl) {
int maxVersions, long oldestUnexpiredTS) {
this.maxVersions = maxVersions;
this.minVersions = minVersions;
this.oldestStamp = System.currentTimeMillis() - ttl;
this.oldestStamp = oldestUnexpiredTS;
this.columns = new ArrayList<ColumnCount>(columns.size());
this.columnsToReuse = new ArrayList<ColumnCount>(columns.size());
for(byte [] column : columns) {
@ -259,6 +261,7 @@ public class ExplicitColumnTracker implements ColumnTracker {
}
}
}
public MatchCode getNextRowOrNextColumn(byte[] bytes, int offset,
int qualLength) {
doneWithColumn(bytes, offset,qualLength);
@ -271,6 +274,6 @@ public class ExplicitColumnTracker implements ColumnTracker {
}
public boolean isDone(long timestamp) {
return minVersions <=0 && isExpired(timestamp);
return minVersions <= 0 && isExpired(timestamp);
}
}

View File

@ -20,8 +20,10 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.SortedSet;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
/**
* Scanner that returns the next KeyValue.
@ -69,6 +71,19 @@ public interface KeyValueScanner {
*/
public void close();
/**
* Allows to filter out scanners (both StoreFile and memstore) that we don't
* want to use based on criteria such as Bloom filters and timestamp ranges.
* @param scan the scan that we are selecting scanners for
* @param columns the set of columns in the current column family, or null if
* not specified by the scan
* @param oldestUnexpiredTS the oldest timestamp we are interested in for
* this query, based on TTL
* @return true if the scanner should be included in the query
*/
public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
long oldestUnexpiredTS);
// "Lazy scanner" optimizations
/**
@ -100,4 +115,10 @@ public interface KeyValueScanner {
* {@link #realSeekDone()} first.
*/
public void enforceSeek() throws IOException;
}
/**
* @return true if this is a file scanner. Otherwise a memory scanner is
* assumed.
*/
public boolean isFileScanner();
}

View File

@ -89,7 +89,7 @@ public class MemStore implements HeapSize {
TimeRangeTracker timeRangeTracker;
TimeRangeTracker snapshotTimeRangeTracker;
MemStoreLAB allocator;
@ -216,11 +216,11 @@ public class MemStore implements HeapSize {
this.lock.readLock().unlock();
}
}
/**
* Internal version of add() that doesn't clone KVs with the
* allocator, and doesn't take the lock.
*
*
* Callers should ensure they already have the read lock taken
*/
private long internalAdd(final KeyValue toAdd) {
@ -251,9 +251,9 @@ public class MemStore implements HeapSize {
/**
* Remove n key from the memstore. Only kvs that have the same key and the
* same memstoreTS are removed. It is ok to not update timeRangeTracker
* in this call. It is possible that we can optimize this method by using
* tailMap/iterator, but since this method is called rarely (only for
* same memstoreTS are removed. It is ok to not update timeRangeTracker
* in this call. It is possible that we can optimize this method by using
* tailMap/iterator, but since this method is called rarely (only for
* error recovery), we can leave those optimization for the future.
* @param kv
*/
@ -544,7 +544,7 @@ public class MemStore implements HeapSize {
* family, and qualifier, they are removed.
* <p>
* Callers must hold the read lock.
*
*
* @param kv
* @return change in size of MemStore
*/
@ -649,9 +649,12 @@ public class MemStore implements HeapSize {
* @param scan
* @return False if the key definitely does not exist in this Memstore
*/
public boolean shouldSeek(Scan scan) {
return timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange());
public boolean shouldSeek(Scan scan, long oldestUnexpiredTS) {
return (timeRangeTracker.includesTimeRange(scan.getTimeRange()) ||
snapshotTimeRangeTracker.includesTimeRange(scan.getTimeRange()))
&& (Math.max(timeRangeTracker.getMaximumTimestamp(),
snapshotTimeRangeTracker.getMaximumTimestamp()) >=
oldestUnexpiredTS);
}
public TimeRangeTracker getSnapshotTimeRangeTracker() {
@ -860,6 +863,12 @@ public class MemStore implements HeapSize {
public long getSequenceID() {
return Long.MAX_VALUE;
}
@Override
public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
long oldestUnexpiredTS) {
return shouldSeek(scan, oldestUnexpiredTS);
}
}
public final static long FIXED_OVERHEAD = ClassSize.align(

View File

@ -20,9 +20,11 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.SortedSet;
import org.apache.commons.lang.NotImplementedException;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
/**
* A "non-lazy" scanner which always does a real seek operation. Most scanners
@ -52,4 +54,16 @@ public abstract class NonLazyKeyValueScanner implements KeyValueScanner {
return forward ? scanner.reseek(kv) : scanner.seek(kv);
}
@Override
public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
long oldestUnexpiredTS) {
// No optimizations implemented by default.
return true;
}
@Override
public boolean isFileScanner() {
// Not a file by default.
return false;
}
}

View File

@ -132,10 +132,12 @@ public class ScanQueryMatcher {
* @param columns
* @param scanType Type of the scan
* @param earliestPutTs Earliest put seen in any of the store files.
* @param oldestUnexpiredTS the oldest timestamp we are interested in,
* based on TTL
*/
public ScanQueryMatcher(Scan scan, Store.ScanInfo scanInfo,
NavigableSet<byte[]> columns, StoreScanner.ScanType scanType,
long readPointToUse, long earliestPutTs) {
long readPointToUse, long earliestPutTs, long oldestUnexpiredTS) {
this.tr = scan.getTimeRange();
this.rowComparator = scanInfo.getComparator().getRawComparator();
this.deletes = new ScanDeleteTracker();
@ -163,15 +165,16 @@ public class ScanQueryMatcher {
hasNullColumn = true;
// use a specialized scan for wildcard column tracker.
this.columns = new ScanWildcardColumnTracker(scanInfo.getMinVersions(), maxVersions, scanInfo.getTtl());
this.columns = new ScanWildcardColumnTracker(
scanInfo.getMinVersions(), maxVersions, oldestUnexpiredTS);
} else {
// whether there is null column in the explicit column query
hasNullColumn = (columns.first().length == 0);
// We can share the ExplicitColumnTracker, diff is we reset
// between rows, not between storefiles.
this.columns = new ExplicitColumnTracker(columns, scanInfo.getMinVersions(), maxVersions,
scanInfo.getTtl());
this.columns = new ExplicitColumnTracker(columns,
scanInfo.getMinVersions(), maxVersions, oldestUnexpiredTS);
}
}
@ -179,10 +182,10 @@ public class ScanQueryMatcher {
* Constructor for tests
*/
ScanQueryMatcher(Scan scan, Store.ScanInfo scanInfo,
NavigableSet<byte[]> columns) {
NavigableSet<byte[]> columns, long oldestUnexpiredTS) {
this(scan, scanInfo, columns, StoreScanner.ScanType.USER_SCAN,
Long.MAX_VALUE, /* max Readpoint to track versions */
HConstants.LATEST_TIMESTAMP);
HConstants.LATEST_TIMESTAMP, oldestUnexpiredTS);
}
/**

View File

@ -26,7 +26,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* Keeps track of the columns for a scan if they are not explicitly specified
@ -49,12 +48,14 @@ public class ScanWildcardColumnTracker implements ColumnTracker {
* Return maxVersions of every row.
* @param minVersion Minimum number of versions to keep
* @param maxVersion Maximum number of versions to return
* @param ttl TimeToLive to enforce
* @param oldestUnexpiredTS oldest timestamp that has not expired according
* to the TTL.
*/
public ScanWildcardColumnTracker(int minVersion, int maxVersion, long ttl) {
public ScanWildcardColumnTracker(int minVersion, int maxVersion,
long oldestUnexpiredTS) {
this.maxVersions = maxVersion;
this.minVersions = minVersion;
this.oldestStamp = EnvironmentEdgeManager.currentTimeMillis() - ttl;
this.oldestStamp = oldestUnexpiredTS;
}
/**
@ -197,7 +198,6 @@ public class ScanWildcardColumnTracker implements ColumnTracker {
}
public boolean isDone(long timestamp) {
return minVersions <=0 && isExpired(timestamp);
return minVersions <= 0 && isExpired(timestamp);
}
}

View File

@ -2142,4 +2142,5 @@ public class Store extends SchemaConfigured implements HeapSize {
return comparator;
}
}
}

View File

@ -1203,20 +1203,20 @@ public class StoreFile extends SchemaConfigured {
reader.close(evictOnClose);
}
public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
return (passesTimerangeFilter(scan) && passesBloomFilter(scan, columns));
}
/**
* Check if this storeFile may contain keys within the TimeRange
* @param scan
* @return False if it definitely does not exist in this StoreFile
* Check if this storeFile may contain keys within the TimeRange that
* have not expired (i.e. not older than oldestUnexpiredTS).
* @param scan the current scan
* @param oldestUnexpiredTS the oldest timestamp that is not expired, as
* determined by the column family's TTL
* @return false if queried keys definitely don't exist in this StoreFile
*/
private boolean passesTimerangeFilter(Scan scan) {
boolean passesTimerangeFilter(Scan scan, long oldestUnexpiredTS) {
if (timeRangeTracker == null) {
return true;
} else {
return timeRangeTracker.includesTimeRange(scan.getTimeRange());
return timeRangeTracker.includesTimeRange(scan.getTimeRange()) &&
timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS;
}
}
@ -1236,7 +1236,7 @@ public class StoreFile extends SchemaConfigured {
* filter, or if the Bloom filter is not applicable for the scan.
* False if the Bloom filter is applicable and the scan fails it.
*/
private boolean passesBloomFilter(Scan scan,
boolean passesBloomFilter(Scan scan,
final SortedSet<byte[]> columns) {
// Multi-column non-get scans will use Bloom filters through the
// lower-level API function that this function calls.

View File

@ -20,21 +20,19 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.SortedSet;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
/**
* KeyValueScanner adaptor over the Reader. It also provides hooks into
@ -250,11 +248,6 @@ class StoreFileScanner implements KeyValueScanner {
}
}
// StoreFile filter hook.
public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
return reader.shouldSeek(scan, columns);
}
@Override
public long getSequenceID() {
return reader.getSequenceID();
@ -362,9 +355,21 @@ class StoreFileScanner implements KeyValueScanner {
this.matcher = matcher;
}
@Override
public boolean isFileScanner() {
return true;
}
// Test methods
static final long getSeekCount() {
return seekCount.get();
}
@Override
public boolean shouldUseScanner(Scan scan, SortedSet<byte[]> columns,
long oldestUnexpiredTS) {
return reader.passesTimerangeFilter(scan, oldestUnexpiredTS) &&
reader.passesBloomFilter(scan, columns);
}
}

View File

@ -34,6 +34,7 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
* Scanner scans both the memstore and the HStore. Coalesce KeyValue stream
@ -55,6 +56,8 @@ class StoreScanner extends NonLazyKeyValueScanner
private final boolean isGet;
private final boolean explicitColumnQuery;
private final boolean useRowColBloom;
private final long oldestUnexpiredTS;
private final int minVersions;
/** We don't ever expect to change this, the constant is just for clarity. */
static final boolean LAZY_SEEK_ENABLED_BY_DEFAULT = true;
@ -68,12 +71,14 @@ class StoreScanner extends NonLazyKeyValueScanner
/** An internal constructor. */
private StoreScanner(Store store, boolean cacheBlocks, Scan scan,
final NavigableSet<byte[]> columns){
final NavigableSet<byte[]> columns, long ttl, int minVersions) {
this.store = store;
this.cacheBlocks = cacheBlocks;
isGet = scan.isGetScan();
int numCol = columns == null ? 0 : columns.size();
explicitColumnQuery = numCol > 0;
oldestUnexpiredTS = EnvironmentEdgeManager.currentTimeMillis() - ttl;
this.minVersions = minVersions;
// We look up row-column Bloom filters for multi-column queries as part of
// the seek operation. However, we also look the row-column Bloom filter
@ -92,14 +97,16 @@ class StoreScanner extends NonLazyKeyValueScanner
*/
StoreScanner(Store store, Scan scan, final NavigableSet<byte[]> columns)
throws IOException {
this(store, scan.getCacheBlocks(), scan, columns);
this(store, scan.getCacheBlocks(), scan, columns, store.scanInfo.getTtl(),
store.scanInfo.getMinVersions());
initializeMetricNames();
if (columns != null && scan.isRaw()) {
throw new DoNotRetryIOException(
"Cannot specify any column for a raw scan");
}
matcher = new ScanQueryMatcher(scan, store.scanInfo, columns,
ScanType.USER_SCAN, Long.MAX_VALUE, HConstants.LATEST_TIMESTAMP);
ScanType.USER_SCAN, Long.MAX_VALUE, HConstants.LATEST_TIMESTAMP,
oldestUnexpiredTS);
// Pass columns to try to filter out unnecessary StoreFiles.
List<KeyValueScanner> scanners = getScanners(scan, columns);
@ -130,17 +137,18 @@ class StoreScanner extends NonLazyKeyValueScanner
* Opens a scanner across specified StoreFiles.
* @param store who we scan
* @param scan the spec
* @param scanners ancilliary scanners
* @param scanners ancillary scanners
* @param smallestReadPoint the readPoint that we should use for tracking versions
* @param retainDeletesInOutput should we retain deletes after compaction?
*/
StoreScanner(Store store, Scan scan,
List<? extends KeyValueScanner> scanners, ScanType scanType,
long smallestReadPoint, long earliestPutTs) throws IOException {
this(store, false, scan, null);
this(store, false, scan, null, store.scanInfo.getTtl(),
store.scanInfo.getMinVersions());
initializeMetricNames();
matcher = new ScanQueryMatcher(scan, store.scanInfo, null, scanType,
smallestReadPoint, earliestPutTs);
smallestReadPoint, earliestPutTs, oldestUnexpiredTS);
// Seek all scanners to the initial key
for(KeyValueScanner scanner : scanners) {
@ -164,10 +172,11 @@ class StoreScanner extends NonLazyKeyValueScanner
StoreScanner.ScanType scanType, final NavigableSet<byte[]> columns,
final List<KeyValueScanner> scanners, long earliestPutTs)
throws IOException {
this(null, scan.getCacheBlocks(), scan, columns);
this(null, scan.getCacheBlocks(), scan, columns, scanInfo.getTtl(),
scanInfo.getMinVersions());
this.initializeMetricNames();
this.matcher = new ScanQueryMatcher(scan, scanInfo, columns, scanType,
Long.MAX_VALUE, earliestPutTs);
Long.MAX_VALUE, earliestPutTs, oldestUnexpiredTS);
// Seek all scanners to the initial key
for (KeyValueScanner scanner : scanners) {
@ -221,18 +230,20 @@ class StoreScanner extends NonLazyKeyValueScanner
List<KeyValueScanner> scanners =
new ArrayList<KeyValueScanner>(allStoreScanners.size());
// We can only exclude store files based on TTL if minVersions is set to 0.
// Otherwise, we might have to return KVs that have technically expired.
long expiredTimestampCutoff = minVersions == 0 ? oldestUnexpiredTS :
Long.MIN_VALUE;
// include only those scan files which pass all filters
for (KeyValueScanner kvs : allStoreScanners) {
if (kvs instanceof StoreFileScanner) {
if (memOnly == false && ((StoreFileScanner)kvs).shouldSeek(scan, columns)) {
scanners.add(kvs);
}
boolean isFile = kvs.isFileScanner();
if ((!isFile && filesOnly) || (isFile && memOnly)) {
continue;
}
else {
// kvs is a MemStoreScanner
if (filesOnly == false && this.store.memstore.shouldSeek(scan)) {
scanners.add(kvs);
}
if (kvs.shouldUseScanner(scan, columns, expiredTimestampCutoff)) {
scanners.add(kvs);
}
}

View File

@ -30,11 +30,11 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Writable;
/**
* Stores the minimum and maximum timestamp values.
* Stores the minimum and maximum timestamp values (both are inclusive).
* Can be used to find if any given time range overlaps with its time range
* MemStores use this class to track its minimum and maximum timestamps.
* When writing StoreFiles, this information is stored in meta blocks and used
* at read time to match against the required TimeRange
* at read time to match against the required TimeRange.
*/
public class TimeRangeTracker implements Writable {
@ -143,5 +143,10 @@ public class TimeRangeTracker implements Writable {
this.maximumTimestamp = in.readLong();
}
@Override
public String toString() {
return "[" + minimumTimestamp + "," + maximumTimestamp + "]";
}
}

View File

@ -241,7 +241,7 @@ public class TestCompaction extends HBaseTestCase {
r.compactStores(true);
int count = count();
assertTrue("Should not see anything after TTL has expired", count == 0);
assertEquals("Should not see anything after TTL has expired", 0, count);
}
public void testMinorCompactionWithDeleteRow() throws Exception {

View File

@ -163,8 +163,7 @@ public class TestCompoundBloomFilter {
* it to the provided threshold.
*
* @param falsePosRate experimental positive rate
* @param nTrials the number of calls to
* {@link StoreFile.Reader#shouldSeek(Scan, java.util.SortedSet)}.
* @param nTrials the number of Bloom filter checks
* @param zValueBoundary z-value boundary, positive for an upper bound and
* negative for a lower bound
* @param cbf the compound Bloom filter we are using
@ -283,7 +282,7 @@ public class TestCompoundBloomFilter {
Scan scan = new Scan(row, row);
TreeSet<byte[]> columns = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
columns.add(qualifier);
return scanner.shouldSeek(scan, columns);
return scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE);
}
private Path writeStoreFile(int t, BloomType bt, List<KeyValue> kvs)

View File

@ -47,7 +47,7 @@ public class TestExplicitColumnTracker extends HBaseTestCase {
List<byte[]> scannerColumns,
List<MatchCode> expected) throws IOException {
ColumnTracker exp = new ExplicitColumnTracker(
trackColumns, 0, maxVersions, Long.MAX_VALUE);
trackColumns, 0, maxVersions, Long.MIN_VALUE);
//Initialize result
@ -166,7 +166,7 @@ public class TestExplicitColumnTracker extends HBaseTestCase {
}
ColumnTracker explicit = new ExplicitColumnTracker(columns, 0, maxVersions,
Long.MAX_VALUE);
Long.MIN_VALUE);
for (int i = 0; i < 100000; i+=2) {
byte [] col = Bytes.toBytes("col"+i);
explicit.checkColumn(col, 0, col.length, 1, KeyValue.Type.Put.getCode(),

View File

@ -779,16 +779,16 @@ public class TestMemStore extends TestCase {
addRows(memstore,timestamp);
scan.setTimeRange(0, 2);
assertTrue(memstore.shouldSeek(scan));
assertTrue(memstore.shouldSeek(scan, Long.MIN_VALUE));
scan.setTimeRange(20, 82);
assertTrue(memstore.shouldSeek(scan));
assertTrue(memstore.shouldSeek(scan, Long.MIN_VALUE));
scan.setTimeRange(10, 20);
assertTrue(memstore.shouldSeek(scan));
assertTrue(memstore.shouldSeek(scan, Long.MIN_VALUE));
scan.setTimeRange(8, 12);
assertTrue(memstore.shouldSeek(scan));
assertTrue(memstore.shouldSeek(scan, Long.MIN_VALUE));
/*This test is not required for correctness but it should pass when
* timestamp range optimization is on*/

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.filter.TimestampsFilter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.junit.experimental.categories.Category;
/**
@ -52,7 +53,8 @@ public class TestMinVersions extends HBaseTestCase {
HTableDescriptor htd = createTableDescriptor(getName(), 1, 1000, 1, false);
HRegion region = createNewHRegion(htd, null, null);
long ts = System.currentTimeMillis() - 2000; // 2s in the past
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
Put p = new Put(T1, ts);
p.add(c0, c0, T1);
@ -95,7 +97,8 @@ public class TestMinVersions extends HBaseTestCase {
HTableDescriptor htd = createTableDescriptor(getName(), 3, 1000, 1, false);
HRegion region = createNewHRegion(htd, null, null);
long ts = System.currentTimeMillis() - 2000; // 2s in the past
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
Put p = new Put(T1, ts-1);
p.add(c0, c0, T2);
@ -144,7 +147,8 @@ public class TestMinVersions extends HBaseTestCase {
HTableDescriptor htd = createTableDescriptor(getName(), 3, 1000, 1, false);
HRegion region = createNewHRegion(htd, null, null);
long ts = System.currentTimeMillis() - 2000; // 2s in the past
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
Put p = new Put(T1, ts-2);
p.add(c0, c0, T1);
@ -196,7 +200,8 @@ public class TestMinVersions extends HBaseTestCase {
HTableDescriptor htd = createTableDescriptor(getName(), 2, 1000, 1, false);
HRegion region = createNewHRegion(htd, null, null);
long ts = System.currentTimeMillis() - 2000; // 2s in the past
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
// 2nd version
Put p = new Put(T1, ts-2);
@ -265,7 +270,8 @@ public class TestMinVersions extends HBaseTestCase {
HTableDescriptor htd = createTableDescriptor(getName(), 2, 1000, 1, false);
HRegion region = createNewHRegion(htd, null, null);
long ts = System.currentTimeMillis() - 2000; // 2s in the past
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
// 1st version
Put p = new Put(T1, ts-3);
@ -351,7 +357,8 @@ public class TestMinVersions extends HBaseTestCase {
HRegion region = createNewHRegion(htd, null, null);
final byte [] c1 = COLUMNS[1];
long ts = System.currentTimeMillis() - 2000; // 2s in the past
// 2s in the past
long ts = EnvironmentEdgeManager.currentTimeMillis() - 2000;
Put p = new Put(T1, ts-3);
p.add(c0, c0, T0);

View File

@ -24,13 +24,15 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.ScanQueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.junit.experimental.categories.Category;
@Category(SmallTests.class)
@ -98,7 +100,8 @@ public class TestQueryMatcher extends HBaseTestCase {
// 2,4,5
ScanQueryMatcher qm = new ScanQueryMatcher(scan, new Store.ScanInfo(fam2,
0, 1, ttl, false, 0, rowComparator), get.getFamilyMap().get(fam2));
0, 1, ttl, false, 0, rowComparator), get.getFamilyMap().get(fam2),
EnvironmentEdgeManager.currentTimeMillis() - ttl);
List<KeyValue> memstore = new ArrayList<KeyValue>();
memstore.add(new KeyValue(row1, fam2, col1, 1, data));
@ -142,7 +145,8 @@ public class TestQueryMatcher extends HBaseTestCase {
expected.add(ScanQueryMatcher.MatchCode.DONE);
ScanQueryMatcher qm = new ScanQueryMatcher(scan, new Store.ScanInfo(fam2,
0, 1, ttl, false, 0, rowComparator), null);
0, 1, ttl, false, 0, rowComparator), null,
EnvironmentEdgeManager.currentTimeMillis() - ttl);
List<KeyValue> memstore = new ArrayList<KeyValue>();
memstore.add(new KeyValue(row1, fam2, col1, 1, data));
@ -192,10 +196,11 @@ public class TestQueryMatcher extends HBaseTestCase {
ScanQueryMatcher.MatchCode.DONE
};
long now = EnvironmentEdgeManager.currentTimeMillis();
ScanQueryMatcher qm = new ScanQueryMatcher(scan, new Store.ScanInfo(fam2,
0, 1, testTTL, false, 0, rowComparator), get.getFamilyMap().get(fam2));
0, 1, testTTL, false, 0, rowComparator), get.getFamilyMap().get(fam2),
now - testTTL);
long now = System.currentTimeMillis();
KeyValue [] kvs = new KeyValue[] {
new KeyValue(row1, fam2, col1, now-100, data),
new KeyValue(row1, fam2, col2, now-50, data),
@ -244,10 +249,11 @@ public class TestQueryMatcher extends HBaseTestCase {
ScanQueryMatcher.MatchCode.DONE
};
long now = EnvironmentEdgeManager.currentTimeMillis();
ScanQueryMatcher qm = new ScanQueryMatcher(scan, new Store.ScanInfo(fam2,
0, 1, testTTL, false, 0, rowComparator), null);
0, 1, testTTL, false, 0, rowComparator), null,
now - testTTL);
long now = System.currentTimeMillis();
KeyValue [] kvs = new KeyValue[] {
new KeyValue(row1, fam2, col1, now-100, data),
new KeyValue(row1, fam2, col2, now-50, data),
@ -258,7 +264,8 @@ public class TestQueryMatcher extends HBaseTestCase {
};
qm.setRow(kvs[0].getRow());
List<ScanQueryMatcher.MatchCode> actual = new ArrayList<ScanQueryMatcher.MatchCode>(kvs.length);
List<ScanQueryMatcher.MatchCode> actual =
new ArrayList<ScanQueryMatcher.MatchCode>(kvs.length);
for (KeyValue kv : kvs) {
actual.add( qm.match(kv) );
}

View File

@ -36,7 +36,7 @@ public class TestScanWildcardColumnTracker extends HBaseTestCase {
public void testCheckColumn_Ok() throws IOException {
ScanWildcardColumnTracker tracker =
new ScanWildcardColumnTracker(0, VERSIONS, Long.MAX_VALUE);
new ScanWildcardColumnTracker(0, VERSIONS, Long.MIN_VALUE);
//Create list of qualifiers
List<byte[]> qualifiers = new ArrayList<byte[]>();
@ -68,7 +68,7 @@ public class TestScanWildcardColumnTracker extends HBaseTestCase {
public void testCheckColumn_EnforceVersions() throws IOException {
ScanWildcardColumnTracker tracker =
new ScanWildcardColumnTracker(0, VERSIONS, Long.MAX_VALUE);
new ScanWildcardColumnTracker(0, VERSIONS, Long.MIN_VALUE);
//Create list of qualifiers
List<byte[]> qualifiers = new ArrayList<byte[]>();
@ -101,7 +101,7 @@ public class TestScanWildcardColumnTracker extends HBaseTestCase {
public void DisabledTestCheckColumn_WrongOrder() {
ScanWildcardColumnTracker tracker =
new ScanWildcardColumnTracker(0, VERSIONS, Long.MAX_VALUE);
new ScanWildcardColumnTracker(0, VERSIONS, Long.MIN_VALUE);
//Create list of qualifiers
List<byte[]> qualifiers = new ArrayList<byte[]>();

View File

@ -27,7 +27,6 @@ import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
@ -355,7 +354,7 @@ public class TestStoreFile extends HBaseTestCase {
Scan scan = new Scan(row.getBytes(),row.getBytes());
scan.addColumn("family".getBytes(), "family:col".getBytes());
boolean exists = scanner.shouldSeek(scan, columns);
boolean exists = scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE);
if (i % 2 == 0) {
if (!exists) falseNeg++;
} else {
@ -501,7 +500,8 @@ public class TestStoreFile extends HBaseTestCase {
Scan scan = new Scan(row.getBytes(),row.getBytes());
scan.addColumn("family".getBytes(), ("col"+col).getBytes());
boolean exists = scanner.shouldSeek(scan, columns);
boolean exists =
scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE);
boolean shouldRowExist = i % 2 == 0;
boolean shouldColExist = j % 2 == 0;
shouldColExist = shouldColExist || bt[x] == StoreFile.BloomType.ROW;
@ -671,21 +671,20 @@ public class TestStoreFile extends HBaseTestCase {
columns.add(qualifier);
scan.setTimeRange(20, 100);
assertTrue(scanner.shouldSeek(scan, columns));
assertTrue(scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE));
scan.setTimeRange(1, 2);
assertTrue(scanner.shouldSeek(scan, columns));
assertTrue(scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE));
scan.setTimeRange(8, 10);
assertTrue(scanner.shouldSeek(scan, columns));
assertTrue(scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE));
scan.setTimeRange(7, 50);
assertTrue(scanner.shouldSeek(scan, columns));
assertTrue(scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE));
/*This test is not required for correctness but it should pass when
* timestamp range optimization is on*/
//scan.setTimeRange(27, 50);
//assertTrue(!scanner.shouldSeek(scan, columns));
// This test relies on the timestamp range optimization
scan.setTimeRange(27, 50);
assertTrue(!scanner.shouldUseScanner(scan, columns, Long.MIN_VALUE));
}
public void testCacheOnWriteEvictOnClose() throws Exception {

View File

@ -20,18 +20,8 @@
package org.apache.hadoop.hbase.regionserver;
import junit.framework.TestCase;
import org.apache.hadoop.hbase.*;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.Store.ScanInfo;
import org.apache.hadoop.hbase.regionserver.StoreScanner.ScanType;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.experimental.categories.Category;
import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.
scanFixture;
import java.io.IOException;
import java.util.ArrayList;
@ -39,7 +29,21 @@ import java.util.Arrays;
import java.util.List;
import java.util.NavigableSet;
import java.util.TreeSet;
import static org.apache.hadoop.hbase.regionserver.KeyValueScanFixture.scanFixture;
import junit.framework.TestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueTestUtil;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.Store.ScanInfo;
import org.apache.hadoop.hbase.regionserver.StoreScanner.ScanType;
import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdge;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.junit.experimental.categories.Category;
// Can't be small as it plays with EnvironmentEdgeManager
@Category(MediumTests.class)