diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java index 3607e7dbe17..f1339ac3b0c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRegionObserver.java @@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -78,7 +78,7 @@ public abstract class BaseRegionObserver implements RegionObserver { @Override public InternalScanner preFlushScannerOpen(final ObserverContext c, - final Store store, final KeyValueScanner memstoreScanner, final InternalScanner s) + final HStore store, final KeyValueScanner memstoreScanner, final InternalScanner s) throws IOException { return null; } @@ -92,13 +92,13 @@ public abstract class BaseRegionObserver implements RegionObserver { } @Override - public InternalScanner preFlush(ObserverContext e, Store store, + public InternalScanner preFlush(ObserverContext e, HStore store, InternalScanner scanner) throws IOException { return scanner; } @Override - public void postFlush(ObserverContext e, Store store, + public void postFlush(ObserverContext e, HStore store, StoreFile resultFile) throws IOException { } @@ -113,27 +113,27 @@ public abstract class BaseRegionObserver implements RegionObserver { @Override public void preCompactSelection(final ObserverContext c, - final Store store, final List candidates) throws IOException { } + final HStore store, final List candidates) throws IOException { } @Override public void postCompactSelection(final ObserverContext c, - final Store store, final ImmutableList selected) { } + final HStore store, final ImmutableList selected) { } @Override public InternalScanner preCompact(ObserverContext e, - final Store store, final InternalScanner scanner) throws IOException { + final HStore store, final InternalScanner scanner) throws IOException { return scanner; } @Override public InternalScanner preCompactScannerOpen(final ObserverContext c, - final Store store, List scanners, final ScanType scanType, + final HStore store, List scanners, final ScanType scanType, final long earliestPutTs, final InternalScanner s) throws IOException { return null; } @Override - public void postCompact(ObserverContext e, final Store store, + public void postCompact(ObserverContext e, final HStore store, final StoreFile resultFile) throws IOException { } @@ -270,7 +270,7 @@ public abstract class BaseRegionObserver implements RegionObserver { @Override public KeyValueScanner preStoreScannerOpen(final ObserverContext c, - final Store store, final Scan scan, final NavigableSet targetCols, + final HStore store, final Scan scan, final NavigableSet targetCols, final KeyValueScanner s) throws IOException { return null; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index c3cfa097bbb..ab973e2030a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -39,7 +39,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; @@ -84,14 +84,14 @@ public interface RegionObserver extends Coprocessor { * @throws IOException if an error occurred on the coprocessor */ InternalScanner preFlushScannerOpen(final ObserverContext c, - final Store store, final KeyValueScanner memstoreScanner, final InternalScanner s) + final HStore store, final KeyValueScanner memstoreScanner, final InternalScanner s) throws IOException; /** * Called before the memstore is flushed to disk. * @param c the environment provided by the region server * @throws IOException if an error occurred on the coprocessor - * @deprecated use {@link #preFlush(ObserverContext, Store, InternalScanner)} instead + * @deprecated use {@link #preFlush(ObserverContext, HStore, InternalScanner)} instead */ void preFlush(final ObserverContext c) throws IOException; @@ -104,14 +104,14 @@ public interface RegionObserver extends Coprocessor { * unless the implementation is writing new store files on its own. * @throws IOException if an error occurred on the coprocessor */ - InternalScanner preFlush(final ObserverContext c, final Store store, + InternalScanner preFlush(final ObserverContext c, final HStore store, final InternalScanner scanner) throws IOException; /** * Called after the memstore is flushed to disk. * @param c the environment provided by the region server * @throws IOException if an error occurred on the coprocessor - * @deprecated use {@link #preFlush(ObserverContext, Store, InternalScanner)} instead. + * @deprecated use {@link #preFlush(ObserverContext, HStore, InternalScanner)} instead. */ void postFlush(final ObserverContext c) throws IOException; @@ -122,7 +122,7 @@ public interface RegionObserver extends Coprocessor { * @param resultFile the new store file written out during compaction * @throws IOException if an error occurred on the coprocessor */ - void postFlush(final ObserverContext c, final Store store, + void postFlush(final ObserverContext c, final HStore store, final StoreFile resultFile) throws IOException; /** @@ -135,7 +135,7 @@ public interface RegionObserver extends Coprocessor { * @throws IOException if an error occurred on the coprocessor */ void preCompactSelection(final ObserverContext c, - final Store store, final List candidates) throws IOException; + final HStore store, final List candidates) throws IOException; /** * Called after the {@link StoreFile}s to compact have been selected from the @@ -145,7 +145,7 @@ public interface RegionObserver extends Coprocessor { * @param selected the store files selected to compact */ void postCompactSelection(final ObserverContext c, - final Store store, final ImmutableList selected); + final HStore store, final ImmutableList selected); /** * Called prior to writing the {@link StoreFile}s selected for compaction into @@ -172,7 +172,7 @@ public interface RegionObserver extends Coprocessor { * @throws IOException if an error occurred on the coprocessor */ InternalScanner preCompact(final ObserverContext c, - final Store store, final InternalScanner scanner) throws IOException; + final HStore store, final InternalScanner scanner) throws IOException; /** * Called prior to writing the {@link StoreFile}s selected for compaction into @@ -194,7 +194,7 @@ public interface RegionObserver extends Coprocessor { * @throws IOException if an error occurred on the coprocessor */ InternalScanner preCompactScannerOpen(final ObserverContext c, - final Store store, List scanners, final ScanType scanType, + final HStore store, List scanners, final ScanType scanType, final long earliestPutTs, final InternalScanner s) throws IOException; /** @@ -205,7 +205,7 @@ public interface RegionObserver extends Coprocessor { * @param resultFile the new store file written out during compaction * @throws IOException if an error occurred on the coprocessor */ - void postCompact(final ObserverContext c, final Store store, + void postCompact(final ObserverContext c, final HStore store, StoreFile resultFile) throws IOException; /** @@ -623,8 +623,8 @@ public interface RegionObserver extends Coprocessor { * Called before a store opens a new scanner. * This hook is called when a "user" scanner is opened. *

- * See {@link #preFlushScannerOpen(ObserverContext, Store, KeyValueScanner, InternalScanner)} - * and {@link #preCompactScannerOpen(ObserverContext, Store, List, ScanType, long, InternalScanner)} + * See {@link #preFlushScannerOpen(ObserverContext, HStore, KeyValueScanner, InternalScanner)} + * and {@link #preCompactScannerOpen(ObserverContext, HStore, List, ScanType, long, InternalScanner)} * to override scanners created for flushes or compactions, resp. *

* Call CoprocessorEnvironment#complete to skip any subsequent chained @@ -640,7 +640,7 @@ public interface RegionObserver extends Coprocessor { * @throws IOException if an error occurred on the coprocessor */ KeyValueScanner preStoreScannerOpen(final ObserverContext c, - final Store store, final Scan scan, final NavigableSet targetCols, + final HStore store, final Scan scan, final NavigableSet targetCols, final KeyValueScanner s) throws IOException; /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java index 8d8b1c4ab07..1a0523c1dc2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.Bytes; @@ -205,8 +205,8 @@ public class HFileOutputFormat extends FileOutputFormat= HStore.PRIORITY_USER) { + if (shouldSplitRegion() && r.getCompactPriority() >= Store.PRIORITY_USER) { byte[] midKey = r.checkSplit(); if (midKey != null) { requestSplit(r, midKey); @@ -186,19 +186,19 @@ public class CompactSplitThread implements CompactionRequestor { public synchronized void requestCompaction(final HRegion r, final String why) throws IOException { - for (HStore s : r.getStores().values()) { - requestCompaction(r, s, why, HStore.NO_PRIORITY); + for (Store s : r.getStores().values()) { + requestCompaction(r, s, why, Store.NO_PRIORITY); } } - public synchronized void requestCompaction(final HRegion r, final HStore s, + public synchronized void requestCompaction(final HRegion r, final Store s, final String why) throws IOException { - requestCompaction(r, s, why, HStore.NO_PRIORITY); + requestCompaction(r, s, why, Store.NO_PRIORITY); } public synchronized void requestCompaction(final HRegion r, final String why, int p) throws IOException { - for (HStore s : r.getStores().values()) { + for (Store s : r.getStores().values()) { requestCompaction(r, s, why, p); } } @@ -209,7 +209,7 @@ public class CompactSplitThread implements CompactionRequestor { * @param why Why compaction requested -- used in debug messages * @param priority override the default priority (NO_PRIORITY == decide) */ - public synchronized void requestCompaction(final HRegion r, final HStore s, + public synchronized void requestCompaction(final HRegion r, final Store s, final String why, int priority) throws IOException { if (this.server.isStopped()) { return; @@ -217,7 +217,7 @@ public class CompactSplitThread implements CompactionRequestor { CompactionRequest cr = s.requestCompaction(priority); if (cr != null) { cr.setServer(server); - if (priority != HStore.NO_PRIORITY) { + if (priority != Store.NO_PRIORITY) { cr.setPriority(priority); } ThreadPoolExecutor pool = s.throttleCompaction(cr.getSize()) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java index 8d957fb30a6..ce7f0ba3568 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionRequestor.java @@ -37,7 +37,7 @@ public interface CompactionRequestor { * @param why Why compaction was requested -- used in debug messages * @throws IOException */ - public void requestCompaction(final HRegion r, final HStore s, final String why) + public void requestCompaction(final HRegion r, final Store s, final String why) throws IOException; /** @@ -55,7 +55,7 @@ public interface CompactionRequestor { * @param pri Priority of this compaction. minHeap. <=0 is critical * @throws IOException */ - public void requestCompaction(final HRegion r, final HStore s, + public void requestCompaction(final HRegion r, final Store s, final String why, int pri) throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Compactor.java index 0847d9f7808..a526345e18c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Compactor.java @@ -60,7 +60,7 @@ class Compactor extends Configured { * nothing made it through the compaction. * @throws IOException */ - StoreFile.Writer compact(final Store store, + StoreFile.Writer compact(final HStore store, final Collection filesToCompact, final boolean majorCompaction, final long maxId) throws IOException { @@ -176,9 +176,9 @@ class Compactor extends Configured { ++progress.currentCompactedKVs; // check periodically to see if a system stop is requested - if (Store.closeCheckInterval > 0) { + if (HStore.closeCheckInterval > 0) { bytesWritten += kv.getLength(); - if (bytesWritten > Store.closeCheckInterval) { + if (bytesWritten > HStore.closeCheckInterval) { bytesWritten = 0; isInterrupted(store, writer); } @@ -201,7 +201,7 @@ class Compactor extends Configured { return writer; } - void isInterrupted(final Store store, final StoreFile.Writer writer) + void isInterrupted(final HStore store, final StoreFile.Writer writer) throws IOException { if (store.getHRegion().areWritesEnabled()) return; // Else cleanup. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java index db6e6b5cecd..a35f2858275 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.java @@ -51,7 +51,7 @@ public class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy { boolean force = region.shouldForceSplit(); boolean foundABigStore = false; - for (HStore store : region.getStores().values()) { + for (Store store : region.getStores().values()) { // If any of the stores are unable to split (eg they contain reference files) // then don't split if ((!store.canSplit())) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java index b4ded11b123..78ef84349b0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/GetClosestRowBeforeTracker.java @@ -84,7 +84,7 @@ class GetClosestRowBeforeTracker { * @return True if this kv is expired. */ boolean isExpired(final KeyValue kv) { - return Store.isExpired(kv, this.oldestts); + return HStore.isExpired(kv, this.oldestts); } /* diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 975109ef082..5188befeaa1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -203,7 +203,7 @@ public class HRegion implements HeapSize { // , Writable{ private final AtomicInteger lockIdGenerator = new AtomicInteger(1); static private Random rand = new Random(); - protected final Map stores = new ConcurrentSkipListMap( + protected final Map stores = new ConcurrentSkipListMap( Bytes.BYTES_RAWCOMPARATOR); // Registered region protocol handlers @@ -545,22 +545,22 @@ public class HRegion implements HeapSize { // , Writable{ ThreadPoolExecutor storeOpenerThreadPool = getStoreOpenAndCloseThreadPool( "StoreOpenerThread-" + this.regionInfo.getRegionNameAsString()); - CompletionService completionService = - new ExecutorCompletionService(storeOpenerThreadPool); + CompletionService completionService = + new ExecutorCompletionService(storeOpenerThreadPool); // initialize each store in parallel for (final HColumnDescriptor family : htableDescriptor.getFamilies()) { status.setStatus("Instantiating store for column family " + family); - completionService.submit(new Callable() { - public Store call() throws IOException { + completionService.submit(new Callable() { + public HStore call() throws IOException { return instantiateHStore(tableDir, family); } }); } try { for (int i = 0; i < htableDescriptor.getFamilies().size(); i++) { - Future future = completionService.take(); - Store store = future.get(); + Future future = completionService.take(); + HStore store = future.get(); this.stores.put(store.getColumnFamilyName().getBytes(), store); long storeSeqId = store.getMaxSequenceId(); @@ -642,7 +642,7 @@ public class HRegion implements HeapSize { // , Writable{ * @return True if this region has references. */ public boolean hasReferences() { - for (HStore store : this.stores.values()) { + for (Store store : this.stores.values()) { for (StoreFile sf : store.getStorefiles()) { // Found a reference, return. if (sf.isReference()) return true; @@ -660,7 +660,7 @@ public class HRegion implements HeapSize { // , Writable{ HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); synchronized (this.stores) { - for (HStore store : this.stores.values()) { + for (Store store : this.stores.values()) { for (StoreFile sf : store.getStorefiles()) { HDFSBlocksDistribution storeFileBlocksDistribution = sf.getHDFSBlockDistribution(); @@ -689,7 +689,7 @@ public class HRegion implements HeapSize { // , Writable{ FileSystem fs = tablePath.getFileSystem(conf); for (HColumnDescriptor family: tableDescriptor.getFamilies()) { - Path storeHomeDir = Store.getStoreHomedir(tablePath, regionEncodedName, + Path storeHomeDir = HStore.getStoreHomedir(tablePath, regionEncodedName, family.getName()); if (!fs.exists(storeHomeDir))continue; @@ -977,7 +977,7 @@ public class HRegion implements HeapSize { // , Writable{ storeCloserThreadPool); // close each store in parallel - for (final HStore store : stores.values()) { + for (final Store store : stores.values()) { completionService .submit(new Callable>() { public ImmutableList call() throws IOException { @@ -1173,7 +1173,7 @@ public class HRegion implements HeapSize { // , Writable{ /** @return returns size of largest HStore. */ public long getLargestHStoreSize() { long size = 0; - for (HStore h : stores.values()) { + for (Store h : stores.values()) { long storeSize = h.getSize(); if (storeSize > size) { size = storeSize; @@ -1205,7 +1205,7 @@ public class HRegion implements HeapSize { // , Writable{ } void triggerMajorCompaction() { - for (HStore h : stores.values()) { + for (Store h : stores.values()) { h.triggerMajorCompaction(); } } @@ -1232,7 +1232,7 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException e */ public void compactStores() throws IOException { - for (HStore s : getStores().values()) { + for (Store s : getStores().values()) { CompactionRequest cr = s.requestCompaction(); if(cr != null) { try { @@ -1500,7 +1500,7 @@ public class HRegion implements HeapSize { // , Writable{ wal.startCacheFlush(this.regionInfo.getEncodedNameAsBytes()); completeSequenceId = this.getCompleteCacheFlushSequenceId(sequenceId); - for (HStore s : stores.values()) { + for (Store s : stores.values()) { storeFlushers.add(s.getStoreFlusher(completeSequenceId)); } @@ -1658,7 +1658,7 @@ public class HRegion implements HeapSize { // , Writable{ startRegionOperation(); this.readRequestsCount.increment(); try { - HStore store = getStore(family); + Store store = getStore(family); // get the closest key. (HStore.getRowKeyAtOrBefore can return null) KeyValue key = store.getRowKeyAtOrBefore(row); Result result = null; @@ -2662,7 +2662,7 @@ public class HRegion implements HeapSize { // , Writable{ byte[] family = e.getKey(); List edits = e.getValue(); - HStore store = getStore(family); + Store store = getStore(family); for (KeyValue kv: edits) { kv.setMemstoreTS(localizedWriteEntry.getWriteNumber()); size += store.add(kv); @@ -2702,7 +2702,7 @@ public class HRegion implements HeapSize { // , Writable{ // Remove those keys from the memstore that matches our // key's (row, cf, cq, timestamp, memstoreTS). The interesting part is // that even the memstoreTS has to match for keys that will be rolleded-back. - HStore store = getStore(family); + Store store = getStore(family); for (KeyValue kv: edits) { store.rollback(kv); kvsRolledback++; @@ -2918,7 +2918,7 @@ public class HRegion implements HeapSize { // , Writable{ long editsCount = 0; long intervalEdits = 0; HLog.Entry entry; - HStore store = null; + Store store = null; boolean reported_once = false; try { @@ -3056,7 +3056,7 @@ public class HRegion implements HeapSize { // , Writable{ * @param kv KeyValue to add. * @return True if we should flush. */ - protected boolean restoreEdit(final HStore s, final KeyValue kv) { + protected boolean restoreEdit(final Store s, final KeyValue kv) { long kvSize = s.add(kv); if (this.rsAccounting != null) { rsAccounting.addAndGetRegionReplayEditsSize(this.regionInfo.getRegionName(), kvSize); @@ -3079,9 +3079,9 @@ public class HRegion implements HeapSize { // , Writable{ return true; } - protected Store instantiateHStore(Path tableDir, HColumnDescriptor c) + protected HStore instantiateHStore(Path tableDir, HColumnDescriptor c) throws IOException { - return new Store(tableDir, this, c, this.fs, this.conf); + return new HStore(tableDir, this, c, this.fs, this.conf); } /** @@ -3091,11 +3091,11 @@ public class HRegion implements HeapSize { // , Writable{ * @return Store that goes with the family on passed column. * TODO: Make this lookup faster. */ - public HStore getStore(final byte[] column) { + public Store getStore(final byte[] column) { return this.stores.get(column); } - public Map getStores() { + public Map getStores() { return this.stores; } @@ -3111,7 +3111,7 @@ public class HRegion implements HeapSize { // , Writable{ List storeFileNames = new ArrayList(); synchronized(closeLock) { for(byte[] column : columns) { - HStore store = this.stores.get(column); + Store store = this.stores.get(column); if (store == null) { throw new IllegalArgumentException("No column family : " + new String(column) + " available"); @@ -3331,7 +3331,7 @@ public class HRegion implements HeapSize { // , Writable{ byte[] familyName = p.getFirst(); String path = p.getSecond(); - HStore store = getStore(familyName); + Store store = getStore(familyName); if (store == null) { IOException ioe = new DoNotRetryIOException( "No such column family " + Bytes.toStringBinary(familyName)); @@ -3373,7 +3373,7 @@ public class HRegion implements HeapSize { // , Writable{ for (Pair p : familyPaths) { byte[] familyName = p.getFirst(); String path = p.getSecond(); - HStore store = getStore(familyName); + Store store = getStore(familyName); try { store.bulkLoadHFile(path); } catch (IOException ioe) { @@ -3474,7 +3474,7 @@ public class HRegion implements HeapSize { // , Writable{ for (Map.Entry> entry : scan.getFamilyMap().entrySet()) { - HStore store = stores.get(entry.getKey()); + Store store = stores.get(entry.getKey()); KeyValueScanner scanner = store.getScanner(scan, entry.getValue()); scanners.add(scanner); } @@ -4054,7 +4054,7 @@ public class HRegion implements HeapSize { // , Writable{ public static void makeColumnFamilyDirs(FileSystem fs, Path tabledir, final HRegionInfo hri, byte [] colFamily) throws IOException { - Path dir = Store.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily); + Path dir = HStore.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily); if (!fs.mkdirs(dir)) { LOG.warn("Failed to create " + dir); } @@ -4195,7 +4195,7 @@ public class HRegion implements HeapSize { // , Writable{ } for (StoreFile hsf: srcFiles) { StoreFile.rename(fs, hsf.getPath(), - StoreFile.getUniqueFile(fs, Store.getStoreHomedir(tableDir, + StoreFile.getUniqueFile(fs, HStore.getStoreHomedir(tableDir, newRegionInfo.getEncodedName(), colFamily))); } } @@ -4252,7 +4252,7 @@ public class HRegion implements HeapSize { // , Writable{ * @throws IOException */ boolean isMajorCompaction() throws IOException { - for (HStore store : this.stores.values()) { + for (Store store : this.stores.values()) { if (store.isMajorCompaction()) { return true; } @@ -4638,7 +4638,7 @@ public class HRegion implements HeapSize { // , Writable{ boolean flush = false; WALEdit walEdits = null; List allKVs = new ArrayList(append.size()); - Map> tempMemstore = new HashMap>(); + Map> tempMemstore = new HashMap>(); long before = EnvironmentEdgeManager.currentTimeMillis(); long size = 0; long txid = 0; @@ -4655,7 +4655,7 @@ public class HRegion implements HeapSize { // , Writable{ for (Map.Entry> family : append.getFamilyMap() .entrySet()) { - HStore store = stores.get(family.getKey()); + Store store = stores.get(family.getKey()); List kvs = new ArrayList(family.getValue().size()); // Get previous values for all columns in this family @@ -4738,8 +4738,8 @@ public class HRegion implements HeapSize { // , Writable{ } //Actually write to Memstore now - for (Map.Entry> entry : tempMemstore.entrySet()) { - HStore store = entry.getKey(); + for (Map.Entry> entry : tempMemstore.entrySet()) { + Store store = entry.getKey(); size += store.upsert(entry.getValue()); allKVs.addAll(entry.getValue()); } @@ -4791,7 +4791,7 @@ public class HRegion implements HeapSize { // , Writable{ boolean flush = false; WALEdit walEdits = null; List allKVs = new ArrayList(increment.numColumns()); - Map> tempMemstore = new HashMap>(); + Map> tempMemstore = new HashMap>(); long before = EnvironmentEdgeManager.currentTimeMillis(); long size = 0; long txid = 0; @@ -4808,7 +4808,7 @@ public class HRegion implements HeapSize { // , Writable{ for (Map.Entry> family : increment.getFamilyMap().entrySet()) { - HStore store = stores.get(family.getKey()); + Store store = stores.get(family.getKey()); List kvs = new ArrayList(family.getValue().size()); // Get previous values for all columns in this family @@ -4860,8 +4860,8 @@ public class HRegion implements HeapSize { // , Writable{ } //Actually write to Memstore now - for (Map.Entry> entry : tempMemstore.entrySet()) { - HStore store = entry.getKey(); + for (Map.Entry> entry : tempMemstore.entrySet()) { + Store store = entry.getKey(); size += store.upsert(entry.getValue()); allKVs.addAll(entry.getValue()); } @@ -4918,7 +4918,7 @@ public class HRegion implements HeapSize { // , Writable{ Integer lid = obtainRowLock(row); this.updatesLock.readLock().lock(); try { - HStore store = stores.get(family); + Store store = stores.get(family); // Get the old value: Get get = new Get(row); @@ -5029,7 +5029,7 @@ public class HRegion implements HeapSize { // , Writable{ @Override public long heapSize() { long heapSize = DEEP_OVERHEAD; - for (HStore store : this.stores.values()) { + for (Store store : this.stores.values()) { heapSize += store.heapSize(); } // this does not take into account row locks, recent flushes, mvcc entries @@ -5274,7 +5274,7 @@ public class HRegion implements HeapSize { // , Writable{ */ public int getCompactPriority() { int count = Integer.MAX_VALUE; - for (HStore store : stores.values()) { + for (Store store : stores.values()) { count = Math.min(count, store.getCompactPriority()); } return count; @@ -5286,7 +5286,7 @@ public class HRegion implements HeapSize { // , Writable{ * @return true if any store has too many store files */ public boolean needsCompaction() { - for (HStore store : stores.values()) { + for (Store store : stores.values()) { if(store.needsCompaction()) { return true; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 665f55a8492..182e173488e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1142,7 +1142,7 @@ public class HRegionServer implements ClientProtocol, long currentCompactedKVs = 0; synchronized (r.stores) { stores += r.stores.size(); - for (HStore store : r.stores.values()) { + for (Store store : r.stores.values()) { storefiles += store.getStorefilesCount(); storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed() / 1024 / 1024); @@ -1228,7 +1228,7 @@ public class HRegionServer implements ClientProtocol, for (HRegion r : this.instance.onlineRegions.values()) { if (r == null) continue; - for (HStore s : r.getStores().values()) { + for (Store s : r.getStores().values()) { try { if (s.needsCompaction()) { // Queue a compaction. Will recognize if major is needed. @@ -1369,8 +1369,8 @@ public class HRegionServer implements ClientProtocol, writeRequestsCount += r.writeRequestsCount.get(); synchronized (r.stores) { stores += r.stores.size(); - for (Map.Entry ee : r.stores.entrySet()) { - final HStore store = ee.getValue(); + for (Map.Entry ee : r.stores.entrySet()) { + final Store store = ee.getValue(); final SchemaMetrics schemaMetrics = store.getSchemaMetrics(); { @@ -1644,7 +1644,7 @@ public class HRegionServer implements ClientProtocol, LOG.info("Post open deploy tasks for region=" + r.getRegionNameAsString() + ", daughter=" + daughter); // Do checks to see if we need to compact (references or too many files) - for (HStore s : r.getStores().values()) { + for (Store s : r.getStores().values()) { if (s.hasReferences() || s.needsCompaction()) { getCompactionRequester().requestCompaction(r, s, "Opening Region"); } @@ -2009,7 +2009,7 @@ public class HRegionServer implements ClientProtocol, int storefileIndexSizeMB = 0; synchronized (r.stores) { stores += r.stores.size(); - for (HStore store : r.stores.values()) { + for (Store store : r.stores.values()) { storefiles += store.getStorefilesCount(); storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024); storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024); @@ -3590,7 +3590,7 @@ public class HRegionServer implements ClientProtocol, region.getRegionNameAsString()); compactSplitThread.requestCompaction(region, "User-triggered " + (major ? "major " : "") + "compaction", - HStore.PRIORITY_USER); + Store.PRIORITY_USER); return CompactRegionResponse.newBuilder().build(); } catch (IOException ie) { throw new ServiceException(ie); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 20a38b053ec..1d84963338e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -1,4 +1,6 @@ /** + * Copyright 2010 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 @@ -18,270 +20,2191 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.NavigableSet; +import java.util.Random; +import java.util.SortedSet; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValue.KVComparator; +import org.apache.hadoop.hbase.RemoteExceptionHandler; +import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.Compression; +import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; +import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; +import org.apache.hadoop.hbase.io.hfile.HFileScanner; +import org.apache.hadoop.hbase.io.hfile.InvalidHFileException; +import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder; +import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; -import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.SchemaAware; +import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured; +import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.ChecksumType; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.hbase.util.CollectionBackedScanner; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.util.StringUtils; +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; /** - * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or - * more StoreFiles, which stretch backwards over time. + * 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 + * and locking is handled at the HRegion level. Store just provides + * services to manage sets of StoreFiles. One of the most important of those + * services is compaction services where files are aggregated once they pass + * a configurable threshold. + * + *

The only thing having to do with logs that Store needs to deal with is + * the reconstructionLog. This is a segment of an HRegion's log that might + * NOT be present upon startup. If the param is NULL, there's nothing to do. + * If the param is non-NULL, we need to process the log to reconstruct + * a TreeMap that might not have been written to disk before the process + * died. + * + *

It's assumed that after this constructor returns, the reconstructionLog + * file will be deleted (by whoever has instantiated the Store). + * + *

Locking and transactions are handled at a higher level. This API should + * not be called directly but by an HRegion manager. */ @InterfaceAudience.Private -@InterfaceStability.Evolving -public interface HStore extends SchemaAware, HeapSize { +public class HStore extends SchemaConfigured implements Store { + static final Log LOG = LogFactory.getLog(HStore.class); - /* The default priority for user-specified compaction requests. - * The user gets top priority unless we have blocking compactions. (Pri <= 0) + protected final MemStore memstore; + // This stores directory in the filesystem. + private final Path homedir; + private final HRegion region; + private final HColumnDescriptor family; + final FileSystem fs; + final Configuration conf; + final CacheConfig cacheConf; + // ttl in milliseconds. + private long ttl; + private final int minFilesToCompact; + private final int maxFilesToCompact; + private final long minCompactSize; + private final long maxCompactSize; + private long lastCompactSize = 0; + volatile boolean forceMajor = false; + /* how many bytes to write between status checks */ + static int closeCheckInterval = 0; + private final int blockingStoreFileCount; + private volatile long storeSize = 0L; + private volatile long totalUncompressedBytes = 0L; + private final Object flushLock = new Object(); + final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + private final boolean verifyBulkLoads; + + // not private for testing + /* package */ScanInfo scanInfo; + /* + * List of store files inside this store. This is an immutable list that + * is atomically replaced when its contents change. */ - public static final int PRIORITY_USER = 1; - public static final int NO_PRIORITY = Integer.MIN_VALUE; + private ImmutableList storefiles = null; - // General Accessors - public KeyValue.KVComparator getComparator(); + List filesCompacting = Lists.newArrayList(); - public List getStorefiles(); + // All access must be synchronized. + private final CopyOnWriteArraySet changedReaderObservers = + new CopyOnWriteArraySet(); + + private final int blocksize; + private HFileDataBlockEncoder dataBlockEncoder; + + /** Checksum configuration */ + private ChecksumType checksumType; + private int bytesPerChecksum; + + // Comparing KeyValues + final KeyValue.KVComparator comparator; + + private final Compactor compactor; /** - * Close all the readers We don't need to worry about subsequent requests because the HRegion - * holds a write lock that will prevent any more reads or writes. - * @return the {@link StoreFile StoreFiles} that were previously being used. - * @throws IOException on failure - */ - public ImmutableList close() throws IOException; - - /** - * Return a scanner for both the memstore and the HStore files. Assumes we are not in a - * compaction. - * @param scan Scan to apply when scanning the stores - * @param targetCols columns to scan - * @return a scanner over the current key values - * @throws IOException on failure - */ - public KeyValueScanner getScanner(Scan scan, final NavigableSet targetCols) - throws IOException; - - /** - * Updates the value for the given row/family/qualifier. This function will always be seen as - * atomic by other readers because it only puts a single KV to memstore. Thus no read/write - * control necessary. - * @param row row to update - * @param f family to update - * @param qualifier qualifier to update - * @param newValue the new value to set into memstore - * @return memstore size delta + * Constructor + * @param basedir qualified path under which the region directory lives; + * generally the table subdirectory + * @param region + * @param family HColumnDescriptor for this column + * @param fs file system object + * @param confParam configuration object + * failed. Can be null. * @throws IOException */ - public long updateColumnValue(byte[] row, byte[] f, byte[] qualifier, long newValue) - throws IOException; + protected HStore(Path basedir, HRegion region, HColumnDescriptor family, + FileSystem fs, Configuration confParam) + throws IOException { + super(new CompoundConfiguration().add(confParam).add( + family.getValues()), region.getRegionInfo().getTableNameAsString(), + Bytes.toString(family.getName())); + HRegionInfo info = region.getRegionInfo(); + this.fs = fs; + // Assemble the store's home directory. + Path p = getStoreHomedir(basedir, info.getEncodedName(), family.getName()); + // Ensure it exists. + this.homedir = createStoreHomeDir(this.fs, p); + this.region = region; + this.family = family; + // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor + this.conf = new CompoundConfiguration() + .add(confParam) + .add(family.getValues()); + this.blocksize = family.getBlocksize(); + + this.dataBlockEncoder = + new HFileDataBlockEncoderImpl(family.getDataBlockEncodingOnDisk(), + family.getDataBlockEncoding()); + + this.comparator = info.getComparator(); + // Get TTL + this.ttl = getTTL(family); + // used by ScanQueryMatcher + long timeToPurgeDeletes = + Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0); + LOG.trace("Time to purge deletes set to " + timeToPurgeDeletes + + "ms in store " + this); + // Why not just pass a HColumnDescriptor in here altogether? Even if have + // to clone it? + scanInfo = new ScanInfo(family, ttl, timeToPurgeDeletes, this.comparator); + this.memstore = new MemStore(conf, this.comparator); + + // By default, compact if storefile.count >= minFilesToCompact + this.minFilesToCompact = Math.max(2, + conf.getInt("hbase.hstore.compaction.min", + /*old name*/ conf.getInt("hbase.hstore.compactionThreshold", 3))); + LOG.info("hbase.hstore.compaction.min = " + this.minFilesToCompact); + + // Setting up cache configuration for this family + this.cacheConf = new CacheConfig(conf, family); + this.blockingStoreFileCount = + conf.getInt("hbase.hstore.blockingStoreFiles", 7); + + this.maxFilesToCompact = conf.getInt("hbase.hstore.compaction.max", 10); + this.minCompactSize = conf.getLong("hbase.hstore.compaction.min.size", + this.region.memstoreFlushSize); + this.maxCompactSize + = conf.getLong("hbase.hstore.compaction.max.size", Long.MAX_VALUE); + + this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false); + + if (HStore.closeCheckInterval == 0) { + HStore.closeCheckInterval = conf.getInt( + "hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */); + } + this.storefiles = sortAndClone(loadStoreFiles()); + + // Initialize checksum type from name. The names are CRC32, CRC32C, etc. + this.checksumType = getChecksumType(conf); + // initilize bytes per checksum + this.bytesPerChecksum = getBytesPerChecksum(conf); + // Create a compaction tool instance + this.compactor = new Compactor(this.conf); + } /** - * Adds or replaces the specified KeyValues. - *

- * For each KeyValue specified, if a cell with the same row, family, and qualifier exists in - * MemStore, it will be replaced. Otherwise, it will just be inserted to MemStore. - *

- * This operation is atomic on each KeyValue (row/family/qualifier) but not necessarily atomic - * across all of them. - * @param kvs - * @return memstore size delta + * @param family + * @return + */ + long getTTL(final HColumnDescriptor family) { + // HCD.getTimeToLive returns ttl in seconds. Convert to milliseconds. + long ttl = family.getTimeToLive(); + if (ttl == HConstants.FOREVER) { + // Default is unlimited ttl. + ttl = Long.MAX_VALUE; + } else if (ttl == -1) { + ttl = Long.MAX_VALUE; + } else { + // Second -> ms adjust for user data + ttl *= 1000; + } + return ttl; + } + + /** + * Create this store's homedir + * @param fs + * @param homedir + * @return Return homedir * @throws IOException */ - public long upsert(Iterable kvs) throws IOException; + Path createStoreHomeDir(final FileSystem fs, + final Path homedir) throws IOException { + if (!fs.exists(homedir)) { + if (!fs.mkdirs(homedir)) + throw new IOException("Failed create of: " + homedir.toString()); + } + return homedir; + } + + FileSystem getFileSystem() { + return this.fs; + } + + /** + * Returns the configured bytesPerChecksum value. + * @param conf The configuration + * @return The bytesPerChecksum that is set in the configuration + */ + public static int getBytesPerChecksum(Configuration conf) { + return conf.getInt(HConstants.BYTES_PER_CHECKSUM, + HFile.DEFAULT_BYTES_PER_CHECKSUM); + } + + /** + * Returns the configured checksum algorithm. + * @param conf The configuration + * @return The checksum algorithm that is set in the configuration + */ + public static ChecksumType getChecksumType(Configuration conf) { + String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME); + if (checksumName == null) { + return HFile.DEFAULT_CHECKSUM_TYPE; + } else { + return ChecksumType.nameToType(checksumName); + } + } + + public HColumnDescriptor getFamily() { + return this.family; + } + + /** + * @return The maximum sequence id in all store files. + */ + long getMaxSequenceId() { + return StoreFile.getMaxSequenceIdInList(this.getStorefiles()); + } + + @Override + public long getMaxMemstoreTS() { + return StoreFile.getMaxMemstoreTSInList(this.getStorefiles()); + } + + /** + * @param tabledir + * @param encodedName Encoded region name. + * @param family + * @return Path to family/Store home directory. + */ + public static Path getStoreHomedir(final Path tabledir, + final String encodedName, final byte [] family) { + return new Path(tabledir, new Path(encodedName, + new Path(Bytes.toString(family)))); + } + + /** + * Return the directory in which this store stores its + * StoreFiles + */ + Path getHomedir() { + return homedir; + } + + @Override + public HFileDataBlockEncoder getDataBlockEncoder() { + return dataBlockEncoder; + } + + /** + * Should be used only in tests. + * @param blockEncoder the block delta encoder to use + */ + void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) { + this.dataBlockEncoder = blockEncoder; + } + + FileStatus[] getStoreFiles() throws IOException { + return FSUtils.listStatus(this.fs, this.homedir, null); + } + + /** + * Creates an unsorted list of StoreFile loaded in parallel + * from the given directory. + * @throws IOException + */ + private List loadStoreFiles() throws IOException { + ArrayList results = new ArrayList(); + FileStatus files[] = getStoreFiles(); + + if (files == null || files.length == 0) { + return results; + } + // initialize the thread pool for opening store files in parallel.. + ThreadPoolExecutor storeFileOpenerThreadPool = + this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" + + this.family.getNameAsString()); + CompletionService completionService = + new ExecutorCompletionService(storeFileOpenerThreadPool); + + int totalValidStoreFile = 0; + for (int i = 0; i < files.length; i++) { + // Skip directories. + if (files[i].isDir()) { + continue; + } + final Path p = files[i].getPath(); + // Check for empty file. Should never be the case but can happen + // after data loss in hdfs for whatever reason (upgrade, etc.): HBASE-646 + if (this.fs.getFileStatus(p).getLen() <= 0) { + LOG.warn("Skipping " + p + " because its empty. HBASE-646 DATA LOSS?"); + continue; + } + + // open each store file in parallel + completionService.submit(new Callable() { + public StoreFile call() throws IOException { + StoreFile storeFile = new StoreFile(fs, p, conf, cacheConf, + family.getBloomFilterType(), dataBlockEncoder); + passSchemaMetricsTo(storeFile); + storeFile.createReader(); + return storeFile; + } + }); + totalValidStoreFile++; + } + + try { + for (int i = 0; i < totalValidStoreFile; i++) { + Future future = completionService.take(); + StoreFile storeFile = future.get(); + long length = storeFile.getReader().length(); + this.storeSize += length; + this.totalUncompressedBytes += + storeFile.getReader().getTotalUncompressedBytes(); + if (LOG.isDebugEnabled()) { + LOG.debug("loaded " + storeFile.toStringDetailed()); + } + results.add(storeFile); + } + } catch (InterruptedException e) { + throw new IOException(e); + } catch (ExecutionException e) { + throw new IOException(e.getCause()); + } finally { + storeFileOpenerThreadPool.shutdownNow(); + } + + return results; + } + + @Override + public long add(final KeyValue kv) { + lock.readLock().lock(); + try { + return this.memstore.add(kv); + } finally { + lock.readLock().unlock(); + } + } /** * Adds a value to the memstore + * * @param kv * @return memstore size delta */ - public long add(KeyValue kv); + protected long delete(final KeyValue kv) { + lock.readLock().lock(); + try { + return this.memstore.delete(kv); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void rollback(final KeyValue kv) { + lock.readLock().lock(); + try { + this.memstore.rollback(kv); + } finally { + lock.readLock().unlock(); + } + } /** - * Removes a kv from the memstore. The KeyValue is removed only if its key & memstoreTS match the - * key & memstoreTS value of the kv parameter. - * @param kv + * @return All store files. */ - public void rollback(final KeyValue kv); + @Override + public List getStorefiles() { + return this.storefiles; + } + + @Override + public void assertBulkLoadHFileOk(Path srcPath) throws IOException { + HFile.Reader reader = null; + try { + LOG.info("Validating hfile at " + srcPath + " for inclusion in " + + "store " + this + " region " + this.region); + reader = HFile.createReader(srcPath.getFileSystem(conf), + srcPath, cacheConf); + reader.loadFileInfo(); + + byte[] firstKey = reader.getFirstRowKey(); + Preconditions.checkState(firstKey != null, "First key can not be null"); + byte[] lk = reader.getLastKey(); + Preconditions.checkState(lk != null, "Last key can not be null"); + byte[] lastKey = KeyValue.createKeyValueFromKey(lk).getRow(); + + LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) + + " last=" + Bytes.toStringBinary(lastKey)); + LOG.debug("Region bounds: first=" + + Bytes.toStringBinary(region.getStartKey()) + + " last=" + Bytes.toStringBinary(region.getEndKey())); + + HRegionInfo hri = region.getRegionInfo(); + if (!hri.containsRange(firstKey, lastKey)) { + throw new WrongRegionException( + "Bulk load file " + srcPath.toString() + " does not fit inside region " + + this.region); + } + + if (verifyBulkLoads) { + KeyValue prevKV = null; + HFileScanner scanner = reader.getScanner(false, false, false); + scanner.seekTo(); + do { + KeyValue kv = scanner.getKeyValue(); + if (prevKV != null) { + if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getRowOffset(), + prevKV.getRowLength(), kv.getBuffer(), kv.getRowOffset(), + kv.getRowLength()) > 0) { + throw new InvalidHFileException("Previous row is greater than" + + " current row: path=" + srcPath + " previous=" + + Bytes.toStringBinary(prevKV.getKey()) + " current=" + + Bytes.toStringBinary(kv.getKey())); + } + if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getFamilyOffset(), + prevKV.getFamilyLength(), kv.getBuffer(), kv.getFamilyOffset(), + kv.getFamilyLength()) != 0) { + throw new InvalidHFileException("Previous key had different" + + " family compared to current key: path=" + srcPath + + " previous=" + Bytes.toStringBinary(prevKV.getFamily()) + + " current=" + Bytes.toStringBinary(kv.getFamily())); + } + } + prevKV = kv; + } while (scanner.next()); + } + } finally { + if (reader != null) reader.close(); + } + } + + @Override + public void bulkLoadHFile(String srcPathStr) throws IOException { + Path srcPath = new Path(srcPathStr); + + // Copy the file if it's on another filesystem + FileSystem srcFs = srcPath.getFileSystem(conf); + FileSystem desFs = fs instanceof HFileSystem ? ((HFileSystem)fs).getBackingFs() : fs; + if (!srcFs.equals(desFs)) { + LOG.info("Bulk-load file " + srcPath + " is on different filesystem than " + + "the destination store. Copying file over to destination filesystem."); + Path tmpPath = getTmpPath(); + FileUtil.copy(srcFs, srcPath, fs, tmpPath, false, conf); + LOG.info("Copied " + srcPath + " to temporary path on destination filesystem: " + tmpPath); + srcPath = tmpPath; + } + + Path dstPath = StoreFile.getRandomFilename(fs, homedir); + LOG.debug("Renaming bulk load file " + srcPath + " to " + dstPath); + StoreFile.rename(fs, srcPath, dstPath); + + StoreFile sf = new StoreFile(fs, dstPath, this.conf, this.cacheConf, + this.family.getBloomFilterType(), this.dataBlockEncoder); + passSchemaMetricsTo(sf); + + StoreFile.Reader r = sf.createReader(); + this.storeSize += r.length(); + this.totalUncompressedBytes += r.getTotalUncompressedBytes(); + + LOG.info("Moved HFile " + srcPath + " into store directory " + + homedir + " - updating store file list."); + + // Append the new storefile into the list + this.lock.writeLock().lock(); + try { + ArrayList newFiles = new ArrayList(storefiles); + newFiles.add(sf); + this.storefiles = sortAndClone(newFiles); + } finally { + // We need the lock, as long as we are updating the storefiles + // or changing the memstore. Let us release it before calling + // notifyChangeReadersObservers. See HBASE-4485 for a possible + // deadlock scenario that could have happened if continue to hold + // the lock. + this.lock.writeLock().unlock(); + } + notifyChangedReadersObservers(); + LOG.info("Successfully loaded store file " + srcPath + + " into store " + this + " (new location: " + dstPath + ")"); + } /** - * Find the key that matches row exactly, or the one that immediately precedes it. WARNING: - * Only use this method on a table where writes occur with strictly increasing timestamps. This - * method assumes this pattern of writes in order to make it reasonably performant. Also our - * search is dependent on the axiom that deletes are for cells that are in the container that - * follows whether a memstore snapshot or a storefile, not for the current container: i.e. we'll - * see deletes before we come across cells we are to delete. Presumption is that the - * memstore#kvset is processed before memstore#snapshot and so on. - * @param row The row key of the targeted row. - * @return Found keyvalue or null if none found. + * Get a temporary path in this region. These temporary files + * will get cleaned up when the region is re-opened if they are + * still around. + */ + private Path getTmpPath() throws IOException { + return StoreFile.getRandomFilename( + fs, region.getTmpDir()); + } + + @Override + public ImmutableList close() throws IOException { + this.lock.writeLock().lock(); + try { + ImmutableList result = storefiles; + + // Clear so metrics doesn't find them. + storefiles = ImmutableList.of(); + + if (!result.isEmpty()) { + // initialize the thread pool for closing store files in parallel. + ThreadPoolExecutor storeFileCloserThreadPool = this.region + .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-" + + this.family.getNameAsString()); + + // close each store file in parallel + CompletionService completionService = + new ExecutorCompletionService(storeFileCloserThreadPool); + for (final StoreFile f : result) { + completionService.submit(new Callable() { + public Void call() throws IOException { + f.closeReader(true); + return null; + } + }); + } + + try { + for (int i = 0; i < result.size(); i++) { + Future future = completionService.take(); + future.get(); + } + } catch (InterruptedException e) { + throw new IOException(e); + } catch (ExecutionException e) { + throw new IOException(e.getCause()); + } finally { + storeFileCloserThreadPool.shutdownNow(); + } + } + LOG.info("Closed " + this); + return result; + } finally { + this.lock.writeLock().unlock(); + } + } + + /** + * Snapshot this stores memstore. Call before running + * {@link #flushCache(long, SortedSet, TimeRangeTracker, AtomicLong, MonitoredTask)} so it has + * some work to do. + */ + void snapshot() { + this.memstore.snapshot(); + } + + /** + * Write out current snapshot. Presumes {@link #snapshot()} has been called + * previously. + * @param logCacheFlushId flush sequence number + * @param snapshot + * @param snapshotTimeRangeTracker + * @param flushedSize The number of bytes flushed + * @param status + * @return Path The path name of the tmp file to which the store was flushed * @throws IOException */ - public KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException; + private Path flushCache(final long logCacheFlushId, + SortedSet snapshot, + TimeRangeTracker snapshotTimeRangeTracker, + AtomicLong flushedSize, + MonitoredTask status) throws IOException { + // If an exception happens flushing, we let it out without clearing + // the memstore snapshot. The old snapshot will be returned when we say + // 'snapshot', the next time flush comes around. + return internalFlushCache( + snapshot, logCacheFlushId, snapshotTimeRangeTracker, flushedSize, status); + } - // Compaction oriented methods + /* + * @param cache + * @param logCacheFlushId + * @param snapshotTimeRangeTracker + * @param flushedSize The number of bytes flushed + * @return Path The path name of the tmp file to which the store was flushed + * @throws IOException + */ + private Path internalFlushCache(final SortedSet set, + final long logCacheFlushId, + TimeRangeTracker snapshotTimeRangeTracker, + AtomicLong flushedSize, + MonitoredTask status) + throws IOException { + StoreFile.Writer writer; + // Find the smallest read point across all the Scanners. + long smallestReadPoint = region.getSmallestReadPoint(); + long flushed = 0; + Path pathName; + // Don't flush if there are no entries. + if (set.size() == 0) { + return null; + } + // Use a store scanner to find which rows to flush. + // Note that we need to retain deletes, hence + // treat this as a minor compaction. + InternalScanner scanner = null; + KeyValueScanner memstoreScanner = new CollectionBackedScanner(set, this.comparator); + if (getHRegion().getCoprocessorHost() != null) { + scanner = getHRegion().getCoprocessorHost().preFlushScannerOpen(this, memstoreScanner); + } + if (scanner == null) { + Scan scan = new Scan(); + scan.setMaxVersions(scanInfo.getMaxVersions()); + scanner = new StoreScanner(this, scanInfo, scan, + Collections.singletonList(memstoreScanner), ScanType.MINOR_COMPACT, + this.region.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP); + } + if (getHRegion().getCoprocessorHost() != null) { + InternalScanner cpScanner = + getHRegion().getCoprocessorHost().preFlush(this, scanner); + // NULL scanner returned from coprocessor hooks means skip normal processing + if (cpScanner == null) { + return null; + } + scanner = cpScanner; + } + try { + int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, 10); + // TODO: We can fail in the below block before we complete adding this + // flush to list of store files. Add cleanup of anything put on filesystem + // if we fail. + synchronized (flushLock) { + status.setStatus("Flushing " + this + ": creating writer"); + // A. Write the map out to the disk + writer = createWriterInTmp(set.size()); + writer.setTimeRangeTracker(snapshotTimeRangeTracker); + pathName = writer.getPath(); + try { + List kvs = new ArrayList(); + boolean hasMore; + do { + hasMore = scanner.next(kvs, compactionKVMax); + if (!kvs.isEmpty()) { + for (KeyValue kv : kvs) { + // If we know that this KV is going to be included always, then let us + // set its memstoreTS to 0. This will help us save space when writing to disk. + if (kv.getMemstoreTS() <= smallestReadPoint) { + // let us not change the original KV. It could be in the memstore + // changing its memstoreTS could affect other threads/scanners. + kv = kv.shallowCopy(); + kv.setMemstoreTS(0); + } + writer.append(kv); + flushed += this.memstore.heapSizeChange(kv, true); + } + kvs.clear(); + } + } while (hasMore); + } finally { + // Write out the log sequence number that corresponds to this output + // hfile. The hfile is current up to and including logCacheFlushId. + status.setStatus("Flushing " + this + ": appending metadata"); + writer.appendMetadata(logCacheFlushId, false); + status.setStatus("Flushing " + this + ": closing flushed file"); + writer.close(); + } + } + } finally { + flushedSize.set(flushed); + scanner.close(); + } + if (LOG.isInfoEnabled()) { + LOG.info("Flushed " + + ", sequenceid=" + logCacheFlushId + + ", memsize=" + StringUtils.humanReadableInt(flushed) + + ", into tmp file " + pathName); + } + return pathName; + } - public boolean throttleCompaction(long compactionSize); + /* + * @param path The pathname of the tmp file into which the store was flushed + * @param logCacheFlushId + * @return StoreFile created. + * @throws IOException + */ + private StoreFile commitFile(final Path path, + final long logCacheFlushId, + TimeRangeTracker snapshotTimeRangeTracker, + AtomicLong flushedSize, + MonitoredTask status) + throws IOException { + // Write-out finished successfully, move into the right spot + String fileName = path.getName(); + Path dstPath = new Path(homedir, fileName); + validateStoreFile(path); + String msg = "Renaming flushed file at " + path + " to " + dstPath; + LOG.debug(msg); + status.setStatus("Flushing " + this + ": " + msg); + if (!fs.rename(path, dstPath)) { + LOG.warn("Unable to rename " + path + " to " + dstPath); + } + + status.setStatus("Flushing " + this + ": reopening flushed file"); + StoreFile sf = new StoreFile(this.fs, dstPath, this.conf, this.cacheConf, + this.family.getBloomFilterType(), this.dataBlockEncoder); + passSchemaMetricsTo(sf); + + StoreFile.Reader r = sf.createReader(); + this.storeSize += r.length(); + this.totalUncompressedBytes += r.getTotalUncompressedBytes(); + + // This increments the metrics associated with total flushed bytes for this + // family. The overall flush count is stored in the static metrics and + // retrieved from HRegion.recentFlushes, which is set within + // HRegion.internalFlushcache, which indirectly calls this to actually do + // the flushing through the StoreFlusherImpl class + getSchemaMetrics().updatePersistentStoreMetric( + SchemaMetrics.StoreMetricType.FLUSH_SIZE, flushedSize.longValue()); + if (LOG.isInfoEnabled()) { + LOG.info("Added " + sf + ", entries=" + r.getEntries() + + ", sequenceid=" + logCacheFlushId + + ", filesize=" + StringUtils.humanReadableInt(r.length())); + } + return sf; + } + + /* + * @param maxKeyCount + * @return Writer for a new StoreFile in the tmp dir. + */ + private StoreFile.Writer createWriterInTmp(int maxKeyCount) + throws IOException { + return createWriterInTmp(maxKeyCount, this.family.getCompression(), false); + } + + /* + * @param maxKeyCount + * @param compression Compression algorithm to use + * @param isCompaction whether we are creating a new file in a compaction + * @return Writer for a new StoreFile in the tmp dir. + */ + StoreFile.Writer createWriterInTmp(int maxKeyCount, + Compression.Algorithm compression, boolean isCompaction) + throws IOException { + final CacheConfig writerCacheConf; + if (isCompaction) { + // Don't cache data on write on compactions. + writerCacheConf = new CacheConfig(cacheConf); + writerCacheConf.setCacheDataOnWrite(false); + } else { + writerCacheConf = cacheConf; + } + StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf, + fs, blocksize) + .withOutputDir(region.getTmpDir()) + .withDataBlockEncoder(dataBlockEncoder) + .withComparator(comparator) + .withBloomType(family.getBloomFilterType()) + .withMaxKeyCount(maxKeyCount) + .withChecksumType(checksumType) + .withBytesPerChecksum(bytesPerChecksum) + .withCompression(compression) + .build(); + // The store file writer's path does not include the CF name, so we need + // to configure the HFile writer directly. + SchemaConfigured sc = (SchemaConfigured) w.writer; + SchemaConfigured.resetSchemaMetricsConf(sc); + passSchemaMetricsTo(sc); + return w; + } + + /* + * Change storefiles adding into place the Reader produced by this new flush. + * @param sf + * @param set That was used to make the passed file p. + * @throws IOException + * @return Whether compaction is required. + */ + private boolean updateStorefiles(final StoreFile sf, + final SortedSet set) + throws IOException { + this.lock.writeLock().lock(); + try { + ArrayList newList = new ArrayList(storefiles); + newList.add(sf); + storefiles = sortAndClone(newList); + + this.memstore.clearSnapshot(set); + } finally { + // We need the lock, as long as we are updating the storefiles + // or changing the memstore. Let us release it before calling + // notifyChangeReadersObservers. See HBASE-4485 for a possible + // deadlock scenario that could have happened if continue to hold + // the lock. + this.lock.writeLock().unlock(); + } + + // Tell listeners of the change in readers. + notifyChangedReadersObservers(); + + return needsCompaction(); + } + + /* + * Notify all observers that set of Readers has changed. + * @throws IOException + */ + private void notifyChangedReadersObservers() throws IOException { + for (ChangedReadersObserver o: this.changedReaderObservers) { + o.updateReaders(); + } + } /** - * getter for CompactionProgress object - * @return CompactionProgress object; can be null + * Get all scanners with no filtering based on TTL (that happens further down + * the line). + * @return all scanners for this store */ - public CompactionProgress getCompactionProgress(); + protected List getScanners(boolean cacheBlocks, + boolean isGet, + boolean isCompaction, + ScanQueryMatcher matcher) throws IOException { + List storeFiles; + List memStoreScanners; + this.lock.readLock().lock(); + try { + storeFiles = this.getStorefiles(); + memStoreScanners = this.memstore.getScanners(); + } finally { + this.lock.readLock().unlock(); + } - public CompactionRequest requestCompaction() throws IOException; + // First the store file scanners - public CompactionRequest requestCompaction(int priority) throws IOException; + // TODO this used to get the store files in descending order, + // but now we get them in ascending order, which I think is + // actually more correct, since memstore get put at the end. + List sfScanners = StoreFileScanner + .getScannersForStoreFiles(storeFiles, cacheBlocks, isGet, isCompaction, matcher); + List scanners = + new ArrayList(sfScanners.size()+1); + scanners.addAll(sfScanners); + // Then the memstore scanners + scanners.addAll(memStoreScanners); + return scanners; + } - public void finishRequest(CompactionRequest cr); + /* + * @param o Observer who wants to know about changes in set of Readers + */ + void addChangedReaderObserver(ChangedReadersObserver o) { + this.changedReaderObservers.add(o); + } + + /* + * @param o Observer no longer interested in changes in set of Readers. + */ + void deleteChangedReaderObserver(ChangedReadersObserver o) { + // We don't check if observer present; it may not be (legitimately) + this.changedReaderObservers.remove(o); + } + + ////////////////////////////////////////////////////////////////////////////// + // Compaction + ////////////////////////////////////////////////////////////////////////////// /** - * @return true if we should run a major compaction. + * Compact the StoreFiles. This method may take some time, so the calling + * thread must be able to block for long periods. + * + *

During this time, the Store can work as usual, getting values from + * StoreFiles and writing new StoreFiles from the memstore. + * + * Existing StoreFiles are not destroyed until the new compacted StoreFile is + * completely written-out to disk. + * + *

The compactLock prevents multiple simultaneous compactions. + * The structureLock prevents us from interfering with other write operations. + * + *

We don't want to hold the structureLock for the whole time, as a compact() + * can be lengthy and we want to allow cache-flushes during this period. + * + * @param cr + * compaction details obtained from requestCompaction() + * @throws IOException + * @return Storefile we compacted into or null if we failed or opted out early. */ - public boolean isMajorCompaction() throws IOException; + StoreFile compact(CompactionRequest cr) throws IOException { + if (cr == null || cr.getFiles().isEmpty()) return null; + Preconditions.checkArgument(cr.getStore().toString().equals(this.toString())); + List filesToCompact = cr.getFiles(); + synchronized (filesCompacting) { + // sanity check: we're compacting files that this store knows about + // TODO: change this to LOG.error() after more debugging + Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact)); + } - public void triggerMajorCompaction(); + // Max-sequenceID is the last key in the files we're compacting + long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact); + + // Ready to go. Have list of files to compact. + LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in " + + this + " of " + + this.region.getRegionInfo().getRegionNameAsString() + + " into tmpdir=" + region.getTmpDir() + ", seqid=" + maxId + ", totalSize=" + + StringUtils.humanReadableInt(cr.getSize())); + + StoreFile sf = null; + try { + StoreFile.Writer writer = + this.compactor.compact(this, filesToCompact, cr.isMajor(), maxId); + // Move the compaction into place. + if (this.conf.getBoolean("hbase.hstore.compaction.complete", true)) { + sf = completeCompaction(filesToCompact, writer); + if (region.getCoprocessorHost() != null) { + region.getCoprocessorHost().postCompact(this, sf); + } + } else { + // Create storefile around what we wrote with a reader on it. + sf = new StoreFile(this.fs, writer.getPath(), this.conf, this.cacheConf, + this.family.getBloomFilterType(), this.dataBlockEncoder); + sf.createReader(); + } + } finally { + synchronized (filesCompacting) { + filesCompacting.removeAll(filesToCompact); + } + } + + LOG.info("Completed" + (cr.isMajor() ? " major " : " ") + "compaction of " + + filesToCompact.size() + " file(s) in " + this + " of " + + this.region.getRegionInfo().getRegionNameAsString() + + " into " + + (sf == null ? "none" : sf.getPath().getName()) + + ", size=" + (sf == null ? "none" : + StringUtils.humanReadableInt(sf.getReader().length())) + + "; total size for store is " + + StringUtils.humanReadableInt(storeSize)); + return sf; + } + + @Override + public void compactRecentForTesting(int N) throws IOException { + List filesToCompact; + long maxId; + boolean isMajor; + + this.lock.readLock().lock(); + try { + synchronized (filesCompacting) { + filesToCompact = Lists.newArrayList(storefiles); + if (!filesCompacting.isEmpty()) { + // exclude all files older than the newest file we're currently + // compacting. this allows us to preserve contiguity (HBASE-2856) + StoreFile last = filesCompacting.get(filesCompacting.size() - 1); + int idx = filesToCompact.indexOf(last); + Preconditions.checkArgument(idx != -1); + filesToCompact.subList(0, idx + 1).clear(); + } + int count = filesToCompact.size(); + if (N > count) { + throw new RuntimeException("Not enough files"); + } + + filesToCompact = filesToCompact.subList(count - N, count); + maxId = StoreFile.getMaxSequenceIdInList(filesToCompact); + isMajor = (filesToCompact.size() == storefiles.size()); + filesCompacting.addAll(filesToCompact); + Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME); + } + } finally { + this.lock.readLock().unlock(); + } + + try { + // Ready to go. Have list of files to compact. + StoreFile.Writer writer = + this.compactor.compact(this, filesToCompact, isMajor, maxId); + // Move the compaction into place. + StoreFile sf = completeCompaction(filesToCompact, writer); + if (region.getCoprocessorHost() != null) { + region.getCoprocessorHost().postCompact(this, sf); + } + } finally { + synchronized (filesCompacting) { + filesCompacting.removeAll(filesToCompact); + } + } + } + + @Override + public boolean hasReferences() { + return hasReferences(this.storefiles); + } + + /* + * @param files + * @return True if any of the files in files are References. + */ + private boolean hasReferences(Collection files) { + if (files != null && files.size() > 0) { + for (StoreFile hsf: files) { + if (hsf.isReference()) { + return true; + } + } + } + return false; + } + + /* + * Gets lowest timestamp from candidate StoreFiles + * + * @param fs + * @param dir + * @throws IOException + */ + public static long getLowestTimestamp(final List candidates) + throws IOException { + long minTs = Long.MAX_VALUE; + for (StoreFile storeFile : candidates) { + minTs = Math.min(minTs, storeFile.getModificationTimeStamp()); + } + return minTs; + } + + @Override + public CompactionProgress getCompactionProgress() { + return this.compactor.getProgress(); + } + + @Override + public boolean isMajorCompaction() throws IOException { + for (StoreFile sf : this.storefiles) { + if (sf.getReader() == null) { + LOG.debug("StoreFile " + sf + " has null Reader"); + return false; + } + } + + List candidates = new ArrayList(this.storefiles); + + // exclude files above the max compaction threshold + // except: save all references. we MUST compact them + int pos = 0; + while (pos < candidates.size() && + candidates.get(pos).getReader().length() > this.maxCompactSize && + !candidates.get(pos).isReference()) ++pos; + candidates.subList(0, pos).clear(); + + return isMajorCompaction(candidates); + } + + /* + * @param filesToCompact Files to compact. Can be null. + * @return True if we should run a major compaction. + */ + private boolean isMajorCompaction(final List filesToCompact) throws IOException { + boolean result = false; + long mcTime = getNextMajorCompactTime(); + if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) { + return result; + } + // TODO: Use better method for determining stamp of last major (HBASE-2990) + long lowTimestamp = getLowestTimestamp(filesToCompact); + long now = System.currentTimeMillis(); + if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) { + // Major compaction time has elapsed. + if (filesToCompact.size() == 1) { + // Single file + StoreFile sf = filesToCompact.get(0); + long oldest = + (sf.getReader().timeRangeTracker == null) ? + Long.MIN_VALUE : + now - sf.getReader().timeRangeTracker.minimumTimestamp; + if (sf.isMajorCompaction() && + (this.ttl == HConstants.FOREVER || oldest < this.ttl)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skipping major compaction of " + this + + " because one (major) compacted file only and oldestTime " + + oldest + "ms is < ttl=" + this.ttl); + } + } else if (this.ttl != HConstants.FOREVER && oldest > this.ttl) { + LOG.debug("Major compaction triggered on store " + this + + ", because keyvalues outdated; time since last major compaction " + + (now - lowTimestamp) + "ms"); + result = true; + } + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Major compaction triggered on store " + this + + "; time since last major compaction " + (now - lowTimestamp) + "ms"); + } + result = true; + } + } + return result; + } + + long getNextMajorCompactTime() { + // default = 24hrs + long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24); + if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) { + String strCompactionTime = + family.getValue(HConstants.MAJOR_COMPACTION_PERIOD); + ret = (new Long(strCompactionTime)).longValue(); + } + + if (ret > 0) { + // default = 20% = +/- 4.8 hrs + double jitterPct = conf.getFloat("hbase.hregion.majorcompaction.jitter", + 0.20F); + if (jitterPct > 0) { + long jitter = Math.round(ret * jitterPct); + // deterministic jitter avoids a major compaction storm on restart + ImmutableList snapshot = storefiles; + if (snapshot != null && !snapshot.isEmpty()) { + String seed = snapshot.get(0).getPath().getName(); + double curRand = new Random(seed.hashCode()).nextDouble(); + ret += jitter - Math.round(2L * jitter * curRand); + } else { + ret = 0; // no storefiles == no major compaction + } + } + } + return ret; + } + + public CompactionRequest requestCompaction() throws IOException { + return requestCompaction(Store.NO_PRIORITY); + } + + public CompactionRequest requestCompaction(int priority) throws IOException { + // don't even select for compaction if writes are disabled + if (!this.region.areWritesEnabled()) { + return null; + } + + CompactionRequest ret = null; + this.lock.readLock().lock(); + try { + synchronized (filesCompacting) { + // candidates = all storefiles not already in compaction queue + List candidates = Lists.newArrayList(storefiles); + if (!filesCompacting.isEmpty()) { + // exclude all files older than the newest file we're currently + // compacting. this allows us to preserve contiguity (HBASE-2856) + StoreFile last = filesCompacting.get(filesCompacting.size() - 1); + int idx = candidates.indexOf(last); + Preconditions.checkArgument(idx != -1); + candidates.subList(0, idx + 1).clear(); + } + + boolean override = false; + if (region.getCoprocessorHost() != null) { + override = region.getCoprocessorHost().preCompactSelection( + this, candidates); + } + CompactSelection filesToCompact; + if (override) { + // coprocessor is overriding normal file selection + filesToCompact = new CompactSelection(conf, candidates); + } else { + filesToCompact = compactSelection(candidates, priority); + } + + if (region.getCoprocessorHost() != null) { + region.getCoprocessorHost().postCompactSelection(this, + ImmutableList.copyOf(filesToCompact.getFilesToCompact())); + } + + // no files to compact + if (filesToCompact.getFilesToCompact().isEmpty()) { + return null; + } + + // basic sanity check: do not try to compact the same StoreFile twice. + if (!Collections.disjoint(filesCompacting, filesToCompact.getFilesToCompact())) { + // TODO: change this from an IAE to LOG.error after sufficient testing + Preconditions.checkArgument(false, "%s overlaps with %s", + filesToCompact, filesCompacting); + } + filesCompacting.addAll(filesToCompact.getFilesToCompact()); + Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME); + + // major compaction iff all StoreFiles are included + boolean isMajor = (filesToCompact.getFilesToCompact().size() == this.storefiles.size()); + if (isMajor) { + // since we're enqueuing a major, update the compaction wait interval + this.forceMajor = false; + } + + // everything went better than expected. create a compaction request + int pri = getCompactPriority(priority); + ret = new CompactionRequest(region, this, filesToCompact, isMajor, pri); + } + } finally { + this.lock.readLock().unlock(); + } + if (ret != null) { + CompactionRequest.preRequest(ret); + } + return ret; + } + + public void finishRequest(CompactionRequest cr) { + CompactionRequest.postRequest(cr); + cr.finishRequest(); + synchronized (filesCompacting) { + filesCompacting.removeAll(cr.getFiles()); + } + } /** - * See if there's too much store files in this store - * @return true if number of store files is greater than the number defined in minFilesToCompact + * Algorithm to choose which files to compact, see {@link #compactSelection(java.util.List, int)} + * @param candidates + * @return + * @throws IOException */ - public boolean needsCompaction(); - - public int getCompactPriority(); + CompactSelection compactSelection(List candidates) throws IOException { + return compactSelection(candidates,Store.NO_PRIORITY); + } /** - * @param priority priority to check against. When priority is {@link HStore#PRIORITY_USER}, - * {@link HStore#PRIORITY_USER} is returned. - * @return The priority that this store has in the compaction queue. + * Algorithm to choose which files to compact + * + * Configuration knobs: + * "hbase.hstore.compaction.ratio" + * normal case: minor compact when file <= sum(smaller_files) * ratio + * "hbase.hstore.compaction.min.size" + * unconditionally compact individual files below this size + * "hbase.hstore.compaction.max.size" + * never compact individual files above this size (unless splitting) + * "hbase.hstore.compaction.min" + * min files needed to minor compact + * "hbase.hstore.compaction.max" + * max files to compact at once (avoids OOM) + * + * @param candidates candidate files, ordered from oldest to newest + * @return subset copy of candidate list that meets compaction criteria + * @throws IOException */ - public int getCompactPriority(int priority); + CompactSelection compactSelection(List candidates, int priority) + throws IOException { + // ASSUMPTION!!! filesCompacting is locked when calling this function - public StoreFlusher getStoreFlusher(long cacheFlushId); + /* normal skew: + * + * older ----> newer + * _ + * | | _ + * | | | | _ + * --|-|- |-|- |-|---_-------_------- minCompactSize + * | | | | | | | | _ | | + * | | | | | | | | | | | | + * | | | | | | | | | | | | + */ + CompactSelection compactSelection = new CompactSelection(conf, candidates); - // Split oriented methods + boolean forcemajor = this.forceMajor && filesCompacting.isEmpty(); + if (!forcemajor) { + // Delete the expired store files before the compaction selection. + if (conf.getBoolean("hbase.store.delete.expired.storefile", true) + && (ttl != Long.MAX_VALUE) && (this.scanInfo.minVersions == 0)) { + CompactSelection expiredSelection = compactSelection + .selectExpiredStoreFilesToCompact( + EnvironmentEdgeManager.currentTimeMillis() - this.ttl); - public boolean canSplit(); + // If there is any expired store files, delete them by compaction. + if (expiredSelection != null) { + return expiredSelection; + } + } + // do not compact old files above a configurable threshold + // save all references. we MUST compact them + int pos = 0; + while (pos < compactSelection.getFilesToCompact().size() && + compactSelection.getFilesToCompact().get(pos).getReader().length() + > maxCompactSize && + !compactSelection.getFilesToCompact().get(pos).isReference()) ++pos; + if (pos != 0) compactSelection.clearSubList(0, pos); + } + + if (compactSelection.getFilesToCompact().isEmpty()) { + LOG.debug(this.getHRegionInfo().getEncodedName() + " - " + + this + ": no store files to compact"); + compactSelection.emptyFileList(); + return compactSelection; + } + + // Force a major compaction if this is a user-requested major compaction, + // or if we do not have too many files to compact and this was requested + // as a major compaction + boolean majorcompaction = (forcemajor && priority == Store.PRIORITY_USER) || + (forcemajor || isMajorCompaction(compactSelection.getFilesToCompact())) && + (compactSelection.getFilesToCompact().size() < this.maxFilesToCompact + ); + LOG.debug(this.getHRegionInfo().getEncodedName() + " - " + + this.getColumnFamilyName() + ": Initiating " + + (majorcompaction ? "major" : "minor") + "compaction"); + + if (!majorcompaction && + !hasReferences(compactSelection.getFilesToCompact())) { + // we're doing a minor compaction, let's see what files are applicable + int start = 0; + double r = compactSelection.getCompactSelectionRatio(); + + // skip selection algorithm if we don't have enough files + if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) { + if(LOG.isDebugEnabled()) { + LOG.debug("Not compacting files because we only have " + + compactSelection.getFilesToCompact().size() + + " files ready for compaction. Need " + this.minFilesToCompact + " to initiate."); + } + compactSelection.emptyFileList(); + return compactSelection; + } + + // remove bulk import files that request to be excluded from minors + compactSelection.getFilesToCompact().removeAll(Collections2.filter( + compactSelection.getFilesToCompact(), + new Predicate() { + public boolean apply(StoreFile input) { + return input.excludeFromMinorCompaction(); + } + })); + + /* TODO: add sorting + unit test back in when HBASE-2856 is fixed + // Sort files by size to correct when normal skew is altered by bulk load. + Collections.sort(filesToCompact, StoreFile.Comparators.FILE_SIZE); + */ + + // get store file sizes for incremental compacting selection. + int countOfFiles = compactSelection.getFilesToCompact().size(); + long [] fileSizes = new long[countOfFiles]; + long [] sumSize = new long[countOfFiles]; + for (int i = countOfFiles-1; i >= 0; --i) { + StoreFile file = compactSelection.getFilesToCompact().get(i); + fileSizes[i] = file.getReader().length(); + // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo + int tooFar = i + this.maxFilesToCompact - 1; + sumSize[i] = fileSizes[i] + + ((i+1 < countOfFiles) ? sumSize[i+1] : 0) + - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0); + } + + /* Start at the oldest file and stop when you find the first file that + * meets compaction criteria: + * (1) a recently-flushed, small file (i.e. <= minCompactSize) + * OR + * (2) within the compactRatio of sum(newer_files) + * Given normal skew, any newer files will also meet this criteria + * + * Additional Note: + * If fileSizes.size() >> maxFilesToCompact, we will recurse on + * compact(). Consider the oldest files first to avoid a + * situation where we always compact [end-threshold,end). Then, the + * last file becomes an aggregate of the previous compactions. + */ + while(countOfFiles - start >= this.minFilesToCompact && + fileSizes[start] > + Math.max(minCompactSize, (long)(sumSize[start+1] * r))) { + ++start; + } + int end = Math.min(countOfFiles, start + this.maxFilesToCompact); + long totalSize = fileSizes[start] + + ((start+1 < countOfFiles) ? sumSize[start+1] : 0); + compactSelection = compactSelection.getSubList(start, end); + + // if we don't have enough files to compact, just wait + if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) { + if (LOG.isDebugEnabled()) { + LOG.debug("Skipped compaction of " + this + + ". Only " + (end - start) + " file(s) of size " + + StringUtils.humanReadableInt(totalSize) + + " have met compaction criteria."); + } + compactSelection.emptyFileList(); + return compactSelection; + } + } else { + if(majorcompaction) { + if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) { + LOG.debug("Warning, compacting more than " + this.maxFilesToCompact + + " files, probably because of a user-requested major compaction"); + if(priority != Store.PRIORITY_USER) { + LOG.error("Compacting more than max files on a non user-requested compaction"); + } + } + } else if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) { + // all files included in this compaction, up to max + int pastMax = compactSelection.getFilesToCompact().size() - this.maxFilesToCompact; + compactSelection.getFilesToCompact().subList(0, pastMax).clear(); + } + } + return compactSelection; + } /** - * Determines if Store should be split - * @return byte[] if store should be split, null otherwise. + * Validates a store file by opening and closing it. In HFileV2 this should + * not be an expensive operation. + * + * @param path the path to the store file */ - public byte[] getSplitPoint(); + private void validateStoreFile(Path path) + throws IOException { + StoreFile storeFile = null; + try { + storeFile = new StoreFile(this.fs, path, this.conf, + this.cacheConf, this.family.getBloomFilterType(), + NoOpDataBlockEncoder.INSTANCE); + passSchemaMetricsTo(storeFile); + storeFile.createReader(); + } catch (IOException e) { + LOG.error("Failed to open store file : " + path + + ", keeping it in tmp location", e); + throw e; + } finally { + if (storeFile != null) { + storeFile.closeReader(false); + } + } + } - // Bulk Load methods + /* + *

It works by processing a compaction that's been written to disk. + * + *

It is usually invoked at the end of a compaction, but might also be + * invoked at HStore startup, if the prior execution died midway through. + * + *

Moving the compacted TreeMap into place means: + *

+   * 1) Moving the new compacted StoreFile into place
+   * 2) Unload all replaced StoreFile, close and collect list to delete.
+   * 3) Loading the new TreeMap.
+   * 4) Compute new store size
+   * 
+ * + * @param compactedFiles list of files that were compacted + * @param compactedFile StoreFile that is the result of the compaction + * @return StoreFile created. May be null. + * @throws IOException + */ + StoreFile completeCompaction(final Collection compactedFiles, + final StoreFile.Writer compactedFile) + throws IOException { + // 1. Moving the new files into place -- if there is a new file (may not + // be if all cells were expired or deleted). + StoreFile result = null; + if (compactedFile != null) { + validateStoreFile(compactedFile.getPath()); + // Move the file into the right spot + Path origPath = compactedFile.getPath(); + Path destPath = new Path(homedir, origPath.getName()); + LOG.info("Renaming compacted file at " + origPath + " to " + destPath); + if (!fs.rename(origPath, destPath)) { + LOG.error("Failed move of compacted file " + origPath + " to " + + destPath); + throw new IOException("Failed move of compacted file " + origPath + + " to " + destPath); + } + result = new StoreFile(this.fs, destPath, this.conf, this.cacheConf, + this.family.getBloomFilterType(), this.dataBlockEncoder); + passSchemaMetricsTo(result); + result.createReader(); + } + try { + this.lock.writeLock().lock(); + try { + // Change this.storefiles so it reflects new state but do not + // delete old store files until we have sent out notification of + // change in case old files are still being accessed by outstanding + // scanners. + ArrayList newStoreFiles = Lists.newArrayList(storefiles); + newStoreFiles.removeAll(compactedFiles); + filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock() + + // If a StoreFile result, move it into place. May be null. + if (result != null) { + newStoreFiles.add(result); + } + + this.storefiles = sortAndClone(newStoreFiles); + } finally { + // We need the lock, as long as we are updating the storefiles + // or changing the memstore. Let us release it before calling + // notifyChangeReadersObservers. See HBASE-4485 for a possible + // deadlock scenario that could have happened if continue to hold + // the lock. + this.lock.writeLock().unlock(); + } + + // Tell observers that list of StoreFiles has changed. + notifyChangedReadersObservers(); + + // let the archive util decide if we should archive or delete the files + LOG.debug("Removing store files after compaction..."); + HFileArchiver.archiveStoreFiles(this.fs, this.region, this.conf, this.family.getName(), + compactedFiles); + + } catch (IOException e) { + e = RemoteExceptionHandler.checkIOException(e); + LOG.error("Failed replacing compacted files in " + this + + ". Compacted file is " + (result == null? "none": result.toString()) + + ". Files replaced " + compactedFiles.toString() + + " some of which may have been already removed", e); + } + + // 4. Compute new store size + this.storeSize = 0L; + this.totalUncompressedBytes = 0L; + for (StoreFile hsf : this.storefiles) { + StoreFile.Reader r = hsf.getReader(); + if (r == null) { + LOG.warn("StoreFile " + hsf + " has a null Reader"); + continue; + } + this.storeSize += r.length(); + this.totalUncompressedBytes += r.getTotalUncompressedBytes(); + } + return result; + } + + public ImmutableList sortAndClone(List storeFiles) { + Collections.sort(storeFiles, StoreFile.Comparators.FLUSH_TIME); + ImmutableList newList = ImmutableList.copyOf(storeFiles); + return newList; + } + + // //////////////////////////////////////////////////////////////////////////// + // Accessors. + // (This is the only section that is directly useful!) + ////////////////////////////////////////////////////////////////////////////// + @Override + public int getNumberOfStoreFiles() { + return this.storefiles.size(); + } + + /* + * @param wantedVersions How many versions were asked for. + * @return wantedVersions or this families' {@link HConstants#VERSIONS}. + */ + int versionsToReturn(final int wantedVersions) { + if (wantedVersions <= 0) { + throw new IllegalArgumentException("Number of versions must be > 0"); + } + // Make sure we do not return more than maximum versions for this store. + int maxVersions = this.family.getMaxVersions(); + return wantedVersions > maxVersions ? maxVersions: wantedVersions; + } + + static boolean isExpired(final KeyValue key, final long oldestTimestamp) { + return key.getTimestamp() < oldestTimestamp; + } + + @Override + public KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException { + // If minVersions is set, we will not ignore expired KVs. + // As we're only looking for the latest matches, that should be OK. + // With minVersions > 0 we guarantee that any KV that has any version + // at all (expired or not) has at least one version that will not expire. + // Note that this method used to take a KeyValue as arguments. KeyValue + // can be back-dated, a row key cannot. + long ttlToUse = scanInfo.getMinVersions() > 0 ? Long.MAX_VALUE : this.ttl; + + KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP); + + GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker( + this.comparator, kv, ttlToUse, this.region.getRegionInfo().isMetaRegion()); + this.lock.readLock().lock(); + try { + // First go to the memstore. Pick up deletes and candidates. + this.memstore.getRowKeyAtOrBefore(state); + // Check if match, if we got a candidate on the asked for 'kv' row. + // Process each store file. Run through from newest to oldest. + for (StoreFile sf : Lists.reverse(storefiles)) { + // Update the candidate keys from the current map file + rowAtOrBeforeFromStoreFile(sf, state); + } + return state.getCandidate(); + } finally { + this.lock.readLock().unlock(); + } + } + + /* + * Check an individual MapFile for the row at or before a given row. + * @param f + * @param state + * @throws IOException + */ + private void rowAtOrBeforeFromStoreFile(final StoreFile f, + final GetClosestRowBeforeTracker state) + throws IOException { + StoreFile.Reader r = f.getReader(); + if (r == null) { + LOG.warn("StoreFile " + f + " has a null Reader"); + return; + } + if (r.getEntries() == 0) { + LOG.warn("StoreFile " + f + " is a empty store file"); + return; + } + // TODO: Cache these keys rather than make each time? + byte [] fk = r.getFirstKey(); + KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length); + byte [] lk = r.getLastKey(); + KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length); + KeyValue firstOnRow = state.getTargetKey(); + if (this.comparator.compareRows(lastKV, firstOnRow) < 0) { + // If last key in file is not of the target table, no candidates in this + // file. Return. + if (!state.isTargetTable(lastKV)) return; + // If the row we're looking for is past the end of file, set search key to + // last key. TODO: Cache last and first key rather than make each time. + firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP); + } + // Get a scanner that caches blocks and that uses pread. + HFileScanner scanner = r.getHFileReader().getScanner(true, true, false); + // Seek scanner. If can't seek it, return. + if (!seekToScanner(scanner, firstOnRow, firstKV)) return; + // If we found candidate on firstOnRow, just return. THIS WILL NEVER HAPPEN! + // Unlikely that there'll be an instance of actual first row in table. + if (walkForwardInSingleRow(scanner, firstOnRow, state)) return; + // If here, need to start backing up. + while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(), + firstOnRow.getKeyLength())) { + KeyValue kv = scanner.getKeyValue(); + if (!state.isTargetTable(kv)) break; + if (!state.isBetterCandidate(kv)) break; + // Make new first on row. + firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP); + // Seek scanner. If can't seek it, break. + if (!seekToScanner(scanner, firstOnRow, firstKV)) break; + // If we find something, break; + if (walkForwardInSingleRow(scanner, firstOnRow, state)) break; + } + } + + /* + * Seek the file scanner to firstOnRow or first entry in file. + * @param scanner + * @param firstOnRow + * @param firstKV + * @return True if we successfully seeked scanner. + * @throws IOException + */ + private boolean seekToScanner(final HFileScanner scanner, + final KeyValue firstOnRow, + final KeyValue firstKV) + throws IOException { + KeyValue kv = firstOnRow; + // If firstOnRow < firstKV, set to firstKV + if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV; + int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(), + kv.getKeyLength()); + return result >= 0; + } + + /* + * When we come in here, we are probably at the kv just before we break into + * the row that firstOnRow is on. Usually need to increment one time to get + * on to the row we are interested in. + * @param scanner + * @param firstOnRow + * @param state + * @return True we found a candidate. + * @throws IOException + */ + private boolean walkForwardInSingleRow(final HFileScanner scanner, + final KeyValue firstOnRow, + final GetClosestRowBeforeTracker state) + throws IOException { + boolean foundCandidate = false; + do { + KeyValue kv = scanner.getKeyValue(); + // If we are not in the row, skip. + if (this.comparator.compareRows(kv, firstOnRow) < 0) continue; + // Did we go beyond the target row? If so break. + if (state.isTooFar(kv, firstOnRow)) break; + if (state.isExpired(kv)) { + continue; + } + // If we added something, this row is a contender. break. + if (state.handle(kv)) { + foundCandidate = true; + break; + } + } while(scanner.next()); + return foundCandidate; + } + + public boolean canSplit() { + this.lock.readLock().lock(); + try { + // Not splitable if we find a reference store file present in the store. + for (StoreFile sf : storefiles) { + if (sf.isReference()) { + if (LOG.isDebugEnabled()) { + LOG.debug(sf + " is not splittable"); + } + return false; + } + } + + return true; + } finally { + this.lock.readLock().unlock(); + } + } + + @Override + public byte[] getSplitPoint() { + this.lock.readLock().lock(); + try { + // sanity checks + if (this.storefiles.isEmpty()) { + return null; + } + // Should already be enforced by the split policy! + assert !this.region.getRegionInfo().isMetaRegion(); + + // Not splitable if we find a reference store file present in the store. + long maxSize = 0L; + StoreFile largestSf = null; + for (StoreFile sf : storefiles) { + if (sf.isReference()) { + // Should already be enforced since we return false in this case + assert false : "getSplitPoint() called on a region that can't split!"; + return null; + } + + StoreFile.Reader r = sf.getReader(); + if (r == null) { + LOG.warn("Storefile " + sf + " Reader is null"); + continue; + } + + long size = r.length(); + if (size > maxSize) { + // This is the largest one so far + maxSize = size; + largestSf = sf; + } + } + + StoreFile.Reader r = largestSf.getReader(); + if (r == null) { + LOG.warn("Storefile " + largestSf + " Reader is null"); + return null; + } + // Get first, last, and mid keys. Midkey is the key that starts block + // in middle of hfile. Has column and timestamp. Need to return just + // the row we want to split on as midkey. + byte [] midkey = r.midkey(); + if (midkey != null) { + KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length); + byte [] fk = r.getFirstKey(); + KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length); + byte [] lk = r.getLastKey(); + KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length); + // if the midkey is the same as the first or last keys, then we cannot + // (ever) split this region. + if (this.comparator.compareRows(mk, firstKey) == 0 || + this.comparator.compareRows(mk, lastKey) == 0) { + if (LOG.isDebugEnabled()) { + LOG.debug("cannot split because midkey is the same as first or " + + "last row"); + } + return null; + } + return mk.getRow(); + } + } catch(IOException e) { + LOG.warn("Failed getting store size for " + this, e); + } finally { + this.lock.readLock().unlock(); + } + return null; + } + + @Override + public long getLastCompactSize() { + return this.lastCompactSize; + } + + @Override + public long getSize() { + return storeSize; + } + + public void triggerMajorCompaction() { + this.forceMajor = true; + } + + boolean getForceMajorCompaction() { + return this.forceMajor; + } + + ////////////////////////////////////////////////////////////////////////////// + // File administration + ////////////////////////////////////////////////////////////////////////////// + + @Override + public KeyValueScanner getScanner(Scan scan, + final NavigableSet targetCols) throws IOException { + lock.readLock().lock(); + try { + KeyValueScanner scanner = null; + if (getHRegion().getCoprocessorHost() != null) { + scanner = getHRegion().getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols); + } + if (scanner == null) { + scanner = new StoreScanner(this, getScanInfo(), scan, targetCols); + } + return scanner; + } finally { + lock.readLock().unlock(); + } + } + + @Override + public String toString() { + return getColumnFamilyName(); + } + + @Override + public int getStorefilesCount() { + return this.storefiles.size(); + } + + @Override + public long getStoreSizeUncompressed() { + return this.totalUncompressedBytes; + } + + @Override + public long getStorefilesSize() { + long size = 0; + for (StoreFile s: storefiles) { + StoreFile.Reader r = s.getReader(); + if (r == null) { + LOG.warn("StoreFile " + s + " has a null Reader"); + continue; + } + size += r.length(); + } + return size; + } + + @Override + public long getStorefilesIndexSize() { + long size = 0; + for (StoreFile s: storefiles) { + StoreFile.Reader r = s.getReader(); + if (r == null) { + LOG.warn("StoreFile " + s + " has a null Reader"); + continue; + } + size += r.indexSize(); + } + return size; + } + + @Override + public long getTotalStaticIndexSize() { + long size = 0; + for (StoreFile s : storefiles) { + size += s.getReader().getUncompressedDataIndexSize(); + } + return size; + } + + @Override + public long getTotalStaticBloomSize() { + long size = 0; + for (StoreFile s : storefiles) { + StoreFile.Reader r = s.getReader(); + size += r.getTotalBloomSize(); + } + return size; + } + + @Override + public long getMemStoreSize() { + return this.memstore.heapSize(); + } + + public int getCompactPriority() { + return getCompactPriority(Store.NO_PRIORITY); + } + + @Override + public int getCompactPriority(int priority) { + // If this is a user-requested compaction, leave this at the highest priority + if(priority == Store.PRIORITY_USER) { + return Store.PRIORITY_USER; + } else { + return this.blockingStoreFileCount - this.storefiles.size(); + } + } + + @Override + public boolean throttleCompaction(long compactionSize) { + // see HBASE-5867 for discussion on the default + long throttlePoint = conf.getLong( + "hbase.regionserver.thread.compaction.throttle", + 2 * this.minFilesToCompact * this.region.memstoreFlushSize); + return compactionSize > throttlePoint; + } + + @Override + public HRegion getHRegion() { + return this.region; + } + + HRegionInfo getHRegionInfo() { + return this.region.regionInfo; + } + + @Override + public long updateColumnValue(byte [] row, byte [] f, + byte [] qualifier, long newValue) + throws IOException { + + this.lock.readLock().lock(); + try { + long now = EnvironmentEdgeManager.currentTimeMillis(); + + return this.memstore.updateColumnValue(row, + f, + qualifier, + newValue, + now); + + } finally { + this.lock.readLock().unlock(); + } + } + + @Override + public long upsert(Iterable kvs) throws IOException { + this.lock.readLock().lock(); + try { + // TODO: Make this operation atomic w/ MVCC + return this.memstore.upsert(kvs); + } finally { + this.lock.readLock().unlock(); + } + } + + public StoreFlusher getStoreFlusher(long cacheFlushId) { + return new StoreFlusherImpl(cacheFlushId); + } + + private class StoreFlusherImpl implements StoreFlusher { + + private long cacheFlushId; + private SortedSet snapshot; + private StoreFile storeFile; + private Path storeFilePath; + private TimeRangeTracker snapshotTimeRangeTracker; + private AtomicLong flushedSize; + + private StoreFlusherImpl(long cacheFlushId) { + this.cacheFlushId = cacheFlushId; + this.flushedSize = new AtomicLong(); + } + + @Override + public void prepare() { + memstore.snapshot(); + this.snapshot = memstore.getSnapshot(); + this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker(); + } + + @Override + public void flushCache(MonitoredTask status) throws IOException { + storeFilePath = HStore.this.flushCache( + cacheFlushId, snapshot, snapshotTimeRangeTracker, flushedSize, status); + } + + @Override + public boolean commit(MonitoredTask status) throws IOException { + if (storeFilePath == null) { + return false; + } + storeFile = HStore.this.commitFile(storeFilePath, cacheFlushId, + snapshotTimeRangeTracker, flushedSize, status); + if (HStore.this.getHRegion().getCoprocessorHost() != null) { + HStore.this.getHRegion() + .getCoprocessorHost() + .postFlush(HStore.this, storeFile); + } + + // Add new file to store files. Clear snapshot too while we have + // the Store write lock. + return HStore.this.updateStorefiles(storeFile, snapshot); + } + } + + @Override + public boolean needsCompaction() { + return (storefiles.size() - filesCompacting.size()) > minFilesToCompact; + } + + @Override + public CacheConfig getCacheConfig() { + return this.cacheConf; + } + + public static final long FIXED_OVERHEAD = + ClassSize.align(SchemaConfigured.SCHEMA_CONFIGURED_UNALIGNED_HEAP_SIZE + + + (17 * ClassSize.REFERENCE) + (6 * Bytes.SIZEOF_LONG) + + (5 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN); + + public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD + + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK + + ClassSize.CONCURRENT_SKIPLISTMAP + + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT + + ScanInfo.FIXED_OVERHEAD); + + @Override + public long heapSize() { + return DEEP_OVERHEAD + this.memstore.heapSize(); + } + + public KeyValue.KVComparator getComparator() { + return comparator; + } + + public ScanInfo getScanInfo() { + return scanInfo; + } /** - * This throws a WrongRegionException if the HFile does not fit in this region, or an - * InvalidHFileException if the HFile is not valid. + * Immutable information for scans over a store. */ - public void assertBulkLoadHFileOk(Path srcPath) throws IOException; + public static class ScanInfo { + private byte[] family; + private int minVersions; + private int maxVersions; + private long ttl; + private boolean keepDeletedCells; + private long timeToPurgeDeletes; + private KVComparator comparator; - /** - * This method should only be called from HRegion. It is assumed that the ranges of values in the - * HFile fit within the stores assigned region. (assertBulkLoadHFileOk checks this) - */ - public void bulkLoadHFile(String srcPathStr) throws IOException; + public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT + + (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT) + + Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN); - // General accessors into the state of the store - // TODO abstract some of this out into a metrics class + /** + * @param family {@link HColumnDescriptor} describing the column family + * @param ttl Store's TTL (in ms) + * @param timeToPurgeDeletes duration in ms after which a delete marker can + * be purged during a major compaction. + * @param comparator The store's comparator + */ + public ScanInfo(HColumnDescriptor family, long ttl, long timeToPurgeDeletes, KVComparator comparator) { + this(family.getName(), family.getMinVersions(), family.getMaxVersions(), ttl, family + .getKeepDeletedCells(), timeToPurgeDeletes, comparator); + } + /** + * @param family Name of this store's column family + * @param minVersions Store's MIN_VERSIONS setting + * @param maxVersions Store's VERSIONS setting + * @param ttl Store's TTL (in ms) + * @param timeToPurgeDeletes duration in ms after which a delete marker can + * be purged during a major compaction. + * @param keepDeletedCells Store's keepDeletedCells setting + * @param comparator The store's comparator + */ + public ScanInfo(byte[] family, int minVersions, int maxVersions, long ttl, + boolean keepDeletedCells, long timeToPurgeDeletes, + KVComparator comparator) { - /** - * @return true if the store has any underlying reference files to older HFiles - */ - public boolean hasReferences(); + this.family = family; + this.minVersions = minVersions; + this.maxVersions = maxVersions; + this.ttl = ttl; + this.keepDeletedCells = keepDeletedCells; + this.timeToPurgeDeletes = timeToPurgeDeletes; + this.comparator = comparator; + } - /** - * @return The size of this store's memstore, in bytes - */ - public long getMemStoreSize(); + public byte[] getFamily() { + return family; + } - public HColumnDescriptor getFamily(); + public int getMinVersions() { + return minVersions; + } - /** - * @return The maximum memstoreTS in all store files. - */ - public long getMaxMemstoreTS(); + public int getMaxVersions() { + return maxVersions; + } - /** - * @return the data block encoder - */ - public HFileDataBlockEncoder getDataBlockEncoder(); + public long getTtl() { + return ttl; + } - /** - * @return the number of files in this store - */ - public int getNumberOfStoreFiles(); + public boolean getKeepDeletedCells() { + return keepDeletedCells; + } - /** @return aggregate size of all HStores used in the last compaction */ - public long getLastCompactSize(); + public long getTimeToPurgeDeletes() { + return timeToPurgeDeletes; + } - /** @return aggregate size of HStore */ - public long getSize(); + public KVComparator getComparator() { + return comparator; + } + } - /** - * @return Count of store files - */ - public int getStorefilesCount(); - - /** - * @return The size of the store files, in bytes, uncompressed. - */ - public long getStoreSizeUncompressed(); - - /** - * @return The size of the store files, in bytes. - */ - public long getStorefilesSize(); - - /** - * @return The size of the store file indexes, in bytes. - */ - public long getStorefilesIndexSize(); - - /** - * Returns the total size of all index blocks in the data block indexes, including the root level, - * intermediate levels, and the leaf level for multi-level indexes, or just the root level for - * single-level indexes. - * @return the total size of block indexes in the store - */ - public long getTotalStaticIndexSize(); - - /** - * Returns the total byte size of all Bloom filter bit arrays. For compound Bloom filters even the - * Bloom blocks currently not loaded into the block cache are counted. - * @return the total size of all Bloom filters in the store - */ - public long getTotalStaticBloomSize(); - - // Test-helper methods - - /** - * Compact the most recent N files. Used in testing. - * @param N number of files to compact. Must be less than or equal to current number of files. - * @throws IOException on failure - */ - public void compactRecentForTesting(int N) throws IOException; - - /** - * Used for tests. - * @return cache configuration for this Store. - */ - public CacheConfig getCacheConfig(); - - /** - * @return the parent region hosting this store - */ - public HRegion getHRegion(); -} \ No newline at end of file +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java index d65a9df7693..e8b54f9d1ae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.java @@ -60,7 +60,7 @@ extends ConstantSizeRegionSplitPolicy { // Get size to check long sizeToCheck = getSizeToCheck(tableRegionsCount); - for (HStore store : region.getStores().values()) { + for (Store store : region.getStores().values()) { // If any of the stores is unable to split (eg they contain reference files) // then don't split if ((!store.canSplit())) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java index 44f1ab447e2..c0f99e75b88 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java @@ -437,7 +437,7 @@ class MemStoreFlusher extends HasThread implements FlushRequester { } private boolean isTooManyStoreFiles(HRegion region) { - for (HStore hstore : region.stores.values()) { + for (Store hstore : region.stores.values()) { if (hstore.getStorefilesCount() > this.blockingStoreFilesNumber) { return true; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java index 0f61539b732..6a4c06a3d22 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java @@ -342,9 +342,9 @@ public class RegionCoprocessorHost /** * See - * {@link RegionObserver#preCompactScannerOpen(ObserverContext, Store, List, ScanType, long, InternalScanner)} + * {@link RegionObserver#preCompactScannerOpen(ObserverContext, HStore, List, ScanType, long, InternalScanner)} */ - public InternalScanner preCompactScannerOpen(Store store, List scanners, + public InternalScanner preCompactScannerOpen(HStore store, List scanners, ScanType scanType, long earliestPutTs) throws IOException { ObserverContext ctx = null; InternalScanner s = null; @@ -373,7 +373,7 @@ public class RegionCoprocessorHost * @return If {@code true}, skip the normal selection process and use the current list * @throws IOException */ - public boolean preCompactSelection(Store store, List candidates) throws IOException { + public boolean preCompactSelection(HStore store, List candidates) throws IOException { ObserverContext ctx = null; boolean bypass = false; for (RegionEnvironment env: coprocessors) { @@ -401,7 +401,7 @@ public class RegionCoprocessorHost * @param store The store where compaction is being requested * @param selected The store files selected to compact */ - public void postCompactSelection(Store store, + public void postCompactSelection(HStore store, ImmutableList selected) { ObserverContext ctx = null; for (RegionEnvironment env: coprocessors) { @@ -426,7 +426,7 @@ public class RegionCoprocessorHost * @param scanner the scanner used to read store data during compaction * @throws IOException */ - public InternalScanner preCompact(Store store, InternalScanner scanner) throws IOException { + public InternalScanner preCompact(HStore store, InternalScanner scanner) throws IOException { ObserverContext ctx = null; boolean bypass = false; for (RegionEnvironment env: coprocessors) { @@ -453,7 +453,7 @@ public class RegionCoprocessorHost * @param resultFile the new store file written during compaction * @throws IOException */ - public void postCompact(Store store, StoreFile resultFile) throws IOException { + public void postCompact(HStore store, StoreFile resultFile) throws IOException { ObserverContext ctx = null; for (RegionEnvironment env: coprocessors) { if (env.getInstance() instanceof RegionObserver) { @@ -474,7 +474,7 @@ public class RegionCoprocessorHost * Invoked before a memstore flush * @throws IOException */ - public InternalScanner preFlush(Store store, InternalScanner scanner) throws IOException { + public InternalScanner preFlush(HStore store, InternalScanner scanner) throws IOException { ObserverContext ctx = null; boolean bypass = false; for (RegionEnvironment env: coprocessors) { @@ -518,9 +518,9 @@ public class RegionCoprocessorHost /** * See - * {@link RegionObserver#preFlush(ObserverContext, Store, KeyValueScanner)} + * {@link RegionObserver#preFlush(ObserverContext, HStore, KeyValueScanner)} */ - public InternalScanner preFlushScannerOpen(Store store, KeyValueScanner memstoreScanner) throws IOException { + public InternalScanner preFlushScannerOpen(HStore store, KeyValueScanner memstoreScanner) throws IOException { ObserverContext ctx = null; InternalScanner s = null; for (RegionEnvironment env : coprocessors) { @@ -564,7 +564,7 @@ public class RegionCoprocessorHost * Invoked after a memstore flush * @throws IOException */ - public void postFlush(final Store store, final StoreFile storeFile) throws IOException { + public void postFlush(final HStore store, final StoreFile storeFile) throws IOException { ObserverContext ctx = null; for (RegionEnvironment env: coprocessors) { if (env.getInstance() instanceof RegionObserver) { @@ -1221,9 +1221,9 @@ public class RegionCoprocessorHost /** * See - * {@link RegionObserver#preStoreScannerOpen(ObserverContext, Store, Scan, NavigableSet, KeyValueScanner)} + * {@link RegionObserver#preStoreScannerOpen(ObserverContext, HStore, Scan, NavigableSet, KeyValueScanner)} */ - public KeyValueScanner preStoreScannerOpen(Store store, Scan scan, + public KeyValueScanner preStoreScannerOpen(HStore store, Scan scan, final NavigableSet targetCols) throws IOException { KeyValueScanner s = null; ObserverContext ctx = null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java index 9ad2c6d7079..518124d49aa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java @@ -73,11 +73,11 @@ public abstract class RegionSplitPolicy extends Configured { if (explicitSplitPoint != null) { return explicitSplitPoint; } - Map stores = region.getStores(); + Map stores = region.getStores(); byte[] splitPointFromLargestStore = null; long largestStoreSize = 0; - for (HStore s : stores.values()) { + for (Store s : stores.values()) { byte[] splitPoint = s.getSplitPoint(); long storeSize = s.getSize(); if (splitPoint != null && largestStoreSize < storeSize) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java index f02afd7171b..68d5cf0095e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java @@ -135,7 +135,7 @@ public class ScanQueryMatcher { * @param oldestUnexpiredTS the oldest timestamp we are interested in, * based on TTL */ - public ScanQueryMatcher(Scan scan, Store.ScanInfo scanInfo, + public ScanQueryMatcher(Scan scan, HStore.ScanInfo scanInfo, NavigableSet columns, ScanType scanType, long readPointToUse, long earliestPutTs, long oldestUnexpiredTS) { this.tr = scan.getTimeRange(); @@ -181,7 +181,7 @@ public class ScanQueryMatcher { /* * Constructor for tests */ - ScanQueryMatcher(Scan scan, Store.ScanInfo scanInfo, + ScanQueryMatcher(Scan scan, HStore.ScanInfo scanInfo, NavigableSet columns, long oldestUnexpiredTS) { this(scan, scanInfo, columns, ScanType.USER_SCAN, Long.MAX_VALUE, /* max Readpoint to track versions */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java index 4791ca2cc70..ed3b2efcfe1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java @@ -640,10 +640,10 @@ public class SplitTransaction { FileSystem fs = this.parent.getFilesystem(); byte [] family = sf.getFamily(); String encoded = this.hri_a.getEncodedName(); - Path storedir = Store.getStoreHomedir(splitdir, encoded, family); + Path storedir = HStore.getStoreHomedir(splitdir, encoded, family); StoreFile.split(fs, storedir, sf, this.splitrow, false); encoded = this.hri_b.getEncodedName(); - storedir = Store.getStoreHomedir(splitdir, encoded, family); + storedir = HStore.getStoreHomedir(splitdir, encoded, family); StoreFile.split(fs, storedir, sf, this.splitrow, true); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index e888d16417f..e37f50441ac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -1,6 +1,4 @@ /** - * Copyright 2010 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 @@ -20,2191 +18,270 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.NavigableSet; -import java.util.Random; -import java.util.SortedSet; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletionService; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.Future; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValue.KVComparator; -import org.apache.hadoop.hbase.RemoteExceptionHandler; -import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.io.hfile.Compression; -import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; -import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl; -import org.apache.hadoop.hbase.io.hfile.HFileScanner; -import org.apache.hadoop.hbase.io.hfile.InvalidHFileException; -import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder; -import org.apache.hadoop.hbase.monitoring.MonitoredTask; -import org.apache.hadoop.hbase.regionserver.compactions.CompactSelection; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; -import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured; -import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.ChecksumType; -import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hadoop.hbase.util.CollectionBackedScanner; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.SchemaAware; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.Collections2; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; /** - * 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 - * and locking is handled at the HRegion level. Store just provides - * services to manage sets of StoreFiles. One of the most important of those - * services is compaction services where files are aggregated once they pass - * a configurable threshold. - * - *

The only thing having to do with logs that Store needs to deal with is - * the reconstructionLog. This is a segment of an HRegion's log that might - * NOT be present upon startup. If the param is NULL, there's nothing to do. - * If the param is non-NULL, we need to process the log to reconstruct - * a TreeMap that might not have been written to disk before the process - * died. - * - *

It's assumed that after this constructor returns, the reconstructionLog - * file will be deleted (by whoever has instantiated the Store). - * - *

Locking and transactions are handled at a higher level. This API should - * not be called directly but by an HRegion manager. + * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or + * more StoreFiles, which stretch backwards over time. */ @InterfaceAudience.Private -public class Store extends SchemaConfigured implements HStore { - static final Log LOG = LogFactory.getLog(Store.class); +@InterfaceStability.Evolving +public interface Store extends SchemaAware, HeapSize { - protected final MemStore memstore; - // This stores directory in the filesystem. - private final Path homedir; - private final HRegion region; - private final HColumnDescriptor family; - final FileSystem fs; - final Configuration conf; - final CacheConfig cacheConf; - // ttl in milliseconds. - private long ttl; - private final int minFilesToCompact; - private final int maxFilesToCompact; - private final long minCompactSize; - private final long maxCompactSize; - private long lastCompactSize = 0; - volatile boolean forceMajor = false; - /* how many bytes to write between status checks */ - static int closeCheckInterval = 0; - private final int blockingStoreFileCount; - private volatile long storeSize = 0L; - private volatile long totalUncompressedBytes = 0L; - private final Object flushLock = new Object(); - final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); - private final boolean verifyBulkLoads; - - // not private for testing - /* package */ScanInfo scanInfo; - /* - * List of store files inside this store. This is an immutable list that - * is atomically replaced when its contents change. + /* The default priority for user-specified compaction requests. + * The user gets top priority unless we have blocking compactions. (Pri <= 0) */ - private ImmutableList storefiles = null; + public static final int PRIORITY_USER = 1; + public static final int NO_PRIORITY = Integer.MIN_VALUE; - List filesCompacting = Lists.newArrayList(); + // General Accessors + public KeyValue.KVComparator getComparator(); - // All access must be synchronized. - private final CopyOnWriteArraySet changedReaderObservers = - new CopyOnWriteArraySet(); - - private final int blocksize; - private HFileDataBlockEncoder dataBlockEncoder; - - /** Checksum configuration */ - private ChecksumType checksumType; - private int bytesPerChecksum; - - // Comparing KeyValues - final KeyValue.KVComparator comparator; - - private final Compactor compactor; + public List getStorefiles(); /** - * Constructor - * @param basedir qualified path under which the region directory lives; - * generally the table subdirectory - * @param region - * @param family HColumnDescriptor for this column - * @param fs file system object - * @param confParam configuration object - * failed. Can be null. + * Close all the readers We don't need to worry about subsequent requests because the HRegion + * holds a write lock that will prevent any more reads or writes. + * @return the {@link StoreFile StoreFiles} that were previously being used. + * @throws IOException on failure + */ + public ImmutableList close() throws IOException; + + /** + * Return a scanner for both the memstore and the HStore files. Assumes we are not in a + * compaction. + * @param scan Scan to apply when scanning the stores + * @param targetCols columns to scan + * @return a scanner over the current key values + * @throws IOException on failure + */ + public KeyValueScanner getScanner(Scan scan, final NavigableSet targetCols) + throws IOException; + + /** + * Updates the value for the given row/family/qualifier. This function will always be seen as + * atomic by other readers because it only puts a single KV to memstore. Thus no read/write + * control necessary. + * @param row row to update + * @param f family to update + * @param qualifier qualifier to update + * @param newValue the new value to set into memstore + * @return memstore size delta * @throws IOException */ - protected Store(Path basedir, HRegion region, HColumnDescriptor family, - FileSystem fs, Configuration confParam) - throws IOException { - super(new CompoundConfiguration().add(confParam).add( - family.getValues()), region.getRegionInfo().getTableNameAsString(), - Bytes.toString(family.getName())); - HRegionInfo info = region.getRegionInfo(); - this.fs = fs; - // Assemble the store's home directory. - Path p = getStoreHomedir(basedir, info.getEncodedName(), family.getName()); - // Ensure it exists. - this.homedir = createStoreHomeDir(this.fs, p); - this.region = region; - this.family = family; - // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor - this.conf = new CompoundConfiguration() - .add(confParam) - .add(family.getValues()); - this.blocksize = family.getBlocksize(); - - this.dataBlockEncoder = - new HFileDataBlockEncoderImpl(family.getDataBlockEncodingOnDisk(), - family.getDataBlockEncoding()); - - this.comparator = info.getComparator(); - // Get TTL - this.ttl = getTTL(family); - // used by ScanQueryMatcher - long timeToPurgeDeletes = - Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0); - LOG.trace("Time to purge deletes set to " + timeToPurgeDeletes + - "ms in store " + this); - // Why not just pass a HColumnDescriptor in here altogether? Even if have - // to clone it? - scanInfo = new ScanInfo(family, ttl, timeToPurgeDeletes, this.comparator); - this.memstore = new MemStore(conf, this.comparator); - - // By default, compact if storefile.count >= minFilesToCompact - this.minFilesToCompact = Math.max(2, - conf.getInt("hbase.hstore.compaction.min", - /*old name*/ conf.getInt("hbase.hstore.compactionThreshold", 3))); - LOG.info("hbase.hstore.compaction.min = " + this.minFilesToCompact); - - // Setting up cache configuration for this family - this.cacheConf = new CacheConfig(conf, family); - this.blockingStoreFileCount = - conf.getInt("hbase.hstore.blockingStoreFiles", 7); - - this.maxFilesToCompact = conf.getInt("hbase.hstore.compaction.max", 10); - this.minCompactSize = conf.getLong("hbase.hstore.compaction.min.size", - this.region.memstoreFlushSize); - this.maxCompactSize - = conf.getLong("hbase.hstore.compaction.max.size", Long.MAX_VALUE); - - this.verifyBulkLoads = conf.getBoolean("hbase.hstore.bulkload.verify", false); - - if (Store.closeCheckInterval == 0) { - Store.closeCheckInterval = conf.getInt( - "hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */); - } - this.storefiles = sortAndClone(loadStoreFiles()); - - // Initialize checksum type from name. The names are CRC32, CRC32C, etc. - this.checksumType = getChecksumType(conf); - // initilize bytes per checksum - this.bytesPerChecksum = getBytesPerChecksum(conf); - // Create a compaction tool instance - this.compactor = new Compactor(this.conf); - } + public long updateColumnValue(byte[] row, byte[] f, byte[] qualifier, long newValue) + throws IOException; /** - * @param family - * @return - */ - long getTTL(final HColumnDescriptor family) { - // HCD.getTimeToLive returns ttl in seconds. Convert to milliseconds. - long ttl = family.getTimeToLive(); - if (ttl == HConstants.FOREVER) { - // Default is unlimited ttl. - ttl = Long.MAX_VALUE; - } else if (ttl == -1) { - ttl = Long.MAX_VALUE; - } else { - // Second -> ms adjust for user data - ttl *= 1000; - } - return ttl; - } - - /** - * Create this store's homedir - * @param fs - * @param homedir - * @return Return homedir + * Adds or replaces the specified KeyValues. + *

+ * For each KeyValue specified, if a cell with the same row, family, and qualifier exists in + * MemStore, it will be replaced. Otherwise, it will just be inserted to MemStore. + *

+ * This operation is atomic on each KeyValue (row/family/qualifier) but not necessarily atomic + * across all of them. + * @param kvs + * @return memstore size delta * @throws IOException */ - Path createStoreHomeDir(final FileSystem fs, - final Path homedir) throws IOException { - if (!fs.exists(homedir)) { - if (!fs.mkdirs(homedir)) - throw new IOException("Failed create of: " + homedir.toString()); - } - return homedir; - } - - FileSystem getFileSystem() { - return this.fs; - } - - /** - * Returns the configured bytesPerChecksum value. - * @param conf The configuration - * @return The bytesPerChecksum that is set in the configuration - */ - public static int getBytesPerChecksum(Configuration conf) { - return conf.getInt(HConstants.BYTES_PER_CHECKSUM, - HFile.DEFAULT_BYTES_PER_CHECKSUM); - } - - /** - * Returns the configured checksum algorithm. - * @param conf The configuration - * @return The checksum algorithm that is set in the configuration - */ - public static ChecksumType getChecksumType(Configuration conf) { - String checksumName = conf.get(HConstants.CHECKSUM_TYPE_NAME); - if (checksumName == null) { - return HFile.DEFAULT_CHECKSUM_TYPE; - } else { - return ChecksumType.nameToType(checksumName); - } - } - - public HColumnDescriptor getFamily() { - return this.family; - } - - /** - * @return The maximum sequence id in all store files. - */ - long getMaxSequenceId() { - return StoreFile.getMaxSequenceIdInList(this.getStorefiles()); - } - - @Override - public long getMaxMemstoreTS() { - return StoreFile.getMaxMemstoreTSInList(this.getStorefiles()); - } - - /** - * @param tabledir - * @param encodedName Encoded region name. - * @param family - * @return Path to family/Store home directory. - */ - public static Path getStoreHomedir(final Path tabledir, - final String encodedName, final byte [] family) { - return new Path(tabledir, new Path(encodedName, - new Path(Bytes.toString(family)))); - } - - /** - * Return the directory in which this store stores its - * StoreFiles - */ - Path getHomedir() { - return homedir; - } - - @Override - public HFileDataBlockEncoder getDataBlockEncoder() { - return dataBlockEncoder; - } - - /** - * Should be used only in tests. - * @param blockEncoder the block delta encoder to use - */ - void setDataBlockEncoderInTest(HFileDataBlockEncoder blockEncoder) { - this.dataBlockEncoder = blockEncoder; - } - - FileStatus[] getStoreFiles() throws IOException { - return FSUtils.listStatus(this.fs, this.homedir, null); - } - - /** - * Creates an unsorted list of StoreFile loaded in parallel - * from the given directory. - * @throws IOException - */ - private List loadStoreFiles() throws IOException { - ArrayList results = new ArrayList(); - FileStatus files[] = getStoreFiles(); - - if (files == null || files.length == 0) { - return results; - } - // initialize the thread pool for opening store files in parallel.. - ThreadPoolExecutor storeFileOpenerThreadPool = - this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" + - this.family.getNameAsString()); - CompletionService completionService = - new ExecutorCompletionService(storeFileOpenerThreadPool); - - int totalValidStoreFile = 0; - for (int i = 0; i < files.length; i++) { - // Skip directories. - if (files[i].isDir()) { - continue; - } - final Path p = files[i].getPath(); - // Check for empty file. Should never be the case but can happen - // after data loss in hdfs for whatever reason (upgrade, etc.): HBASE-646 - if (this.fs.getFileStatus(p).getLen() <= 0) { - LOG.warn("Skipping " + p + " because its empty. HBASE-646 DATA LOSS?"); - continue; - } - - // open each store file in parallel - completionService.submit(new Callable() { - public StoreFile call() throws IOException { - StoreFile storeFile = new StoreFile(fs, p, conf, cacheConf, - family.getBloomFilterType(), dataBlockEncoder); - passSchemaMetricsTo(storeFile); - storeFile.createReader(); - return storeFile; - } - }); - totalValidStoreFile++; - } - - try { - for (int i = 0; i < totalValidStoreFile; i++) { - Future future = completionService.take(); - StoreFile storeFile = future.get(); - long length = storeFile.getReader().length(); - this.storeSize += length; - this.totalUncompressedBytes += - storeFile.getReader().getTotalUncompressedBytes(); - if (LOG.isDebugEnabled()) { - LOG.debug("loaded " + storeFile.toStringDetailed()); - } - results.add(storeFile); - } - } catch (InterruptedException e) { - throw new IOException(e); - } catch (ExecutionException e) { - throw new IOException(e.getCause()); - } finally { - storeFileOpenerThreadPool.shutdownNow(); - } - - return results; - } - - @Override - public long add(final KeyValue kv) { - lock.readLock().lock(); - try { - return this.memstore.add(kv); - } finally { - lock.readLock().unlock(); - } - } + public long upsert(Iterable kvs) throws IOException; /** * Adds a value to the memstore - * * @param kv * @return memstore size delta */ - protected long delete(final KeyValue kv) { - lock.readLock().lock(); - try { - return this.memstore.delete(kv); - } finally { - lock.readLock().unlock(); - } - } - - @Override - public void rollback(final KeyValue kv) { - lock.readLock().lock(); - try { - this.memstore.rollback(kv); - } finally { - lock.readLock().unlock(); - } - } + public long add(KeyValue kv); /** - * @return All store files. + * Removes a kv from the memstore. The KeyValue is removed only if its key & memstoreTS match the + * key & memstoreTS value of the kv parameter. + * @param kv */ - @Override - public List getStorefiles() { - return this.storefiles; - } - - @Override - public void assertBulkLoadHFileOk(Path srcPath) throws IOException { - HFile.Reader reader = null; - try { - LOG.info("Validating hfile at " + srcPath + " for inclusion in " - + "store " + this + " region " + this.region); - reader = HFile.createReader(srcPath.getFileSystem(conf), - srcPath, cacheConf); - reader.loadFileInfo(); - - byte[] firstKey = reader.getFirstRowKey(); - Preconditions.checkState(firstKey != null, "First key can not be null"); - byte[] lk = reader.getLastKey(); - Preconditions.checkState(lk != null, "Last key can not be null"); - byte[] lastKey = KeyValue.createKeyValueFromKey(lk).getRow(); - - LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) + - " last=" + Bytes.toStringBinary(lastKey)); - LOG.debug("Region bounds: first=" + - Bytes.toStringBinary(region.getStartKey()) + - " last=" + Bytes.toStringBinary(region.getEndKey())); - - HRegionInfo hri = region.getRegionInfo(); - if (!hri.containsRange(firstKey, lastKey)) { - throw new WrongRegionException( - "Bulk load file " + srcPath.toString() + " does not fit inside region " - + this.region); - } - - if (verifyBulkLoads) { - KeyValue prevKV = null; - HFileScanner scanner = reader.getScanner(false, false, false); - scanner.seekTo(); - do { - KeyValue kv = scanner.getKeyValue(); - if (prevKV != null) { - if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getRowOffset(), - prevKV.getRowLength(), kv.getBuffer(), kv.getRowOffset(), - kv.getRowLength()) > 0) { - throw new InvalidHFileException("Previous row is greater than" - + " current row: path=" + srcPath + " previous=" - + Bytes.toStringBinary(prevKV.getKey()) + " current=" - + Bytes.toStringBinary(kv.getKey())); - } - if (Bytes.compareTo(prevKV.getBuffer(), prevKV.getFamilyOffset(), - prevKV.getFamilyLength(), kv.getBuffer(), kv.getFamilyOffset(), - kv.getFamilyLength()) != 0) { - throw new InvalidHFileException("Previous key had different" - + " family compared to current key: path=" + srcPath - + " previous=" + Bytes.toStringBinary(prevKV.getFamily()) - + " current=" + Bytes.toStringBinary(kv.getFamily())); - } - } - prevKV = kv; - } while (scanner.next()); - } - } finally { - if (reader != null) reader.close(); - } - } - - @Override - public void bulkLoadHFile(String srcPathStr) throws IOException { - Path srcPath = new Path(srcPathStr); - - // Copy the file if it's on another filesystem - FileSystem srcFs = srcPath.getFileSystem(conf); - FileSystem desFs = fs instanceof HFileSystem ? ((HFileSystem)fs).getBackingFs() : fs; - if (!srcFs.equals(desFs)) { - LOG.info("Bulk-load file " + srcPath + " is on different filesystem than " + - "the destination store. Copying file over to destination filesystem."); - Path tmpPath = getTmpPath(); - FileUtil.copy(srcFs, srcPath, fs, tmpPath, false, conf); - LOG.info("Copied " + srcPath + " to temporary path on destination filesystem: " + tmpPath); - srcPath = tmpPath; - } - - Path dstPath = StoreFile.getRandomFilename(fs, homedir); - LOG.debug("Renaming bulk load file " + srcPath + " to " + dstPath); - StoreFile.rename(fs, srcPath, dstPath); - - StoreFile sf = new StoreFile(fs, dstPath, this.conf, this.cacheConf, - this.family.getBloomFilterType(), this.dataBlockEncoder); - passSchemaMetricsTo(sf); - - StoreFile.Reader r = sf.createReader(); - this.storeSize += r.length(); - this.totalUncompressedBytes += r.getTotalUncompressedBytes(); - - LOG.info("Moved HFile " + srcPath + " into store directory " + - homedir + " - updating store file list."); - - // Append the new storefile into the list - this.lock.writeLock().lock(); - try { - ArrayList newFiles = new ArrayList(storefiles); - newFiles.add(sf); - this.storefiles = sortAndClone(newFiles); - } finally { - // We need the lock, as long as we are updating the storefiles - // or changing the memstore. Let us release it before calling - // notifyChangeReadersObservers. See HBASE-4485 for a possible - // deadlock scenario that could have happened if continue to hold - // the lock. - this.lock.writeLock().unlock(); - } - notifyChangedReadersObservers(); - LOG.info("Successfully loaded store file " + srcPath - + " into store " + this + " (new location: " + dstPath + ")"); - } + public void rollback(final KeyValue kv); /** - * Get a temporary path in this region. These temporary files - * will get cleaned up when the region is re-opened if they are - * still around. + * Find the key that matches row exactly, or the one that immediately precedes it. WARNING: + * Only use this method on a table where writes occur with strictly increasing timestamps. This + * method assumes this pattern of writes in order to make it reasonably performant. Also our + * search is dependent on the axiom that deletes are for cells that are in the container that + * follows whether a memstore snapshot or a storefile, not for the current container: i.e. we'll + * see deletes before we come across cells we are to delete. Presumption is that the + * memstore#kvset is processed before memstore#snapshot and so on. + * @param row The row key of the targeted row. + * @return Found keyvalue or null if none found. + * @throws IOException */ - private Path getTmpPath() throws IOException { - return StoreFile.getRandomFilename( - fs, region.getTmpDir()); - } + public KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException; - @Override - public ImmutableList close() throws IOException { - this.lock.writeLock().lock(); - try { - ImmutableList result = storefiles; + // Compaction oriented methods - // Clear so metrics doesn't find them. - storefiles = ImmutableList.of(); - - if (!result.isEmpty()) { - // initialize the thread pool for closing store files in parallel. - ThreadPoolExecutor storeFileCloserThreadPool = this.region - .getStoreFileOpenAndCloseThreadPool("StoreFileCloserThread-" - + this.family.getNameAsString()); - - // close each store file in parallel - CompletionService completionService = - new ExecutorCompletionService(storeFileCloserThreadPool); - for (final StoreFile f : result) { - completionService.submit(new Callable() { - public Void call() throws IOException { - f.closeReader(true); - return null; - } - }); - } - - try { - for (int i = 0; i < result.size(); i++) { - Future future = completionService.take(); - future.get(); - } - } catch (InterruptedException e) { - throw new IOException(e); - } catch (ExecutionException e) { - throw new IOException(e.getCause()); - } finally { - storeFileCloserThreadPool.shutdownNow(); - } - } - LOG.info("Closed " + this); - return result; - } finally { - this.lock.writeLock().unlock(); - } - } + public boolean throttleCompaction(long compactionSize); /** - * Snapshot this stores memstore. Call before running - * {@link #flushCache(long, SortedSet, TimeRangeTracker, AtomicLong, MonitoredTask)} so it has - * some work to do. + * getter for CompactionProgress object + * @return CompactionProgress object; can be null */ - void snapshot() { - this.memstore.snapshot(); - } + public CompactionProgress getCompactionProgress(); + + public CompactionRequest requestCompaction() throws IOException; + + public CompactionRequest requestCompaction(int priority) throws IOException; + + public void finishRequest(CompactionRequest cr); /** - * Write out current snapshot. Presumes {@link #snapshot()} has been called - * previously. - * @param logCacheFlushId flush sequence number - * @param snapshot - * @param snapshotTimeRangeTracker - * @param flushedSize The number of bytes flushed - * @param status - * @return Path The path name of the tmp file to which the store was flushed - * @throws IOException + * @return true if we should run a major compaction. */ - private Path flushCache(final long logCacheFlushId, - SortedSet snapshot, - TimeRangeTracker snapshotTimeRangeTracker, - AtomicLong flushedSize, - MonitoredTask status) throws IOException { - // If an exception happens flushing, we let it out without clearing - // the memstore snapshot. The old snapshot will be returned when we say - // 'snapshot', the next time flush comes around. - return internalFlushCache( - snapshot, logCacheFlushId, snapshotTimeRangeTracker, flushedSize, status); - } + public boolean isMajorCompaction() throws IOException; - /* - * @param cache - * @param logCacheFlushId - * @param snapshotTimeRangeTracker - * @param flushedSize The number of bytes flushed - * @return Path The path name of the tmp file to which the store was flushed - * @throws IOException - */ - private Path internalFlushCache(final SortedSet set, - final long logCacheFlushId, - TimeRangeTracker snapshotTimeRangeTracker, - AtomicLong flushedSize, - MonitoredTask status) - throws IOException { - StoreFile.Writer writer; - // Find the smallest read point across all the Scanners. - long smallestReadPoint = region.getSmallestReadPoint(); - long flushed = 0; - Path pathName; - // Don't flush if there are no entries. - if (set.size() == 0) { - return null; - } - // Use a store scanner to find which rows to flush. - // Note that we need to retain deletes, hence - // treat this as a minor compaction. - InternalScanner scanner = null; - KeyValueScanner memstoreScanner = new CollectionBackedScanner(set, this.comparator); - if (getHRegion().getCoprocessorHost() != null) { - scanner = getHRegion().getCoprocessorHost().preFlushScannerOpen(this, memstoreScanner); - } - if (scanner == null) { - Scan scan = new Scan(); - scan.setMaxVersions(scanInfo.getMaxVersions()); - scanner = new StoreScanner(this, scanInfo, scan, - Collections.singletonList(memstoreScanner), ScanType.MINOR_COMPACT, - this.region.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP); - } - if (getHRegion().getCoprocessorHost() != null) { - InternalScanner cpScanner = - getHRegion().getCoprocessorHost().preFlush(this, scanner); - // NULL scanner returned from coprocessor hooks means skip normal processing - if (cpScanner == null) { - return null; - } - scanner = cpScanner; - } - try { - int compactionKVMax = conf.getInt(HConstants.COMPACTION_KV_MAX, 10); - // TODO: We can fail in the below block before we complete adding this - // flush to list of store files. Add cleanup of anything put on filesystem - // if we fail. - synchronized (flushLock) { - status.setStatus("Flushing " + this + ": creating writer"); - // A. Write the map out to the disk - writer = createWriterInTmp(set.size()); - writer.setTimeRangeTracker(snapshotTimeRangeTracker); - pathName = writer.getPath(); - try { - List kvs = new ArrayList(); - boolean hasMore; - do { - hasMore = scanner.next(kvs, compactionKVMax); - if (!kvs.isEmpty()) { - for (KeyValue kv : kvs) { - // If we know that this KV is going to be included always, then let us - // set its memstoreTS to 0. This will help us save space when writing to disk. - if (kv.getMemstoreTS() <= smallestReadPoint) { - // let us not change the original KV. It could be in the memstore - // changing its memstoreTS could affect other threads/scanners. - kv = kv.shallowCopy(); - kv.setMemstoreTS(0); - } - writer.append(kv); - flushed += this.memstore.heapSizeChange(kv, true); - } - kvs.clear(); - } - } while (hasMore); - } finally { - // Write out the log sequence number that corresponds to this output - // hfile. The hfile is current up to and including logCacheFlushId. - status.setStatus("Flushing " + this + ": appending metadata"); - writer.appendMetadata(logCacheFlushId, false); - status.setStatus("Flushing " + this + ": closing flushed file"); - writer.close(); - } - } - } finally { - flushedSize.set(flushed); - scanner.close(); - } - if (LOG.isInfoEnabled()) { - LOG.info("Flushed " + - ", sequenceid=" + logCacheFlushId + - ", memsize=" + StringUtils.humanReadableInt(flushed) + - ", into tmp file " + pathName); - } - return pathName; - } - - /* - * @param path The pathname of the tmp file into which the store was flushed - * @param logCacheFlushId - * @return StoreFile created. - * @throws IOException - */ - private StoreFile commitFile(final Path path, - final long logCacheFlushId, - TimeRangeTracker snapshotTimeRangeTracker, - AtomicLong flushedSize, - MonitoredTask status) - throws IOException { - // Write-out finished successfully, move into the right spot - String fileName = path.getName(); - Path dstPath = new Path(homedir, fileName); - validateStoreFile(path); - String msg = "Renaming flushed file at " + path + " to " + dstPath; - LOG.debug(msg); - status.setStatus("Flushing " + this + ": " + msg); - if (!fs.rename(path, dstPath)) { - LOG.warn("Unable to rename " + path + " to " + dstPath); - } - - status.setStatus("Flushing " + this + ": reopening flushed file"); - StoreFile sf = new StoreFile(this.fs, dstPath, this.conf, this.cacheConf, - this.family.getBloomFilterType(), this.dataBlockEncoder); - passSchemaMetricsTo(sf); - - StoreFile.Reader r = sf.createReader(); - this.storeSize += r.length(); - this.totalUncompressedBytes += r.getTotalUncompressedBytes(); - - // This increments the metrics associated with total flushed bytes for this - // family. The overall flush count is stored in the static metrics and - // retrieved from HRegion.recentFlushes, which is set within - // HRegion.internalFlushcache, which indirectly calls this to actually do - // the flushing through the StoreFlusherImpl class - getSchemaMetrics().updatePersistentStoreMetric( - SchemaMetrics.StoreMetricType.FLUSH_SIZE, flushedSize.longValue()); - if (LOG.isInfoEnabled()) { - LOG.info("Added " + sf + ", entries=" + r.getEntries() + - ", sequenceid=" + logCacheFlushId + - ", filesize=" + StringUtils.humanReadableInt(r.length())); - } - return sf; - } - - /* - * @param maxKeyCount - * @return Writer for a new StoreFile in the tmp dir. - */ - private StoreFile.Writer createWriterInTmp(int maxKeyCount) - throws IOException { - return createWriterInTmp(maxKeyCount, this.family.getCompression(), false); - } - - /* - * @param maxKeyCount - * @param compression Compression algorithm to use - * @param isCompaction whether we are creating a new file in a compaction - * @return Writer for a new StoreFile in the tmp dir. - */ - StoreFile.Writer createWriterInTmp(int maxKeyCount, - Compression.Algorithm compression, boolean isCompaction) - throws IOException { - final CacheConfig writerCacheConf; - if (isCompaction) { - // Don't cache data on write on compactions. - writerCacheConf = new CacheConfig(cacheConf); - writerCacheConf.setCacheDataOnWrite(false); - } else { - writerCacheConf = cacheConf; - } - StoreFile.Writer w = new StoreFile.WriterBuilder(conf, writerCacheConf, - fs, blocksize) - .withOutputDir(region.getTmpDir()) - .withDataBlockEncoder(dataBlockEncoder) - .withComparator(comparator) - .withBloomType(family.getBloomFilterType()) - .withMaxKeyCount(maxKeyCount) - .withChecksumType(checksumType) - .withBytesPerChecksum(bytesPerChecksum) - .withCompression(compression) - .build(); - // The store file writer's path does not include the CF name, so we need - // to configure the HFile writer directly. - SchemaConfigured sc = (SchemaConfigured) w.writer; - SchemaConfigured.resetSchemaMetricsConf(sc); - passSchemaMetricsTo(sc); - return w; - } - - /* - * Change storefiles adding into place the Reader produced by this new flush. - * @param sf - * @param set That was used to make the passed file p. - * @throws IOException - * @return Whether compaction is required. - */ - private boolean updateStorefiles(final StoreFile sf, - final SortedSet set) - throws IOException { - this.lock.writeLock().lock(); - try { - ArrayList newList = new ArrayList(storefiles); - newList.add(sf); - storefiles = sortAndClone(newList); - - this.memstore.clearSnapshot(set); - } finally { - // We need the lock, as long as we are updating the storefiles - // or changing the memstore. Let us release it before calling - // notifyChangeReadersObservers. See HBASE-4485 for a possible - // deadlock scenario that could have happened if continue to hold - // the lock. - this.lock.writeLock().unlock(); - } - - // Tell listeners of the change in readers. - notifyChangedReadersObservers(); - - return needsCompaction(); - } - - /* - * Notify all observers that set of Readers has changed. - * @throws IOException - */ - private void notifyChangedReadersObservers() throws IOException { - for (ChangedReadersObserver o: this.changedReaderObservers) { - o.updateReaders(); - } - } + public void triggerMajorCompaction(); /** - * Get all scanners with no filtering based on TTL (that happens further down - * the line). - * @return all scanners for this store + * See if there's too much store files in this store + * @return true if number of store files is greater than the number defined in minFilesToCompact */ - protected List getScanners(boolean cacheBlocks, - boolean isGet, - boolean isCompaction, - ScanQueryMatcher matcher) throws IOException { - List storeFiles; - List memStoreScanners; - this.lock.readLock().lock(); - try { - storeFiles = this.getStorefiles(); - memStoreScanners = this.memstore.getScanners(); - } finally { - this.lock.readLock().unlock(); - } + public boolean needsCompaction(); - // First the store file scanners - - // TODO this used to get the store files in descending order, - // but now we get them in ascending order, which I think is - // actually more correct, since memstore get put at the end. - List sfScanners = StoreFileScanner - .getScannersForStoreFiles(storeFiles, cacheBlocks, isGet, isCompaction, matcher); - List scanners = - new ArrayList(sfScanners.size()+1); - scanners.addAll(sfScanners); - // Then the memstore scanners - scanners.addAll(memStoreScanners); - return scanners; - } - - /* - * @param o Observer who wants to know about changes in set of Readers - */ - void addChangedReaderObserver(ChangedReadersObserver o) { - this.changedReaderObservers.add(o); - } - - /* - * @param o Observer no longer interested in changes in set of Readers. - */ - void deleteChangedReaderObserver(ChangedReadersObserver o) { - // We don't check if observer present; it may not be (legitimately) - this.changedReaderObservers.remove(o); - } - - ////////////////////////////////////////////////////////////////////////////// - // Compaction - ////////////////////////////////////////////////////////////////////////////// + public int getCompactPriority(); /** - * Compact the StoreFiles. This method may take some time, so the calling - * thread must be able to block for long periods. - * - *

During this time, the Store can work as usual, getting values from - * StoreFiles and writing new StoreFiles from the memstore. - * - * Existing StoreFiles are not destroyed until the new compacted StoreFile is - * completely written-out to disk. - * - *

The compactLock prevents multiple simultaneous compactions. - * The structureLock prevents us from interfering with other write operations. - * - *

We don't want to hold the structureLock for the whole time, as a compact() - * can be lengthy and we want to allow cache-flushes during this period. - * - * @param cr - * compaction details obtained from requestCompaction() - * @throws IOException - * @return Storefile we compacted into or null if we failed or opted out early. + * @param priority priority to check against. When priority is {@link Store#PRIORITY_USER}, + * {@link Store#PRIORITY_USER} is returned. + * @return The priority that this store has in the compaction queue. */ - StoreFile compact(CompactionRequest cr) throws IOException { - if (cr == null || cr.getFiles().isEmpty()) return null; - Preconditions.checkArgument(cr.getStore().toString().equals(this.toString())); - List filesToCompact = cr.getFiles(); - synchronized (filesCompacting) { - // sanity check: we're compacting files that this store knows about - // TODO: change this to LOG.error() after more debugging - Preconditions.checkArgument(filesCompacting.containsAll(filesToCompact)); - } + public int getCompactPriority(int priority); - // Max-sequenceID is the last key in the files we're compacting - long maxId = StoreFile.getMaxSequenceIdInList(filesToCompact); + public StoreFlusher getStoreFlusher(long cacheFlushId); - // Ready to go. Have list of files to compact. - LOG.info("Starting compaction of " + filesToCompact.size() + " file(s) in " - + this + " of " - + this.region.getRegionInfo().getRegionNameAsString() - + " into tmpdir=" + region.getTmpDir() + ", seqid=" + maxId + ", totalSize=" - + StringUtils.humanReadableInt(cr.getSize())); + // Split oriented methods - StoreFile sf = null; - try { - StoreFile.Writer writer = - this.compactor.compact(this, filesToCompact, cr.isMajor(), maxId); - // Move the compaction into place. - if (this.conf.getBoolean("hbase.hstore.compaction.complete", true)) { - sf = completeCompaction(filesToCompact, writer); - if (region.getCoprocessorHost() != null) { - region.getCoprocessorHost().postCompact(this, sf); - } - } else { - // Create storefile around what we wrote with a reader on it. - sf = new StoreFile(this.fs, writer.getPath(), this.conf, this.cacheConf, - this.family.getBloomFilterType(), this.dataBlockEncoder); - sf.createReader(); - } - } finally { - synchronized (filesCompacting) { - filesCompacting.removeAll(filesToCompact); - } - } - - LOG.info("Completed" + (cr.isMajor() ? " major " : " ") + "compaction of " - + filesToCompact.size() + " file(s) in " + this + " of " - + this.region.getRegionInfo().getRegionNameAsString() - + " into " + - (sf == null ? "none" : sf.getPath().getName()) + - ", size=" + (sf == null ? "none" : - StringUtils.humanReadableInt(sf.getReader().length())) - + "; total size for store is " - + StringUtils.humanReadableInt(storeSize)); - return sf; - } - - @Override - public void compactRecentForTesting(int N) throws IOException { - List filesToCompact; - long maxId; - boolean isMajor; - - this.lock.readLock().lock(); - try { - synchronized (filesCompacting) { - filesToCompact = Lists.newArrayList(storefiles); - if (!filesCompacting.isEmpty()) { - // exclude all files older than the newest file we're currently - // compacting. this allows us to preserve contiguity (HBASE-2856) - StoreFile last = filesCompacting.get(filesCompacting.size() - 1); - int idx = filesToCompact.indexOf(last); - Preconditions.checkArgument(idx != -1); - filesToCompact.subList(0, idx + 1).clear(); - } - int count = filesToCompact.size(); - if (N > count) { - throw new RuntimeException("Not enough files"); - } - - filesToCompact = filesToCompact.subList(count - N, count); - maxId = StoreFile.getMaxSequenceIdInList(filesToCompact); - isMajor = (filesToCompact.size() == storefiles.size()); - filesCompacting.addAll(filesToCompact); - Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME); - } - } finally { - this.lock.readLock().unlock(); - } - - try { - // Ready to go. Have list of files to compact. - StoreFile.Writer writer = - this.compactor.compact(this, filesToCompact, isMajor, maxId); - // Move the compaction into place. - StoreFile sf = completeCompaction(filesToCompact, writer); - if (region.getCoprocessorHost() != null) { - region.getCoprocessorHost().postCompact(this, sf); - } - } finally { - synchronized (filesCompacting) { - filesCompacting.removeAll(filesToCompact); - } - } - } - - @Override - public boolean hasReferences() { - return hasReferences(this.storefiles); - } - - /* - * @param files - * @return True if any of the files in files are References. - */ - private boolean hasReferences(Collection files) { - if (files != null && files.size() > 0) { - for (StoreFile hsf: files) { - if (hsf.isReference()) { - return true; - } - } - } - return false; - } - - /* - * Gets lowest timestamp from candidate StoreFiles - * - * @param fs - * @param dir - * @throws IOException - */ - public static long getLowestTimestamp(final List candidates) - throws IOException { - long minTs = Long.MAX_VALUE; - for (StoreFile storeFile : candidates) { - minTs = Math.min(minTs, storeFile.getModificationTimeStamp()); - } - return minTs; - } - - @Override - public CompactionProgress getCompactionProgress() { - return this.compactor.getProgress(); - } - - @Override - public boolean isMajorCompaction() throws IOException { - for (StoreFile sf : this.storefiles) { - if (sf.getReader() == null) { - LOG.debug("StoreFile " + sf + " has null Reader"); - return false; - } - } - - List candidates = new ArrayList(this.storefiles); - - // exclude files above the max compaction threshold - // except: save all references. we MUST compact them - int pos = 0; - while (pos < candidates.size() && - candidates.get(pos).getReader().length() > this.maxCompactSize && - !candidates.get(pos).isReference()) ++pos; - candidates.subList(0, pos).clear(); - - return isMajorCompaction(candidates); - } - - /* - * @param filesToCompact Files to compact. Can be null. - * @return True if we should run a major compaction. - */ - private boolean isMajorCompaction(final List filesToCompact) throws IOException { - boolean result = false; - long mcTime = getNextMajorCompactTime(); - if (filesToCompact == null || filesToCompact.isEmpty() || mcTime == 0) { - return result; - } - // TODO: Use better method for determining stamp of last major (HBASE-2990) - long lowTimestamp = getLowestTimestamp(filesToCompact); - long now = System.currentTimeMillis(); - if (lowTimestamp > 0l && lowTimestamp < (now - mcTime)) { - // Major compaction time has elapsed. - if (filesToCompact.size() == 1) { - // Single file - StoreFile sf = filesToCompact.get(0); - long oldest = - (sf.getReader().timeRangeTracker == null) ? - Long.MIN_VALUE : - now - sf.getReader().timeRangeTracker.minimumTimestamp; - if (sf.isMajorCompaction() && - (this.ttl == HConstants.FOREVER || oldest < this.ttl)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Skipping major compaction of " + this + - " because one (major) compacted file only and oldestTime " + - oldest + "ms is < ttl=" + this.ttl); - } - } else if (this.ttl != HConstants.FOREVER && oldest > this.ttl) { - LOG.debug("Major compaction triggered on store " + this + - ", because keyvalues outdated; time since last major compaction " + - (now - lowTimestamp) + "ms"); - result = true; - } - } else { - if (LOG.isDebugEnabled()) { - LOG.debug("Major compaction triggered on store " + this + - "; time since last major compaction " + (now - lowTimestamp) + "ms"); - } - result = true; - } - } - return result; - } - - long getNextMajorCompactTime() { - // default = 24hrs - long ret = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24); - if (family.getValue(HConstants.MAJOR_COMPACTION_PERIOD) != null) { - String strCompactionTime = - family.getValue(HConstants.MAJOR_COMPACTION_PERIOD); - ret = (new Long(strCompactionTime)).longValue(); - } - - if (ret > 0) { - // default = 20% = +/- 4.8 hrs - double jitterPct = conf.getFloat("hbase.hregion.majorcompaction.jitter", - 0.20F); - if (jitterPct > 0) { - long jitter = Math.round(ret * jitterPct); - // deterministic jitter avoids a major compaction storm on restart - ImmutableList snapshot = storefiles; - if (snapshot != null && !snapshot.isEmpty()) { - String seed = snapshot.get(0).getPath().getName(); - double curRand = new Random(seed.hashCode()).nextDouble(); - ret += jitter - Math.round(2L * jitter * curRand); - } else { - ret = 0; // no storefiles == no major compaction - } - } - } - return ret; - } - - public CompactionRequest requestCompaction() throws IOException { - return requestCompaction(HStore.NO_PRIORITY); - } - - public CompactionRequest requestCompaction(int priority) throws IOException { - // don't even select for compaction if writes are disabled - if (!this.region.areWritesEnabled()) { - return null; - } - - CompactionRequest ret = null; - this.lock.readLock().lock(); - try { - synchronized (filesCompacting) { - // candidates = all storefiles not already in compaction queue - List candidates = Lists.newArrayList(storefiles); - if (!filesCompacting.isEmpty()) { - // exclude all files older than the newest file we're currently - // compacting. this allows us to preserve contiguity (HBASE-2856) - StoreFile last = filesCompacting.get(filesCompacting.size() - 1); - int idx = candidates.indexOf(last); - Preconditions.checkArgument(idx != -1); - candidates.subList(0, idx + 1).clear(); - } - - boolean override = false; - if (region.getCoprocessorHost() != null) { - override = region.getCoprocessorHost().preCompactSelection( - this, candidates); - } - CompactSelection filesToCompact; - if (override) { - // coprocessor is overriding normal file selection - filesToCompact = new CompactSelection(conf, candidates); - } else { - filesToCompact = compactSelection(candidates, priority); - } - - if (region.getCoprocessorHost() != null) { - region.getCoprocessorHost().postCompactSelection(this, - ImmutableList.copyOf(filesToCompact.getFilesToCompact())); - } - - // no files to compact - if (filesToCompact.getFilesToCompact().isEmpty()) { - return null; - } - - // basic sanity check: do not try to compact the same StoreFile twice. - if (!Collections.disjoint(filesCompacting, filesToCompact.getFilesToCompact())) { - // TODO: change this from an IAE to LOG.error after sufficient testing - Preconditions.checkArgument(false, "%s overlaps with %s", - filesToCompact, filesCompacting); - } - filesCompacting.addAll(filesToCompact.getFilesToCompact()); - Collections.sort(filesCompacting, StoreFile.Comparators.FLUSH_TIME); - - // major compaction iff all StoreFiles are included - boolean isMajor = (filesToCompact.getFilesToCompact().size() == this.storefiles.size()); - if (isMajor) { - // since we're enqueuing a major, update the compaction wait interval - this.forceMajor = false; - } - - // everything went better than expected. create a compaction request - int pri = getCompactPriority(priority); - ret = new CompactionRequest(region, this, filesToCompact, isMajor, pri); - } - } finally { - this.lock.readLock().unlock(); - } - if (ret != null) { - CompactionRequest.preRequest(ret); - } - return ret; - } - - public void finishRequest(CompactionRequest cr) { - CompactionRequest.postRequest(cr); - cr.finishRequest(); - synchronized (filesCompacting) { - filesCompacting.removeAll(cr.getFiles()); - } - } + public boolean canSplit(); /** - * Algorithm to choose which files to compact, see {@link #compactSelection(java.util.List, int)} - * @param candidates - * @return - * @throws IOException + * Determines if Store should be split + * @return byte[] if store should be split, null otherwise. */ - CompactSelection compactSelection(List candidates) throws IOException { - return compactSelection(candidates,HStore.NO_PRIORITY); - } + public byte[] getSplitPoint(); + + // Bulk Load methods /** - * Algorithm to choose which files to compact - * - * Configuration knobs: - * "hbase.hstore.compaction.ratio" - * normal case: minor compact when file <= sum(smaller_files) * ratio - * "hbase.hstore.compaction.min.size" - * unconditionally compact individual files below this size - * "hbase.hstore.compaction.max.size" - * never compact individual files above this size (unless splitting) - * "hbase.hstore.compaction.min" - * min files needed to minor compact - * "hbase.hstore.compaction.max" - * max files to compact at once (avoids OOM) - * - * @param candidates candidate files, ordered from oldest to newest - * @return subset copy of candidate list that meets compaction criteria - * @throws IOException + * This throws a WrongRegionException if the HFile does not fit in this region, or an + * InvalidHFileException if the HFile is not valid. */ - CompactSelection compactSelection(List candidates, int priority) - throws IOException { - // ASSUMPTION!!! filesCompacting is locked when calling this function - - /* normal skew: - * - * older ----> newer - * _ - * | | _ - * | | | | _ - * --|-|- |-|- |-|---_-------_------- minCompactSize - * | | | | | | | | _ | | - * | | | | | | | | | | | | - * | | | | | | | | | | | | - */ - CompactSelection compactSelection = new CompactSelection(conf, candidates); - - boolean forcemajor = this.forceMajor && filesCompacting.isEmpty(); - if (!forcemajor) { - // Delete the expired store files before the compaction selection. - if (conf.getBoolean("hbase.store.delete.expired.storefile", true) - && (ttl != Long.MAX_VALUE) && (this.scanInfo.minVersions == 0)) { - CompactSelection expiredSelection = compactSelection - .selectExpiredStoreFilesToCompact( - EnvironmentEdgeManager.currentTimeMillis() - this.ttl); - - // If there is any expired store files, delete them by compaction. - if (expiredSelection != null) { - return expiredSelection; - } - } - // do not compact old files above a configurable threshold - // save all references. we MUST compact them - int pos = 0; - while (pos < compactSelection.getFilesToCompact().size() && - compactSelection.getFilesToCompact().get(pos).getReader().length() - > maxCompactSize && - !compactSelection.getFilesToCompact().get(pos).isReference()) ++pos; - if (pos != 0) compactSelection.clearSubList(0, pos); - } - - if (compactSelection.getFilesToCompact().isEmpty()) { - LOG.debug(this.getHRegionInfo().getEncodedName() + " - " + - this + ": no store files to compact"); - compactSelection.emptyFileList(); - return compactSelection; - } - - // Force a major compaction if this is a user-requested major compaction, - // or if we do not have too many files to compact and this was requested - // as a major compaction - boolean majorcompaction = (forcemajor && priority == HStore.PRIORITY_USER) || - (forcemajor || isMajorCompaction(compactSelection.getFilesToCompact())) && - (compactSelection.getFilesToCompact().size() < this.maxFilesToCompact - ); - LOG.debug(this.getHRegionInfo().getEncodedName() + " - " + - this.getColumnFamilyName() + ": Initiating " + - (majorcompaction ? "major" : "minor") + "compaction"); - - if (!majorcompaction && - !hasReferences(compactSelection.getFilesToCompact())) { - // we're doing a minor compaction, let's see what files are applicable - int start = 0; - double r = compactSelection.getCompactSelectionRatio(); - - // skip selection algorithm if we don't have enough files - if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) { - if(LOG.isDebugEnabled()) { - LOG.debug("Not compacting files because we only have " + - compactSelection.getFilesToCompact().size() + - " files ready for compaction. Need " + this.minFilesToCompact + " to initiate."); - } - compactSelection.emptyFileList(); - return compactSelection; - } - - // remove bulk import files that request to be excluded from minors - compactSelection.getFilesToCompact().removeAll(Collections2.filter( - compactSelection.getFilesToCompact(), - new Predicate() { - public boolean apply(StoreFile input) { - return input.excludeFromMinorCompaction(); - } - })); - - /* TODO: add sorting + unit test back in when HBASE-2856 is fixed - // Sort files by size to correct when normal skew is altered by bulk load. - Collections.sort(filesToCompact, StoreFile.Comparators.FILE_SIZE); - */ - - // get store file sizes for incremental compacting selection. - int countOfFiles = compactSelection.getFilesToCompact().size(); - long [] fileSizes = new long[countOfFiles]; - long [] sumSize = new long[countOfFiles]; - for (int i = countOfFiles-1; i >= 0; --i) { - StoreFile file = compactSelection.getFilesToCompact().get(i); - fileSizes[i] = file.getReader().length(); - // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo - int tooFar = i + this.maxFilesToCompact - 1; - sumSize[i] = fileSizes[i] - + ((i+1 < countOfFiles) ? sumSize[i+1] : 0) - - ((tooFar < countOfFiles) ? fileSizes[tooFar] : 0); - } - - /* Start at the oldest file and stop when you find the first file that - * meets compaction criteria: - * (1) a recently-flushed, small file (i.e. <= minCompactSize) - * OR - * (2) within the compactRatio of sum(newer_files) - * Given normal skew, any newer files will also meet this criteria - * - * Additional Note: - * If fileSizes.size() >> maxFilesToCompact, we will recurse on - * compact(). Consider the oldest files first to avoid a - * situation where we always compact [end-threshold,end). Then, the - * last file becomes an aggregate of the previous compactions. - */ - while(countOfFiles - start >= this.minFilesToCompact && - fileSizes[start] > - Math.max(minCompactSize, (long)(sumSize[start+1] * r))) { - ++start; - } - int end = Math.min(countOfFiles, start + this.maxFilesToCompact); - long totalSize = fileSizes[start] - + ((start+1 < countOfFiles) ? sumSize[start+1] : 0); - compactSelection = compactSelection.getSubList(start, end); - - // if we don't have enough files to compact, just wait - if (compactSelection.getFilesToCompact().size() < this.minFilesToCompact) { - if (LOG.isDebugEnabled()) { - LOG.debug("Skipped compaction of " + this - + ". Only " + (end - start) + " file(s) of size " - + StringUtils.humanReadableInt(totalSize) - + " have met compaction criteria."); - } - compactSelection.emptyFileList(); - return compactSelection; - } - } else { - if(majorcompaction) { - if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) { - LOG.debug("Warning, compacting more than " + this.maxFilesToCompact + - " files, probably because of a user-requested major compaction"); - if(priority != HStore.PRIORITY_USER) { - LOG.error("Compacting more than max files on a non user-requested compaction"); - } - } - } else if (compactSelection.getFilesToCompact().size() > this.maxFilesToCompact) { - // all files included in this compaction, up to max - int pastMax = compactSelection.getFilesToCompact().size() - this.maxFilesToCompact; - compactSelection.getFilesToCompact().subList(0, pastMax).clear(); - } - } - return compactSelection; - } + public void assertBulkLoadHFileOk(Path srcPath) throws IOException; /** - * Validates a store file by opening and closing it. In HFileV2 this should - * not be an expensive operation. - * - * @param path the path to the store file + * This method should only be called from HRegion. It is assumed that the ranges of values in the + * HFile fit within the stores assigned region. (assertBulkLoadHFileOk checks this) */ - private void validateStoreFile(Path path) - throws IOException { - StoreFile storeFile = null; - try { - storeFile = new StoreFile(this.fs, path, this.conf, - this.cacheConf, this.family.getBloomFilterType(), - NoOpDataBlockEncoder.INSTANCE); - passSchemaMetricsTo(storeFile); - storeFile.createReader(); - } catch (IOException e) { - LOG.error("Failed to open store file : " + path - + ", keeping it in tmp location", e); - throw e; - } finally { - if (storeFile != null) { - storeFile.closeReader(false); - } - } - } + public void bulkLoadHFile(String srcPathStr) throws IOException; - /* - *

It works by processing a compaction that's been written to disk. - * - *

It is usually invoked at the end of a compaction, but might also be - * invoked at HStore startup, if the prior execution died midway through. - * - *

Moving the compacted TreeMap into place means: - *

-   * 1) Moving the new compacted StoreFile into place
-   * 2) Unload all replaced StoreFile, close and collect list to delete.
-   * 3) Loading the new TreeMap.
-   * 4) Compute new store size
-   * 
- * - * @param compactedFiles list of files that were compacted - * @param compactedFile StoreFile that is the result of the compaction - * @return StoreFile created. May be null. - * @throws IOException - */ - StoreFile completeCompaction(final Collection compactedFiles, - final StoreFile.Writer compactedFile) - throws IOException { - // 1. Moving the new files into place -- if there is a new file (may not - // be if all cells were expired or deleted). - StoreFile result = null; - if (compactedFile != null) { - validateStoreFile(compactedFile.getPath()); - // Move the file into the right spot - Path origPath = compactedFile.getPath(); - Path destPath = new Path(homedir, origPath.getName()); - LOG.info("Renaming compacted file at " + origPath + " to " + destPath); - if (!fs.rename(origPath, destPath)) { - LOG.error("Failed move of compacted file " + origPath + " to " + - destPath); - throw new IOException("Failed move of compacted file " + origPath + - " to " + destPath); - } - result = new StoreFile(this.fs, destPath, this.conf, this.cacheConf, - this.family.getBloomFilterType(), this.dataBlockEncoder); - passSchemaMetricsTo(result); - result.createReader(); - } - try { - this.lock.writeLock().lock(); - try { - // Change this.storefiles so it reflects new state but do not - // delete old store files until we have sent out notification of - // change in case old files are still being accessed by outstanding - // scanners. - ArrayList newStoreFiles = Lists.newArrayList(storefiles); - newStoreFiles.removeAll(compactedFiles); - filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock() - - // If a StoreFile result, move it into place. May be null. - if (result != null) { - newStoreFiles.add(result); - } - - this.storefiles = sortAndClone(newStoreFiles); - } finally { - // We need the lock, as long as we are updating the storefiles - // or changing the memstore. Let us release it before calling - // notifyChangeReadersObservers. See HBASE-4485 for a possible - // deadlock scenario that could have happened if continue to hold - // the lock. - this.lock.writeLock().unlock(); - } - - // Tell observers that list of StoreFiles has changed. - notifyChangedReadersObservers(); - - // let the archive util decide if we should archive or delete the files - LOG.debug("Removing store files after compaction..."); - HFileArchiver.archiveStoreFiles(this.fs, this.region, this.conf, this.family.getName(), - compactedFiles); - - } catch (IOException e) { - e = RemoteExceptionHandler.checkIOException(e); - LOG.error("Failed replacing compacted files in " + this + - ". Compacted file is " + (result == null? "none": result.toString()) + - ". Files replaced " + compactedFiles.toString() + - " some of which may have been already removed", e); - } - - // 4. Compute new store size - this.storeSize = 0L; - this.totalUncompressedBytes = 0L; - for (StoreFile hsf : this.storefiles) { - StoreFile.Reader r = hsf.getReader(); - if (r == null) { - LOG.warn("StoreFile " + hsf + " has a null Reader"); - continue; - } - this.storeSize += r.length(); - this.totalUncompressedBytes += r.getTotalUncompressedBytes(); - } - return result; - } - - public ImmutableList sortAndClone(List storeFiles) { - Collections.sort(storeFiles, StoreFile.Comparators.FLUSH_TIME); - ImmutableList newList = ImmutableList.copyOf(storeFiles); - return newList; - } - - // //////////////////////////////////////////////////////////////////////////// - // Accessors. - // (This is the only section that is directly useful!) - ////////////////////////////////////////////////////////////////////////////// - @Override - public int getNumberOfStoreFiles() { - return this.storefiles.size(); - } - - /* - * @param wantedVersions How many versions were asked for. - * @return wantedVersions or this families' {@link HConstants#VERSIONS}. - */ - int versionsToReturn(final int wantedVersions) { - if (wantedVersions <= 0) { - throw new IllegalArgumentException("Number of versions must be > 0"); - } - // Make sure we do not return more than maximum versions for this store. - int maxVersions = this.family.getMaxVersions(); - return wantedVersions > maxVersions ? maxVersions: wantedVersions; - } - - static boolean isExpired(final KeyValue key, final long oldestTimestamp) { - return key.getTimestamp() < oldestTimestamp; - } - - @Override - public KeyValue getRowKeyAtOrBefore(final byte[] row) throws IOException { - // If minVersions is set, we will not ignore expired KVs. - // As we're only looking for the latest matches, that should be OK. - // With minVersions > 0 we guarantee that any KV that has any version - // at all (expired or not) has at least one version that will not expire. - // Note that this method used to take a KeyValue as arguments. KeyValue - // can be back-dated, a row key cannot. - long ttlToUse = scanInfo.getMinVersions() > 0 ? Long.MAX_VALUE : this.ttl; - - KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP); - - GetClosestRowBeforeTracker state = new GetClosestRowBeforeTracker( - this.comparator, kv, ttlToUse, this.region.getRegionInfo().isMetaRegion()); - this.lock.readLock().lock(); - try { - // First go to the memstore. Pick up deletes and candidates. - this.memstore.getRowKeyAtOrBefore(state); - // Check if match, if we got a candidate on the asked for 'kv' row. - // Process each store file. Run through from newest to oldest. - for (StoreFile sf : Lists.reverse(storefiles)) { - // Update the candidate keys from the current map file - rowAtOrBeforeFromStoreFile(sf, state); - } - return state.getCandidate(); - } finally { - this.lock.readLock().unlock(); - } - } - - /* - * Check an individual MapFile for the row at or before a given row. - * @param f - * @param state - * @throws IOException - */ - private void rowAtOrBeforeFromStoreFile(final StoreFile f, - final GetClosestRowBeforeTracker state) - throws IOException { - StoreFile.Reader r = f.getReader(); - if (r == null) { - LOG.warn("StoreFile " + f + " has a null Reader"); - return; - } - if (r.getEntries() == 0) { - LOG.warn("StoreFile " + f + " is a empty store file"); - return; - } - // TODO: Cache these keys rather than make each time? - byte [] fk = r.getFirstKey(); - KeyValue firstKV = KeyValue.createKeyValueFromKey(fk, 0, fk.length); - byte [] lk = r.getLastKey(); - KeyValue lastKV = KeyValue.createKeyValueFromKey(lk, 0, lk.length); - KeyValue firstOnRow = state.getTargetKey(); - if (this.comparator.compareRows(lastKV, firstOnRow) < 0) { - // If last key in file is not of the target table, no candidates in this - // file. Return. - if (!state.isTargetTable(lastKV)) return; - // If the row we're looking for is past the end of file, set search key to - // last key. TODO: Cache last and first key rather than make each time. - firstOnRow = new KeyValue(lastKV.getRow(), HConstants.LATEST_TIMESTAMP); - } - // Get a scanner that caches blocks and that uses pread. - HFileScanner scanner = r.getHFileReader().getScanner(true, true, false); - // Seek scanner. If can't seek it, return. - if (!seekToScanner(scanner, firstOnRow, firstKV)) return; - // If we found candidate on firstOnRow, just return. THIS WILL NEVER HAPPEN! - // Unlikely that there'll be an instance of actual first row in table. - if (walkForwardInSingleRow(scanner, firstOnRow, state)) return; - // If here, need to start backing up. - while (scanner.seekBefore(firstOnRow.getBuffer(), firstOnRow.getKeyOffset(), - firstOnRow.getKeyLength())) { - KeyValue kv = scanner.getKeyValue(); - if (!state.isTargetTable(kv)) break; - if (!state.isBetterCandidate(kv)) break; - // Make new first on row. - firstOnRow = new KeyValue(kv.getRow(), HConstants.LATEST_TIMESTAMP); - // Seek scanner. If can't seek it, break. - if (!seekToScanner(scanner, firstOnRow, firstKV)) break; - // If we find something, break; - if (walkForwardInSingleRow(scanner, firstOnRow, state)) break; - } - } - - /* - * Seek the file scanner to firstOnRow or first entry in file. - * @param scanner - * @param firstOnRow - * @param firstKV - * @return True if we successfully seeked scanner. - * @throws IOException - */ - private boolean seekToScanner(final HFileScanner scanner, - final KeyValue firstOnRow, - final KeyValue firstKV) - throws IOException { - KeyValue kv = firstOnRow; - // If firstOnRow < firstKV, set to firstKV - if (this.comparator.compareRows(firstKV, firstOnRow) == 0) kv = firstKV; - int result = scanner.seekTo(kv.getBuffer(), kv.getKeyOffset(), - kv.getKeyLength()); - return result >= 0; - } - - /* - * When we come in here, we are probably at the kv just before we break into - * the row that firstOnRow is on. Usually need to increment one time to get - * on to the row we are interested in. - * @param scanner - * @param firstOnRow - * @param state - * @return True we found a candidate. - * @throws IOException - */ - private boolean walkForwardInSingleRow(final HFileScanner scanner, - final KeyValue firstOnRow, - final GetClosestRowBeforeTracker state) - throws IOException { - boolean foundCandidate = false; - do { - KeyValue kv = scanner.getKeyValue(); - // If we are not in the row, skip. - if (this.comparator.compareRows(kv, firstOnRow) < 0) continue; - // Did we go beyond the target row? If so break. - if (state.isTooFar(kv, firstOnRow)) break; - if (state.isExpired(kv)) { - continue; - } - // If we added something, this row is a contender. break. - if (state.handle(kv)) { - foundCandidate = true; - break; - } - } while(scanner.next()); - return foundCandidate; - } - - public boolean canSplit() { - this.lock.readLock().lock(); - try { - // Not splitable if we find a reference store file present in the store. - for (StoreFile sf : storefiles) { - if (sf.isReference()) { - if (LOG.isDebugEnabled()) { - LOG.debug(sf + " is not splittable"); - } - return false; - } - } - - return true; - } finally { - this.lock.readLock().unlock(); - } - } - - @Override - public byte[] getSplitPoint() { - this.lock.readLock().lock(); - try { - // sanity checks - if (this.storefiles.isEmpty()) { - return null; - } - // Should already be enforced by the split policy! - assert !this.region.getRegionInfo().isMetaRegion(); - - // Not splitable if we find a reference store file present in the store. - long maxSize = 0L; - StoreFile largestSf = null; - for (StoreFile sf : storefiles) { - if (sf.isReference()) { - // Should already be enforced since we return false in this case - assert false : "getSplitPoint() called on a region that can't split!"; - return null; - } - - StoreFile.Reader r = sf.getReader(); - if (r == null) { - LOG.warn("Storefile " + sf + " Reader is null"); - continue; - } - - long size = r.length(); - if (size > maxSize) { - // This is the largest one so far - maxSize = size; - largestSf = sf; - } - } - - StoreFile.Reader r = largestSf.getReader(); - if (r == null) { - LOG.warn("Storefile " + largestSf + " Reader is null"); - return null; - } - // Get first, last, and mid keys. Midkey is the key that starts block - // in middle of hfile. Has column and timestamp. Need to return just - // the row we want to split on as midkey. - byte [] midkey = r.midkey(); - if (midkey != null) { - KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length); - byte [] fk = r.getFirstKey(); - KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length); - byte [] lk = r.getLastKey(); - KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length); - // if the midkey is the same as the first or last keys, then we cannot - // (ever) split this region. - if (this.comparator.compareRows(mk, firstKey) == 0 || - this.comparator.compareRows(mk, lastKey) == 0) { - if (LOG.isDebugEnabled()) { - LOG.debug("cannot split because midkey is the same as first or " + - "last row"); - } - return null; - } - return mk.getRow(); - } - } catch(IOException e) { - LOG.warn("Failed getting store size for " + this, e); - } finally { - this.lock.readLock().unlock(); - } - return null; - } - - @Override - public long getLastCompactSize() { - return this.lastCompactSize; - } - - @Override - public long getSize() { - return storeSize; - } - - public void triggerMajorCompaction() { - this.forceMajor = true; - } - - boolean getForceMajorCompaction() { - return this.forceMajor; - } - - ////////////////////////////////////////////////////////////////////////////// - // File administration - ////////////////////////////////////////////////////////////////////////////// - - @Override - public KeyValueScanner getScanner(Scan scan, - final NavigableSet targetCols) throws IOException { - lock.readLock().lock(); - try { - KeyValueScanner scanner = null; - if (getHRegion().getCoprocessorHost() != null) { - scanner = getHRegion().getCoprocessorHost().preStoreScannerOpen(this, scan, targetCols); - } - if (scanner == null) { - scanner = new StoreScanner(this, getScanInfo(), scan, targetCols); - } - return scanner; - } finally { - lock.readLock().unlock(); - } - } - - @Override - public String toString() { - return getColumnFamilyName(); - } - - @Override - public int getStorefilesCount() { - return this.storefiles.size(); - } - - @Override - public long getStoreSizeUncompressed() { - return this.totalUncompressedBytes; - } - - @Override - public long getStorefilesSize() { - long size = 0; - for (StoreFile s: storefiles) { - StoreFile.Reader r = s.getReader(); - if (r == null) { - LOG.warn("StoreFile " + s + " has a null Reader"); - continue; - } - size += r.length(); - } - return size; - } - - @Override - public long getStorefilesIndexSize() { - long size = 0; - for (StoreFile s: storefiles) { - StoreFile.Reader r = s.getReader(); - if (r == null) { - LOG.warn("StoreFile " + s + " has a null Reader"); - continue; - } - size += r.indexSize(); - } - return size; - } - - @Override - public long getTotalStaticIndexSize() { - long size = 0; - for (StoreFile s : storefiles) { - size += s.getReader().getUncompressedDataIndexSize(); - } - return size; - } - - @Override - public long getTotalStaticBloomSize() { - long size = 0; - for (StoreFile s : storefiles) { - StoreFile.Reader r = s.getReader(); - size += r.getTotalBloomSize(); - } - return size; - } - - @Override - public long getMemStoreSize() { - return this.memstore.heapSize(); - } - - public int getCompactPriority() { - return getCompactPriority(HStore.NO_PRIORITY); - } - - @Override - public int getCompactPriority(int priority) { - // If this is a user-requested compaction, leave this at the highest priority - if(priority == HStore.PRIORITY_USER) { - return HStore.PRIORITY_USER; - } else { - return this.blockingStoreFileCount - this.storefiles.size(); - } - } - - @Override - public boolean throttleCompaction(long compactionSize) { - // see HBASE-5867 for discussion on the default - long throttlePoint = conf.getLong( - "hbase.regionserver.thread.compaction.throttle", - 2 * this.minFilesToCompact * this.region.memstoreFlushSize); - return compactionSize > throttlePoint; - } - - @Override - public HRegion getHRegion() { - return this.region; - } - - HRegionInfo getHRegionInfo() { - return this.region.regionInfo; - } - - @Override - public long updateColumnValue(byte [] row, byte [] f, - byte [] qualifier, long newValue) - throws IOException { - - this.lock.readLock().lock(); - try { - long now = EnvironmentEdgeManager.currentTimeMillis(); - - return this.memstore.updateColumnValue(row, - f, - qualifier, - newValue, - now); - - } finally { - this.lock.readLock().unlock(); - } - } - - @Override - public long upsert(Iterable kvs) throws IOException { - this.lock.readLock().lock(); - try { - // TODO: Make this operation atomic w/ MVCC - return this.memstore.upsert(kvs); - } finally { - this.lock.readLock().unlock(); - } - } - - public StoreFlusher getStoreFlusher(long cacheFlushId) { - return new StoreFlusherImpl(cacheFlushId); - } - - private class StoreFlusherImpl implements StoreFlusher { - - private long cacheFlushId; - private SortedSet snapshot; - private StoreFile storeFile; - private Path storeFilePath; - private TimeRangeTracker snapshotTimeRangeTracker; - private AtomicLong flushedSize; - - private StoreFlusherImpl(long cacheFlushId) { - this.cacheFlushId = cacheFlushId; - this.flushedSize = new AtomicLong(); - } - - @Override - public void prepare() { - memstore.snapshot(); - this.snapshot = memstore.getSnapshot(); - this.snapshotTimeRangeTracker = memstore.getSnapshotTimeRangeTracker(); - } - - @Override - public void flushCache(MonitoredTask status) throws IOException { - storeFilePath = Store.this.flushCache( - cacheFlushId, snapshot, snapshotTimeRangeTracker, flushedSize, status); - } - - @Override - public boolean commit(MonitoredTask status) throws IOException { - if (storeFilePath == null) { - return false; - } - storeFile = Store.this.commitFile(storeFilePath, cacheFlushId, - snapshotTimeRangeTracker, flushedSize, status); - if (Store.this.getHRegion().getCoprocessorHost() != null) { - Store.this.getHRegion() - .getCoprocessorHost() - .postFlush(Store.this, storeFile); - } - - // Add new file to store files. Clear snapshot too while we have - // the Store write lock. - return Store.this.updateStorefiles(storeFile, snapshot); - } - } - - @Override - public boolean needsCompaction() { - return (storefiles.size() - filesCompacting.size()) > minFilesToCompact; - } - - @Override - public CacheConfig getCacheConfig() { - return this.cacheConf; - } - - public static final long FIXED_OVERHEAD = - ClassSize.align(SchemaConfigured.SCHEMA_CONFIGURED_UNALIGNED_HEAP_SIZE + - + (17 * ClassSize.REFERENCE) + (6 * Bytes.SIZEOF_LONG) - + (5 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_BOOLEAN); - - public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD - + ClassSize.OBJECT + ClassSize.REENTRANT_LOCK - + ClassSize.CONCURRENT_SKIPLISTMAP - + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + ClassSize.OBJECT - + ScanInfo.FIXED_OVERHEAD); - - @Override - public long heapSize() { - return DEEP_OVERHEAD + this.memstore.heapSize(); - } - - public KeyValue.KVComparator getComparator() { - return comparator; - } - - public ScanInfo getScanInfo() { - return scanInfo; - } + // General accessors into the state of the store + // TODO abstract some of this out into a metrics class /** - * Immutable information for scans over a store. + * @return true if the store has any underlying reference files to older HFiles */ - public static class ScanInfo { - private byte[] family; - private int minVersions; - private int maxVersions; - private long ttl; - private boolean keepDeletedCells; - private long timeToPurgeDeletes; - private KVComparator comparator; + public boolean hasReferences(); - public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT - + (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT) - + Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN); + /** + * @return The size of this store's memstore, in bytes + */ + public long getMemStoreSize(); - /** - * @param family {@link HColumnDescriptor} describing the column family - * @param ttl Store's TTL (in ms) - * @param timeToPurgeDeletes duration in ms after which a delete marker can - * be purged during a major compaction. - * @param comparator The store's comparator - */ - public ScanInfo(HColumnDescriptor family, long ttl, long timeToPurgeDeletes, KVComparator comparator) { - this(family.getName(), family.getMinVersions(), family.getMaxVersions(), ttl, family - .getKeepDeletedCells(), timeToPurgeDeletes, comparator); - } - /** - * @param family Name of this store's column family - * @param minVersions Store's MIN_VERSIONS setting - * @param maxVersions Store's VERSIONS setting - * @param ttl Store's TTL (in ms) - * @param timeToPurgeDeletes duration in ms after which a delete marker can - * be purged during a major compaction. - * @param keepDeletedCells Store's keepDeletedCells setting - * @param comparator The store's comparator - */ - public ScanInfo(byte[] family, int minVersions, int maxVersions, long ttl, - boolean keepDeletedCells, long timeToPurgeDeletes, - KVComparator comparator) { + public HColumnDescriptor getFamily(); - this.family = family; - this.minVersions = minVersions; - this.maxVersions = maxVersions; - this.ttl = ttl; - this.keepDeletedCells = keepDeletedCells; - this.timeToPurgeDeletes = timeToPurgeDeletes; - this.comparator = comparator; - } + /** + * @return The maximum memstoreTS in all store files. + */ + public long getMaxMemstoreTS(); - public byte[] getFamily() { - return family; - } + /** + * @return the data block encoder + */ + public HFileDataBlockEncoder getDataBlockEncoder(); - public int getMinVersions() { - return minVersions; - } + /** + * @return the number of files in this store + */ + public int getNumberOfStoreFiles(); - public int getMaxVersions() { - return maxVersions; - } + /** @return aggregate size of all HStores used in the last compaction */ + public long getLastCompactSize(); - public long getTtl() { - return ttl; - } + /** @return aggregate size of HStore */ + public long getSize(); - public boolean getKeepDeletedCells() { - return keepDeletedCells; - } + /** + * @return Count of store files + */ + public int getStorefilesCount(); - public long getTimeToPurgeDeletes() { - return timeToPurgeDeletes; - } + /** + * @return The size of the store files, in bytes, uncompressed. + */ + public long getStoreSizeUncompressed(); - public KVComparator getComparator() { - return comparator; - } - } + /** + * @return The size of the store files, in bytes. + */ + public long getStorefilesSize(); -} + /** + * @return The size of the store file indexes, in bytes. + */ + public long getStorefilesIndexSize(); + + /** + * Returns the total size of all index blocks in the data block indexes, including the root level, + * intermediate levels, and the leaf level for multi-level indexes, or just the root level for + * single-level indexes. + * @return the total size of block indexes in the store + */ + public long getTotalStaticIndexSize(); + + /** + * Returns the total byte size of all Bloom filter bit arrays. For compound Bloom filters even the + * Bloom blocks currently not loaded into the block cache are counted. + * @return the total size of all Bloom filters in the store + */ + public long getTotalStaticBloomSize(); + + // Test-helper methods + + /** + * Compact the most recent N files. Used in testing. + * @param N number of files to compact. Must be less than or equal to current number of files. + * @throws IOException on failure + */ + public void compactRecentForTesting(int N) throws IOException; + + /** + * Used for tests. + * @return cache configuration for this Store. + */ + public CacheConfig getCacheConfig(); + + /** + * @return the parent region hosting this store + */ + public HRegion getHRegion(); +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index 5682d940e23..aa40b242c1e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.regionserver.StoreFile.Reader; */ @InterfaceAudience.LimitedPrivate("Coprocessor") public class StoreFileScanner implements KeyValueScanner { - static final Log LOG = LogFactory.getLog(Store.class); + static final Log LOG = LogFactory.getLog(HStore.class); // the reader it comes from: private final StoreFile.Reader reader; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index 5533dcd70f9..21251d27ee7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.Filter; -import org.apache.hadoop.hbase.regionserver.Store.ScanInfo; +import org.apache.hadoop.hbase.regionserver.HStore.ScanInfo; import org.apache.hadoop.hbase.regionserver.metrics.RegionMetricsStorage; import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; import org.apache.hadoop.hbase.util.Bytes; @@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; public class StoreScanner extends NonLazyKeyValueScanner implements KeyValueScanner, InternalScanner, ChangedReadersObserver { static final Log LOG = LogFactory.getLog(StoreScanner.class); - private Store store; + private HStore store; private ScanQueryMatcher matcher; private KeyValueHeap heap; private boolean cacheBlocks; @@ -79,7 +79,7 @@ public class StoreScanner extends NonLazyKeyValueScanner private KeyValue lastTop = null; /** An internal constructor. */ - private StoreScanner(Store store, boolean cacheBlocks, Scan scan, + private StoreScanner(HStore store, boolean cacheBlocks, Scan scan, final NavigableSet columns, long ttl, int minVersions) { this.store = store; this.cacheBlocks = cacheBlocks; @@ -107,7 +107,7 @@ public class StoreScanner extends NonLazyKeyValueScanner * @param columns which columns we are scanning * @throws IOException */ - public StoreScanner(Store store, ScanInfo scanInfo, Scan scan, final NavigableSet columns) + public StoreScanner(HStore store, ScanInfo scanInfo, Scan scan, final NavigableSet columns) throws IOException { this(store, scan.getCacheBlocks(), scan, columns, scanInfo.getTtl(), scanInfo.getMinVersions()); @@ -159,7 +159,7 @@ public class StoreScanner extends NonLazyKeyValueScanner * @param smallestReadPoint the readPoint that we should use for tracking * versions */ - public StoreScanner(Store store, ScanInfo scanInfo, Scan scan, + public StoreScanner(HStore store, ScanInfo scanInfo, Scan scan, List scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException { this(store, false, scan, null, scanInfo.getTtl(), @@ -181,7 +181,7 @@ public class StoreScanner extends NonLazyKeyValueScanner } /** Constructor for testing. */ - StoreScanner(final Scan scan, Store.ScanInfo scanInfo, + StoreScanner(final Scan scan, HStore.ScanInfo scanInfo, ScanType scanType, final NavigableSet columns, final List scanners) throws IOException { this(scan, scanInfo, scanType, columns, scanners, @@ -189,7 +189,7 @@ public class StoreScanner extends NonLazyKeyValueScanner } // Constructor for testing. - StoreScanner(final Scan scan, Store.ScanInfo scanInfo, + StoreScanner(final Scan scan, HStore.ScanInfo scanInfo, ScanType scanType, final NavigableSet columns, final List scanners, long earliestPutTs) throws IOException { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java index 78e73b506c4..56ed2902041 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.RemoteExceptionHandler; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.StringUtils; @@ -52,7 +52,7 @@ public class CompactionRequest implements Comparable, Runnable { static final Log LOG = LogFactory.getLog(CompactionRequest.class); private final HRegion r; - private final Store s; + private final HStore s; private final CompactSelection compactSelection; private final long totalSize; private final boolean isMajor; @@ -68,7 +68,7 @@ public class CompactionRequest implements Comparable, private static final ConcurrentHashMap minorCompactions = new ConcurrentHashMap(); - public CompactionRequest(HRegion r, Store s, + public CompactionRequest(HRegion r, HStore s, CompactSelection files, boolean isMajor, int p) { Preconditions.checkNotNull(r); Preconditions.checkNotNull(files); @@ -181,7 +181,7 @@ public class CompactionRequest implements Comparable, } /** Gets the Store for the request */ - public Store getStore() { + public HStore getStore() { return s; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 2007299fad7..92ca7b1f099 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.ipc.RequestContext; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.security.AccessDeniedException; @@ -802,14 +802,14 @@ public class AccessController extends BaseRegionObserver @Override public InternalScanner preCompact(ObserverContext e, - final Store store, final InternalScanner scanner) throws IOException { + final HStore store, final InternalScanner scanner) throws IOException { requirePermission(getTableName(e.getEnvironment()), null, null, Action.ADMIN); return scanner; } @Override public void preCompactSelection(final ObserverContext e, - final Store store, final List candidates) throws IOException { + final HStore store, final List candidates) throws IOException { requirePermission(getTableName(e.getEnvironment()), null, null, Action.ADMIN); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java index 5d712a71ec3..9f05d1f5864 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HFileArchiveUtil.java @@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; /** * Helper class for all utilities related to archival/retrieval of HFiles @@ -64,7 +64,7 @@ public class HFileArchiveUtil { public static Path getStoreArchivePath(Configuration conf, HRegionInfo region, Path tabledir, byte[] family) { Path tableArchiveDir = getTableArchivePath(conf, tabledir); - return Store.getStoreHomedir(tableArchiveDir, + return HStore.getStoreHomedir(tableArchiveDir, HRegionInfo.encodeRegionName(region.getRegionName()), family); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java index 934587bc2ae..bc73ea5e340 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RegionSplitter.java @@ -57,7 +57,7 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.NoServerForRegionException; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import com.google.common.base.Preconditions; @@ -123,7 +123,7 @@ import com.google.common.collect.Sets; *

* The more complicated answer is that this depends upon the largest storefile * in your region. With a growing data size, this will get larger over time. You - * want the largest region to be just big enough that the {@link Store} compact + * want the largest region to be just big enough that the {@link HStore} compact * selection algorithm only compacts it due to a timed major. If you don't, your * cluster can be prone to compaction storms as the algorithm decides to run * major compactions on a large series of regions all at once. Note that @@ -671,7 +671,7 @@ public class RegionSplitter { HTableDescriptor htd = table.getTableDescriptor(); // check every Column Family for that region for (HColumnDescriptor c : htd.getFamilies()) { - Path cfDir = Store.getStoreHomedir(tableDir, hri.getEncodedName(), + Path cfDir = HStore.getStoreHomedir(tableDir, hri.getEncodedName(), c.getName()); if (fs.exists(cfDir)) { for (FileStatus file : fs.listStatus(cfDir)) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 34a67b6396e..37053287a07 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -73,7 +73,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -1815,7 +1815,7 @@ public class HBaseTestingUtility { * Do a small get/scan against one store. This is required because store * has no actual methods of querying itself, and relies on StoreScanner. */ - public static List getFromStoreFile(Store store, + public static List getFromStoreFile(HStore store, Get get) throws IOException { MultiVersionConsistencyControl.resetThreadReadPoint(); Scan scan = new Scan(get); @@ -1839,7 +1839,7 @@ public class HBaseTestingUtility { * Do a small get/scan against one store. This is required because store * has no actual methods of querying itself, and relies on StoreScanner. */ - public static List getFromStoreFile(Store store, + public static List getFromStoreFile(HStore store, byte [] row, NavigableSet columns ) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java index 1f1545b4d5e..db11fd0438a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java @@ -40,8 +40,8 @@ import org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner; import org.apache.hadoop.hbase.regionserver.CheckedArchivingHFileCleaner; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveTestingUtil; @@ -178,7 +178,7 @@ public class TestZooKeeperTableArchiveClient { loadAndCompact(region); // check that we actually have some store files that were archived - HStore store = region.getStore(TEST_FAM); + Store store = region.getStore(TEST_FAM); Path storeArchiveDir = HFileArchiveTestingUtil.getStoreArchivePath(UTIL.getConfiguration(), region, store); @@ -338,7 +338,7 @@ public class TestZooKeeperTableArchiveClient { loadAndCompact(region); // check that we actually have some store files that were archived - HStore store = region.getStore(TEST_FAM); + Store store = region.getStore(TEST_FAM); Path storeArchiveDir = HFileArchiveTestingUtil.getStoreArchivePath(UTIL.getConfiguration(), region, store); @@ -376,7 +376,7 @@ public class TestZooKeeperTableArchiveClient { * Compact all the store files in a given region. */ private void compactRegion(HRegion region, byte[] family) throws IOException { - HStore store = region.getStores().get(TEST_FAM); + Store store = region.getStores().get(TEST_FAM); store.compactRecentForTesting(store.getStorefiles().size()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index 9a6a3778ba6..96d3c040ead 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -67,8 +67,8 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.io.DataInputBuffer; @@ -4594,7 +4594,7 @@ public class TestFromClientSide { String regionName = table.getRegionLocations().firstKey().getEncodedName(); HRegion region = TEST_UTIL.getRSForFirstRegionInTable( tableName).getFromOnlineRegions(regionName); - HStore store = region.getStores().values().iterator().next(); + Store store = region.getStores().values().iterator().next(); CacheConfig cacheConf = store.getCacheConfig(); cacheConf.setCacheDataOnWrite(true); cacheConf.setEvictOnClose(true); @@ -4669,7 +4669,7 @@ public class TestFromClientSide { assertEquals(++expectedBlockMiss, cache.getStats().getMissCount()); } - private void waitForStoreFileCount(HStore store, int count, int timeout) + private void waitForStoreFileCount(Store store, int count, int timeout) throws InterruptedException { long start = System.currentTimeMillis(); while (start + timeout > System.currentTimeMillis() && diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java index 51e9e6ab8a1..7f0078f8eda 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.Leases; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; @@ -140,20 +140,20 @@ public class SimpleRegionObserver extends BaseRegionObserver { } @Override - public InternalScanner preFlush(ObserverContext c, Store store, InternalScanner scanner) { + public InternalScanner preFlush(ObserverContext c, HStore store, InternalScanner scanner) { hadPreFlush = true; return scanner; } @Override public InternalScanner preFlushScannerOpen(final ObserverContext c, - Store store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException { + HStore store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException { hadPreFlushScannerOpen = true; return null; } @Override - public void postFlush(ObserverContext c, Store store, StoreFile resultFile) { + public void postFlush(ObserverContext c, HStore store, StoreFile resultFile) { hadPostFlush = true; } @@ -177,26 +177,26 @@ public class SimpleRegionObserver extends BaseRegionObserver { @Override public void preCompactSelection(ObserverContext c, - Store store, List candidates) { + HStore store, List candidates) { hadPreCompactSelect = true; } @Override public void postCompactSelection(ObserverContext c, - Store store, ImmutableList selected) { + HStore store, ImmutableList selected) { hadPostCompactSelect = true; } @Override public InternalScanner preCompact(ObserverContext e, - Store store, InternalScanner scanner) { + HStore store, InternalScanner scanner) { hadPreCompact = true; return scanner; } @Override public InternalScanner preCompactScannerOpen(final ObserverContext c, - Store store, List scanners, ScanType scanType, long earliestPutTs, + HStore store, List scanners, ScanType scanType, long earliestPutTs, InternalScanner s) throws IOException { hadPreCompactScanner = true; return null; @@ -204,7 +204,7 @@ public class SimpleRegionObserver extends BaseRegionObserver { @Override public void postCompact(ObserverContext e, - Store store, StoreFile resultFile) { + HStore store, StoreFile resultFile) { hadPostCompact = true; } @@ -222,7 +222,7 @@ public class SimpleRegionObserver extends BaseRegionObserver { @Override public KeyValueScanner preStoreScannerOpen(final ObserverContext c, - final Store store, final Scan scan, final NavigableSet targetCols, + final HStore store, final Scan scan, final NavigableSet targetCols, final KeyValueScanner s) throws IOException { hadPreStoreScannerOpen = true; return null; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java index 99b8e1d2bdb..990a0c65ae6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorInterface.java @@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.SplitTransaction; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; @@ -174,13 +174,13 @@ public class TestCoprocessorInterface extends HBaseTestCase { } @Override public InternalScanner preCompact(ObserverContext e, - Store store, InternalScanner scanner) { + HStore store, InternalScanner scanner) { preCompactCalled = true; return scanner; } @Override public void postCompact(ObserverContext e, - Store store, StoreFile resultFile) { + HStore store, StoreFile resultFile) { postCompactCalled = true; } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java index 91deda489a8..4c9103bf226 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverInterface.java @@ -61,7 +61,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -316,7 +316,7 @@ public class TestRegionObserverInterface { @Override public InternalScanner preCompact(ObserverContext e, - Store store, final InternalScanner scanner) { + HStore store, final InternalScanner scanner) { return new InternalScanner() { @Override public boolean next(List results) throws IOException { @@ -368,7 +368,7 @@ public class TestRegionObserverInterface { @Override public void postCompact(ObserverContext e, - Store store, StoreFile resultFile) { + HStore store, StoreFile resultFile) { lastCompaction = EnvironmentEdgeManager.currentTimeMillis(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java index a3c9ae9f5cd..d60ccc99d0e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHeapSize.java @@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.io.hfile.CachedBlock; import org.apache.hadoop.hbase.io.hfile.LruBlockCache; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.MemStore; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ClassSize; @@ -303,8 +303,8 @@ public class TestHeapSize extends TestCase { sc.heapSize()); // Store Overhead - cl = Store.class; - actual = Store.FIXED_OVERHEAD; + cl = HStore.class; + actual = HStore.FIXED_OVERHEAD; expected = ClassSize.estimateBase(cl, false); if(expected != actual) { ClassSize.estimateBase(cl, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java index 0d672ca409b..bac7a80d667 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat.java @@ -64,7 +64,7 @@ import org.apache.hadoop.hbase.io.hfile.Compression; import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile.Reader; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -706,7 +706,7 @@ public class TestHFileOutputFormat { assertEquals("Should start with empty table", 0, util.countRows(table)); // deep inspection: get the StoreFile dir - final Path storePath = Store.getStoreHomedir( + final Path storePath = HStore.getStoreHomedir( HTableDescriptor.getTableDir(FSUtils.getRootDir(conf), TABLE_NAME), admin.getTableRegions(TABLE_NAME).get(0).getEncodedName(), FAMILIES[0]); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java index a54b1cf5063..b79cfc53b67 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java @@ -65,7 +65,7 @@ import org.apache.hadoop.hbase.master.CatalogJanitor.SplitParentFirstComparator; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; @@ -353,7 +353,7 @@ public class TestCatalogJanitor { Path rootdir = services.getMasterFileSystem().getRootDir(); Path tabledir = HTableDescriptor.getTableDir(rootdir, htd.getName()); - Path storedir = Store.getStoreHomedir(tabledir, splita.getEncodedName(), + Path storedir = HStore.getStoreHomedir(tabledir, splita.getEncodedName(), htd.getColumnFamilies()[0].getName()); Reference ref = Reference.createTopReference(Bytes.toBytes("ccc")); long now = System.currentTimeMillis(); @@ -599,7 +599,7 @@ public class TestCatalogJanitor { // the single test passes, but when the full suite is run, things get borked). FSUtils.setRootDir(fs.getConf(), rootdir); Path tabledir = HTableDescriptor.getTableDir(rootdir, htd.getName()); - Path storedir = Store.getStoreHomedir(tabledir, parent.getEncodedName(), + Path storedir = HStore.getStoreHomedir(tabledir, parent.getEncodedName(), htd.getColumnFamilies()[0].getName()); // delete the file and ensure that the files have been archived @@ -665,7 +665,7 @@ public class TestCatalogJanitor { // the single test passes, but when the full suite is run, things get borked). FSUtils.setRootDir(fs.getConf(), rootdir); Path tabledir = HTableDescriptor.getTableDir(rootdir, parent.getTableName()); - Path storedir = Store.getStoreHomedir(tabledir, parent.getEncodedName(), + Path storedir = HStore.getStoreHomedir(tabledir, parent.getEncodedName(), htd.getColumnFamilies()[0].getName()); System.out.println("Old root:" + rootdir); System.out.println("Old table:" + tabledir); @@ -772,7 +772,7 @@ public class TestCatalogJanitor { throws IOException { Path rootdir = services.getMasterFileSystem().getRootDir(); Path tabledir = HTableDescriptor.getTableDir(rootdir, parent.getTableName()); - Path storedir = Store.getStoreHomedir(tabledir, daughter.getEncodedName(), + Path storedir = HStore.getStoreHomedir(tabledir, daughter.getEncodedName(), htd.getColumnFamilies()[0].getName()); Reference ref = top? Reference.createTopReference(midkey): Reference.createBottomReference(midkey); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java index 3860ed1db9a..851006759ce 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java @@ -111,7 +111,7 @@ public class CompactionTool implements Tool { * @return * @throws IOException */ - private Store getStore(final FileSystem fs, final Path storedir, final Path tmpdir) + private HStore getStore(final FileSystem fs, final Path storedir, final Path tmpdir) throws IOException { // TODO: Let config on table and column family be configurable from // command-line setting versions, etc. For now do defaults @@ -121,7 +121,7 @@ public class CompactionTool implements Tool { HRegion region = createHRegion(hri, tmpdir); // Create a Store w/ check of hbase.rootdir blanked out and return our // list of files instead of have Store search its home dir. - return new Store(tmpdir, region, hcd, fs, getConf()) { + return new HStore(tmpdir, region, hcd, fs, getConf()) { @Override public FileStatus[] getStoreFiles() throws IOException { return this.fs.listStatus(getHomedir()); @@ -145,7 +145,7 @@ public class CompactionTool implements Tool { errCode = checkdir(fs, tmpdir); if (errCode != 0) return errCode; // Get a Store that wraps the inputdir of files to compact. - Store store = getStore(fs, inputdir, tmpdir); + HStore store = getStore(fs, inputdir, tmpdir); // Now we have a Store, run a compaction of passed files. try { CompactionRequest cr = store.requestCompaction(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java index e6ff17305ec..5e2032c2d4e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/HFileReadWriteTest.java @@ -346,7 +346,7 @@ public class HFileReadWriteTest { HTableDescriptor htd = new HTableDescriptor(TABLE_NAME); HRegion region = new HRegion(outputDir, null, fs, conf, regionInfo, htd, null); - Store store = new Store(outputDir, region, columnDescriptor, fs, conf); + HStore store = new HStore(outputDir, region, columnDescriptor, fs, conf); StoreFile.Writer writer = new StoreFile.WriterBuilder(conf, new CacheConfig(conf), fs, blockSize) @@ -400,7 +400,7 @@ public class HFileReadWriteTest { return resultPath; } - private void performMerge(List scanners, Store store, + private void performMerge(List scanners, HStore store, StoreFile.Writer writer) throws IOException { InternalScanner scanner = null; try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java index 668c04372c2..2e9f1fccea3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java @@ -25,9 +25,9 @@ public class NoOpScanPolicyObserver extends BaseRegionObserver { */ @Override public InternalScanner preFlushScannerOpen(final ObserverContext c, - Store store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException { - Store.ScanInfo oldSI = store.getScanInfo(); - Store.ScanInfo scanInfo = new Store.ScanInfo(store.getFamily(), oldSI.getTtl(), + HStore store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException { + HStore.ScanInfo oldSI = store.getScanInfo(); + HStore.ScanInfo scanInfo = new HStore.ScanInfo(store.getFamily(), oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); Scan scan = new Scan(); scan.setMaxVersions(oldSI.getMaxVersions()); @@ -41,11 +41,11 @@ public class NoOpScanPolicyObserver extends BaseRegionObserver { */ @Override public InternalScanner preCompactScannerOpen(final ObserverContext c, - Store store, List scanners, ScanType scanType, long earliestPutTs, + HStore store, List scanners, ScanType scanType, long earliestPutTs, InternalScanner s) throws IOException { // this demonstrates how to override the scanners default behavior - Store.ScanInfo oldSI = store.getScanInfo(); - Store.ScanInfo scanInfo = new Store.ScanInfo(store.getFamily(), oldSI.getTtl(), + HStore.ScanInfo oldSI = store.getScanInfo(); + HStore.ScanInfo scanInfo = new HStore.ScanInfo(store.getFamily(), oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); Scan scan = new Scan(); scan.setMaxVersions(oldSI.getMaxVersions()); @@ -55,7 +55,7 @@ public class NoOpScanPolicyObserver extends BaseRegionObserver { @Override public KeyValueScanner preStoreScannerOpen(final ObserverContext c, - Store store, final Scan scan, final NavigableSet targetCols, KeyValueScanner s) + HStore store, final Scan scan, final NavigableSet targetCols, KeyValueScanner s) throws IOException { return new StoreScanner(store, store.getScanInfo(), scan, targetCols); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java index b0453f501cd..cf5df1cbd64 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java @@ -127,7 +127,7 @@ public class TestAtomicOperation extends HBaseTestCase { assertEquals(value+amount, result); - Store store = (Store) region.getStore(fam1); + HStore store = (HStore) region.getStore(fam1); // ICV removes any extra values floating around in there. assertEquals(1, store.memstore.kvset.size()); assertTrue(store.memstore.snapshot.isEmpty()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java index 29d28c31e4c..706dd2725d0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java @@ -121,7 +121,7 @@ public class TestCacheOnWriteInSchema { private final CacheOnWriteType cowType; private Configuration conf; private final String testDescription; - private Store store; + private HStore store; private FileSystem fs; public TestCacheOnWriteInSchema(CacheOnWriteType cowType) { @@ -164,7 +164,7 @@ public class TestCacheOnWriteInSchema { HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false); HLog hlog = new HLog(fs, logdir, oldLogDir, conf); HRegion region = new HRegion(basedir, hlog, fs, conf, info, htd, null); - store = new Store(basedir, region, hcd, fs, conf); + store = new HStore(basedir, region, hcd, fs, conf); } @After diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java index 451ada65d48..45d02e8f3bd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactSelection.java @@ -49,7 +49,7 @@ public class TestCompactSelection extends TestCase { private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private Configuration conf; - private Store store; + private HStore store; private static final String DIR= TEST_UTIL.getDataTestDir("TestCompactSelection").toString(); private static Path TEST_FILE; @@ -91,7 +91,7 @@ public class TestCompactSelection extends TestCase { Path tableDir = new Path(basedir, Bytes.toString(htd.getName())); region = new HRegion(tableDir, hlog, fs, conf, info, htd, null); - store = new Store(basedir, region, hcd, fs, conf); + store = new HStore(basedir, region, hcd, fs, conf); TEST_FILE = StoreFile.getRandomFilename(fs, store.getHomedir()); fs.create(TEST_FILE); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java index 92f62ae1d20..be5f39f8586 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -168,10 +168,10 @@ public class TestCompaction extends HBaseTestCase { public void majorCompactionWithDataBlockEncoding(boolean inCacheOnly) throws Exception { - Map replaceBlockCache = - new HashMap(); - for (Entry pair : r.getStores().entrySet()) { - Store store = (Store) pair.getValue(); + Map replaceBlockCache = + new HashMap(); + for (Entry pair : r.getStores().entrySet()) { + HStore store = (HStore) pair.getValue(); HFileDataBlockEncoder blockEncoder = store.getDataBlockEncoder(); replaceBlockCache.put(store, blockEncoder); final DataBlockEncoding inCache = DataBlockEncoding.PREFIX; @@ -184,7 +184,7 @@ public class TestCompaction extends HBaseTestCase { majorCompaction(); // restore settings - for (Entry entry : + for (Entry entry : replaceBlockCache.entrySet()) { entry.getKey().setDataBlockEncoderInTest(entry.getValue()); } @@ -206,7 +206,7 @@ public class TestCompaction extends HBaseTestCase { assertEquals(compactionThreshold, result.size()); // see if CompactionProgress is in place but null - for (HStore store : this.r.stores.values()) { + for (Store store : this.r.stores.values()) { assertNull(store.getCompactionProgress()); } @@ -215,7 +215,7 @@ public class TestCompaction extends HBaseTestCase { // see if CompactionProgress has done its thing on at least one store int storeCount = 0; - for (HStore store : this.r.stores.values()) { + for (Store store : this.r.stores.values()) { CompactionProgress progress = store.getCompactionProgress(); if( progress != null ) { ++storeCount; @@ -281,10 +281,10 @@ public class TestCompaction extends HBaseTestCase { // Multiple versions allowed for an entry, so the delete isn't enough // Lower TTL and expire to ensure that all our entries have been wiped final int ttl = 1000; - for (HStore hstore : this.r.stores.values()) { - Store store = ((Store) hstore); - Store.ScanInfo old = store.scanInfo; - Store.ScanInfo si = new Store.ScanInfo(old.getFamily(), + for (Store hstore : this.r.stores.values()) { + HStore store = ((HStore) hstore); + HStore.ScanInfo old = store.scanInfo; + HStore.ScanInfo si = new HStore.ScanInfo(old.getFamily(), old.getMinVersions(), old.getMaxVersions(), ttl, old.getKeepDeletedCells(), 0, old.getComparator()); store.scanInfo = si; @@ -303,7 +303,7 @@ public class TestCompaction extends HBaseTestCase { conf.setLong(HConstants.MAJOR_COMPACTION_PERIOD, delay); conf.setFloat("hbase.hregion.majorcompaction.jitter", jitterPct); - Store s = ((Store) r.getStore(COLUMN_FAMILY)); + HStore s = ((HStore) r.getStore(COLUMN_FAMILY)); try { createStoreFile(r); createStoreFile(r); @@ -436,7 +436,7 @@ public class TestCompaction extends HBaseTestCase { assertEquals(compactionThreshold, result.size()); // do a compaction - HStore store2 = this.r.stores.get(fam2); + Store store2 = this.r.stores.get(fam2); int numFiles1 = store2.getStorefiles().size(); assertTrue("Was expecting to see 4 store files", numFiles1 > compactionThreshold); // > 3 store2.compactRecentForTesting(compactionThreshold); // = 3 @@ -482,8 +482,8 @@ public class TestCompaction extends HBaseTestCase { assertEquals(0, count()); // lower the polling interval for this test - int origWI = Store.closeCheckInterval; - Store.closeCheckInterval = 10*1000; // 10 KB + int origWI = HStore.closeCheckInterval; + HStore.closeCheckInterval = 10*1000; // 10 KB try { // Create a couple store files w/ 15KB (over 10KB interval) @@ -513,7 +513,7 @@ public class TestCompaction extends HBaseTestCase { spyR.compactStores(); // ensure that the compaction stopped, all old files are intact, - HStore s = r.stores.get(COLUMN_FAMILY); + Store s = r.stores.get(COLUMN_FAMILY); assertEquals(compactionThreshold, s.getStorefilesCount()); assertTrue(s.getStorefilesSize() > 15*1000); // and no new store files persisted past compactStores() @@ -523,7 +523,7 @@ public class TestCompaction extends HBaseTestCase { } finally { // don't mess up future tests r.writestate.writesEnabled = true; - Store.closeCheckInterval = origWI; + HStore.closeCheckInterval = origWI; // Delete all Store information once done using for (int i = 0; i < compactionThreshold; i++) { @@ -537,10 +537,10 @@ public class TestCompaction extends HBaseTestCase { // Multiple versions allowed for an entry, so the delete isn't enough // Lower TTL and expire to ensure that all our entries have been wiped final int ttl = 1000; - for (HStore hstore: this.r.stores.values()) { - Store store = (Store)hstore; - Store.ScanInfo old = store.scanInfo; - Store.ScanInfo si = new Store.ScanInfo(old.getFamily(), + for (Store hstore: this.r.stores.values()) { + HStore store = (HStore)hstore; + HStore.ScanInfo old = store.scanInfo; + HStore.ScanInfo si = new HStore.ScanInfo(old.getFamily(), old.getMinVersions(), old.getMaxVersions(), ttl, old.getKeepDeletedCells(), 0, old.getComparator()); store.scanInfo = si; @@ -585,7 +585,7 @@ public class TestCompaction extends HBaseTestCase { for (int i = 0; i < nfiles; i++) { createStoreFile(r); } - Store store = (Store) r.getStore(COLUMN_FAMILY); + HStore store = (HStore) r.getStore(COLUMN_FAMILY); List storeFiles = store.getStorefiles(); long maxId = StoreFile.getMaxSequenceIdInList(storeFiles); @@ -623,14 +623,14 @@ public class TestCompaction extends HBaseTestCase { * Test for HBASE-5920 - Test user requested major compactions always occurring */ public void testNonUserMajorCompactionRequest() throws Exception { - HStore store = r.getStore(COLUMN_FAMILY); + Store store = r.getStore(COLUMN_FAMILY); createStoreFile(r); for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) { createStoreFile(r); } store.triggerMajorCompaction(); - CompactionRequest request = store.requestCompaction(HStore.NO_PRIORITY); + CompactionRequest request = store.requestCompaction(Store.NO_PRIORITY); assertNotNull("Expected to receive a compaction request", request); assertEquals( "System-requested major compaction should not occur if there are too many store files", @@ -642,13 +642,13 @@ public class TestCompaction extends HBaseTestCase { * Test for HBASE-5920 */ public void testUserMajorCompactionRequest() throws IOException{ - HStore store = r.getStore(COLUMN_FAMILY); + Store store = r.getStore(COLUMN_FAMILY); createStoreFile(r); for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) { createStoreFile(r); } store.triggerMajorCompaction(); - CompactionRequest request = store.requestCompaction(HStore.PRIORITY_USER); + CompactionRequest request = store.requestCompaction(Store.PRIORITY_USER); assertNotNull("Expected to receive a compaction request", request); assertEquals( "User-requested major compaction should always occur, even if there are too many store files", diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 85f8e4f3f81..83f0f8da7b3 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -236,7 +236,7 @@ public class TestHRegion extends HBaseTestCase { MonitoredTask status = TaskMonitor.get().createStatus(method); Map maxSeqIdInStores = new TreeMap( Bytes.BYTES_COMPARATOR); - for (HStore store : region.getStores().values()) { + for (Store store : region.getStores().values()) { maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId - 1); } @@ -288,7 +288,7 @@ public class TestHRegion extends HBaseTestCase { MonitoredTask status = TaskMonitor.get().createStatus(method); Map maxSeqIdInStores = new TreeMap( Bytes.BYTES_COMPARATOR); - for (HStore store : region.getStores().values()) { + for (Store store : region.getStores().values()) { maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1); } @@ -336,7 +336,7 @@ public class TestHRegion extends HBaseTestCase { Map maxSeqIdInStores = new TreeMap( Bytes.BYTES_COMPARATOR); - for (HStore store : region.getStores().values()) { + for (Store store : region.getStores().values()) { maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId); } long seqId = region.replayRecoveredEditsIfAny(regiondir, @@ -864,7 +864,7 @@ public class TestHRegion extends HBaseTestCase { put.add(kv); //checkAndPut with wrong value - Store store = (Store) region.getStore(fam1); + HStore store = (HStore) region.getStore(fam1); store.memstore.kvset.size(); boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, @@ -1379,10 +1379,10 @@ public class TestHRegion extends HBaseTestCase { // extract the key values out the memstore: // This is kinda hacky, but better than nothing... long now = System.currentTimeMillis(); - KeyValue firstKv = ((Store) region.getStore(fam1)).memstore.kvset.first(); + KeyValue firstKv = ((HStore) region.getStore(fam1)).memstore.kvset.first(); assertTrue(firstKv.getTimestamp() <= now); now = firstKv.getTimestamp(); - for (KeyValue kv : ((Store) region.getStore(fam1)).memstore.kvset) { + for (KeyValue kv : ((HStore) region.getStore(fam1)).memstore.kvset) { assertTrue(kv.getTimestamp() <= now); now = kv.getTimestamp(); } @@ -2320,7 +2320,7 @@ public class TestHRegion extends HBaseTestCase { assertEquals(value+amount, result); - Store store = (Store) region.getStore(fam1); + HStore store = (HStore) region.getStore(fam1); // ICV removes any extra values floating around in there. assertEquals(1, store.memstore.kvset.size()); assertTrue(store.memstore.snapshot.isEmpty()); @@ -2346,7 +2346,7 @@ public class TestHRegion extends HBaseTestCase { region.put(put); // get the store in question: - Store s = (Store) region.getStore(fam1); + HStore s = (HStore) region.getStore(fam1); s.snapshot(); //bam // now increment: @@ -2490,7 +2490,7 @@ public class TestHRegion extends HBaseTestCase { // flush to disk. region.flushcache(); - Store store = (Store) region.getStore(fam1); + HStore store = (HStore) region.getStore(fam1); assertEquals(0, store.memstore.kvset.size()); long r = region.incrementColumnValue(row, fam1, qual1, amount, true); @@ -2516,7 +2516,7 @@ public class TestHRegion extends HBaseTestCase { region.put(put); region.flushcache(); - Store store = (Store) region.getStore(fam1); + HStore store = (HStore) region.getStore(fam1); assertEquals(0, store.memstore.kvset.size()); long r = region.incrementColumnValue(row, fam1, qual3, amount, true); @@ -2562,7 +2562,7 @@ public class TestHRegion extends HBaseTestCase { assertEquals(value+amount, result); - Store store = (Store) region.getStore(fam1); + HStore store = (HStore) region.getStore(fam1); // ICV should update the existing Put with the same timestamp assertEquals(1, store.memstore.kvset.size()); assertTrue(store.memstore.snapshot.isEmpty()); @@ -2578,7 +2578,7 @@ public class TestHRegion extends HBaseTestCase { assertEquals(value+amount, result); - store = (Store) region.getStore(fam1); + store = (HStore) region.getStore(fam1); // ICV should update the existing Put with the same timestamp assertEquals(2, store.memstore.kvset.size()); assertTrue(store.memstore.snapshot.isEmpty()); @@ -3397,7 +3397,7 @@ public class TestHRegion extends HBaseTestCase { region.flushcache(); } //before compaction - Store store = (Store) region.getStore(fam1); + HStore store = (HStore) region.getStore(fam1); List storeFiles = store.getStorefiles(); for (StoreFile storefile : storeFiles) { StoreFile.Reader reader = storefile.getReader(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java index 0da62dfc17c..024bfe759df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMemStore.java @@ -35,7 +35,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.Store.ScanInfo; +import org.apache.hadoop.hbase.regionserver.HStore.ScanInfo; import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; import org.apache.hadoop.hbase.util.Bytes; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java index 79db715467e..832bf1492b9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestQueryMatcher.java @@ -99,7 +99,7 @@ public class TestQueryMatcher extends HBaseTestCase { // 2,4,5 - ScanQueryMatcher qm = new ScanQueryMatcher(scan, new Store.ScanInfo(fam2, + ScanQueryMatcher qm = new ScanQueryMatcher(scan, new HStore.ScanInfo(fam2, 0, 1, ttl, false, 0, rowComparator), get.getFamilyMap().get(fam2), EnvironmentEdgeManager.currentTimeMillis() - ttl); @@ -144,7 +144,7 @@ public class TestQueryMatcher extends HBaseTestCase { expected.add(ScanQueryMatcher.MatchCode.INCLUDE); expected.add(ScanQueryMatcher.MatchCode.DONE); - ScanQueryMatcher qm = new ScanQueryMatcher(scan, new Store.ScanInfo(fam2, + ScanQueryMatcher qm = new ScanQueryMatcher(scan, new HStore.ScanInfo(fam2, 0, 1, ttl, false, 0, rowComparator), null, EnvironmentEdgeManager.currentTimeMillis() - ttl); @@ -197,7 +197,7 @@ public class TestQueryMatcher extends HBaseTestCase { }; long now = EnvironmentEdgeManager.currentTimeMillis(); - ScanQueryMatcher qm = new ScanQueryMatcher(scan, new Store.ScanInfo(fam2, + ScanQueryMatcher qm = new ScanQueryMatcher(scan, new HStore.ScanInfo(fam2, 0, 1, testTTL, false, 0, rowComparator), get.getFamilyMap().get(fam2), now - testTTL); @@ -250,7 +250,7 @@ public class TestQueryMatcher extends HBaseTestCase { }; long now = EnvironmentEdgeManager.currentTimeMillis(); - ScanQueryMatcher qm = new ScanQueryMatcher(scan, new Store.ScanInfo(fam2, + ScanQueryMatcher qm = new ScanQueryMatcher(scan, new HStore.ScanInfo(fam2, 0, 1, testTTL, false, 0, rowComparator), null, now - testTTL); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java index d845231223e..a383a4842fb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java @@ -240,7 +240,7 @@ public class TestRegionServerMetrics { rs.doMetrics(); for (HRegion r : TEST_UTIL.getMiniHBaseCluster().getRegions( Bytes.toBytes(TABLE_NAME))) { - for (Map.Entry storeEntry : r.getStores().entrySet()) { + for (Map.Entry storeEntry : r.getStores().entrySet()) { LOG.info("For region " + r.getRegionNameAsString() + ", CF " + Bytes.toStringBinary(storeEntry.getKey()) + " found store files " + ": " + storeEntry.getValue().getStorefiles()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java index 300155b20a2..00f2d08ec4c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java @@ -45,7 +45,7 @@ public class TestRegionSplitPolicy { private Configuration conf; private HTableDescriptor htd; private HRegion mockRegion; - private TreeMap stores; + private TreeMap stores; private static final byte [] TABLENAME = new byte [] {'t'}; @Before @@ -57,7 +57,7 @@ public class TestRegionSplitPolicy { Mockito.doReturn(htd).when(mockRegion).getTableDesc(); Mockito.doReturn(hri).when(mockRegion).getRegionInfo(); - stores = new TreeMap(Bytes.BYTES_COMPARATOR); + stores = new TreeMap(Bytes.BYTES_COMPARATOR); Mockito.doReturn(stores).when(mockRegion).getStores(); } @@ -90,7 +90,7 @@ public class TestRegionSplitPolicy { // Add a store in excess of split size. Because there are "no regions" // on this server -- rss.getOnlineRegions is 0 -- then we should split // like a constantsizeregionsplitpolicy would - Store mockStore = Mockito.mock(Store.class); + HStore mockStore = Mockito.mock(HStore.class); Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); stores.put(new byte[]{1}, mockStore); @@ -152,7 +152,7 @@ public class TestRegionSplitPolicy { Mockito.doReturn(myHtd).when(myMockRegion).getTableDesc(); Mockito.doReturn(stores).when(myMockRegion).getStores(); - Store mockStore = Mockito.mock(Store.class); + HStore mockStore = Mockito.mock(HStore.class); Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); Mockito.doReturn(Bytes.toBytes("abcd")).when(mockStore).getSplitPoint(); @@ -190,7 +190,7 @@ public class TestRegionSplitPolicy { assertFalse(policy.shouldSplit()); // Add a store above the requisite size. Should split. - Store mockStore = Mockito.mock(Store.class); + HStore mockStore = Mockito.mock(HStore.class); Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); stores.put(new byte[]{1}, mockStore); @@ -228,7 +228,7 @@ public class TestRegionSplitPolicy { assertNull(policy.getSplitPoint()); // Add a store above the requisite size. Should split. - Store mockStore = Mockito.mock(Store.class); + HStore mockStore = Mockito.mock(HStore.class); Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); Mockito.doReturn(Bytes.toBytes("store 1 split")) @@ -239,7 +239,7 @@ public class TestRegionSplitPolicy { Bytes.toString(policy.getSplitPoint())); // Add a bigger store. The split point should come from that one - Store mockStore2 = Mockito.mock(Store.class); + HStore mockStore2 = Mockito.mock(HStore.class); Mockito.doReturn(4000L).when(mockStore2).getSize(); Mockito.doReturn(true).when(mockStore2).canSplit(); Mockito.doReturn(Bytes.toBytes("store 2 split")) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java index e6546182287..bd4e0e4786c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransaction.java @@ -148,7 +148,7 @@ public class TestSplitTransaction { when(storeFileMock.isReference()).thenReturn(true); // add the mock to the parent stores - Store storeMock = Mockito.mock(Store.class); + HStore storeMock = Mockito.mock(HStore.class); List storeFileList = new ArrayList(1); storeFileList.add(storeFileMock); when(storeMock.getStorefiles()).thenReturn(storeFileList); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java index c0718bcc136..616f6f9ff57 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -74,7 +74,7 @@ import com.google.common.base.Joiner; public class TestStore extends TestCase { public static final Log LOG = LogFactory.getLog(TestStore.class); - Store store; + HStore store; byte [] table = Bytes.toBytes("table"); byte [] family = Bytes.toBytes("family"); @@ -147,7 +147,7 @@ public class TestStore extends TestCase { HLog hlog = new HLog(fs, logdir, oldLogDir, conf); HRegion region = new HRegion(basedir, hlog, fs, conf, info, htd, null); - store = new Store(basedir, region, hcd, fs, conf); + store = new HStore(basedir, region, hcd, fs, conf); } public void testDeleteExpiredStoreFiles() throws Exception { @@ -216,14 +216,14 @@ public class TestStore extends TestCase { } // after flush; check the lowest time stamp long lowestTimeStampFromStore = - Store.getLowestTimestamp(store.getStorefiles()); + HStore.getLowestTimestamp(store.getStorefiles()); long lowestTimeStampFromFS = getLowestTimeStampFromFS(fs,store.getStorefiles()); assertEquals(lowestTimeStampFromStore,lowestTimeStampFromFS); // after compact; check the lowest time stamp store.compact(store.requestCompaction()); - lowestTimeStampFromStore = Store.getLowestTimestamp(store.getStorefiles()); + lowestTimeStampFromStore = HStore.getLowestTimestamp(store.getStorefiles()); lowestTimeStampFromFS = getLowestTimeStampFromFS(fs,store.getStorefiles()); assertEquals(lowestTimeStampFromStore,lowestTimeStampFromFS); } @@ -278,7 +278,7 @@ public class TestStore extends TestCase { w.close(); this.store.close(); // Reopen it... should pick up two files - this.store = new Store(storedir.getParent().getParent(), + this.store = new HStore(storedir.getParent().getParent(), this.store.getHRegion(), this.store.getFamily(), fs, c); System.out.println(this.store.getHRegionInfo().getEncodedName()); @@ -688,7 +688,7 @@ public class TestStore extends TestCase { - private static void flushStore(Store store, long id) throws IOException { + private static void flushStore(HStore store, long id) throws IOException { StoreFlusher storeFlusher = store.getStoreFlusher(id); storeFlusher.prepare(); storeFlusher.flushCache(Mockito.mock(MonitoredTask.class)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index b62f5c263cf..e299e1e738d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -178,14 +178,14 @@ public class TestStoreFile extends HBaseTestCase { KeyValue midKV = KeyValue.createKeyValueFromKey(midkey); byte [] midRow = midKV.getRow(); // Create top split. - Path topDir = Store.getStoreHomedir(this.testDir, "1", + Path topDir = HStore.getStoreHomedir(this.testDir, "1", Bytes.toBytes(f.getPath().getParent().getName())); if (this.fs.exists(topDir)) { this.fs.delete(topDir, true); } Path topPath = StoreFile.split(this.fs, topDir, f, midRow, true); // Create bottom split. - Path bottomDir = Store.getStoreHomedir(this.testDir, "2", + Path bottomDir = HStore.getStoreHomedir(this.testDir, "2", Bytes.toBytes(f.getPath().getParent().getName())); if (this.fs.exists(bottomDir)) { this.fs.delete(bottomDir, true); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java index 01f0731549f..133a3377621 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueTestUtil; import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.regionserver.Store.ScanInfo; +import org.apache.hadoop.hbase.regionserver.HStore.ScanInfo; import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdge; @@ -550,7 +550,7 @@ public class TestStoreScanner extends TestCase { List scanners = scanFixture(kvs); Scan scan = new Scan(); scan.setMaxVersions(2); - Store.ScanInfo scanInfo = new Store.ScanInfo(Bytes.toBytes("cf"), + HStore.ScanInfo scanInfo = new HStore.ScanInfo(Bytes.toBytes("cf"), 0 /* minVersions */, 2 /* maxVersions */, 500 /* ttl */, false /* keepDeletedCells */, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java index bc68a864753..ff0866b5c42 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALReplay.java @@ -48,8 +48,8 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.FlushRequester; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdge; @@ -182,7 +182,7 @@ public class TestWALReplay { // flush region and make major compaction destServer.getOnlineRegion(destRegion.getRegionName()).flushcache(); // wait to complete major compaction - for (HStore store : destServer.getOnlineRegion(destRegion.getRegionName()) + for (Store store : destServer.getOnlineRegion(destRegion.getRegionName()) .getStores().values()) { store.triggerMajorCompaction(); } @@ -422,7 +422,7 @@ public class TestWALReplay { final AtomicInteger countOfRestoredEdits = new AtomicInteger(0); HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) { @Override - protected boolean restoreEdit(HStore s, KeyValue kv) { + protected boolean restoreEdit(Store s, KeyValue kv) { boolean b = super.restoreEdit(s, kv); countOfRestoredEdits.incrementAndGet(); return b; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java index 654e5583612..1b48cb7db92 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java @@ -34,8 +34,8 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; /** * Test helper for testing archiving of HFiles @@ -222,7 +222,7 @@ public class HFileArchiveTestingUtil { * @param store store that is archiving files * @return {@link Path} to the store archive directory for the given region */ - public static Path getStoreArchivePath(Configuration conf, HRegion region, HStore store) { + public static Path getStoreArchivePath(Configuration conf, HRegion region, Store store) { return HFileArchiveUtil.getStoreArchivePath(conf, region, store.getFamily().getName()); } @@ -234,7 +234,7 @@ public class HFileArchiveTestingUtil { HRegion region = servingRegions.get(0); // check that we actually have some store files that were archived - HStore store = region.getStore(storeName); + Store store = region.getStore(storeName); return HFileArchiveTestingUtil.getStoreArchivePath(util.getConfiguration(), region, store); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index 1915ca37276..4aad86ee6cd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -47,7 +47,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.util.Bytes; @@ -203,15 +203,15 @@ public class TestCoprocessorScanPolicy { @Override public InternalScanner preFlushScannerOpen(final ObserverContext c, - Store store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException { + HStore store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException { Long newTtl = ttls.get(store.getTableName()); if (newTtl != null) { System.out.println("PreFlush:" + newTtl); } Integer newVersions = versions.get(store.getTableName()); - Store.ScanInfo oldSI = store.getScanInfo(); + HStore.ScanInfo oldSI = store.getScanInfo(); HColumnDescriptor family = store.getFamily(); - Store.ScanInfo scanInfo = new Store.ScanInfo(family.getName(), family.getMinVersions(), + HStore.ScanInfo scanInfo = new HStore.ScanInfo(family.getName(), family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions, newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); @@ -224,13 +224,13 @@ public class TestCoprocessorScanPolicy { @Override public InternalScanner preCompactScannerOpen(final ObserverContext c, - Store store, List scanners, ScanType scanType, + HStore store, List scanners, ScanType scanType, long earliestPutTs, InternalScanner s) throws IOException { Long newTtl = ttls.get(store.getTableName()); Integer newVersions = versions.get(store.getTableName()); - Store.ScanInfo oldSI = store.getScanInfo(); + HStore.ScanInfo oldSI = store.getScanInfo(); HColumnDescriptor family = store.getFamily(); - Store.ScanInfo scanInfo = new Store.ScanInfo(family.getName(), family.getMinVersions(), + HStore.ScanInfo scanInfo = new HStore.ScanInfo(family.getName(), family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions, newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); @@ -242,13 +242,13 @@ public class TestCoprocessorScanPolicy { @Override public KeyValueScanner preStoreScannerOpen( - final ObserverContext c, Store store, final Scan scan, + final ObserverContext c, HStore store, final Scan scan, final NavigableSet targetCols, KeyValueScanner s) throws IOException { Long newTtl = ttls.get(store.getTableName()); Integer newVersions = versions.get(store.getTableName()); - Store.ScanInfo oldSI = store.getScanInfo(); + HStore.ScanInfo oldSI = store.getScanInfo(); HColumnDescriptor family = store.getFamily(); - Store.ScanInfo scanInfo = new Store.ScanInfo(family.getName(), family.getMinVersions(), + HStore.ScanInfo scanInfo = new HStore.ScanInfo(family.getName(), family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions, newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());