HBASE-6599 Rename Store (the implementations) as HStore, and HStore (the Interface) as Store

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1374354 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2012-08-17 16:46:07 +00:00
parent 7036d7a4e8
commit 093c6fa19c
54 changed files with 2558 additions and 2558 deletions

View File

@ -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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> e, Store store,
public InternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> e, HStore store,
InternalScanner scanner) throws IOException {
return scanner;
}
@Override
public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e, Store store,
public void postFlush(ObserverContext<RegionCoprocessorEnvironment> e, HStore store,
StoreFile resultFile) throws IOException {
}
@ -113,27 +113,27 @@ public abstract class BaseRegionObserver implements RegionObserver {
@Override
public void preCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
final Store store, final List<StoreFile> candidates) throws IOException { }
final HStore store, final List<StoreFile> candidates) throws IOException { }
@Override
public void postCompactSelection(final ObserverContext<RegionCoprocessorEnvironment> c,
final Store store, final ImmutableList<StoreFile> selected) { }
final HStore store, final ImmutableList<StoreFile> selected) { }
@Override
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c,
final Store store, List<? extends KeyValueScanner> scanners, final ScanType scanType,
final HStore store, List<? extends KeyValueScanner> scanners, final ScanType scanType,
final long earliestPutTs, final InternalScanner s) throws IOException {
return null;
}
@Override
public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e, final Store store,
public void postCompact(ObserverContext<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c,
final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
final HStore store, final Scan scan, final NavigableSet<byte[]> targetCols,
final KeyValueScanner s) throws IOException {
return null;
}

View File

