From 7886526f30642f8826c80c8c0d6b8e472b236dbb Mon Sep 17 00:00:00 2001 From: Michael Stack Date: Wed, 24 Jun 2009 19:56:51 +0000 Subject: [PATCH] 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 --- CHANGES.txt | 2 + bin/HBase.rb | 6 +- bin/hirb.rb | 2 +- conf/hbase-default.xml | 30 +-- .../documentation/content/xdocs/metrics.xml | 2 +- .../org/apache/hadoop/hbase/HServerLoad.java | 42 ++-- .../apache/hadoop/hbase/HTableDescriptor.java | 24 +- .../org/apache/hadoop/hbase/KeyValue.java | 4 +- .../client/UnmodifyableHTableDescriptor.java | 4 +- .../hbase/regionserver/ColumnTracker.java | 2 +- .../hbase/regionserver/DeleteCompare.java | 8 +- .../regionserver/ExplicitColumnTracker.java | 4 +- .../hbase/regionserver/FlushRequester.java | 2 +- .../hadoop/hbase/regionserver/HLog.java | 2 +- .../hadoop/hbase/regionserver/HRegion.java | 94 +++---- .../hbase/regionserver/HRegionServer.java | 32 +-- .../hbase/regionserver/KeyValueHeap.java | 2 +- .../{Memcache.java => MemStore.java} | 102 ++++---- ...cacheFlusher.java => MemStoreFlusher.java} | 84 +++---- .../MinorCompactingStoreScanner.java | 16 +- .../hbase/regionserver/QueryMatcher.java | 10 +- .../hadoop/hbase/regionserver/Store.java | 54 ++-- .../hadoop/hbase/regionserver/StoreFile.java | 2 +- .../hbase/regionserver/StoreScanner.java | 10 +- .../regionserver/WildcardColumnTracker.java | 4 +- .../metrics/RegionServerMetrics.java | 14 +- .../org/apache/hadoop/hbase/DFSAbort.java | 73 ------ .../hadoop/hbase/TestClassMigration.java | 2 +- .../hadoop/hbase/client/TestClient.java | 74 +++--- .../hadoop/hbase/client/TestScannerTimes.java | 10 +- .../hbase/regionserver/TestCompaction.java | 8 +- .../hbase/regionserver/TestDeleteCompare.java | 66 ++--- .../hbase/regionserver/TestHRegion.java | 27 +- .../hbase/regionserver/TestLogRolling.java | 15 +- .../{TestMemcache.java => TestMemStore.java} | 236 +++++++++--------- .../hbase/regionserver/TestQueryMatcher.java | 37 ++- .../hbase/regionserver/TestScanner.java | 6 +- .../hadoop/hbase/regionserver/TestStore.java | 32 +-- 38 files changed, 530 insertions(+), 614 deletions(-) rename src/java/org/apache/hadoop/hbase/regionserver/{Memcache.java => MemStore.java} (89%) rename src/java/org/apache/hadoop/hbase/regionserver/{MemcacheFlusher.java => MemStoreFlusher.java} (82%) delete mode 100644 src/test/org/apache/hadoop/hbase/DFSAbort.java rename src/test/org/apache/hadoop/hbase/regionserver/{TestMemcache.java => TestMemStore.java} (77%) diff --git a/CHANGES.txt b/CHANGES.txt index df661492327..e957a8fccdc 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -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 diff --git a/bin/HBase.rb b/bin/HBase.rb index f982a2ae8b2..f01eec954b2 100644 --- a/bin/HBase.rb +++ b/bin/HBase.rb @@ -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) diff --git a/bin/hirb.rb b/bin/hirb.rb index 8b5736a34ac..9acfe984a9a 100644 --- a/bin/hirb.rb +++ b/bin/hirb.rb @@ -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'} diff --git a/conf/hbase-default.xml b/conf/hbase-default.xml index c69ecd62026..0e2a70e7f91 100644 --- a/conf/hbase-default.xml +++ b/conf/hbase-default.xml @@ -230,20 +230,20 @@ - hbase.regionserver.globalMemcache.upperLimit + hbase.regionserver.global.memstore.upperLimit 0.4 - Maximum size of all memcaches in a region server before new - updates are blocked and flushes are forced. Defaults to 40% of heap. + Maximum size of all memstores in a region server before new + updates are blocked and flushes are forced. Defaults to 40% of heap - hbase.regionserver.globalMemcache.lowerLimit + hbase.regionserver.global.memstore.lowerLimit 0.25 - When memcaches are being forced to flush to make room in + 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. @@ -268,22 +268,22 @@ - hbase.hregion.memcache.flush.size + hbase.hregion.memstore.flush.size 67108864 - 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. + every hbase.server.thread.wakefrequency. - hbase.hregion.memcache.block.multiplier + hbase.hregion.memstore.block.multiplier 2 - 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. @@ -302,7 +302,7 @@ 3 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 diff --git a/src/docs/src/documentation/content/xdocs/metrics.xml b/src/docs/src/documentation/content/xdocs/metrics.xml index 01acae8f480..c8744f438de 100644 --- a/src/docs/src/documentation/content/xdocs/metrics.xml +++ b/src/docs/src/documentation/content/xdocs/metrics.xml @@ -49,7 +49,7 @@

