HBASE-1578 Change the name of the in-memory updates from 'memcache' to 'memtable' or....

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@788160 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2009-06-24 19:56:51 +00:00
parent 0f0a133abf
commit 7886526f30
38 changed files with 530 additions and 614 deletions

View File

@ -397,6 +397,8 @@ Release 0.20.0 - Unreleased
row tables & performance
HBASE-1577 Move memcache to ConcurrentSkipListMap from
ConcurrentSkipListSet
HBASE-1578 Change the name of the in-memory updates from 'memcache' to
'memtable' or....
OPTIMIZATIONS
HBASE-1412 Change values for delete column and column family in KeyValue

View File

@ -217,9 +217,9 @@ module HBase
htd.setMaxFileSize(HTableDescriptor::DEFAULT_MAX_FILESIZE);
args[READONLY]? htd.setReadOnly(JBoolean.valueOf(args[READONLY])) :
htd.setReadOnly(HTableDescriptor::DEFAULT_READONLY);
args[MEMCACHE_FLUSHSIZE]?
htd.setMemcacheFlushSize(JLong.valueOf(args[MEMCACHE_FLUSHSIZE])) :
htd.setMemcacheFlushSize(HTableDescriptor::DEFAULT_MEMCACHE_FLUSH_SIZE);
args[MEMSTORE_FLUSHSIZE]?
htd.setMemStoreFlushSize(JLong.valueOf(args[MEMSTORE_FLUSHSIZE])) :
htd.setMemStoreFlushSize(HTableDescriptor::DEFAULT_MEMSTORE_FLUSH_SIZE);
@admin.modifyTable(tableName.to_java_bytes, htd)
else
descriptor = hcd(args)

View File

@ -156,7 +156,7 @@ HBASE SHELL COMMANDS:
hbase> alter 't1', {NAME => 'f1', METHOD => 'delete'}
You can also change table-scope attributes like MAX_FILESIZE
MEMCACHE_FLUSHSIZE and READONLY.
MEMSTORE_FLUSHSIZE and READONLY.
For example, to change the max size of a family to 128MB, do:
hbase> alter 't1', {METHOD => 'table_att', MAX_FILESIZE => '134217728'}

View File

@ -230,20 +230,20 @@
</description>
</property>
<property>
<name>hbase.regionserver.globalMemcache.upperLimit</name>
<name>hbase.regionserver.global.memstore.upperLimit</name>
<value>0.4</value>
<description>Maximum size of all memcaches in a region server before new
updates are blocked and flushes are forced. Defaults to 40% of heap.
<description>Maximum size of all memstores in a region server before new
updates are blocked and flushes are forced. Defaults to 40% of heap
</description>
</property>
<property>
<name>hbase.regionserver.globalMemcache.lowerLimit</name>
<name>hbase.regionserver.global.memstore.lowerLimit</name>
<value>0.25</value>
<description>When memcaches are being forced to flush to make room in
<description>When memstores are being forced to flush to make room in
memory, keep flushing until we hit this mark. Defaults to 30% of heap.
This value equal to hbase.regionserver.globalmemcache.upperLimit causes
This value equal to hbase.regionserver.global.memstore.upperLimit causes
the minimum possible flushing to occur when updates are blocked due to
memcache limiting.
memstore limiting.
</description>
</property>
<property>
@ -268,22 +268,22 @@
</description>
</property>
<property>
<name>hbase.hregion.memcache.flush.size</name>
<name>hbase.hregion.memstore.flush.size</name>
<value>67108864</value>
<description>
A HRegion memcache will be flushed to disk if size of the memcache
Memstore will be flushed to disk if size of the memstore
exceeds this number of bytes. Value is checked by a thread that runs
every hbase.server.thread.wakefrequency.
</description>
</property>
<property>
<name>hbase.hregion.memcache.block.multiplier</name>
<name>hbase.hregion.memstore.block.multiplier</name>
<value>2</value>
<description>
Block updates if memcache has hbase.hregion.block.memcache
Block updates if memstore has hbase.hregion.block.memstore
time hbase.hregion.flush.size bytes. Useful preventing
runaway memcache during spikes in update traffic. Without an
upper-bound, memcache fills such that when it flushes the
runaway memstore during spikes in update traffic. Without an
upper-bound, memstore fills such that when it flushes the
resultant flush files take a long time to compact or split, or
worse, we OOME.
</description>
@ -302,7 +302,7 @@
<value>3</value>
<description>
If more than this number of HStoreFiles in any one HStore
(one HStoreFile is written per flush of memcache) then a compaction
(one HStoreFile is written per flush of memstore) then a compaction
is run to rewrite all HStoreFiles files as one. Larger numbers
put off compaction but when it runs, it takes longer to complete.
During a compaction, updates cannot be flushed to disk. Long

View File

@ -49,7 +49,7 @@
</p>
<p>
If you enable the <em>hbase</em> context, on regionservers you'll see total requests since last
metric emission, count of regions and storefiles as well as a count of memcache size.
metric emission, count of regions and storefiles as well as a count of memstore size.
On the master, you'll see a count of the cluster's requests.
</p>
<p>

View File