@ -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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c, final Store store,
InternalScanner preFlush(final ObserverContext<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c, final Store store,
void postFlush(final ObserverContext<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c,
final Store store, final List<StoreFile> candidates) throws IOException;
final HStore store, final List<StoreFile> 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<RegionCoprocessorEnvironment> c,
final Store store, final ImmutableList<StoreFile> selected);
final HStore store, final ImmutableList<StoreFile> 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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c,
final Store store, List<? extends KeyValueScanner> scanners, final ScanType scanType,
final HStore store, List<? extends KeyValueScanner> 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<RegionCoprocessorEnvironment> c, final Store store,
void postCompact(final ObserverContext<RegionCoprocessorEnvironment> 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.
* <p>
* 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.
* <p>
* 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<RegionCoprocessorEnvironment> c,
final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
final HStore store, final Scan scan, final NavigableSet<byte[]> targetCols,
final KeyValueScanner s) throws IOException;
/**

View File

@ -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<ImmutableBytesWritable,
.withCompression(compression)
.withComparator(KeyValue.KEY_COMPARATOR)
.withDataBlockEncoder(encoder)
.withChecksumType(Store.getChecksumType(conf))
.withBytesPerChecksum(Store.getBytesPerChecksum(conf))
.withChecksumType(HStore.getChecksumType(conf))
.withBytesPerChecksum(HStore.getBytesPerChecksum(conf))
.create();
this.writers.put(family, wl);
return wl;

View File

@ -72,7 +72,7 @@ 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.protobuf.ProtobufUtil;
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.StoreFile.BloomType;
import org.apache.hadoop.hbase.util.Bytes;
@ -551,8 +551,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
.withCompression(compression)
.withDataBlockEncoder(dataBlockEncoder)
.withBloomType(bloomFilterType)
.withChecksumType(Store.getChecksumType(conf))
.withBytesPerChecksum(Store.getBytesPerChecksum(conf))
.withChecksumType(HStore.getChecksumType(conf))
.withBytesPerChecksum(HStore.getBytesPerChecksum(conf))
.build();
HFileScanner scanner = halfReader.getScanner(false, false, false);
scanner.seekTo();

View File

@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Result;
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.FSUtils;
@ -336,7 +336,7 @@ class CatalogJanitor extends Chore {
HTableDescriptor parentDescriptor = getTableDescriptor(parent.getTableName());
for (HColumnDescriptor family: parentDescriptor.getFamilies()) {
Path p = Store.getStoreHomedir(tabledir, split.getEncodedName(),
Path p = HStore.getStoreHomedir(tabledir, split.getEncodedName(),
family.getName());
if (!fs.exists(p)) continue;
// Look for reference files. Call listStatus with anonymous instance of PathFilter.

View File

@ -158,7 +158,7 @@ public class CompactSplitThread implements CompactionRequestor {
public synchronized boolean requestSplit(final HRegion r) {
// don't split regions that are blocking
if (shouldSplitRegion() && r.getCompactPriority() >= 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())

View File

@ -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;
}

View File

@ -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<StoreFile> 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.

View File

@ -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())) {

View File

@ -84,7 +84,7 @@ class GetClosestRowBeforeTracker {
* @return True if this <code>kv</code> is expired.
*/
boolean isExpired(final KeyValue kv) {
return Store.isExpired(kv, this.oldestts);
return HStore.isExpired(kv, this.oldestts);
}
/*

View File

@ -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<byte[], HStore> stores = new ConcurrentSkipListMap<byte[], HStore>(
protected final Map<byte[], Store> stores = new ConcurrentSkipListMap<byte[], Store>(
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<Store> completionService =
new ExecutorCompletionService<Store>(storeOpenerThreadPool);
CompletionService<HStore> completionService =
new ExecutorCompletionService<HStore>(storeOpenerThreadPool);
// initialize each store in parallel
for (final HColumnDescriptor family : htableDescriptor.getFamilies()) {
status.setStatus("Instantiating store for column family " + family);
completionService.submit(new Callable<Store>() {
public Store call() throws IOException {
completionService.submit(new Callable<HStore>() {
public HStore call() throws IOException {
return instantiateHStore(tableDir, family);
}
});
}
try {
for (int i = 0; i < htableDescriptor.getFamilies().size(); i++) {
Future<Store> future = completionService.take();
Store store = future.get();
Future<HStore> 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<ImmutableList<StoreFile>>() {
public ImmutableList<StoreFile> 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<KeyValue> 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 <code>column</code>.
* TODO: Make this lookup faster.
*/
public HStore getStore(final byte[] column) {
public Store getStore(final byte[] column) {
return this.stores.get(column);
}
public Map<byte[], HStore> getStores() {
public Map<byte[], Store> getStores() {
return this.stores;
}
@ -3111,7 +3111,7 @@ public class HRegion implements HeapSize { // , Writable{
List<String> storeFileNames = new ArrayList<String>();
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<byte[], String> 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<byte[], NavigableSet<byte[]>> 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<KeyValue> allKVs = new ArrayList<KeyValue>(append.size());
Map<HStore, List<KeyValue>> tempMemstore = new HashMap<HStore, List<KeyValue>>();
Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
long before = EnvironmentEdgeManager.currentTimeMillis();
long size = 0;
long txid = 0;
@ -4655,7 +4655,7 @@ public class HRegion implements HeapSize { // , Writable{
for (Map.Entry<byte[], List<KeyValue>> family : append.getFamilyMap()
.entrySet()) {
HStore store = stores.get(family.getKey());
Store store = stores.get(family.getKey());
List<KeyValue> kvs = new ArrayList<KeyValue>(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<HStore, List<KeyValue>> entry : tempMemstore.entrySet()) {
HStore store = entry.getKey();
for (Map.Entry<Store, List<KeyValue>> 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<KeyValue> allKVs = new ArrayList<KeyValue>(increment.numColumns());
Map<HStore, List<KeyValue>> tempMemstore = new HashMap<HStore, List<KeyValue>>();
Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
long before = EnvironmentEdgeManager.currentTimeMillis();
long size = 0;
long txid = 0;
@ -4808,7 +4808,7 @@ public class HRegion implements HeapSize { // , Writable{
for (Map.Entry<byte [], NavigableMap<byte [], Long>> family :
increment.getFamilyMap().entrySet()) {
HStore store = stores.get(family.getKey());
Store store = stores.get(family.getKey());
List<KeyValue> kvs = new ArrayList<KeyValue>(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<HStore, List<KeyValue>> entry : tempMemstore.entrySet()) {
HStore store = entry.getKey();
for (Map.Entry<Store, List<KeyValue>> 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;
}

View File

@ -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<byte[], HStore> ee : r.stores.entrySet()) {
final HStore store = ee.getValue();
for (Map.Entry<byte[], Store> 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);

View File

@ -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())) {

View File

@ -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;
}

View File

@ -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<StoreFileScanner> scanners,
public InternalScanner preCompactScannerOpen(HStore store, List<StoreFileScanner> scanners,
ScanType scanType, long earliestPutTs) throws IOException {
ObserverContext<RegionCoprocessorEnvironment> 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<StoreFile> candidates) throws IOException {
public boolean preCompactSelection(HStore store, List<StoreFile> candidates) throws IOException {
ObserverContext<RegionCoprocessorEnvironment> 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<StoreFile> selected) {
ObserverContext<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> 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<byte[]> targetCols) throws IOException {
KeyValueScanner s = null;
ObserverContext<RegionCoprocessorEnvironment> ctx = null;

View File

@ -73,11 +73,11 @@ public abstract class RegionSplitPolicy extends Configured {
if (explicitSplitPoint != null) {
return explicitSplitPoint;
}
Map<byte[], HStore> stores = region.getStores();
Map<byte[], Store> 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) {

View File

@ -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<byte[]> 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<byte[]> columns, long oldestUnexpiredTS) {
this(scan, scanInfo, columns, ScanType.USER_SCAN,
Long.MAX_VALUE, /* max Readpoint to track versions */

View File

@ -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);
}

View File

@ -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;

View File

@ -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<byte[]> 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<byte[]> columns)
public StoreScanner(HStore store, ScanInfo scanInfo, Scan scan, final NavigableSet<byte[]> 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<? extends KeyValueScanner> 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<byte[]> columns,
final List<KeyValueScanner> 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<byte[]> columns,
final List<KeyValueScanner> scanners, long earliestPutTs)
throws IOException {

View File

@ -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<CompactionRequest>,
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<CompactionRequest>,
private static final ConcurrentHashMap<Long, AtomicInteger>
minorCompactions = new ConcurrentHashMap<Long, AtomicInteger>();
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<CompactionRequest>,
}
/** Gets the Store for the request */
public Store getStore() {
public HStore getStore() {
return s;
}

View File

@ -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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> e,
final Store store, final List<StoreFile> candidates) throws IOException {
final HStore store, final List<StoreFile> candidates) throws IOException {
requirePermission(getTableName(e.getEnvironment()), null, null, Action.ADMIN);
}

View File

@ -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);
}

View File

@ -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;
* <p>
* 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)) {

View File

@ -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<KeyValue> getFromStoreFile(Store store,
public static List<KeyValue> 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<KeyValue> getFromStoreFile(Store store,
public static List<KeyValue> getFromStoreFile(HStore store,
byte [] row,
NavigableSet<byte[]> columns
) throws IOException {

View File

@ -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());
}
}

View File

@ -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() &&

View File

@ -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<RegionCoprocessorEnvironment> c, Store store, InternalScanner scanner) {
public InternalScanner preFlush(ObserverContext<RegionCoprocessorEnvironment> c, HStore store, InternalScanner scanner) {
hadPreFlush = true;
return scanner;
}
@Override
public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c, Store store, StoreFile resultFile) {
public void postFlush(ObserverContext<RegionCoprocessorEnvironment> c, HStore store, StoreFile resultFile) {
hadPostFlush = true;
}
@ -177,26 +177,26 @@ public class SimpleRegionObserver extends BaseRegionObserver {
@Override
public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
Store store, List<StoreFile> candidates) {
HStore store, List<StoreFile> candidates) {
hadPreCompactSelect = true;
}
@Override
public void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c,
Store store, ImmutableList<StoreFile> selected) {
HStore store, ImmutableList<StoreFile> selected) {
hadPostCompactSelect = true;
}
@Override
public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> e,
Store store, InternalScanner scanner) {
HStore store, InternalScanner scanner) {
hadPreCompact = true;
return scanner;
}
@Override
public InternalScanner preCompactScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
HStore store, List<? extends KeyValueScanner> 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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c,
final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
final HStore store, final Scan scan, final NavigableSet<byte[]> targetCols,
final KeyValueScanner s) throws IOException {
hadPreStoreScannerOpen = true;
return null;

View File

@ -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<RegionCoprocessorEnvironment> e,
Store store, InternalScanner scanner) {
HStore store, InternalScanner scanner) {
preCompactCalled = true;
return scanner;
}
@Override
public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
Store store, StoreFile resultFile) {
HStore store, StoreFile resultFile) {
postCompactCalled = true;
}
@Override

View File

@ -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<RegionCoprocessorEnvironment> e,
Store store, final InternalScanner scanner) {
HStore store, final InternalScanner scanner) {
return new InternalScanner() {
@Override
public boolean next(List<KeyValue> results) throws IOException {
@ -368,7 +368,7 @@ public class TestRegionObserverInterface {
@Override
public void postCompact(ObserverContext<RegionCoprocessorEnvironment> e,
Store store, StoreFile resultFile) {
HStore store, StoreFile resultFile) {
lastCompaction = EnvironmentEdgeManager.currentTimeMillis();
}

View File

@ -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);

View File

@ -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]);

View File

@ -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);

View File

@ -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();

View File

@ -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<StoreFileScanner> scanners, Store store,
private void performMerge(List<StoreFileScanner> scanners, HStore store,
StoreFile.Writer writer) throws IOException {
InternalScanner scanner = null;
try {

View File

@ -25,9 +25,9 @@ public class NoOpScanPolicyObserver extends BaseRegionObserver {
*/
@Override
public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c,
Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
HStore store, List<? extends KeyValueScanner> 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<RegionCoprocessorEnvironment> c,
Store store, final Scan scan, final NavigableSet<byte[]> targetCols, KeyValueScanner s)
HStore store, final Scan scan, final NavigableSet<byte[]> targetCols, KeyValueScanner s)
throws IOException {
return new StoreScanner(store, store.getScanInfo(), scan, targetCols);
}

View File

@ -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());

View File

@ -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

View File

@ -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);
}