If you enable the hbase 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.

diff --git a/src/java/org/apache/hadoop/hbase/HServerLoad.java b/src/java/org/apache/hadoop/hbase/HServerLoad.java index d013657d9c5..8f5285b81b5 100644 --- a/src/java/org/apache/hadoop/hbase/HServerLoad.java +++ b/src/java/org/apache/hadoop/hbase/HServerLoad.java @@ -59,8 +59,8 @@ public class HServerLoad implements WritableComparable { 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 { * @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 { } /** - * @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 { } /** - * @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 { 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 { 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 { 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 { } /** - * @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 { * @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 diff --git a/src/java/org/apache/hadoop/hbase/HTableDescriptor.java b/src/java/org/apache/hadoop/hbase/HTableDescriptor.java index 9ddc9d5b776..a4258e7e149 100644 --- a/src/java/org/apache/hadoop/hbase/HTableDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/HTableDescriptor.java @@ -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 getIndexes() { diff --git a/src/java/org/apache/hadoop/hbase/KeyValue.java b/src/java/org/apache/hadoop/hbase/KeyValue.java index 13b02e6b915..67ff09183ab 100644 --- a/src/java/org/apache/hadoop/hbase/KeyValue.java +++ b/src/java/org/apache/hadoop/hbase/KeyValue.java @@ -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; diff --git a/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java index 8dd4c6fe4cb..52e5c1ded77 100644 --- a/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java +++ b/src/java/org/apache/hadoop/hbase/client/UnmodifyableHTableDescriptor.java @@ -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"); } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java index 4d68c86d9cd..dfb30261ca3 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/ColumnTracker.java @@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.regionserver.QueryMatcher.MatchCode; *

  • {@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. - *
  • {@link #update} is called at the end of every StoreFile or Memcache. + *
  • {@link #update} is called at the end of every StoreFile or memstore. *

    * This class is NOT thread-safe as queries are never multi-threaded */ diff --git a/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java b/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java index ff0fe3eb7e7..03cb1896275 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java @@ -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 diff --git a/src/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java index bb02a0ba260..6a2b4243b93 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/ExplicitColumnTracker.java @@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.util.Bytes; *

    • {@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. - *
    • {@link #update} is called at the end of every StoreFile or Memcache. + *
    • {@link #update} is called at the end of every StoreFile or memstore. *

      * 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) { diff --git a/src/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java b/src/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java index 8e962a5e708..e1db18f614d 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java @@ -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 { /** diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HLog.java b/src/java/org/apache/hadoop/hbase/regionserver/HLog.java index 90d8607f738..9b72c88d153 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HLog.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HLog.java @@ -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(); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java index d72f820ef82..c96d88a8860 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -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.) * *

      So, we have a three-step process: * - *

      • A. Flush the memcache to the on-disk stores, noting the current + *
        • A. Flush the memstore to the on-disk stores, noting the current * sequence ID for the log.
        • * *
        • B. Write a FLUSHCACHE-COMPLETE message to the log, using the sequence - * ID that was current at the time of memcache-flush.
        • + * ID that was current at the time of memstore-flush. * - *
        • C. Get rid of the memcache structures that are now redundant, as + *
        • C. Get rid of the memstore structures that are now redundant, as * they've been flushed to the on-disk HStores.
        • *
        *

        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); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 6a66d3abd7f..42aa43d8fa8 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -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 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 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; diff --git a/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java b/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java index daea26aa42e..a9f8ce5be1d 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/KeyValueHeap.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.KeyValue.KVComparator; * Implements KeyValueScanner itself. *

        * 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. *

        * 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 diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java b/src/java/org/apache/hadoop/hbase/regionserver/MemStore.java similarity index 89% rename from src/java/org/apache/hadoop/hbase/regionserver/Memcache.java rename to src/java/org/apache/hadoop/hbase/regionserver/MemStore.java index be91d72398b..2fb2e6fbf88 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/Memcache.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/MemStore.java @@ -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 memcache; + volatile ConcurrentSkipListMap memstore; - // Snapshot of memcache. Made for flusher. + // Snapshot of memstore. Made for flusher. volatile ConcurrentSkipListMap 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 entry: this.memcache.entrySet()) { + for (Map.Entry entry: this.memstore.entrySet()) { LOG.info(entry.getKey()); } for (Map.Entry 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 deletes = new ArrayList(); - SortedMap tail = this.memcache.tailMap(delete); + SortedMap 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 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 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 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. *

        * 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 mc; private KeyValue current = null; private List result = new ArrayList(); private int idx = 0; - MemcacheScanner(final NavigableMap mc) { + MemStoreScanner(final NavigableMap 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++) { diff --git a/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java b/src/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java similarity index 82% rename from src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java rename to src/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java index f5a24dcd813..6826e0eb560 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/MemcacheFlusher.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java @@ -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 flushQueue = new LinkedBlockingQueue(); @@ -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 regionsToCompact = new ArrayList(); for (SortedMap 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()); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/MinorCompactingStoreScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/MinorCompactingStoreScanner.java index 1c5c2660a07..e6a31de4203 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/MinorCompactingStoreScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/MinorCompactingStoreScanner.java @@ -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 { diff --git a/src/java/org/apache/hadoop/hbase/regionserver/QueryMatcher.java b/src/java/org/apache/hadoop/hbase/regionserver/QueryMatcher.java index d140fc9d3a7..75f45aaf605 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/QueryMatcher.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/QueryMatcher.java @@ -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. *

        * 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). *

        * This method will update the internal structures to be accurate for * the next section. diff --git a/src/java/org/apache/hadoop/hbase/regionserver/Store.java b/src/java/org/apache/hadoop/hbase/regionserver/Store.java index 8d8bbc06f28..25729306043 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -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. * *

        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 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 cache = this.memcache.getSnapshot(); + // this.memstore.snapshot() has happened earlier up in the chain. + ConcurrentSkipListMap 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. * *

        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 m = this.storefiles.descendingMap(); for (Map.Entry 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 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(); diff --git a/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 1cb5847a72c..a26be98c401 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -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 diff --git a/src/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/src/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 812f21fe0d4..0ad65f3419f 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -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 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 columns) { this.store = store; @@ -90,9 +90,9 @@ class StoreScanner implements KeyValueScanner, InternalScanner, ChangedReadersOb */ private List getScanners() { List 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; } diff --git a/src/java/org/apache/hadoop/hbase/regionserver/WildcardColumnTracker.java b/src/java/org/apache/hadoop/hbase/regionserver/WildcardColumnTracker.java index c1f4b312c6e..0fe73ab316e 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/WildcardColumnTracker.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/WildcardColumnTracker.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes; *

        • {@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. - *
        • {@link #update} is called at the end of every StoreFile or Memcache. + *
        • {@link #update} is called at the end of every StoreFile or memstore. *

          * 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 diff --git a/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java b/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java index 2a723efb57f..52ab21ffffe 100644 --- a/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java +++ b/src/java/org/apache/hadoop/hbase/regionserver/metrics/RegionServerMetrics.java @@ -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 = @@ -197,4 +197,4 @@ public class RegionServerMetrics implements Updater { Long.valueOf(this.blockCacheHitRatio.get())); return sb.toString(); } -} +} \ No newline at end of file diff --git a/src/test/org/apache/hadoop/hbase/DFSAbort.java b/src/test/org/apache/hadoop/hbase/DFSAbort.java deleted file mode 100644 index 19c2891775f..00000000000 --- a/src/test/org/apache/hadoop/hbase/DFSAbort.java +++ /dev/null @@ -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)); - } -} diff --git a/src/test/org/apache/hadoop/hbase/TestClassMigration.java b/src/test/org/apache/hadoop/hbase/TestClassMigration.java index fb7f45beed3..4805d6621d1 100644 --- a/src/test/org/apache/hadoop/hbase/TestClassMigration.java +++ b/src/test/org/apache/hadoop/hbase/TestClassMigration.java @@ -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; diff --git a/src/test/org/apache/hadoop/hbase/client/TestClient.java b/src/test/org/apache/hadoop/hbase/client/TestClient.java index da10ecf4b5a..4d855e97fd7 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestClient.java +++ b/src/test/org/apache/hadoop/hbase/client/TestClient.java @@ -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]); diff --git a/src/test/org/apache/hadoop/hbase/client/TestScannerTimes.java b/src/test/org/apache/hadoop/hbase/client/TestScannerTimes.java index bb938de484f..6046d24fc54 100644 --- a/src/test/org/apache/hadoop/hbase/client/TestScannerTimes.java +++ b/src/test/org/apache/hadoop/hbase/client/TestScannerTimes.java @@ -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); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java index 4a46fdded2a..e03a9dc608b 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -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. diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteCompare.java b/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteCompare.java index d08941214e1..21d919478a6 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteCompare.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestDeleteCompare.java @@ -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 memcache = new TreeSet(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 memstore = new TreeSet(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 expected = new ArrayList(); @@ -55,7 +55,7 @@ public class TestDeleteCompare extends TestCase { byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG]; List actual = new ArrayList(); - 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 memcache = new TreeSet(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 memstore = new TreeSet(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 actual = new ArrayList(); - 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 memcache = new TreeSet(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 memstore = new TreeSet(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 expected = new ArrayList(); @@ -133,7 +133,7 @@ public class TestDeleteCompare extends TestCase { byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG]; List actual = new ArrayList(); - 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 memcache = new TreeSet(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 memstore = new TreeSet(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 expected = new ArrayList(); @@ -176,7 +176,7 @@ public class TestDeleteCompare extends TestCase { byte deleteType = deleteBuffer[deleteTimestampOffset +Bytes.SIZEOF_LONG]; List actual = new ArrayList(); - for(KeyValue mem : memcache){ + for(KeyValue mem : memstore){ actual.add(DeleteCompare.deleteCompare(mem, deleteBuffer, deleteRowOffset, deleteRowLen, deleteQualifierOffset, deleteQualifierLen, deleteTimestampOffset, deleteType, KeyValue.KEY_COMPARATOR)); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 32581fd4da3..f3a742c4ef9 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -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 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"); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java b/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java index 34fc8d13416..e8d78d5bfd8 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestLogRolling.java @@ -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); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestMemcache.java b/src/test/org/apache/hadoop/hbase/regionserver/TestMemStore.java similarity index 77% rename from src/test/org/apache/hadoop/hbase/regionserver/TestMemcache.java rename to src/test/org/apache/hadoop/hbase/regionserver/TestMemStore.java index a13c2a5c88f..a7002aa1494 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestMemcache.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestMemStore.java @@ -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 result = new ArrayList(); 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 ss = this.memcache.getSnapshot(); + addRows(this.memstore); + runSnapshot(this.memstore); + Map 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 entry: mc.memcache.entrySet()) { + for (Map.Entry 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 results = null; // try finding "015" results = - new TreeSet(this.memcache.comparator.getComparatorIgnoringType()); + new TreeSet(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(this.memcache.comparator.getComparatorIgnoringType()); - memcache.getRowKeyAtOrBefore(new KeyValue(t20, System.currentTimeMillis()), + new TreeSet(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(this.memcache.comparator.getComparatorIgnoringType()); - memcache.getRowKeyAtOrBefore(new KeyValue(t30, System.currentTimeMillis()), + new TreeSet(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(this.memcache.comparator.getComparatorIgnoringType()); + new TreeSet(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(this.memcache.comparator.getComparatorIgnoringType()); + new TreeSet(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 results = new ArrayList(); 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 result = new ArrayList(); - 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 result = new ArrayList(); - 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 result = new ArrayList(); - 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 result = new ArrayList(); - memcache.get(matcher, result); + memstore.get(matcher, result); assertEquals(expected.size(), result.size()); for(int i=0; i expected = new ArrayList(); 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 entry : memcache.memcache.entrySet()) { + for(Map.Entry 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 expected = new ArrayList(); expected.add(put3); expected.add(del2); - assertEquals(2, memcache.memcache.size()); + assertEquals(2, memstore.memstore.size()); int i = 0; - for(Map.Entry entry : memcache.memcache.entrySet()) { + for(Map.Entry 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 expected = new ArrayList(); expected.add(del); expected.add(put4); - assertEquals(2, memcache.memcache.size()); + assertEquals(2, memstore.memstore.size()); int i = 0; - for(Map.Entry entry : memcache.memcache.entrySet()) { + for(Map.Entry 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(); diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java b/src/test/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java index 8336f28234a..45354e1d6e5 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java @@ -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 memCache = new ArrayList(); - 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 memstore = new ArrayList(); + 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 actual = new ArrayList(); - 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 memCache = new ArrayList(); - 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 memstore = new ArrayList(); + 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 actual = new ArrayList(); - for(KeyValue kv : memCache){ + for(KeyValue kv : memstore){ actual.add(qm.match(kv)); } diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java b/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java index 23200442ced..f5e8b2e55de 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestScanner.java @@ -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 */ diff --git a/src/test/org/apache/hadoop/hbase/regionserver/TestStore.java b/src/test/org/apache/hadoop/hbase/regionserver/TestStore.java index b8618d57dc1..36a2cdd9e34 100644 --- a/src/test/org/apache/hadoop/hbase/regionserver/TestStore.java +++ b/src/test/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -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)));