@ -59,8 +59,8 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
private int stores;
/** the number of storefiles for the region */
private int storefiles;
/** the current size of the memcache for the region, in MB */
private int memcacheSizeMB;
/** the current size of the memstore for the region, in MB */
private int memstoreSizeMB;
/** the current total size of storefile indexes for the region, in MB */
private int storefileIndexSizeMB;
@ -75,16 +75,16 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
* @param name
* @param stores
* @param storefiles
* @param memcacheSizeMB
* @param memstoreSizeMB
* @param storefileIndexSizeMB
*/
public RegionLoad(final byte[] name, final int stores,
final int storefiles, final int memcacheSizeMB,
final int storefiles, final int memstoreSizeMB,
final int storefileIndexSizeMB) {
this.name = name;
this.stores = stores;
this.storefiles = storefiles;
this.memcacheSizeMB = memcacheSizeMB;
this.memstoreSizeMB = memstoreSizeMB;
this.storefileIndexSizeMB = storefileIndexSizeMB;
}
@ -119,10 +119,10 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
}
/**
* @return the memcache size, in MB
* @return the memstore size, in MB
*/
public int getMemcacheSizeMB() {
return memcacheSizeMB;
public int getMemStoreSizeMB() {
return memstoreSizeMB;
}
/**
@ -156,10 +156,10 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
}
/**
* @param memcacheSizeMB the memcache size, in MB
* @param memstoreSizeMB the memstore size, in MB
*/
public void setMemcacheSizeMB(int memcacheSizeMB) {
this.memcacheSizeMB = memcacheSizeMB;
public void setMemStoreSizeMB(int memstoreSizeMB) {
this.memstoreSizeMB = memstoreSizeMB;
}
/**
@ -177,7 +177,7 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
in.readFully(this.name);
this.stores = in.readInt();
this.storefiles = in.readInt();
this.memcacheSizeMB = in.readInt();
this.memstoreSizeMB = in.readInt();
this.storefileIndexSizeMB = in.readInt();
}
@ -186,7 +186,7 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
out.write(name);
out.writeInt(stores);
out.writeInt(storefiles);
out.writeInt(memcacheSizeMB);
out.writeInt(memstoreSizeMB);
out.writeInt(storefileIndexSizeMB);
}
@ -199,8 +199,8 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
Integer.valueOf(this.stores));
sb = Strings.appendKeyValue(sb, "storefiles",
Integer.valueOf(this.storefiles));
sb = Strings.appendKeyValue(sb, "memcacheSize",
Integer.valueOf(this.memcacheSizeMB));
sb = Strings.appendKeyValue(sb, "memstoreSize",
Integer.valueOf(this.memstoreSizeMB));
sb = Strings.appendKeyValue(sb, "storefileIndexSize",
Integer.valueOf(this.storefileIndexSizeMB));
return sb.toString();
@ -350,12 +350,12 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
}
/**
* @return Size of memcaches in MB
* @return Size of memstores in MB
*/
public int getMemcacheSizeInMB() {
public int getMemStoreSizeInMB() {
int count = 0;
for (RegionLoad info: regionLoad)
count += info.getMemcacheSizeMB();
count += info.getMemStoreSizeMB();
return count;
}
@ -411,16 +411,16 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
* @param name
* @param stores
* @param storefiles
* @param memcacheSizeMB
* @param memstoreSizeMB
* @param storefileIndexSizeMB
* @deprecated Use {@link #addRegionInfo(RegionLoad)}
*/
@Deprecated
public void addRegionInfo(final byte[] name, final int stores,
final int storefiles, final int memcacheSizeMB,
final int storefiles, final int memstoreSizeMB,
final int storefileIndexSizeMB) {
this.regionLoad.add(new HServerLoad.RegionLoad(name, stores, storefiles,
memcacheSizeMB, storefileIndexSizeMB));
memstoreSizeMB, storefileIndexSizeMB));
}
// Writable

View File

@ -70,9 +70,9 @@ ISerializable {
public static final String READONLY = "READONLY";
public static final ImmutableBytesWritable READONLY_KEY =
new ImmutableBytesWritable(Bytes.toBytes(READONLY));
public static final String MEMCACHE_FLUSHSIZE = "MEMCACHE_FLUSHSIZE";
public static final ImmutableBytesWritable MEMCACHE_FLUSHSIZE_KEY =
new ImmutableBytesWritable(Bytes.toBytes(MEMCACHE_FLUSHSIZE));
public static final String MEMSTORE_FLUSHSIZE = "MEMSTORE_FLUSHSIZE";
public static final ImmutableBytesWritable MEMSTORE_FLUSHSIZE_KEY =
new ImmutableBytesWritable(Bytes.toBytes(MEMSTORE_FLUSHSIZE));
public static final String IS_ROOT = "IS_ROOT";
public static final ImmutableBytesWritable IS_ROOT_KEY =
new ImmutableBytesWritable(Bytes.toBytes(IS_ROOT));
@ -94,7 +94,7 @@ ISerializable {
public static final boolean DEFAULT_READONLY = false;
public static final int DEFAULT_MEMCACHE_FLUSH_SIZE = 1024*1024*64;
public static final int DEFAULT_MEMSTORE_FLUSH_SIZE = 1024*1024*64;
public static final int DEFAULT_MAX_FILESIZE = 1024*1024*256;
@ -122,7 +122,7 @@ ISerializable {
for(HColumnDescriptor descriptor : families) {
this.families.put(descriptor.getName(), descriptor);
}
setMemcacheFlushSize(16 * 1024);
setMemStoreFlushSize(16 * 1024);
}
/**
@ -440,19 +440,19 @@ ISerializable {
/**
* @return memory cache flush size for each hregion
*/
public int getMemcacheFlushSize() {
byte [] value = getValue(MEMCACHE_FLUSHSIZE_KEY);
public int getMemStoreFlushSize() {
byte [] value = getValue(MEMSTORE_FLUSHSIZE_KEY);
if (value != null)
return Integer.valueOf(Bytes.toString(value)).intValue();
return DEFAULT_MEMCACHE_FLUSH_SIZE;
return DEFAULT_MEMSTORE_FLUSH_SIZE;
}
/**
* @param memcacheFlushSize memory cache flush size for each hregion
* @param memstoreFlushSize memory cache flush size for each hregion
*/
public void setMemcacheFlushSize(int memcacheFlushSize) {
setValue(MEMCACHE_FLUSHSIZE_KEY,
Bytes.toBytes(Integer.toString(memcacheFlushSize)));
public void setMemStoreFlushSize(int memstoreFlushSize) {
setValue(MEMSTORE_FLUSHSIZE_KEY,
Bytes.toBytes(Integer.toString(memstoreFlushSize)));
}
// public Collection<IndexSpecification> getIndexes() {

View File

@ -214,7 +214,7 @@ public class KeyValue implements Writable, HeapSize {
/**
* Lowest possible key.
* Makes a Key with highest possible Timestamp, empty row and column. No
* key can be equal or lower than this one in memcache or in store file.
* key can be equal or lower than this one in memstore or in store file.
*/
public static final KeyValue LOWESTKEY =
new KeyValue(HConstants.EMPTY_BYTE_ARRAY, HConstants.LATEST_TIMESTAMP);
@ -1771,7 +1771,7 @@ public class KeyValue implements Writable, HeapSize {
int compareTimestamps(final long ltimestamp, final long rtimestamp) {
// The below older timestamps sorting ahead of newer timestamps looks
// wrong but it is intentional. This way, newer timestamps are first
// found when we iterate over a memcache and newer versions are the
// found when we iterate over a memstore and newer versions are the
// first we trip over when reading from a store file.
if (ltimestamp < rtimestamp) {
return 1;

View File

@ -119,10 +119,10 @@ public class UnmodifyableHTableDescriptor extends HTableDescriptor {
}
/**
* @see org.apache.hadoop.hbase.HTableDescriptor#setMemcacheFlushSize(int)
* @see org.apache.hadoop.hbase.HTableDescriptor#setMemStoreFlushSize(int)
*/
@Override
public void setMemcacheFlushSize(int memcacheFlushSize) {
public void setMemStoreFlushSize(int memstoreFlushSize) {
throw new UnsupportedOperationException("HTableDescriptor is read-only");
}

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
* <ul><li>{@link #checkColumn} is called when a Put satisfies all other
* conditions of the query. This method returns a {@link MatchCode} to define
* what action should be taken.
* <li>{@link #update} is called at the end of every StoreFile or Memcache.
* <li>{@link #update} is called at the end of every StoreFile or memstore.
* <p>
* This class is NOT thread-safe as queries are never multi-threaded
*/

View File

@ -5,7 +5,7 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* Class that provides static method needed when putting deletes into memcache
* Class that provides static method needed when putting deletes into memstore
*/
public class DeleteCompare {
@ -14,7 +14,7 @@ public class DeleteCompare {
*/
enum DeleteCode {
/**
* Do nothing. Move to next KV in Memcache
* Do nothing. Move to next KV in memstore
*/
SKIP,
@ -24,13 +24,13 @@ public class DeleteCompare {
DELETE,
/**
* Stop looking at KVs in Memcache. Finalize.
* Stop looking at KVs in memstore. Finalize.
*/
DONE
}
/**
* Method used when putting deletes into memcache to remove all the previous
* Method used when putting deletes into memstore to remove all the previous
* entries that are affected by this Delete
* @param mem
* @param deleteBuffer

View File

@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.util.Bytes;
* <ul><li>{@link #checkColumn} is called when a Put satisfies all other
* conditions of the query. This method returns a {@link MatchCode} to define
* what action should be taken.
* <li>{@link #update} is called at the end of every StoreFile or Memcache.
* <li>{@link #update} is called at the end of every StoreFile or memstore.
* <p>
* This class is NOT thread-safe as queries are never multi-threaded
*/
@ -129,7 +129,7 @@ public class ExplicitColumnTracker implements ColumnTracker {
}
/**
* Called at the end of every StoreFile or Memcache.
* Called at the end of every StoreFile or memstore.
*/
public void update() {
if(this.columns.size() != 0) {

View File

@ -24,7 +24,7 @@ package org.apache.hadoop.hbase.regionserver;
* Implementors of this interface want to be notified when an HRegion
* determines that a cache flush is needed. A FlushRequester (or null)
* must be passed to the HRegion constructor so it knows who to call when it
* has a filled memcache.
* has a filled memstore.
*/
public interface FlushRequester {
/**

View File

@ -709,7 +709,7 @@ public class HLog implements HConstants, Syncable {
* Abort a cache flush.
* Call if the flush fails. Note that the only recovery for an aborted flush
* currently is a restart of the regionserver so the snapshot content dropped
* by the failure gets restored to the memcache.
* by the failure gets restored to the memstore.
*/
void abortCacheFlush() {
this.cacheFlushLock.unlock();

View File

@ -125,7 +125,7 @@ public class HRegion implements HConstants { // , Writable{
// private int [] storeSize = null;
// private byte [] name = null;
final AtomicLong memcacheSize = new AtomicLong(0);
final AtomicLong memstoreSize = new AtomicLong(0);
// This is the table subdirectory.
final Path basedir;
@ -148,7 +148,7 @@ public class HRegion implements HConstants { // , Writable{
* compactions and closes.
*/
static class WriteState {
// Set while a memcache flush is happening.
// Set while a memstore flush is happening.
volatile boolean flushing = false;
// Set when a flush has been requested.
volatile boolean flushRequested = false;
@ -178,10 +178,10 @@ public class HRegion implements HConstants { // , Writable{
private volatile WriteState writestate = new WriteState();
final int memcacheFlushSize;
final int memstoreFlushSize;
private volatile long lastFlushTime;
final FlushRequester flushListener;
private final int blockingMemcacheSize;
private final int blockingMemStoreSize;
final long threadWakeFrequency;
// Used to guard splits and closes
private final ReentrantReadWriteLock splitsAndClosesLock =
@ -212,12 +212,12 @@ public class HRegion implements HConstants { // , Writable{
*/
public HRegion(){
this.basedir = null;
this.blockingMemcacheSize = 0;
this.blockingMemStoreSize = 0;
this.conf = null;
this.flushListener = null;
this.fs = null;
this.historian = null;
this.memcacheFlushSize = 0;
this.memstoreFlushSize = 0;
this.log = null;
this.regionCompactionDir = null;
this.regiondir = null;
@ -265,14 +265,14 @@ public class HRegion implements HConstants { // , Writable{
}
this.regionCompactionDir =
new Path(getCompactionDir(basedir), encodedNameStr);
int flushSize = regionInfo.getTableDesc().getMemcacheFlushSize();
if (flushSize == HTableDescriptor.DEFAULT_MEMCACHE_FLUSH_SIZE) {
flushSize = conf.getInt("hbase.hregion.memcache.flush.size",
HTableDescriptor.DEFAULT_MEMCACHE_FLUSH_SIZE);
int flushSize = regionInfo.getTableDesc().getMemStoreFlushSize();
if (flushSize == HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE) {
flushSize = conf.getInt("hbase.hregion.memstore.flush.size",
HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
}
this.memcacheFlushSize = flushSize;
this.blockingMemcacheSize = this.memcacheFlushSize *
conf.getInt("hbase.hregion.memcache.block.multiplier", 1);
this.memstoreFlushSize = flushSize;
this.blockingMemStoreSize = this.memstoreFlushSize *
conf.getInt("hbase.hregion.memstore.block.multiplier", 1);
}
/**
@ -811,7 +811,7 @@ public class HRegion implements HConstants { // , Writable{
this.writestate.flushing = true;
} else {
if(LOG.isDebugEnabled()) {
LOG.debug("NOT flushing memcache for region " + this +
LOG.debug("NOT flushing memstore for region " + this +
", flushing=" +
writestate.flushing + ", writesEnabled=" +
writestate.writesEnabled);
@ -838,23 +838,23 @@ public class HRegion implements HConstants { // , Writable{
/**
* Flushing the cache is a little tricky. We have a lot of updates in the
* HMemcache, all of which have also been written to the log. We need to
* write those updates in the HMemcache out to disk, while being able to
* memstore, all of which have also been written to the log. We need to
* write those updates in the memstore out to disk, while being able to
* process reads/writes as much as possible during the flush operation. Also,
* the log has to state clearly the point in time at which the HMemcache was
* the log has to state clearly the point in time at which the memstore was
* flushed. (That way, during recovery, we know when we can rely on the
* on-disk flushed structures and when we have to recover the HMemcache from
* on-disk flushed structures and when we have to recover the memstore from
* the log.)
*
* <p>So, we have a three-step process:
*
* <ul><li>A. Flush the memcache to the on-disk stores, noting the current
* <ul><li>A. Flush the memstore to the on-disk stores, noting the current
* sequence ID for the log.<li>
*
* <li>B. Write a FLUSHCACHE-COMPLETE message to the log, using the sequence
* ID that was current at the time of memcache-flush.</li>
* ID that was current at the time of memstore-flush.</li>
*
* <li>C. Get rid of the memcache structures that are now redundant, as
* <li>C. Get rid of the memstore structures that are now redundant, as
* they've been flushed to the on-disk HStores.</li>
* </ul>
* <p>This method is protected, but can be accessed via several public
@ -874,27 +874,27 @@ public class HRegion implements HConstants { // , Writable{
// Record latest flush time
this.lastFlushTime = startTime;
// If nothing to flush, return and avoid logging start/stop flush.
if (this.memcacheSize.get() <= 0) {
if (this.memstoreSize.get() <= 0) {
return false;
}
if (LOG.isDebugEnabled()) {
LOG.debug("Started memcache flush for region " + this +
". Current region memcache size " +
StringUtils.humanReadableInt(this.memcacheSize.get()));
LOG.debug("Started memstore flush for region " + this +
". Current region memstore size " +
StringUtils.humanReadableInt(this.memstoreSize.get()));
}
// Stop updates while we snapshot the memcache of all stores. We only have
// Stop updates while we snapshot the memstore of all stores. We only have
// to do this for a moment. Its quick. The subsequent sequence id that
// goes into the HLog after we've flushed all these snapshots also goes
// into the info file that sits beside the flushed files.
// We also set the memcache size to zero here before we allow updates
// We also set the memstore size to zero here before we allow updates
// again so its value will represent the size of the updates received
// during the flush
long sequenceId = -1L;
long completeSequenceId = -1L;
this.updatesLock.writeLock().lock();
// Get current size of memcaches.
final long currentMemcacheSize = this.memcacheSize.get();
// Get current size of memstores.
final long currentMemStoreSize = this.memstoreSize.get();
try {
for (Store s: stores.values()) {
s.snapshot();
@ -906,12 +906,12 @@ public class HRegion implements HConstants { // , Writable{
}
// Any failure from here on out will be catastrophic requiring server
// restart so hlog content can be replayed and put back into the memcache.
// restart so hlog content can be replayed and put back into the memstore.
// Otherwise, the snapshot content while backed up in the hlog, it will not
// be part of the current running servers state.
boolean compactionRequested = false;
try {
// A. Flush memcache to all the HStores.
// A. Flush memstore to all the HStores.
// Keep running vector of all store files that includes both old and the
// just-made new flush store file.
for (Store hstore: stores.values()) {
@ -920,11 +920,11 @@ public class HRegion implements HConstants { // , Writable{
compactionRequested = true;
}
}
// Set down the memcache size by amount of flush.
this.memcacheSize.addAndGet(-currentMemcacheSize);
// Set down the memstore size by amount of flush.
this.memstoreSize.addAndGet(-currentMemStoreSize);
} catch (Throwable t) {
// An exception here means that the snapshot was not persisted.
// The hlog needs to be replayed so its content is restored to memcache.
// The hlog needs to be replayed so its content is restored to memstore.
// Currently, only a server restart will do this.
// We used to only catch IOEs but its possible that we'd get other
// exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
@ -946,7 +946,7 @@ public class HRegion implements HConstants { // , Writable{
this.log.completeCacheFlush(getRegionName(),
regionInfo.getTableDesc().getName(), completeSequenceId);
// C. Finally notify anyone waiting on memcache to clear:
// C. Finally notify anyone waiting on memstore to clear:
// e.g. checkResources().
synchronized (this) {
notifyAll();
@ -955,8 +955,8 @@ public class HRegion implements HConstants { // , Writable{
if (LOG.isDebugEnabled()) {
long now = System.currentTimeMillis();
String timeTaken = StringUtils.formatTimeDiff(now, startTime);
LOG.debug("Finished memcache flush of ~" +
StringUtils.humanReadableInt(currentMemcacheSize) + " for region " +
LOG.debug("Finished memstore flush of ~" +
StringUtils.humanReadableInt(currentMemStoreSize) + " for region " +
this + " in " + (now - startTime) + "ms, sequence id=" + sequenceId +
", compaction requested=" + compactionRequested);
if (!regionInfo.isMetaRegion()) {
@ -1161,7 +1161,7 @@ public class HRegion implements HConstants { // , Writable{
kv.updateLatestStamp(byteNow);
}
size = this.memcacheSize.addAndGet(store.delete(kv));
size = this.memstoreSize.addAndGet(store.delete(kv));
}
flush = isFlushSize(size);
} finally {
@ -1369,15 +1369,15 @@ public class HRegion implements HConstants { // , Writable{
*/
private void checkResources() {
boolean blocked = false;
while (this.memcacheSize.get() > this.blockingMemcacheSize) {
while (this.memstoreSize.get() > this.blockingMemStoreSize) {
requestFlush();
if (!blocked) {
LOG.info("Blocking updates for '" + Thread.currentThread().getName() +
"' on region " + Bytes.toString(getRegionName()) +
": Memcache size " +
StringUtils.humanReadableInt(this.memcacheSize.get()) +
": memstore size " +
StringUtils.humanReadableInt(this.memstoreSize.get()) +
" is >= than blocking " +
StringUtils.humanReadableInt(this.blockingMemcacheSize) + " size");
StringUtils.humanReadableInt(this.blockingMemStoreSize) + " size");
}
blocked = true;
synchronized(this) {
@ -1404,7 +1404,7 @@ public class HRegion implements HConstants { // , Writable{
}
/**
* Add updates first to the hlog and then add values to memcache.
* Add updates first to the hlog and then add values to memstore.
* Warning: Assumption is caller has lock on passed in row.
* @param edits Cell updates by column
* @praram now
@ -1416,7 +1416,7 @@ public class HRegion implements HConstants { // , Writable{
}
/**
* Add updates first to the hlog (if writeToWal) and then add values to memcache.
* Add updates first to the hlog (if writeToWal) and then add values to memstore.
* Warning: Assumption is caller has lock on passed in row.
* @param family
* @param edits
@ -1440,7 +1440,7 @@ public class HRegion implements HConstants { // , Writable{
long size = 0;
Store store = getStore(family);
for (KeyValue kv: edits) {
size = this.memcacheSize.addAndGet(store.add(kv));
size = this.memstoreSize.addAndGet(store.add(kv));
}
flush = isFlushSize(size);
} finally {
@ -1474,7 +1474,7 @@ public class HRegion implements HConstants { // , Writable{
* @return True if size is over the flush threshold
*/
private boolean isFlushSize(final long size) {
return size > this.memcacheFlushSize;
return size > this.memstoreFlushSize;
}
protected Store instantiateHStore(Path baseDir,
@ -2271,7 +2271,7 @@ public class HRegion implements HConstants { // , Writable{
store.incrementColumnValue(row, family, qualifier, amount);
result = vas.value;
long size = this.memcacheSize.addAndGet(vas.sizeAdded);
long size = this.memstoreSize.addAndGet(vas.sizeAdded);
flush = isFlushSize(size);
} finally {
releaseRowLock(lid);

View File

@ -196,7 +196,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
CompactSplitThread compactSplitThread;
// Cache flushing
MemcacheFlusher cacheFlusher;
MemStoreFlusher cacheFlusher;
/* Check for major compactions.
*/
@ -322,7 +322,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
this.workerThread = new Thread(worker);
// Cache flushing thread.
this.cacheFlusher = new MemcacheFlusher(conf, this);
this.cacheFlusher = new MemStoreFlusher(conf, this);
// Compaction thread
this.compactSplitThread = new CompactSplitThread(this);
@ -756,7 +756,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
byte[] name = r.getRegionName();
int stores = 0;
int storefiles = 0;
int memcacheSizeMB = (int)(r.memcacheSize.get()/1024/1024);
int memstoreSizeMB = (int)(r.memstoreSize.get()/1024/1024);
int storefileIndexSizeMB = 0;
synchronized (r.stores) {
stores += r.stores.size();
@ -766,7 +766,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
(int)(store.getStorefilesIndexSize()/1024/1024);
}
}
return new HServerLoad.RegionLoad(name, stores, storefiles, memcacheSizeMB,
return new HServerLoad.RegionLoad(name, stores, storefiles, memstoreSizeMB,
storefileIndexSizeMB);
}
@ -1061,12 +1061,12 @@ public class HRegionServer implements HConstants, HRegionInterface,
// the synchronizations?
int stores = 0;
int storefiles = 0;
long memcacheSize = 0;
long memstoreSize = 0;
long storefileIndexSize = 0;
synchronized (this.onlineRegions) {
for (Map.Entry<Integer, HRegion> e: this.onlineRegions.entrySet()) {
HRegion r = e.getValue();
memcacheSize += r.memcacheSize.get();
memstoreSize += r.memstoreSize.get();
synchronized (r.stores) {
stores += r.stores.size();
for(Map.Entry<byte [], Store> ee: r.stores.entrySet()) {
@ -1079,7 +1079,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
}
this.metrics.stores.set(stores);
this.metrics.storefiles.set(storefiles);
this.metrics.memcacheSizeMB.set((int)(memcacheSize/(1024*1024)));
this.metrics.memstoreSizeMB.set((int)(memstoreSize/(1024*1024)));
this.metrics.storefileIndexSizeMB.set((int)(storefileIndexSize/(1024*1024)));
LruBlockCache lruBlockCache = (LruBlockCache)StoreFile.getBlockCache(conf);
@ -1741,7 +1741,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
this.requestCount.incrementAndGet();
HRegion region = getRegion(regionName);
try {
cacheFlusher.reclaimMemcacheMemory();
cacheFlusher.reclaimMemStoreMemory();
region.put(put, getLockFromId(put.getLockId()));
} catch (Throwable t) {
throw convertThrowableToIOE(cleanup(t));
@ -1754,7 +1754,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
checkOpen();
try {
HRegion region = getRegion(regionName);
this.cacheFlusher.reclaimMemcacheMemory();
this.cacheFlusher.reclaimMemStoreMemory();
Integer[] locks = new Integer[puts.length];
for (i = 0; i < puts.length; i++) {
this.requestCount.incrementAndGet();
@ -1794,7 +1794,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
this.requestCount.incrementAndGet();
HRegion region = getRegion(regionName);
try {
cacheFlusher.reclaimMemcacheMemory();
cacheFlusher.reclaimMemStoreMemory();
return region.checkAndPut(row, family, qualifier, value, put,
getLockFromId(put.getLockId()), true);
} catch (Throwable t) {
@ -1932,7 +1932,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
checkOpen();
try {
boolean writeToWAL = true;
this.cacheFlusher.reclaimMemcacheMemory();
this.cacheFlusher.reclaimMemStoreMemory();
this.requestCount.incrementAndGet();
Integer lock = getLockFromId(delete.getLockId());
HRegion region = getRegion(regionName);
@ -2164,7 +2164,7 @@ public class HRegionServer implements HConstants, HRegionInterface,
// Copy over all regions. Regions are sorted by size with biggest first.
synchronized (this.onlineRegions) {
for (HRegion region : this.onlineRegions.values()) {
sortedRegions.put(Long.valueOf(region.memcacheSize.get()), region);
sortedRegions.put(Long.valueOf(region.memstoreSize.get()), region);
}
}
return sortedRegions;
@ -2288,14 +2288,14 @@ public class HRegionServer implements HConstants, HRegionInterface,
}
/**
* Return the total size of all memcaches in every region.
* @return memcache size in bytes
* Return the total size of all memstores in every region.
* @return memstore size in bytes
*/
public long getGlobalMemcacheSize() {
public long getGlobalMemStoreSize() {
long total = 0;
synchronized (onlineRegions) {
for (HRegion region : onlineRegions.values()) {
total += region.memcacheSize.get();
total += region.memstoreSize.get();
}
}
return total;

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator;
* Implements KeyValueScanner itself.
* <p>
* This class is used at the Region level to merge across Stores
* and at the Store level to merge across the Memcache and StoreFiles.
* and at the Store level to merge across the memstore and StoreFiles.
* <p>
* In the Region case, we also need InternalScanner.next(List), so this class
* also implements InternalScanner. WARNING: As is, if you try to use this

View File

@ -43,27 +43,27 @@ import org.apache.hadoop.hbase.regionserver.DeleteCompare.DeleteCode;
import org.apache.hadoop.hbase.util.Bytes;
/**
* The Memcache holds in-memory modifications to the HRegion. Modifications
* are {@link KeyValue}s. When asked to flush, current memcache is moved
* to snapshot and is cleared. We continue to serve edits out of new memcache
* The MemStore holds in-memory modifications to the Store. Modifications
* are {@link KeyValue}s. When asked to flush, current memstore is moved
* to snapshot and is cleared. We continue to serve edits out of new memstore
* and backing snapshot until flusher reports in that the flush succeeded. At
* this point we let the snapshot go.
* TODO: Adjust size of the memcache when we remove items because they have
* TODO: Adjust size of the memstore when we remove items because they have
* been deleted.
*/
class Memcache {
private static final Log LOG = LogFactory.getLog(Memcache.class);
class MemStore {
private static final Log LOG = LogFactory.getLog(MemStore.class);
private final long ttl;
// Memcache. Use a SkipListMap rather than SkipListSet because of the
// MemStore. Use a SkipListMap rather than SkipListSet because of the
// better semantics. The Map will overwrite if passed a key it already had
// whereas the Set will not add new KV if key is same though value might be
// different. Value is not important -- just make sure always same
// reference passed.
volatile ConcurrentSkipListMap<KeyValue, Object> memcache;
volatile ConcurrentSkipListMap<KeyValue, Object> memstore;
// Snapshot of memcache. Made for flusher.
// Snapshot of memstore. Made for flusher.
volatile ConcurrentSkipListMap<KeyValue, Object> snapshot;
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
@ -87,7 +87,7 @@ class Memcache {
/**
* Default constructor. Used for tests.
*/
public Memcache() {
public MemStore() {
this(HConstants.FOREVER, KeyValue.COMPARATOR);
}
@ -96,13 +96,13 @@ class Memcache {
* @param ttl The TTL for cache entries, in milliseconds.
* @param c
*/
public Memcache(final long ttl, final KeyValue.KVComparator c) {
public MemStore(final long ttl, final KeyValue.KVComparator c) {
this.ttl = ttl;
this.comparator = c;
this.comparatorIgnoreTimestamp =
this.comparator.getComparatorIgnoringTimestamps();
this.comparatorIgnoreType = this.comparator.getComparatorIgnoringType();
this.memcache = createMap(c);
this.memstore = createMap(c);
this.snapshot = createMap(c);
}
@ -111,7 +111,7 @@ class Memcache {
}
void dump() {
for (Map.Entry<KeyValue, ?> entry: this.memcache.entrySet()) {
for (Map.Entry<KeyValue, ?> entry: this.memstore.entrySet()) {
LOG.info(entry.getKey());
}
for (Map.Entry<KeyValue, ?> entry: this.snapshot.entrySet()) {
@ -120,7 +120,7 @@ class Memcache {
}
/**
* Creates a snapshot of the current Memcache.
* Creates a snapshot of the current memstore.
* Snapshot must be cleared by call to {@link #clearSnapshot(java.util.Map)}
* To get the snapshot made by this method, use {@link #getSnapshot()}
*/
@ -133,12 +133,12 @@ class Memcache {
LOG.warn("Snapshot called again without clearing previous. " +
"Doing nothing. Another ongoing flush or did we fail last attempt?");
} else {
// We used to synchronize on the memcache here but we're inside a
// We used to synchronize on the memstore here but we're inside a
// write lock so removed it. Comment is left in case removal was a
// mistake. St.Ack
if (!this.memcache.isEmpty()) {
this.snapshot = this.memcache;
this.memcache = createMap(this.comparator);
if (!this.memstore.isEmpty()) {
this.snapshot = this.memstore;
this.memstore = createMap(this.comparator);
}
}
} finally {
@ -193,7 +193,7 @@ class Memcache {
try {
// Add anything as value as long as same instance each time.
size = heapSize(kv,
this.memcache.put(kv, NULL) == null);
this.memstore.put(kv, NULL) == null);
} finally {
this.lock.readLock().unlock();
}
@ -211,7 +211,7 @@ class Memcache {
//Have to find out what we want to do here, to find the fastest way of
//removing things that are under a delete.
//Actions that will take place here are:
//1. Insert a delete and remove all the affected entries already in memcache
//1. Insert a delete and remove all the affected entries already in memstore
//2. In the case of a Delete and the matching put is found then don't insert
// the delete
//TODO Would be nice with if we had an iterator for this, so we could remove
@ -221,7 +221,7 @@ class Memcache {
try {
boolean notpresent = false;
List<KeyValue> deletes = new ArrayList<KeyValue>();
SortedMap<KeyValue, Object> tail = this.memcache.tailMap(delete);
SortedMap<KeyValue, Object> tail = this.memstore.tailMap(delete);
//Parse the delete, so that it is only done once
byte [] deleteBuffer = delete.getBuffer();
@ -250,7 +250,7 @@ class Memcache {
deleteOffset += Bytes.SIZEOF_LONG;
byte deleteType = deleteBuffer[deleteOffset];
//Comparing with tail from memcache
//Comparing with tail from memstore
for (Map.Entry<KeyValue, ?> entry : tail.entrySet()) {
DeleteCode res = DeleteCompare.deleteCompare(entry.getKey(),
deleteBuffer,
@ -266,13 +266,13 @@ class Memcache {
//Delete all the entries effected by the last added delete
for(KeyValue del : deletes) {
notpresent = this.memcache.remove(del) == null;
notpresent = this.memstore.remove(del) == null;
size -= heapSize(del, notpresent);
}
// Adding the delete to memcache. Add any value, as long as
// Adding the delete to memstore. Add any value, as long as
// same instance each time.
size += heapSize(delete, this.memcache.put(delete, NULL) == null);
size += heapSize(delete, this.memstore.put(delete, NULL) == null);
} finally {
this.lock.readLock().unlock();
}
@ -280,7 +280,7 @@ class Memcache {
}
/*
* Calculate how the memcache size has changed, approximately. Be careful.
* Calculate how the memstore size has changed, approximately. Be careful.
* If class changes, be sure to change the size calculation.
* Add in tax of Map.Entry.
* @param kv
@ -302,7 +302,7 @@ class Memcache {
KeyValue getNextRow(final KeyValue kv) {
this.lock.readLock().lock();
try {
return getLowest(getNextRow(kv, this.memcache),
return getLowest(getNextRow(kv, this.memstore),
getNextRow(kv, this.snapshot));
} finally {
this.lock.readLock().unlock();
@ -350,7 +350,7 @@ class Memcache {
/**
* @param row Row to look for.
* @param candidateKeys Map of candidate keys (Accumulation over lots of
* lookup over stores and memcaches)
* lookup over stores and memstores)
*/
void getRowKeyAtOrBefore(final KeyValue row,
final NavigableSet<KeyValue> candidateKeys) {
@ -361,7 +361,7 @@ class Memcache {
/**
* @param kv Row to look for.
* @param candidates Map of candidate keys (Accumulation over lots of
* lookup over stores and memcaches). Pass a Set with a Comparator that
* lookup over stores and memstores). Pass a Set with a Comparator that
* ignores key Type so we can do Set.remove using a delete, i.e. a KeyValue
* with a different Type to the candidate key.
* @param deletes Pass a Set that has a Comparator that ignores key type.
@ -372,7 +372,7 @@ class Memcache {
final NavigableSet<KeyValue> deletes, final long now) {
this.lock.readLock().lock();
try {
getRowKeyAtOrBefore(memcache, kv, candidates, deletes, now);
getRowKeyAtOrBefore(memstore, kv, candidates, deletes, now);
getRowKeyAtOrBefore(snapshot, kv, candidates, deletes, now);
} finally {
this.lock.readLock().unlock();
@ -544,14 +544,14 @@ class Memcache {
}
/**
* @return scanner on memcache and snapshot in this order.
* @return scanner on memstore and snapshot in this order.
*/
KeyValueScanner [] getScanners() {
this.lock.readLock().lock();
try {
KeyValueScanner [] scanners = new KeyValueScanner[2];
scanners[0] = new MemcacheScanner(this.memcache);
scanners[1] = new MemcacheScanner(this.snapshot);
scanners[0] = new MemStoreScanner(this.memstore);
scanners[1] = new MemStoreScanner(this.snapshot);
return scanners;
} finally {
this.lock.readLock().unlock();
@ -563,7 +563,7 @@ class Memcache {
//
/**
* Perform a single-row Get on the memcache and snapshot, placing results
* Perform a single-row Get on the and snapshot, placing results
* into the specified KV list.
* <p>
* This will return true if it is determined that the query is complete
@ -579,7 +579,7 @@ class Memcache {
throws IOException {
this.lock.readLock().lock();
try {
if(internalGet(this.memcache, matcher, result) || matcher.isDone()) {
if(internalGet(this.memstore, matcher, result) || matcher.isDone()) {
return true;
}
matcher.update();
@ -591,7 +591,7 @@ class Memcache {
/**
*
* @param map memcache or snapshot
* @param map memstore or snapshot
* @param matcher query matcher
* @param result list to add results to
* @return true if done with store (early-out), false if not
@ -624,18 +624,18 @@ class Memcache {
/*
* MemcacheScanner implements the KeyValueScanner.
* It lets the caller scan the contents of a memcache.
* MemStoreScanner implements the KeyValueScanner.
* It lets the caller scan the contents of a memstore.
* This behaves as if it were a real scanner but does not maintain position
* in the passed memcache tree.
* in the passed memstore tree.
*/
protected class MemcacheScanner implements KeyValueScanner {
protected class MemStoreScanner implements KeyValueScanner {
private final NavigableMap<KeyValue, Object> mc;
private KeyValue current = null;
private List<KeyValue> result = new ArrayList<KeyValue>();
private int idx = 0;
MemcacheScanner(final NavigableMap<KeyValue, Object> mc) {
MemStoreScanner(final NavigableMap<KeyValue, Object> mc) {
this.mc = mc;
}
@ -713,7 +713,7 @@ class Memcache {
/**
* Code to help figure if our approximation of object heap sizes is close
* enough. See hbase-900. Fills memcaches then waits so user can heap
* enough. See hbase-900. Fills memstores then waits so user can heap
* dump and bring up resultant hprof in something like jprofiler which
* allows you get 'deep size' on objects.
* @param args
@ -723,27 +723,27 @@ class Memcache {
LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
LOG.info("vmInputArguments=" + runtime.getInputArguments());
Memcache memcache1 = new Memcache();
MemStore memstore1 = new MemStore();
// TODO: x32 vs x64
long size = 0;
final int count = 10000;
byte [] column = Bytes.toBytes("col:umn");
for (int i = 0; i < count; i++) {
// Give each its own ts
size += memcache1.add(new KeyValue(Bytes.toBytes(i), column, i));
size += memstore1.add(new KeyValue(Bytes.toBytes(i), column, i));
}
LOG.info("memcache1 estimated size=" + size);
LOG.info("memstore1 estimated size=" + size);
for (int i = 0; i < count; i++) {
size += memcache1.add(new KeyValue(Bytes.toBytes(i), column, i));
size += memstore1.add(new KeyValue(Bytes.toBytes(i), column, i));
}
LOG.info("memcache1 estimated size (2nd loading of same data)=" + size);
// Make a variably sized memcache.
Memcache memcache2 = new Memcache();
LOG.info("memstore1 estimated size (2nd loading of same data)=" + size);
// Make a variably sized memstore.
MemStore memstore2 = new MemStore();
for (int i = 0; i < count; i++) {
size += memcache2.add(new KeyValue(Bytes.toBytes(i), column, i,
size += memstore2.add(new KeyValue(Bytes.toBytes(i), column, i,
new byte[i]));
}
LOG.info("memcache2 estimated size=" + size);
LOG.info("memstore2 estimated size=" + size);
final int seconds = 30;
LOG.info("Waiting " + seconds + " seconds while heap dump is taken");
for (int i = 0; i < seconds; i++) {

View File

@ -48,8 +48,8 @@ import org.apache.hadoop.util.StringUtils;
*
* @see FlushRequester
*/
class MemcacheFlusher extends Thread implements FlushRequester {
static final Log LOG = LogFactory.getLog(MemcacheFlusher.class);
class MemStoreFlusher extends Thread implements FlushRequester {
static final Log LOG = LogFactory.getLog(MemStoreFlusher.class);
private final BlockingQueue<HRegion> flushQueue =
new LinkedBlockingQueue<HRegion>();
@ -59,15 +59,15 @@ class MemcacheFlusher extends Thread implements FlushRequester {
private final HRegionServer server;
private final ReentrantLock lock = new ReentrantLock();
protected final long globalMemcacheLimit;
protected final long globalMemcacheLimitLowMark;
protected final long globalMemStoreLimit;
protected final long globalMemStoreLimitLowMark;
private static final float DEFAULT_UPPER = 0.4f;
private static final float DEFAULT_LOWER = 0.25f;
private static final String UPPER_KEY =
"hbase.regionserver.globalMemcache.upperLimit";
"hbase.regionserver.global.memstore.upperLimit";
private static final String LOWER_KEY =
"hbase.regionserver.globalMemcache.lowerLimit";
"hbase.regionserver.global.memstore.lowerLimit";
private long blockingStoreFilesNumber;
private long blockingWaitTime;
@ -75,22 +75,22 @@ class MemcacheFlusher extends Thread implements FlushRequester {
* @param conf
* @param server
*/
public MemcacheFlusher(final HBaseConfiguration conf,
public MemStoreFlusher(final HBaseConfiguration conf,
final HRegionServer server) {
super();
this.server = server;
this.threadWakeFrequency =
conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
long max = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getMax();
this.globalMemcacheLimit = globalMemcacheLimit(max, DEFAULT_UPPER,
this.globalMemStoreLimit = globalMemStoreLimit(max, DEFAULT_UPPER,
UPPER_KEY, conf);
long lower = globalMemcacheLimit(max, DEFAULT_LOWER, LOWER_KEY, conf);
if (lower > this.globalMemcacheLimit) {
lower = this.globalMemcacheLimit;
LOG.info("Setting globalMemcacheLimitLowMark == globalMemcacheLimit " +
long lower = globalMemStoreLimit(max, DEFAULT_LOWER, LOWER_KEY, conf);
if (lower > this.globalMemStoreLimit) {
lower = this.globalMemStoreLimit;
LOG.info("Setting globalMemStoreLimitLowMark == globalMemStoreLimit " +
"because supplied " + LOWER_KEY + " was > " + UPPER_KEY);
}
this.globalMemcacheLimitLowMark = lower;
this.globalMemStoreLimitLowMark = lower;
this.blockingStoreFilesNumber =
conf.getInt("hbase.hstore.blockingStoreFiles", -1);
if (this.blockingStoreFilesNumber == -1) {
@ -99,10 +99,10 @@ class MemcacheFlusher extends Thread implements FlushRequester {
}
this.blockingWaitTime = conf.getInt("hbase.hstore.blockingWaitTime",
90000); // default of 180 seconds
LOG.info("globalMemcacheLimit=" +
StringUtils.humanReadableInt(this.globalMemcacheLimit) +
", globalMemcacheLimitLowMark=" +
StringUtils.humanReadableInt(this.globalMemcacheLimitLowMark) +
LOG.info("globalMemStoreLimit=" +
StringUtils.humanReadableInt(this.globalMemStoreLimit) +
", globalMemStoreLimitLowMark=" +
StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark) +
", maxHeap=" + StringUtils.humanReadableInt(max));
}
@ -115,16 +115,16 @@ class MemcacheFlusher extends Thread implements FlushRequester {
* @param c
* @return Limit.
*/
static long globalMemcacheLimit(final long max,
static long globalMemStoreLimit(final long max,
final float defaultLimit, final String key, final HBaseConfiguration c) {
float limit = c.getFloat(key, defaultLimit);
return getMemcacheLimit(max, limit, defaultLimit);
return getMemStoreLimit(max, limit, defaultLimit);
}
static long getMemcacheLimit(final long max, final float limit,
static long getMemStoreLimit(final long max, final float limit,
final float defaultLimit) {
if (limit >= 0.9f || limit < 0.1f) {
LOG.warn("Setting global memcache limit to default of " + defaultLimit +
LOG.warn("Setting global memstore limit to default of " + defaultLimit +
" because supplied value outside allowed range of 0.1 -> 0.9");
}
return (long)(max * limit);
@ -292,13 +292,13 @@ class MemcacheFlusher extends Thread implements FlushRequester {
}
/**
* Check if the regionserver's memcache memory usage is greater than the
* limit. If so, flush regions with the biggest memcaches until we're down
* Check if the regionserver's memstore memory usage is greater than the
* limit. If so, flush regions with the biggest memstores until we're down
* to the lower limit. This method blocks callers until we're down to a safe
* amount of memcache consumption.
* amount of memstore consumption.
*/
public synchronized void reclaimMemcacheMemory() {
if (server.getGlobalMemcacheSize() >= globalMemcacheLimit) {
public synchronized void reclaimMemStoreMemory() {
if (server.getGlobalMemStoreSize() >= globalMemStoreLimit) {
flushSomeRegions();
}
}
@ -308,33 +308,33 @@ class MemcacheFlusher extends Thread implements FlushRequester {
*/
private synchronized void flushSomeRegions() {
// keep flushing until we hit the low water mark
long globalMemcacheSize = -1;
long globalMemStoreSize = -1;
ArrayList<HRegion> regionsToCompact = new ArrayList<HRegion>();
for (SortedMap<Long, HRegion> m =
this.server.getCopyOfOnlineRegionsSortedBySize();
(globalMemcacheSize = server.getGlobalMemcacheSize()) >=
this.globalMemcacheLimitLowMark;) {
// flush the region with the biggest memcache
(globalMemStoreSize = server.getGlobalMemStoreSize()) >=
this.globalMemStoreLimitLowMark;) {
// flush the region with the biggest memstore
if (m.size() <= 0) {
LOG.info("No online regions to flush though we've been asked flush " +
"some; globalMemcacheSize=" +
StringUtils.humanReadableInt(globalMemcacheSize) +
", globalMemcacheLimitLowMark=" +
StringUtils.humanReadableInt(this.globalMemcacheLimitLowMark));
"some; globalMemStoreSize=" +
StringUtils.humanReadableInt(globalMemStoreSize) +
", globalMemStoreLimitLowMark=" +
StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark));
break;
}
HRegion biggestMemcacheRegion = m.remove(m.firstKey());
LOG.info("Forced flushing of " + biggestMemcacheRegion.toString() +
" because global memcache limit of " +
StringUtils.humanReadableInt(this.globalMemcacheLimit) +
HRegion biggestMemStoreRegion = m.remove(m.firstKey());
LOG.info("Forced flushing of " + biggestMemStoreRegion.toString() +
" because global memstore limit of " +
StringUtils.humanReadableInt(this.globalMemStoreLimit) +
" exceeded; currently " +
StringUtils.humanReadableInt(globalMemcacheSize) + " and flushing till " +
StringUtils.humanReadableInt(this.globalMemcacheLimitLowMark));
if (!flushRegion(biggestMemcacheRegion, true)) {
StringUtils.humanReadableInt(globalMemStoreSize) + " and flushing till " +
StringUtils.humanReadableInt(this.globalMemStoreLimitLowMark));
if (!flushRegion(biggestMemStoreRegion, true)) {
LOG.warn("Flush failed");
break;
}
regionsToCompact.add(biggestMemcacheRegion);
regionsToCompact.add(biggestMemStoreRegion);
}
for (HRegion region : regionsToCompact) {
server.compactSplitThread.compactionRequested(region, getName());

View File

@ -20,19 +20,17 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.util.Bytes;
import java.util.List;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.hfile.HFile;
/**
* A scanner that does a minor compaction at the same time. Doesn't need to
* implement ChangedReadersObserver, since it doesn't scan memcache, only store files
* and optionally the memcache-snapshot.
* implement ChangedReadersObserver, since it doesn't scan memstore, only store files
* and optionally the memstore-snapshot.
*/
public class MinorCompactingStoreScanner implements KeyValueScanner, InternalScanner {

View File

@ -49,10 +49,10 @@ import org.apache.hadoop.hbase.util.Bytes;
public class QueryMatcher {
/**
* {@link #match} return codes. These instruct the scanner moving through
* Memcaches and StoreFiles what to do with the current KeyValue.
* memstores and StoreFiles what to do with the current KeyValue.
* <p>
* Additionally, this contains "early-out" language to tell the scanner to
* move on to the next File (Memcache or Storefile), or to return immediately.
* move on to the next File (memstore or Storefile), or to return immediately.
*/
static enum MatchCode {
/**
@ -66,7 +66,7 @@ public class QueryMatcher {
SKIP,
/**
* Do not include, jump to next StoreFile or Memcache (in time order)
* Do not include, jump to next StoreFile or memstore (in time order)
*/
NEXT,
@ -100,7 +100,7 @@ public class QueryMatcher {
/** Keeps track of columns and versions */
protected ColumnTracker columns;
/** Key to seek to in Memcache and StoreFiles */
/** Key to seek to in memstore and StoreFiles */
protected KeyValue startKey;
/** Row comparator for the region this query is for */
@ -309,7 +309,7 @@ public class QueryMatcher {
}
/**
* Called after reading each section (memcache, snapshot, storefiles).
* Called after reading each section (memstore, snapshot, storefiles).
* <p>
* This method will update the internal structures to be accurate for
* the next section.

View File

@ -59,7 +59,7 @@ import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.StringUtils;
/**
* A Store holds a column family in a Region. Its a memcache and a set of zero
* A Store holds a column family in a Region. Its a memstore and a set of zero
* or more StoreFiles, which stretch backwards over time.
*
* <p>There's no reason to consider append-logging at this level; all logging
@ -89,7 +89,7 @@ public class Store implements HConstants {
* If no delimiter present, presume the buffer holds a store name so no need
* of a delimiter.
*/
protected final Memcache memcache;
protected final MemStore memstore;
// This stores directory in the filesystem.
private final Path homedir;
private final HRegionInfo regioninfo;
@ -176,7 +176,7 @@ public class Store implements HConstants {
// second -> ms adjust for user data
this.ttl *= 1000;
}
this.memcache = new Memcache(this.ttl, this.comparator);
this.memstore = new MemStore(this.ttl, this.comparator);
this.regionCompactionDir = new Path(HRegion.getCompactionDir(basedir),
Integer.toString(info.getEncodedName()));
this.storeName = this.family.getName();
@ -289,7 +289,7 @@ public class Store implements HConstants {
long maxSeqIdInLog = -1;
// TODO: Move this memstoring over into MemStore.
ConcurrentSkipListMap<KeyValue, Object> reconstructedCache =
Memcache.createMap(this.comparator);
MemStore.createMap(this.comparator);
SequenceFile.Reader logReader = new SequenceFile.Reader(this.fs,
reconstructionLog, this.conf);
try {
@ -387,30 +387,30 @@ public class Store implements HConstants {
}
/**
* Adds a value to the memcache
* Adds a value to the memstore
*
* @param kv
* @return memcache size delta
* @return memstore size delta
*/
protected long add(final KeyValue kv) {
lock.readLock().lock();
try {
return this.memcache.add(kv);
return this.memstore.add(kv);
} finally {
lock.readLock().unlock();
}
}
/**
* Adds a value to the memcache
* Adds a value to the memstore
*
* @param kv
* @return memcache size delta
* @return memstore size delta
*/
protected long delete(final KeyValue kv) {
lock.readLock().lock();
try {
return this.memcache.delete(kv);
return this.memstore.delete(kv);
} finally {
lock.readLock().unlock();
}
@ -449,11 +449,11 @@ public class Store implements HConstants {
}
/**
* Snapshot this stores memcache. Call before running
* Snapshot this stores memstore. Call before running
* {@link #flushCache(long)} so it has some work to do.
*/
void snapshot() {
this.memcache.snapshot();
this.memstore.snapshot();
}
/**
@ -465,10 +465,10 @@ public class Store implements HConstants {
*/
boolean flushCache(final long logCacheFlushId) throws IOException {
// Get the snapshot to flush. Presumes that a call to
// this.memcache.snapshot() has happened earlier up in the chain.
ConcurrentSkipListMap<KeyValue, ?> cache = this.memcache.getSnapshot();
// this.memstore.snapshot() has happened earlier up in the chain.
ConcurrentSkipListMap<KeyValue, ?> cache = this.memstore.getSnapshot();
// If an exception happens flushing, we let it out without clearing
// the memcache snapshot. The old snapshot will be returned when we say
// the memstore snapshot. The old snapshot will be returned when we say
// 'snapshot', the next time flush comes around.
StoreFile sf = internalFlushCache(cache, logCacheFlushId);
if (sf == null) {
@ -509,7 +509,7 @@ public class Store implements HConstants {
if (!isExpired(kv, oldestTimestamp)) {
writer.append(kv);
entries++;
flushed += this.memcache.heapSize(kv, true);
flushed += this.memstore.heapSize(kv, true);
}
}
// B. Write out the log sequence number that corresponds to this output
@ -568,7 +568,7 @@ public class Store implements HConstants {
count = this.storefiles.size();
// Tell listeners of the change in readers.
notifyChangedReadersObservers();
this.memcache.clearSnapshot(cache);
this.memstore.clearSnapshot(cache);
return count;
} finally {
this.lock.writeLock().unlock();
@ -612,7 +612,7 @@ public class Store implements HConstants {
* thread must be able to block for long periods.
*
* <p>During this time, the Store can work as usual, getting values from
* MapFiles and writing new MapFiles from the Memcache.
* MapFiles and writing new MapFiles from the memstore.
*
* Existing MapFiles are not destroyed until the new compacted TreeMap is
* completely written-out to disk.
@ -1014,8 +1014,8 @@ public class Store implements HConstants {
long now = System.currentTimeMillis();
this.lock.readLock().lock();
try {
// First go to the memcache. Pick up deletes and candidates.
this.memcache.getRowKeyAtOrBefore(targetkey, candidates, deletes, now);
// First go to the memstore. Pick up deletes and candidates.
this.memstore.getRowKeyAtOrBefore(targetkey, candidates, deletes, now);
// Process each store file. Run through from newest to oldest.
Map<Long, StoreFile> m = this.storefiles.descendingMap();
for (Map.Entry<Long, StoreFile> e: m.entrySet()) {
@ -1340,7 +1340,7 @@ public class Store implements HConstants {
//////////////////////////////////////////////////////////////////////////////
/**
* Return a scanner for both the memcache and the HStore files
* Return a scanner for both the memstore and the HStore files
*/
protected KeyValueScanner getScanner(Scan scan,
final NavigableSet<byte []> targetCols) {
@ -1470,9 +1470,9 @@ public class Store implements HConstants {
QueryMatcher matcher = new QueryMatcher(get, this.family.getName(), columns,
this.ttl, keyComparator, versionsToReturn(get.getMaxVersions()));
// Read from Memcache
if(this.memcache.get(matcher, result)) {
// Received early-out from memcache
// Read from memstore
if(this.memstore.get(matcher, result)) {
// Received early-out from memstore
return;
}
@ -1526,9 +1526,9 @@ public class Store implements HConstants {
QueryMatcher matcher = new QueryMatcher(get, f, qualifiers, this.ttl,
keyComparator, 1);
// Read from Memcache
if(this.memcache.get(matcher, result)) {
// Received early-out from memcache
// Read from memstore
if(this.memstore.get(matcher, result)) {
// Received early-out from memstore
KeyValue kv = result.get(0);
byte [] buffer = kv.getBuffer();
int valueOffset = kv.getValueOffset();

View File

@ -44,7 +44,7 @@ import java.util.regex.Pattern;
/**
* A Store data file. Stores usually have one or more of these files. They
* are produced by flushing the memcache to disk. To
* are produced by flushing the memstore to disk. To
* create, call {@link #getWriter(FileSystem, Path)} and append data. Be
* sure to add any metadata before calling close on the Writer
* (Use the appendMetadata convenience methods). On close, a StoreFile is

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
/**
* Scanner scans both the memcache and the HStore. Coaleace KeyValue stream
* Scanner scans both the memstore and the HStore. Coaleace KeyValue stream
* into List<KeyValue> for a single row.
*/
class StoreScanner implements KeyValueScanner, InternalScanner, ChangedReadersObserver {
@ -47,7 +47,7 @@ class StoreScanner implements KeyValueScanner, InternalScanner, ChangedReadersOb
private final AtomicBoolean closing = new AtomicBoolean(false);
/**
* Opens a scanner across memcache, snapshot, and all StoreFiles.
* Opens a scanner across memstore, snapshot, and all StoreFiles.
*/
StoreScanner(Store store, Scan scan, final NavigableSet<byte[]> columns) {
this.store = store;
@ -90,9 +90,9 @@ class StoreScanner implements KeyValueScanner, InternalScanner, ChangedReadersOb
*/
private List<KeyValueScanner> getScanners() {
List<KeyValueScanner> scanners = getStoreFileScanners();
KeyValueScanner [] memcachescanners = this.store.memcache.getScanners();
for (int i = memcachescanners.length - 1; i >= 0; i--) {
scanners.add(memcachescanners[i]);
KeyValueScanner [] memstorescanners = this.store.memstore.getScanners();
for (int i = memstorescanners.length - 1; i >= 0; i--) {
scanners.add(memstorescanners[i]);
}
return scanners;
}

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes;
* <ul><li>{@link #checkColumn} is called when a Put satisfies all other
* conditions of the query. This method returns a {@link MatchCode} to define
* what action should be taken.
* <li>{@link #update} is called at the end of every StoreFile or Memcache.
* <li>{@link #update} is called at the end of every StoreFile or memstore.
* <p>
* This class is NOT thread-safe as queries are never multi-threaded
*/
@ -236,7 +236,7 @@ public class WildcardColumnTracker implements ColumnTracker {
}
/**
* Called at the end of every StoreFile or Memcache.
* Called at the end of every StoreFile or memstore.
*/
public void update() {
// If no previous columns, use new columns and return

View File

@ -99,10 +99,10 @@ public class RegionServerMetrics implements Updater {
new MetricsIntValue("storefileIndexSizeMB", registry);
/**
* Sum of all the memcache sizes in this regionserver in MB
* Sum of all the memstore sizes in this regionserver in MB
*/
public final MetricsIntValue memcacheSizeMB =
new MetricsIntValue("memcacheSizeMB", registry);
public final MetricsIntValue memstoreSizeMB =
new MetricsIntValue("memstoreSizeMB", registry);
public RegionServerMetrics() {
MetricsContext context = MetricsUtil.getContext("hbase");
@ -129,7 +129,7 @@ public class RegionServerMetrics implements Updater {
this.stores.pushMetric(this.metricsRecord);
this.storefiles.pushMetric(this.metricsRecord);
this.storefileIndexSizeMB.pushMetric(this.metricsRecord);
this.memcacheSizeMB.pushMetric(this.metricsRecord);
this.memstoreSizeMB.pushMetric(this.metricsRecord);
this.regions.pushMetric(this.metricsRecord);
this.requests.pushMetric(this.metricsRecord);
@ -177,8 +177,8 @@ public class RegionServerMetrics implements Updater {
Integer.valueOf(this.storefiles.get()));
sb = Strings.appendKeyValue(sb, "storefileIndexSize",
Integer.valueOf(this.storefileIndexSizeMB.get()));
sb = Strings.appendKeyValue(sb, "memcacheSize",
Integer.valueOf(this.memcacheSizeMB.get()));
sb = Strings.appendKeyValue(sb, "memstoreSize",
Integer.valueOf(this.memstoreSizeMB.get()));
// Duplicate from jvmmetrics because metrics are private there so
// inaccessible.
MemoryUsage memory =

View File

@ -1,73 +0,0 @@
/**
* Copyright 2007 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;
import junit.framework.TestSuite;
import junit.textui.TestRunner;
import org.apache.hadoop.hbase.client.HBaseAdmin;
/**
* Test ability of HBase to handle DFS failure
*/
public class DFSAbort extends HBaseClusterTestCase {
/** constructor */
public DFSAbort() {
super();
// For less frequently updated regions flush after every 2 flushes
conf.setInt("hbase.hregion.memcache.optionalflushcount", 2);
}
@Override
public void setUp() throws Exception {
try {
super.setUp();
HTableDescriptor desc = new HTableDescriptor(getName());
desc.addFamily(new HColumnDescriptor(HConstants.CATALOG_FAMILY));
HBaseAdmin admin = new HBaseAdmin(conf);
admin.createTable(desc);
} catch (Exception e) {
e.printStackTrace();
throw e;
}
}
/**
* @throws Exception
*/
public void testDFSAbort() throws Exception {
try {
// By now the Mini DFS is running, Mini HBase is running and we have
// created a table. Now let's yank the rug out from HBase
dfsCluster.shutdown();
threadDumpingJoin();
} catch (Exception e) {
e.printStackTrace();
throw e;
}
}
/**
* @param args unused
*/
public static void main(String[] args) {
TestRunner.run(new TestSuite(DFSAbort.class));
}
}

View File

@ -221,7 +221,7 @@ public class TestClassMigration extends TestCase {
}
// The below older timestamps sorting ahead of newer timestamps looks
// wrong but it is intentional. This way, newer timestamps are first
// found when we iterate over a memcache and newer versions are the
// found when we iterate over a memstore and newer versions are the
// first we trip over when reading from a store file.
if (this.timestamp < other.timestamp) {
result = 1;

View File

@ -215,10 +215,10 @@ public class TestClient extends HBaseClusterTestCase {
scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
////////////////////////////////////////////////////////////////////////////
// Flush memcache and run same tests from storefiles
// Flush memstore and run same tests from storefiles
////////////////////////////////////////////////////////////////////////////
flushMemcache(TABLE);
flushMemStore(TABLE);
// Redo get and scan tests from storefile
@ -228,7 +228,7 @@ public class TestClient extends HBaseClusterTestCase {
scanVerifySingleEmpty(ht, ROWS, 0, FAMILIES, 4, QUALIFIERS, 0);
////////////////////////////////////////////////////////////////////////////
// Now, Test reading from memcache and storefiles at once
// Now, Test reading from memstore and storefiles at once
////////////////////////////////////////////////////////////////////////////
// Insert multiple columns to two other families
@ -253,14 +253,14 @@ public class TestClient extends HBaseClusterTestCase {
// Flush the table again
////////////////////////////////////////////////////////////////////////////
flushMemcache(TABLE);
flushMemStore(TABLE);
// Redo tests again
singleRowGetTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
singleRowScanTest(ht, ROWS, FAMILIES, QUALIFIERS, VALUES);
// Insert more data to memcache
// Insert more data to memstore
put = new Put(ROWS[0]);
put.add(FAMILIES[6], QUALIFIERS[5], VALUES[5]);
@ -315,7 +315,7 @@ public class TestClient extends HBaseClusterTestCase {
assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[8], VALUES[8]);
////////////////////////////////////////////////////////////////////////////
// Delete a memcache column
// Delete a memstore column
////////////////////////////////////////////////////////////////////////////
delete = new Delete(ROWS[0]);
delete.deleteColumns(FAMILIES[6], QUALIFIERS[8]);
@ -360,7 +360,7 @@ public class TestClient extends HBaseClusterTestCase {
assertSingleResult(result, ROWS[0], FAMILIES[6], QUALIFIERS[9], VALUES[9]);
////////////////////////////////////////////////////////////////////////////
// Delete joint storefile/memcache family
// Delete joint storefile/memstore family
////////////////////////////////////////////////////////////////////////////
delete = new Delete(ROWS[0]);
@ -374,7 +374,7 @@ public class TestClient extends HBaseClusterTestCase {
result = ht.get(get);
assertEmptyResult(result);
// Try to get memcache column in deleted family
// Try to get memstore column in deleted family
get = new Get(ROWS[0]);
get.addColumn(FAMILIES[4], QUALIFIERS[3]);
result = ht.get(get);
@ -393,7 +393,7 @@ public class TestClient extends HBaseClusterTestCase {
result = getSingleScanResult(ht, scan);
assertNullResult(result);
// Try to scan memcache column in deleted family
// Try to scan memstore column in deleted family
scan = new Scan();
scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
result = getSingleScanResult(ht, scan);
@ -433,7 +433,7 @@ public class TestClient extends HBaseClusterTestCase {
// Flush everything and rerun delete tests
////////////////////////////////////////////////////////////////////////////
flushMemcache(TABLE);
flushMemStore(TABLE);
// Try to get storefile column in deleted family
@ -442,7 +442,7 @@ public class TestClient extends HBaseClusterTestCase {
result = ht.get(get);
assertEmptyResult(result);
// Try to get memcache column in deleted family
// Try to get memstore column in deleted family
get = new Get(ROWS[0]);
get.addColumn(FAMILIES[4], QUALIFIERS[3]);
result = ht.get(get);
@ -461,7 +461,7 @@ public class TestClient extends HBaseClusterTestCase {
result = getSingleScanResult(ht, scan);
assertNullResult(result);
// Try to scan memcache column in deleted family
// Try to scan memstore column in deleted family
scan = new Scan();
scan.addColumn(FAMILIES[4], QUALIFIERS[3]);
result = getSingleScanResult(ht, scan);
@ -572,7 +572,7 @@ public class TestClient extends HBaseClusterTestCase {
// Flush and try again
flushMemcache(TABLE2);
flushMemStore(TABLE2);
getTestNull(ht, ROW, FAMILY, VALUE);
@ -681,7 +681,7 @@ public class TestClient extends HBaseClusterTestCase {
// Flush and redo
flushMemcache(TABLE);
flushMemStore(TABLE);
// Verify we can get each one properly
getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
@ -721,7 +721,7 @@ public class TestClient extends HBaseClusterTestCase {
0, 1);
// Add some memcache and retest
// Add some memstore and retest
// Insert 4 more versions of same column and a dupe
put = new Put(ROW);
@ -784,7 +784,7 @@ public class TestClient extends HBaseClusterTestCase {
// Ensure maxVersions of table is respected
flushMemcache(TABLE);
flushMemStore(TABLE);
// Insert 4 more versions of same column and a dupe
put = new Put(ROW);
@ -812,7 +812,7 @@ public class TestClient extends HBaseClusterTestCase {
new byte[][] {VALUES[3], VALUES[4], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
0, 9);
// Delete a version in the memcache and a version in a storefile
// Delete a version in the memstore and a version in a storefile
Delete delete = new Delete(ROW);
delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]);
delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]);
@ -1406,7 +1406,7 @@ public class TestClient extends HBaseClusterTestCase {
// flush and try again
flushMemcache(TABLE);
flushMemStore(TABLE);
// Get a row
get = new Get(ROWS[numRows-1]);
@ -1466,12 +1466,12 @@ public class TestClient extends HBaseClusterTestCase {
getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
// Try same from storefile
flushMemcache(TABLE);
flushMemStore(TABLE);
getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[5]);
// Insert two more versions surrounding others, into memcache
// Insert two more versions surrounding others, into memstore
put = new Put(ROW);
put.add(FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
put.add(FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
@ -1487,7 +1487,7 @@ public class TestClient extends HBaseClusterTestCase {
getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[6], VALUES[6]);
// Try same from two storefiles
flushMemcache(TABLE);
flushMemStore(TABLE);
getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[0], VALUES[0]);
getVersionAndVerifyMissing(ht, ROW, FAMILY, QUALIFIER, STAMPS[1]);
getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[2], VALUES[2]);
@ -1533,7 +1533,7 @@ public class TestClient extends HBaseClusterTestCase {
scanVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 3);
// Try same from storefile
flushMemcache(TABLE);
flushMemStore(TABLE);
getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
getVersionRangeAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 2);
@ -1603,7 +1603,7 @@ public class TestClient extends HBaseClusterTestCase {
scanVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 4, 5);
// Try same from storefile
flushMemcache(TABLE);
flushMemStore(TABLE);
getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
getVersionRangeAndVerifyGreaterThan(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 2, 5);
@ -1644,7 +1644,7 @@ public class TestClient extends HBaseClusterTestCase {
scanAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
// Try same from storefile
flushMemcache(TABLE);
flushMemStore(TABLE);
getAllVersionsAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS, VALUES, 0, 5);
@ -1821,7 +1821,7 @@ public class TestClient extends HBaseClusterTestCase {
byte [][] QUALIFIERS, byte [][] VALUES)
throws Exception {
// Single column from memcache
// Single column from memstore
Get get = new Get(ROWS[0]);
get.addColumn(FAMILIES[4], QUALIFIERS[0]);
Result result = ht.get(get);
@ -1839,7 +1839,7 @@ public class TestClient extends HBaseClusterTestCase {
result = ht.get(get);
assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
// Two columns, one from memcache one from storefile, same family,
// Two columns, one from memstore one from storefile, same family,
// wildcard match
get = new Get(ROWS[0]);
get.addFamily(FAMILIES[4]);
@ -1847,7 +1847,7 @@ public class TestClient extends HBaseClusterTestCase {
assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
FAMILIES[4], QUALIFIERS[4], VALUES[4]);
// Two columns, one from memcache one from storefile, same family,
// Two columns, one from memstore one from storefile, same family,
// explicit match
get = new Get(ROWS[0]);
get.addColumn(FAMILIES[4], QUALIFIERS[0]);
@ -1856,7 +1856,7 @@ public class TestClient extends HBaseClusterTestCase {
assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
FAMILIES[4], QUALIFIERS[4], VALUES[4]);
// Three column, one from memcache two from storefile, different families,
// Three column, one from memstore two from storefile, different families,
// wildcard match
get = new Get(ROWS[0]);
get.addFamily(FAMILIES[4]);
@ -1919,7 +1919,7 @@ public class TestClient extends HBaseClusterTestCase {
byte [][] QUALIFIERS, byte [][] VALUES)
throws Exception {
// Single column from memcache
// Single column from memstore
Scan scan = new Scan();
scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
Result result = getSingleScanResult(ht, scan);
@ -1937,7 +1937,7 @@ public class TestClient extends HBaseClusterTestCase {
result = getSingleScanResult(ht, scan);
assertSingleResult(result, ROWS[0], FAMILIES[7], QUALIFIERS[7], VALUES[7]);
// Two columns, one from memcache one from storefile, same family,
// Two columns, one from memstore one from storefile, same family,
// wildcard match
scan = new Scan();
scan.addFamily(FAMILIES[4]);
@ -1945,7 +1945,7 @@ public class TestClient extends HBaseClusterTestCase {
assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
FAMILIES[4], QUALIFIERS[4], VALUES[4]);
// Two columns, one from memcache one from storefile, same family,
// Two columns, one from memstore one from storefile, same family,
// explicit match
scan = new Scan();
scan.addColumn(FAMILIES[4], QUALIFIERS[0]);
@ -1954,7 +1954,7 @@ public class TestClient extends HBaseClusterTestCase {
assertDoubleResult(result, ROWS[0], FAMILIES[4], QUALIFIERS[0], VALUES[0],
FAMILIES[4], QUALIFIERS[4], VALUES[4]);
// Three column, one from memcache two from storefile, different families,
// Three column, one from memstore two from storefile, different families,
// wildcard match
scan = new Scan();
scan.addFamily(FAMILIES[4]);
@ -2378,7 +2378,7 @@ public class TestClient extends HBaseClusterTestCase {
// Helpers
//
private void flushMemcache(byte [] tableName) throws Exception {
private void flushMemStore(byte [] tableName) throws Exception {
System.out.println("\n\nFlushing table [" + Bytes.toString(tableName) + "]...\n");
// HBaseAdmin hba = new HBaseAdmin(conf);
// hba.flush(tableName);
@ -2561,7 +2561,7 @@ public class TestClient extends HBaseClusterTestCase {
// Flush and redo
flushMemcache(TABLE);
flushMemStore(TABLE);
// Verify we can get each one properly
getVersionAndVerify(ht, ROW, FAMILY, QUALIFIER, STAMPS[1], VALUES[1]);
@ -2601,7 +2601,7 @@ public class TestClient extends HBaseClusterTestCase {
0, 1);
// Add some memcache and retest
// Add some memstore and retest
// Insert 4 more versions of same column and a dupe
put = new Put(ROW);
@ -2665,7 +2665,7 @@ public class TestClient extends HBaseClusterTestCase {
// Ensure maxVersions of table is respected
flushMemcache(TABLE);
flushMemStore(TABLE);
// Insert 4 more versions of same column and a dupe
put = new Put(ROW);
@ -2693,7 +2693,7 @@ public class TestClient extends HBaseClusterTestCase {
new byte[][] {VALUES[3], VALUES[14], VALUES[5], VALUES[6], VALUES[7], VALUES[8], VALUES[9], VALUES[11], VALUES[13], VALUES[15]},
0, 9);
// Delete a version in the memcache and a version in a storefile
// Delete a version in the memstore and a version in a storefile
Delete delete = new Delete(ROW);
delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[11]);
delete.deleteColumn(FAMILY, QUALIFIER, STAMPS[7]);

View File

@ -22,14 +22,10 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.io.Cell;
import org.apache.hadoop.hbase.io.RowResult;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -85,7 +81,7 @@ public class TestScannerTimes extends HBaseClusterTestCase {
long times[] = new long[3];
// First scan the memcache
// First scan the memstore
Scan scan = new Scan();
scan.addFamily(FAM1);

View File

@ -54,8 +54,8 @@ public class TestCompaction extends HBaseTestCase {
super();
// Set cache flush size to 1MB
conf.setInt("hbase.hregion.memcache.flush.size", 1024*1024);
conf.setInt("hbase.hregion.memcache.block.multiplier", 10);
conf.setInt("hbase.hregion.memstore.flush.size", 1024*1024);
conf.setInt("hbase.hregion.memstore.block.multiplier", 10);
this.cluster = null;
}
@ -85,7 +85,7 @@ public class TestCompaction extends HBaseTestCase {
}
/**
* Run compaction and flushing memcache
* Run compaction and flushing memstore
* Assert deletes get cleaned up.
* @throws Exception
*/
@ -129,7 +129,7 @@ public class TestCompaction extends HBaseTestCase {
// cellValues.length);
// assertTrue(cellValues.length == 3);
// Now add deletes to memcache and then flush it. That will put us over
// Now add deletes to memstore and then flush it. That will put us over
// the compaction threshold of 3 store files. Compacting these store files
// should result in a compacted store file that has no references to the
// deleted row.

View File

@ -20,18 +20,18 @@ public class TestDeleteCompare extends TestCase {
//3. Delete and the matching put
//4. Big test that include starting on the wrong row and qualifier
public void testDeleteCompare_DeleteFamily() {
//Creating memcache
Set<KeyValue> memcache = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col2", 1, "d-c"));
//Creating memstore
Set<KeyValue> memstore = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col2", 1, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col3", 3, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col3", 2, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col3", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col3", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col3", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col3", 1, "d-c"));
memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 1, "d-c"));
//Creating expected result
List<DeleteCode> expected = new ArrayList<DeleteCode>();
@ -55,7 +55,7 @@ public class TestDeleteCompare extends TestCase {
byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG];
List<DeleteCode> actual = new ArrayList<DeleteCode>();
for(KeyValue mem : memcache){
for(KeyValue mem : memstore){
actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset,
deleteRowLen, deleteQualifierOffset, deleteQualifierLen,
deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR));
@ -69,12 +69,12 @@ public class TestDeleteCompare extends TestCase {
}
public void testDeleteCompare_DeleteColumn() {
//Creating memcache
Set<KeyValue> memcache = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 1, "d-c"));
//Creating memstore
Set<KeyValue> memstore = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 1, "d-c"));
//Creating expected result
@ -95,7 +95,7 @@ public class TestDeleteCompare extends TestCase {
byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG];
List<DeleteCode> actual = new ArrayList<DeleteCode>();
for(KeyValue mem : memcache){
for(KeyValue mem : memstore){
actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset,
deleteRowLen, deleteQualifierOffset, deleteQualifierLen,
deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR));
@ -110,11 +110,11 @@ public class TestDeleteCompare extends TestCase {
public void testDeleteCompare_Delete() {
//Creating memcache
Set<KeyValue> memcache = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c"));
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
//Creating memstore
Set<KeyValue> memstore = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
//Creating expected result
List<DeleteCode> expected = new ArrayList<DeleteCode>();
@ -133,7 +133,7 @@ public class TestDeleteCompare extends TestCase {
byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG];
List<DeleteCode> actual = new ArrayList<DeleteCode>();
for(KeyValue mem : memcache){
for(KeyValue mem : memstore){
actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset,
deleteRowLen, deleteQualifierOffset, deleteQualifierLen,
deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR));
@ -146,15 +146,15 @@ public class TestDeleteCompare extends TestCase {
}
public void testDeleteCompare_Multiple() {
//Creating memcache
Set<KeyValue> memcache = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memcache.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 4, "d-c"));
memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 3, "d-c"));
memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 2, "d-c"));
memcache.add(KeyValueTestUtil.create("row21", "fam", "col1", 1,
//Creating memstore
Set<KeyValue> memstore = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
memstore.add(KeyValueTestUtil.create("row11", "fam", "col1", 1, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 4, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 3, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 2, "d-c"));
memstore.add(KeyValueTestUtil.create("row21", "fam", "col1", 1,
KeyValue.Type.Delete, "dont-care"));
memcache.add(KeyValueTestUtil.create("row31", "fam", "col1", 1, "dont-care"));
memstore.add(KeyValueTestUtil.create("row31", "fam", "col1", 1, "dont-care"));
//Creating expected result
List<DeleteCode> expected = new ArrayList<DeleteCode>();
@ -176,7 +176,7 @@ public class TestDeleteCompare extends TestCase {
byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG];
List<DeleteCode> actual = new ArrayList<DeleteCode>();
for(KeyValue mem : memcache){
for(KeyValue mem : memstore){
actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset,
deleteRowLen, deleteQualifierOffset, deleteQualifierLen,
deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR));

View File

@ -24,7 +24,6 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.Iterator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -37,11 +36,11 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
import org.apache.hadoop.hbase.regionserver.HRegion.RegionScanner;
import org.apache.hadoop.hbase.util.Bytes;
@ -192,11 +191,11 @@ public class TestHRegion extends HBaseTestCase {
//checkAndPut with wrong value
Store store = region.getStore(fam1);
int size = store.memcache.memcache.size();
int size = store.memstore.memstore.size();
boolean res = region.checkAndPut(row1, fam1, qf1, val1, put, lockId, true);
assertEquals(true, res);
size = store.memcache.memcache.size();
size = store.memstore.memstore.size();
Get get = new Get(row1);
get.addColumn(fam2, qf1);
@ -412,14 +411,14 @@ public class TestHRegion extends HBaseTestCase {
region.delete(fam1, kvs, true);
// extract the key values out the memcache:
// extract the key values out the memstore:
// This is kinda hacky, but better than nothing...
long now = System.currentTimeMillis();
KeyValue firstKv = region.getStore(fam1).memcache.memcache.firstKey();
KeyValue firstKv = region.getStore(fam1).memstore.memstore.firstKey();
assertTrue(firstKv.getTimestamp() <= now);
now = firstKv.getTimestamp();
for (Map.Entry<KeyValue, ?> entry:
region.getStore(fam1).memcache.memcache.entrySet()) {
region.getStore(fam1).memstore.memstore.entrySet()) {
KeyValue kv = entry.getKey();
assertTrue(kv.getTimestamp() <= now);
now = kv.getTimestamp();
@ -467,7 +466,7 @@ public class TestHRegion extends HBaseTestCase {
String method = this.getName();
initHRegion(tableName, method, fam1);
//Add to memcache
//Add to memstore
Put put = new Put(row1);
put.add(fam1, col1, null);
put.add(fam1, col2, null);
@ -529,7 +528,7 @@ public class TestHRegion extends HBaseTestCase {
String method = this.getName();
initHRegion(HConstants.ROOT_TABLE_NAME, method, HConstants.CATALOG_FAMILY);
//Add to memcache
//Add to memstore
Put put = new Put(HConstants.EMPTY_START_ROW);
put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, null);
region.put(put);
@ -542,7 +541,7 @@ public class TestHRegion extends HBaseTestCase {
HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
KeyValue [] expected = {kv1};
//Test from memcache
//Test from memstore
Result res = region.get(get, null);
assertEquals(expected.length, res.size());
@ -871,7 +870,7 @@ public class TestHRegion extends HBaseTestCase {
}
public void testScanner_ExplicitColumns_FromMemcache_EnforceVersions()
public void testScanner_ExplicitColumns_FromMemStore_EnforceVersions()
throws IOException {
byte [] tableName = Bytes.toBytes("testtable");
byte [] row1 = Bytes.toBytes("row1");
@ -987,7 +986,7 @@ public class TestHRegion extends HBaseTestCase {
}
}
public void testScanner_ExplicitColumns_FromMemcacheAndFiles_EnforceVersions()
public void testScanner_ExplicitColumns_FromMemStoreAndFiles_EnforceVersions()
throws IOException {
byte [] tableName = Bytes.toBytes("testtable");
byte [] row1 = Bytes.toBytes("row1");
@ -1066,7 +1065,7 @@ public class TestHRegion extends HBaseTestCase {
}
}
public void testScanner_Wildcard_FromMemcache_EnforceVersions()
public void testScanner_Wildcard_FromMemStore_EnforceVersions()
throws IOException {
byte [] tableName = Bytes.toBytes("testtable");
byte [] row1 = Bytes.toBytes("row1");
@ -1232,7 +1231,7 @@ public class TestHRegion extends HBaseTestCase {
}
public void testScanner_Wildcard_FromMemcacheAndFiles_EnforceVersions()
public void testScanner_Wildcard_FromMemStoreAndFiles_EnforceVersions()
throws IOException {
byte [] tableName = Bytes.toBytes("testtable");
byte [] row1 = Bytes.toBytes("row1");

View File

@ -24,16 +24,13 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.HBaseClusterTestCase;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.BatchUpdate;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.util.Bytes;
/**
@ -83,10 +80,10 @@ public class TestLogRolling extends HBaseClusterTestCase {
conf.setInt("hbase.regionserver.maxlogentries", 32);
// For less frequently updated regions flush after every 2 flushes
conf.setInt("hbase.hregion.memcache.optionalflushcount", 2);
conf.setInt("hbase.hregion.memstore.optionalflushcount", 2);
// We flush the cache after every 8192 bytes
conf.setInt("hbase.hregion.memcache.flush.size", 8192);
conf.setInt("hbase.hregion.memstore.flush.size", 8192);
// Make lease timeout longer, lease checks less frequent
conf.setInt("hbase.master.lease.period", 10 * 1000);

View File

@ -38,10 +38,10 @@ import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.util.Bytes;
/** memcache test case */
public class TestMemcache extends TestCase {
/** memstore test case */
public class TestMemStore extends TestCase {
private final Log LOG = LogFactory.getLog(this.getClass());
private Memcache memcache;
private MemStore memstore;
private static final int ROW_COUNT = 10;
private static final int QUALIFIER_COUNT = 10;
private static final byte [] FAMILY = Bytes.toBytes("column");
@ -51,20 +51,20 @@ public class TestMemcache extends TestCase {
@Override
public void setUp() throws Exception {
super.setUp();
this.memcache = new Memcache();
this.memstore = new MemStore();
}
/**
* Test memcache snapshot happening while scanning.
* Test memstore snapshot happening while scanning.
* @throws IOException
*/
public void testScanAcrossSnapshot() throws IOException {
int rowCount = addRows(this.memcache);
KeyValueScanner [] memcachescanners = this.memcache.getScanners();
int rowCount = addRows(this.memstore);
KeyValueScanner [] memstorescanners = this.memstore.getScanners();
Scan scan = new Scan();
List<KeyValue> result = new ArrayList<KeyValue>();
StoreScanner s = new StoreScanner(scan, null, HConstants.LATEST_TIMESTAMP,
this.memcache.comparator, null, memcachescanners);
this.memstore.comparator, null, memstorescanners);
int count = 0;
try {
while (s.next(result)) {
@ -78,7 +78,7 @@ public class TestMemcache extends TestCase {
assertEquals(rowCount, count);
// Now assert can count same number even if a snapshot mid-scan.
s = new StoreScanner(scan, null, HConstants.LATEST_TIMESTAMP,
this.memcache.comparator, null, memcachescanners);
this.memstore.comparator, null, memstorescanners);
count = 0;
try {
while (s.next(result)) {
@ -87,7 +87,7 @@ public class TestMemcache extends TestCase {
assertTrue(Bytes.compareTo(Bytes.toBytes(count), result.get(0).getRow()) == 0);
count++;
if (count == 2) {
this.memcache.snapshot();
this.memstore.snapshot();
LOG.info("Snapshotted");
}
result.clear();
@ -99,22 +99,22 @@ public class TestMemcache extends TestCase {
}
/**
* Test memcache snapshots
* Test memstore snapshots
* @throws IOException
*/
public void testSnapshotting() throws IOException {
final int snapshotCount = 5;
// Add some rows, run a snapshot. Do it a few times.
for (int i = 0; i < snapshotCount; i++) {
addRows(this.memcache);
runSnapshot(this.memcache);
Map<KeyValue, ?> ss = this.memcache.getSnapshot();
addRows(this.memstore);
runSnapshot(this.memstore);
Map<KeyValue, ?> ss = this.memstore.getSnapshot();
assertEquals("History not being cleared", 0, ss.size());
}
}
public void testMultipleVersionsSimple() throws Exception {
Memcache m = new Memcache(HConstants.FOREVER, KeyValue.COMPARATOR);
MemStore m = new MemStore(HConstants.FOREVER, KeyValue.COMPARATOR);
byte [] row = Bytes.toBytes("testRow");
byte [] family = Bytes.toBytes("testFamily");
byte [] qf = Bytes.toBytes("testQualifier");
@ -129,12 +129,12 @@ public class TestMemcache extends TestCase {
m.add(key1);
m.add(key2);
assertTrue("Expected memcache to hold 3 values, actually has " +
m.memcache.size(), m.memcache.size() == 3);
assertTrue("Expected memstore to hold 3 values, actually has " +
m.memstore.size(), m.memstore.size() == 3);
}
public void testBinary() throws IOException {
Memcache mc = new Memcache(HConstants.FOREVER, KeyValue.ROOT_COMPARATOR);
MemStore mc = new MemStore(HConstants.FOREVER, KeyValue.ROOT_COMPARATOR);
final int start = 43;
final int end = 46;
for (int k = start; k <= end; k++) {
@ -153,7 +153,7 @@ public class TestMemcache extends TestCase {
// System.out.println(key);
}
int index = start;
for (Map.Entry<KeyValue, ?> entry: mc.memcache.entrySet()) {
for (Map.Entry<KeyValue, ?> entry: mc.memstore.entrySet()) {
System.out.println(entry);
byte [] b = entry.getKey().getRow();
// Hardcoded offsets into String
@ -177,70 +177,70 @@ public class TestMemcache extends TestCase {
byte [] t35 = Bytes.toBytes("035");
byte [] t40 = Bytes.toBytes("040");
memcache.add(getKV(t10, "t10 bytes".getBytes()));
memcache.add(getKV(t20, "t20 bytes".getBytes()));
memcache.add(getKV(t30, "t30 bytes".getBytes()));
memcache.add(getKV(t35, "t35 bytes".getBytes()));
memstore.add(getKV(t10, "t10 bytes".getBytes()));
memstore.add(getKV(t20, "t20 bytes".getBytes()));
memstore.add(getKV(t30, "t30 bytes".getBytes()));
memstore.add(getKV(t35, "t35 bytes".getBytes()));
// write a delete in there to see if things still work ok
memcache.add(getDeleteKV(t35));
memcache.add(getKV(t40, "t40 bytes".getBytes()));
memstore.add(getDeleteKV(t35));
memstore.add(getKV(t40, "t40 bytes".getBytes()));
NavigableSet<KeyValue> results = null;
// try finding "015"
results =
new TreeSet<KeyValue>(this.memcache.comparator.getComparatorIgnoringType());
new TreeSet<KeyValue>(this.memstore.comparator.getComparatorIgnoringType());
KeyValue t15 = new KeyValue(Bytes.toBytes("015"),
System.currentTimeMillis());
memcache.getRowKeyAtOrBefore(t15, results);
memstore.getRowKeyAtOrBefore(t15, results);
KeyValue kv = results.last();
assertTrue(KeyValue.COMPARATOR.compareRows(kv, t10) == 0);
// try "020", we should get that row exactly
results =
new TreeSet<KeyValue>(this.memcache.comparator.getComparatorIgnoringType());
memcache.getRowKeyAtOrBefore(new KeyValue(t20, System.currentTimeMillis()),
new TreeSet<KeyValue>(this.memstore.comparator.getComparatorIgnoringType());
memstore.getRowKeyAtOrBefore(new KeyValue(t20, System.currentTimeMillis()),
results);
assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t20) == 0);
// try "030", we should get that row exactly
results =
new TreeSet<KeyValue>(this.memcache.comparator.getComparatorIgnoringType());
memcache.getRowKeyAtOrBefore(new KeyValue(t30, System.currentTimeMillis()),
new TreeSet<KeyValue>(this.memstore.comparator.getComparatorIgnoringType());
memstore.getRowKeyAtOrBefore(new KeyValue(t30, System.currentTimeMillis()),
results);
assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t30) == 0);
// try "038", should skip the deleted "035" and give "030"
results =
new TreeSet<KeyValue>(this.memcache.comparator.getComparatorIgnoringType());
new TreeSet<KeyValue>(this.memstore.comparator.getComparatorIgnoringType());
byte [] t38 = Bytes.toBytes("038");
memcache.getRowKeyAtOrBefore(new KeyValue(t38, System.currentTimeMillis()),
memstore.getRowKeyAtOrBefore(new KeyValue(t38, System.currentTimeMillis()),
results);
assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t30) == 0);
// try "050", should get stuff from "040"
results =
new TreeSet<KeyValue>(this.memcache.comparator.getComparatorIgnoringType());
new TreeSet<KeyValue>(this.memstore.comparator.getComparatorIgnoringType());
byte [] t50 = Bytes.toBytes("050");
memcache.getRowKeyAtOrBefore(new KeyValue(t50, System.currentTimeMillis()),
memstore.getRowKeyAtOrBefore(new KeyValue(t50, System.currentTimeMillis()),
results);
assertTrue(KeyValue.COMPARATOR.compareRows(results.last(), t40) == 0);
}
/** Test getNextRow from memcache
/** Test getNextRow from memstore
* @throws InterruptedException
*/
public void testGetNextRow() throws Exception {
addRows(this.memcache);
addRows(this.memstore);
// Add more versions to make it a little more interesting.
Thread.sleep(1);
addRows(this.memcache);
KeyValue closestToEmpty = this.memcache.getNextRow(KeyValue.LOWESTKEY);
addRows(this.memstore);
KeyValue closestToEmpty = this.memstore.getNextRow(KeyValue.LOWESTKEY);
assertTrue(KeyValue.COMPARATOR.compareRows(closestToEmpty,
new KeyValue(Bytes.toBytes(0), System.currentTimeMillis())) == 0);
for (int i = 0; i < ROW_COUNT; i++) {
KeyValue nr = this.memcache.getNextRow(new KeyValue(Bytes.toBytes(i),
KeyValue nr = this.memstore.getNextRow(new KeyValue(Bytes.toBytes(i),
System.currentTimeMillis()));
if (i + 1 == ROW_COUNT) {
assertEquals(nr, null);
@ -253,8 +253,8 @@ public class TestMemcache extends TestCase {
for (int startRowId = 0; startRowId < ROW_COUNT; startRowId++) {
InternalScanner scanner =
new StoreScanner(new Scan(Bytes.toBytes(startRowId)), FAMILY,
Integer.MAX_VALUE, this.memcache.comparator, null,
new KeyValueScanner[]{memcache.getScanners()[0]});
Integer.MAX_VALUE, this.memstore.comparator, null,
new KeyValueScanner[]{memstore.getScanners()[0]});
List<KeyValue> results = new ArrayList<KeyValue>();
for (int i = 0; scanner.next(results); i++) {
int rowId = startRowId + i;
@ -281,13 +281,13 @@ public class TestMemcache extends TestCase {
byte [] qf3 = Bytes.toBytes("testqualifier3");
byte [] val = Bytes.toBytes("testval");
//Setting up memcache
//Setting up memstore
KeyValue add1 = new KeyValue(row, fam ,qf1, val);
KeyValue add2 = new KeyValue(row, fam ,qf2, val);
KeyValue add3 = new KeyValue(row, fam ,qf3, val);
memcache.add(add1);
memcache.add(add2);
memcache.add(add3);
memstore.add(add1);
memstore.add(add2);
memstore.add(add3);
//test
Get get = new Get(row);
@ -299,7 +299,7 @@ public class TestMemcache extends TestCase {
new QueryMatcher(get, fam, columns, ttl, KeyValue.KEY_COMPARATOR, 1);
List<KeyValue> result = new ArrayList<KeyValue>();
boolean res = memcache.get(matcher, result);
boolean res = memstore.get(matcher, result);
assertEquals(true, res);
}
@ -311,11 +311,11 @@ public class TestMemcache extends TestCase {
byte [] qf3 = Bytes.toBytes("testqualifier3");
byte [] val = Bytes.toBytes("testval");
//Setting up memcache
//Setting up memstore
KeyValue add1 = new KeyValue(row, fam ,qf1, val);
KeyValue add3 = new KeyValue(row, fam ,qf3, val);
memcache.add(add1);
memcache.add(add3);
memstore.add(add1);
memstore.add(add3);
//test
Get get = new Get(row);
@ -327,11 +327,11 @@ public class TestMemcache extends TestCase {
new QueryMatcher(get, fam, columns, ttl, KeyValue.KEY_COMPARATOR, 1);
List<KeyValue> result = new ArrayList<KeyValue>();
boolean res = memcache.get(matcher, result);
boolean res = memstore.get(matcher, result);
assertEquals(false, res);
}
public void testGet_MemcacheAndSnapShot() throws IOException {
public void testGet_memstoreAndSnapShot() throws IOException {
byte [] row = Bytes.toBytes("testrow");
byte [] fam = Bytes.toBytes("testfamily");
byte [] qf1 = Bytes.toBytes("testqualifier1");
@ -351,21 +351,21 @@ public class TestMemcache extends TestCase {
QueryMatcher matcher =
new QueryMatcher(get, fam, columns, ttl, KeyValue.KEY_COMPARATOR, 1);
//Setting up memcache
memcache.add(new KeyValue(row, fam ,qf1, val));
memcache.add(new KeyValue(row, fam ,qf2, val));
memcache.add(new KeyValue(row, fam ,qf3, val));
//Setting up memstore
memstore.add(new KeyValue(row, fam ,qf1, val));
memstore.add(new KeyValue(row, fam ,qf2, val));
memstore.add(new KeyValue(row, fam ,qf3, val));
//Creating a snapshot
memcache.snapshot();
assertEquals(3, memcache.snapshot.size());
//Adding value to "new" memcache
assertEquals(0, memcache.memcache.size());
memcache.add(new KeyValue(row, fam ,qf4, val));
memcache.add(new KeyValue(row, fam ,qf5, val));
assertEquals(2, memcache.memcache.size());
memstore.snapshot();
assertEquals(3, memstore.snapshot.size());
//Adding value to "new" memstore
assertEquals(0, memstore.memstore.size());
memstore.add(new KeyValue(row, fam ,qf4, val));
memstore.add(new KeyValue(row, fam ,qf5, val));
assertEquals(2, memstore.memstore.size());
List<KeyValue> result = new ArrayList<KeyValue>();
boolean res = memcache.get(matcher, result);
boolean res = memstore.get(matcher, result);
assertEquals(true, res);
}
@ -402,20 +402,20 @@ public class TestMemcache extends TestCase {
expected.add(kv2);
expected.add(kv3);
//Setting up memcache
memcache.add(new KeyValue(row, fam ,qf1, ts1, val));
memcache.add(new KeyValue(row, fam ,qf2, ts1, val));
memcache.add(new KeyValue(row, fam ,qf3, ts1, val));
memcache.add(kv1);
memcache.add(kv2);
memcache.add(kv3);
memcache.add(new KeyValue(row, fam ,qf1, ts3, val));
memcache.add(new KeyValue(row, fam ,qf2, ts3, val));
memcache.add(new KeyValue(row, fam ,qf3, ts3, val));
//Setting up memstore
memstore.add(new KeyValue(row, fam ,qf1, ts1, val));
memstore.add(new KeyValue(row, fam ,qf2, ts1, val));
memstore.add(new KeyValue(row, fam ,qf3, ts1, val));
memstore.add(kv1);
memstore.add(kv2);
memstore.add(kv3);
memstore.add(new KeyValue(row, fam ,qf1, ts3, val));
memstore.add(new KeyValue(row, fam ,qf2, ts3, val));
memstore.add(new KeyValue(row, fam ,qf3, ts3, val));
//Get
List<KeyValue> result = new ArrayList<KeyValue>();
memcache.get(matcher, result);
memstore.get(matcher, result);
assertEquals(expected.size(), result.size());
for(int i=0; i<expected.size(); i++){
@ -439,23 +439,23 @@ public class TestMemcache extends TestCase {
KeyValue put2 = new KeyValue(row, fam, qf1, ts2, val);
long ts3 = ts2 +1;
KeyValue put3 = new KeyValue(row, fam, qf1, ts3, val);
memcache.add(put1);
memcache.add(put2);
memcache.add(put3);
memstore.add(put1);
memstore.add(put2);
memstore.add(put3);
assertEquals(3, memcache.memcache.size());
assertEquals(3, memstore.memstore.size());
KeyValue del2 = new KeyValue(row, fam, qf1, ts2, KeyValue.Type.Delete, val);
memcache.delete(del2);
memstore.delete(del2);
List<KeyValue> expected = new ArrayList<KeyValue>();
expected.add(put3);
expected.add(del2);
expected.add(put1);
assertEquals(3, memcache.memcache.size());
assertEquals(3, memstore.memstore.size());
int i = 0;
for(Map.Entry<KeyValue, ?> entry : memcache.memcache.entrySet()) {
for(Map.Entry<KeyValue, ?> entry : memstore.memstore.entrySet()) {
assertEquals(expected.get(i++), entry.getKey());
}
}
@ -472,23 +472,23 @@ public class TestMemcache extends TestCase {
KeyValue put2 = new KeyValue(row, fam, qf1, ts2, val);
long ts3 = ts2 +1;
KeyValue put3 = new KeyValue(row, fam, qf1, ts3, val);
memcache.add(put1);
memcache.add(put2);
memcache.add(put3);
memstore.add(put1);
memstore.add(put2);
memstore.add(put3);
assertEquals(3, memcache.memcache.size());
assertEquals(3, memstore.memstore.size());
KeyValue del2 =
new KeyValue(row, fam, qf1, ts2, KeyValue.Type.DeleteColumn, val);
memcache.delete(del2);
memstore.delete(del2);
List<KeyValue> expected = new ArrayList<KeyValue>();
expected.add(put3);
expected.add(del2);
assertEquals(2, memcache.memcache.size());
assertEquals(2, memstore.memstore.size());
int i = 0;
for(Map.Entry<KeyValue, ?> entry : memcache.memcache.entrySet()) {
for(Map.Entry<KeyValue, ?> entry : memstore.memstore.entrySet()) {
assertEquals(expected.get(i++), entry.getKey());
}
}
@ -508,74 +508,74 @@ public class TestMemcache extends TestCase {
KeyValue put3 = new KeyValue(row, fam, qf3, ts, val);
KeyValue put4 = new KeyValue(row, fam, qf3, ts+1, val);
memcache.add(put1);
memcache.add(put2);
memcache.add(put3);
memcache.add(put4);
memstore.add(put1);
memstore.add(put2);
memstore.add(put3);
memstore.add(put4);
KeyValue del =
new KeyValue(row, fam, null, ts, KeyValue.Type.DeleteFamily, val);
memcache.delete(del);
memstore.delete(del);
List<KeyValue> expected = new ArrayList<KeyValue>();
expected.add(del);
expected.add(put4);
assertEquals(2, memcache.memcache.size());
assertEquals(2, memstore.memstore.size());
int i = 0;
for(Map.Entry<KeyValue, ?> entry : memcache.memcache.entrySet()) {
for(Map.Entry<KeyValue, ?> entry : memstore.memstore.entrySet()) {
assertEquals(expected.get(i++), entry.getKey());
}
}
public void testKeepDeleteInMemcache() {
public void testKeepDeleteInmemstore() {
byte [] row = Bytes.toBytes("testrow");
byte [] fam = Bytes.toBytes("testfamily");
byte [] qf = Bytes.toBytes("testqualifier");
byte [] val = Bytes.toBytes("testval");
long ts = System.nanoTime();
memcache.add(new KeyValue(row, fam, qf, ts, val));
memstore.add(new KeyValue(row, fam, qf, ts, val));
KeyValue delete = new KeyValue(row, fam, qf, ts, KeyValue.Type.Delete, val);
memcache.delete(delete);
assertEquals(1, memcache.memcache.size());
assertEquals(delete, memcache.memcache.firstKey());
memstore.delete(delete);
assertEquals(1, memstore.memstore.size());
assertEquals(delete, memstore.memstore.firstKey());
}
public void testRetainsDeleteVersion() throws IOException {
// add a put to memcache
memcache.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"));
// add a put to memstore
memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"));
// now process a specific delete:
KeyValue delete = KeyValueTestUtil.create(
"row1", "fam", "a", 100, KeyValue.Type.Delete, "dont-care");
memcache.delete(delete);
memstore.delete(delete);
assertEquals(1, memcache.memcache.size());
assertEquals(delete, memcache.memcache.firstKey());
assertEquals(1, memstore.memstore.size());
assertEquals(delete, memstore.memstore.firstKey());
}
public void testRetainsDeleteColumn() throws IOException {
// add a put to memcache
memcache.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"));
// add a put to memstore
memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"));
// now process a specific delete:
KeyValue delete = KeyValueTestUtil.create("row1", "fam", "a", 100,
KeyValue.Type.DeleteColumn, "dont-care");
memcache.delete(delete);
memstore.delete(delete);
assertEquals(1, memcache.memcache.size());
assertEquals(delete, memcache.memcache.firstKey());
assertEquals(1, memstore.memstore.size());
assertEquals(delete, memstore.memstore.firstKey());
}
public void testRetainsDeleteFamily() throws IOException {
// add a put to memcache
memcache.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"));
// add a put to memstore
memstore.add(KeyValueTestUtil.create("row1", "fam", "a", 100, "dont-care"));
// now process a specific delete:
KeyValue delete = KeyValueTestUtil.create("row1", "fam", "a", 100,
KeyValue.Type.DeleteFamily, "dont-care");
memcache.delete(delete);
memstore.delete(delete);
assertEquals(1, memcache.memcache.size());
assertEquals(delete, memcache.memcache.firstKey());
assertEquals(1, memstore.memstore.size());
assertEquals(delete, memstore.memstore.firstKey());
}
@ -593,7 +593,7 @@ public class TestMemcache extends TestCase {
* @return How many rows we added.
* @throws IOException
*/
private int addRows(final Memcache hmc) {
private int addRows(final MemStore hmc) {
for (int i = 0; i < ROW_COUNT; i++) {
long timestamp = System.currentTimeMillis();
for (int ii = 0; ii < QUALIFIER_COUNT; ii++) {
@ -605,7 +605,7 @@ public class TestMemcache extends TestCase {
return ROW_COUNT;
}
private void runSnapshot(final Memcache hmc) throws UnexpectedException {
private void runSnapshot(final MemStore hmc) throws UnexpectedException {
// Save off old state.
int oldHistorySize = hmc.getSnapshot().size();
hmc.snapshot();

View File

@ -23,17 +23,14 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
import java.util.TreeSet;
import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KeyComparator;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode;
import org.apache.hadoop.hbase.util.Bytes;
public class TestQueryMatcher extends HBaseTestCase
@ -100,18 +97,18 @@ implements HConstants {
QueryMatcher qm = new QueryMatcher(get, fam2,
get.getFamilyMap().get(fam2), ttl, rowComparator, 1);
List<KeyValue> memCache = new ArrayList<KeyValue>();
memCache.add(new KeyValue(row1, fam2, col1, data));
memCache.add(new KeyValue(row1, fam2, col2, data));
memCache.add(new KeyValue(row1, fam2, col3, data));
memCache.add(new KeyValue(row1, fam2, col4, data));
memCache.add(new KeyValue(row1, fam2, col5, data));
List<KeyValue> memstore = new ArrayList<KeyValue>();
memstore.add(new KeyValue(row1, fam2, col1, data));
memstore.add(new KeyValue(row1, fam2, col2, data));
memstore.add(new KeyValue(row1, fam2, col3, data));
memstore.add(new KeyValue(row1, fam2, col4, data));
memstore.add(new KeyValue(row1, fam2, col5, data));
memCache.add(new KeyValue(row2, fam1, col1, data));
memstore.add(new KeyValue(row2, fam1, col1, data));
List<MatchCode> actual = new ArrayList<MatchCode>();
for(KeyValue kv : memCache){
for(KeyValue kv : memstore){
actual.add(qm.match(kv));
}
@ -142,17 +139,17 @@ implements HConstants {
QueryMatcher qm = new QueryMatcher(get, fam2, null, ttl, rowComparator, 1);
List<KeyValue> memCache = new ArrayList<KeyValue>();
memCache.add(new KeyValue(row1, fam2, col1, data));
memCache.add(new KeyValue(row1, fam2, col2, data));
memCache.add(new KeyValue(row1, fam2, col3, data));
memCache.add(new KeyValue(row1, fam2, col4, data));
memCache.add(new KeyValue(row1, fam2, col5, data));
memCache.add(new KeyValue(row2, fam1, col1, data));
List<KeyValue> memstore = new ArrayList<KeyValue>();
memstore.add(new KeyValue(row1, fam2, col1, data));
memstore.add(new KeyValue(row1, fam2, col2, data));
memstore.add(new KeyValue(row1, fam2, col3, data));
memstore.add(new KeyValue(row1, fam2, col4, data));
memstore.add(new KeyValue(row1, fam2, col5, data));
memstore.add(new KeyValue(row2, fam1, col1, data));
List<MatchCode> actual = new ArrayList<MatchCode>();
for(KeyValue kv : memCache){
for(KeyValue kv : memstore){
actual.add(qm.match(kv));
}

View File

@ -157,7 +157,7 @@ public class TestScanner extends HBaseTestCase {
byteStream.toByteArray());
region.put(put);
// What we just committed is in the memcache. Verify that we can get
// What we just committed is in the memstore. Verify that we can get
// it back both with scanning and get
scan(false, null);
@ -188,7 +188,7 @@ public class TestScanner extends HBaseTestCase {
region.put(put);
// Validate that we can still get the HRegionInfo, even though it is in
// an older row on disk and there is a newer row in the memcache
// an older row on disk and there is a newer row in the memstore
scan(true, address.toString());
getRegionInfo();
@ -383,7 +383,7 @@ public class TestScanner extends HBaseTestCase {
/**
* Tests to do a concurrent flush (using a 2nd thread) while scanning. This tests both
* the StoreScanner update readers and the transition from memcache -> snapshot -> store file.
* the StoreScanner update readers and the transition from memstore -> snapshot -> store file.
*
* @throws Exception
*/

View File

@ -88,13 +88,13 @@ public class TestStore extends TestCase {
// Get tests
//////////////////////////////////////////////////////////////////////////////
/**
* Getting data from memcache only
* Getting data from memstore only
* @throws IOException
*/
public void testGet_FromMemCacheOnly() throws IOException {
public void testGet_FromMemStoreOnly() throws IOException {
init(this.getName());
//Put data in memcache
//Put data in memstore
this.store.add(new KeyValue(row, family, qf1, null));
this.store.add(new KeyValue(row, family, qf2, null));
this.store.add(new KeyValue(row, family, qf3, null));
@ -116,7 +116,7 @@ public class TestStore extends TestCase {
public void testGet_FromFilesOnly() throws IOException {
init(this.getName());
//Put data in memcache
//Put data in memstore
this.store.add(new KeyValue(row, family, qf1, null));
this.store.add(new KeyValue(row, family, qf2, null));
//flush
@ -145,13 +145,13 @@ public class TestStore extends TestCase {
}
/**
* Getting data from memcache and files
* Getting data from memstore and files
* @throws IOException
*/
public void testGet_FromMemCacheAndFiles() throws IOException {
public void testGet_FromMemStoreAndFiles() throws IOException {
init(this.getName());
//Put data in memcache
//Put data in memstore
this.store.add(new KeyValue(row, family, qf1, null));
this.store.add(new KeyValue(row, family, qf2, null));
//flush
@ -181,7 +181,7 @@ public class TestStore extends TestCase {
this.store.snapshot();
this.store.flushCache(id++);
assertEquals(storeFilessize, this.store.getStorefiles().size());
assertEquals(0, this.store.memcache.memcache.size());
assertEquals(0, this.store.memstore.memstore.size());
}
private void assertCheck() {
@ -196,14 +196,14 @@ public class TestStore extends TestCase {
//////////////////////////////////////////////////////////////////////////////
/**
* Testing if the update in place works. When you want to update a value that
* is already in memcache, you don't delete it and put a new one, but just
* is already in memstore, you don't delete it and put a new one, but just
* update the value in the original KeyValue
* @throws IOException
*/
public void testIncrementColumnValue_UpdatingInPlace() throws IOException {
init(this.getName());
//Put data in memcache
//Put data in memstore
long value = 1L;
long amount = 3L;
this.store.add(new KeyValue(row, family, qf1, Bytes.toBytes(value)));
@ -227,7 +227,7 @@ public class TestStore extends TestCase {
throws IOException {
init(this.getName());
//Put data in memcache
//Put data in memstore
long value = 3L;
long amount = -1L;
this.store.add(new KeyValue(row, family, qf1, Bytes.toBytes(value)));
@ -250,7 +250,7 @@ public class TestStore extends TestCase {
public void testIncrementColumnValue_AddingNew() throws IOException {
init(this.getName());
//Put data in memcache
//Put data in memstore
long value = 1L;
long amount = 3L;
this.store.add(new KeyValue(row, family, qf1, Bytes.toBytes(value)));
@ -268,14 +268,14 @@ public class TestStore extends TestCase {
}
/**
* When we have the key in a file add a new key + value to memcache with the
* When we have the key in a file add a new key + value to memstore with the
* updates value.
* @throws IOException
*/
public void testIncrementColumnValue_UpdatingFromSF() throws IOException {
init(this.getName());
//Put data in memcache
//Put data in memstore
long value = 1L;
long amount = 3L;
this.store.add(new KeyValue(row, family, qf1, Bytes.toBytes(value)));
@ -296,14 +296,14 @@ public class TestStore extends TestCase {
/**
* Same as testIncrementColumnValue_AddingNew() except that the keys are
* checked in file not in memcache
* checked in file not in memstore
* @throws IOException
*/
public void testIncrementColumnValue_AddingNewAfterSFCheck()
throws IOException {
init(this.getName());
//Put data in memcache
//Put data in memstore
long value = 1L;
long amount = 3L;
this.store.add(new KeyValue(row, family, qf1, Bytes.toBytes(value)));