View File

@ -168,10 +168,10 @@ public class TestCompaction extends HBaseTestCase {
public void majorCompactionWithDataBlockEncoding(boolean inCacheOnly)
throws Exception {
Map<Store, HFileDataBlockEncoder> replaceBlockCache =
new HashMap<Store, HFileDataBlockEncoder>();
for (Entry<byte[], HStore> pair : r.getStores().entrySet()) {
Store store = (Store) pair.getValue();
Map<HStore, HFileDataBlockEncoder> replaceBlockCache =
new HashMap<HStore, HFileDataBlockEncoder>();
for (Entry<byte[], Store> 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<Store, HFileDataBlockEncoder> entry :
for (Entry<HStore, HFileDataBlockEncoder> 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<StoreFile> 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",

View File

@ -236,7 +236,7 @@ public class TestHRegion extends HBaseTestCase {
MonitoredTask status = TaskMonitor.get().createStatus(method);
Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(
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<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(
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<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(
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<StoreFile> storeFiles = store.getStorefiles();
for (StoreFile storefile : storeFiles) {
StoreFile.Reader reader = storefile.getReader();

View File

@ -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;

View File

@ -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);

View File

@ -240,7 +240,7 @@ public class TestRegionServerMetrics {
rs.doMetrics();
for (HRegion r : TEST_UTIL.getMiniHBaseCluster().getRegions(
Bytes.toBytes(TABLE_NAME))) {
for (Map.Entry<byte[], HStore> storeEntry : r.getStores().entrySet()) {
for (Map.Entry<byte[], Store> storeEntry : r.getStores().entrySet()) {
LOG.info("For region " + r.getRegionNameAsString() + ", CF " +
Bytes.toStringBinary(storeEntry.getKey()) + " found store files " +
": " + storeEntry.getValue().getStorefiles());

View File

@ -45,7 +45,7 @@ public class TestRegionSplitPolicy {
private Configuration conf;
private HTableDescriptor htd;
private HRegion mockRegion;
private TreeMap<byte[], Store> stores;
private TreeMap<byte[], HStore> 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<byte[], Store>(Bytes.BYTES_COMPARATOR);
stores = new TreeMap<byte[], HStore>(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"))

View File

@ -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<StoreFile> storeFileList = new ArrayList<StoreFile>(1);
storeFileList.add(storeFileMock);
when(storeMock.getStorefiles()).thenReturn(storeFileList);

View File

@ -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));

View File

@ -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);

View File

@ -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<KeyValueScanner> 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 */,

View File

@ -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;

View File

@ -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);
}
}

View File

@ -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<RegionCoprocessorEnvironment> 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<RegionCoprocessorEnvironment> c,
Store store, List<? extends KeyValueScanner> scanners, ScanType scanType,
HStore store, List<? extends KeyValueScanner> 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<RegionCoprocessorEnvironment> c, Store store, final Scan scan,
final ObserverContext<RegionCoprocessorEnvironment> c, HStore store, final Scan scan,
final NavigableSet<byte[]> 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());