HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface

This commit is contained in:
zhangduo 2017-09-28 15:24:41 +08:00
parent 0cf15fadd8
commit 7f4c3b3564
62 changed files with 770 additions and 877 deletions

View File

@ -583,6 +583,11 @@ public class ColumnFamilyDescriptorBuilder {
return this; return this;
} }
public ColumnFamilyDescriptorBuilder setValue(final String key, final String value) {
desc.setValue(key, value);
return this;
}
/** /**
* An ModifyableFamilyDescriptor contains information about a column family such as the * An ModifyableFamilyDescriptor contains information about a column family such as the
* number of versions, compression settings, etc. * number of versions, compression settings, etc.

View File

@ -203,7 +203,7 @@ public class ZooKeeperScanPolicyObserver implements RegionCoprocessor, RegionObs
if (data == null) { if (data == null) {
return null; return null;
} }
ScanInfo oldSI = store.getScanInfo(); ScanInfo oldSI = ((HStore) store).getScanInfo();
if (oldSI.getTtl() == Long.MAX_VALUE) { if (oldSI.getTtl() == Long.MAX_VALUE) {
return null; return null;
} }

View File

@ -15,9 +15,10 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
@ -161,12 +162,12 @@ public class CompactionTool extends Configured implements Tool {
} }
do { do {
Optional<CompactionContext> compaction = Optional<CompactionContext> compaction =
store.requestCompaction(Store.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null); store.requestCompaction(PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null);
if (!compaction.isPresent()) { if (!compaction.isPresent()) {
break; break;
} }
List<HStoreFile> storeFiles = List<HStoreFile> storeFiles =
store.compact(compaction.get(), NoLimitThroughputController.INSTANCE); store.compact(compaction.get(), NoLimitThroughputController.INSTANCE, null);
if (storeFiles != null && !storeFiles.isEmpty()) { if (storeFiles != null && !storeFiles.isEmpty()) {
if (keepCompactedFiles && deleteCompacted) { if (keepCompactedFiles && deleteCompacted) {
for (HStoreFile storeFile: storeFiles) { for (HStoreFile storeFile: storeFiles) {

View File

@ -85,7 +85,7 @@ import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem; import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -1258,7 +1258,7 @@ public class TestHFileOutputFormat2 {
public Boolean call() throws Exception { public Boolean call() throws Exception {
List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]); List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]);
for (HRegion region : regions) { for (HRegion region : regions) {
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
store.closeAndArchiveCompactedFiles(); store.closeAndArchiveCompactedFiles();
} }
} }
@ -1277,7 +1277,7 @@ public class TestHFileOutputFormat2 {
public Boolean call() throws Exception { public Boolean call() throws Exception {
List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]); List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]);
for (HRegion region : regions) { for (HRegion region : regions) {
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
store.closeAndArchiveCompactedFiles(); store.closeAndArchiveCompactedFiles();
} }
} }

View File

@ -105,7 +105,7 @@ public class BusyRegionSplitPolicy extends IncreasingToUpperBoundRegionSplitPoli
return false; return false;
} }
for (Store store: region.getStores()) { for (HStore store: region.getStores()) {
if (!store.canSplit()) { if (!store.canSplit()) {
return false; return false;
} }

View File

@ -18,6 +18,9 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.regionserver.Store.NO_PRIORITY;
import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER;
import java.io.IOException; import java.io.IOException;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.io.StringWriter; import java.io.StringWriter;
@ -35,7 +38,6 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.conf.ConfigurationManager; import org.apache.hadoop.hbase.conf.ConfigurationManager;
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver; import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager; import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
@ -45,12 +47,14 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory; import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.StealJobQueue; import org.apache.hadoop.hbase.util.StealJobQueue;
import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
/** /**
* Compact region on request and then run split if appropriate * Compact region on request and then run split if appropriate
@ -195,7 +199,7 @@ public class CompactSplit implements PropagatingConfigurationObserver {
public synchronized boolean requestSplit(final Region r) { public synchronized boolean requestSplit(final Region r) {
// don't split regions that are blocking // don't split regions that are blocking
if (shouldSplitRegion() && ((HRegion)r).getCompactPriority() >= Store.PRIORITY_USER) { if (shouldSplitRegion() && ((HRegion)r).getCompactPriority() >= PRIORITY_USER) {
byte[] midKey = ((HRegion)r).checkSplit(); byte[] midKey = ((HRegion)r).checkSplit();
if (midKey != null) { if (midKey != null) {
requestSplit(r, midKey); requestSplit(r, midKey);
@ -298,13 +302,13 @@ public class CompactSplit implements PropagatingConfigurationObserver {
} }
public synchronized void requestSystemCompaction(HRegion region, String why) throws IOException { public synchronized void requestSystemCompaction(HRegion region, String why) throws IOException {
requestCompactionInternal(region, why, Store.NO_PRIORITY, false, requestCompactionInternal(region, why, NO_PRIORITY, false,
CompactionLifeCycleTracker.DUMMY, null); CompactionLifeCycleTracker.DUMMY, null);
} }
public synchronized void requestSystemCompaction(HRegion region, HStore store, String why) public synchronized void requestSystemCompaction(HRegion region, HStore store, String why)
throws IOException { throws IOException {
requestCompactionInternal(region, store, why, Store.NO_PRIORITY, false, requestCompactionInternal(region, store, why, NO_PRIORITY, false,
CompactionLifeCycleTracker.DUMMY, null); CompactionLifeCycleTracker.DUMMY, null);
} }

View File

@ -24,11 +24,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.ScheduledChore; import org.apache.hadoop.hbase.ScheduledChore;
import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.Stoppable;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.executor.EventType; import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -95,12 +92,11 @@ public class CompactedHFilesDischarger extends ScheduledChore {
if (LOG.isTraceEnabled()) { if (LOG.isTraceEnabled()) {
LOG.trace("Started compacted hfiles cleaner on " + region.getRegionInfo()); LOG.trace("Started compacted hfiles cleaner on " + region.getRegionInfo());
} }
for (Store store : region.getStores()) { for (HStore store : ((HRegion) region).getStores()) {
try { try {
if (useExecutor && regionServerServices != null) { if (useExecutor && regionServerServices != null) {
CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler( CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler(
(Server) regionServerServices, EventType.RS_COMPACTED_FILES_DISCHARGER, (Server) regionServerServices, EventType.RS_COMPACTED_FILES_DISCHARGER, store);
(HStore) store);
regionServerServices.getExecutorService().submit(handler); regionServerServices.getExecutorService().submit(handler);
} else { } else {
// call synchronously if the RegionServerServices are not // call synchronously if the RegionServerServices are not

View File

@ -72,7 +72,7 @@ public class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy {
boolean force = region.shouldForceSplit(); boolean force = region.shouldForceSplit();
boolean foundABigStore = false; boolean foundABigStore = false;
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
// If any of the stores are unable to split (eg they contain reference files) // If any of the stores are unable to split (eg they contain reference files)
// then don't split // then don't split
if ((!store.canSplit())) { if ((!store.canSplit())) {

View File

@ -78,11 +78,11 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
} }
protected boolean shouldFlush(HStore store) { protected boolean shouldFlush(HStore store) {
if (store.getSizeOfMemStore().getDataSize() > this.flushSizeLowerBound) { if (store.getMemStoreSize().getDataSize() > this.flushSizeLowerBound) {
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Flush Column Family " + store.getColumnFamilyName() + " of " + LOG.debug("Flush Column Family " + store.getColumnFamilyName() + " of " +
region.getRegionInfo().getEncodedName() + " because memstoreSize=" + region.getRegionInfo().getEncodedName() + " because memstoreSize=" +
store.getSizeOfMemStore().getDataSize() + " > lower bound=" store.getMemStoreSize().getDataSize() + " > lower bound="
+ this.flushSizeLowerBound); + this.flushSizeLowerBound);
} }
return true; return true;

View File

@ -1011,13 +1011,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
hasSloppyStores = true; hasSloppyStores = true;
} }
long storeMaxSequenceId = store.getMaxSequenceId(); long storeMaxSequenceId = store.getMaxSequenceId().orElse(0L);
maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(),
storeMaxSequenceId); storeMaxSequenceId);
if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) { if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) {
maxSeqId = storeMaxSequenceId; maxSeqId = storeMaxSequenceId;
} }
long maxStoreMemstoreTS = store.getMaxMemstoreTS(); long maxStoreMemstoreTS = store.getMaxMemstoreTS().orElse(0L);
if (maxStoreMemstoreTS > maxMemstoreTS) { if (maxStoreMemstoreTS > maxMemstoreTS) {
maxMemstoreTS = maxStoreMemstoreTS; maxMemstoreTS = maxStoreMemstoreTS;
} }
@ -1645,7 +1645,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// close each store in parallel // close each store in parallel
for (HStore store : stores.values()) { for (HStore store : stores.values()) {
MemstoreSize flushableSize = store.getSizeToFlush(); MemstoreSize flushableSize = store.getFlushableSize();
if (!(abort || flushableSize.getDataSize() == 0 || writestate.readOnly)) { if (!(abort || flushableSize.getDataSize() == 0 || writestate.readOnly)) {
if (getRegionServerServices() != null) { if (getRegionServerServices() != null) {
getRegionServerServices().abort("Assertion failed while closing store " getRegionServerServices().abort("Assertion failed while closing store "
@ -1717,7 +1717,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
private long getMemstoreHeapSize() { private long getMemstoreHeapSize() {
return stores.values().stream().mapToLong(s -> s.getSizeOfMemStore().getHeapSize()).sum(); return stores.values().stream().mapToLong(s -> s.getMemStoreSize().getHeapSize()).sum();
} }
@Override @Override
@ -2320,7 +2320,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
//since we didn't flush in the recent past, flush now if certain conditions //since we didn't flush in the recent past, flush now if certain conditions
//are met. Return true on first such memstore hit. //are met. Return true on first such memstore hit.
for (Store s : stores.values()) { for (HStore s : stores.values()) {
if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) { if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
// we have an old enough edit in the memstore, flush // we have an old enough edit in the memstore, flush
whyFlush.append(s.toString() + " has an old edit so flush to free WALs"); whyFlush.append(s.toString() + " has an old edit so flush to free WALs");
@ -2481,7 +2481,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
} }
for (HStore s : storesToFlush) { for (HStore s : storesToFlush) {
MemstoreSize flushableSize = s.getSizeToFlush(); MemstoreSize flushableSize = s.getFlushableSize();
totalSizeOfFlushableStores.incMemstoreSize(flushableSize); totalSizeOfFlushableStores.incMemstoreSize(flushableSize);
storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(), s.createFlushContext(flushOpSeqId)); storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(), s.createFlushContext(flushOpSeqId));
committedFiles.put(s.getColumnFamilyDescriptor().getName(), null); // for writing stores to WAL committedFiles.put(s.getColumnFamilyDescriptor().getName(), null); // for writing stores to WAL
@ -2529,7 +2529,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
for (HStore store: storesToFlush) { for (HStore store: storesToFlush) {
perCfExtras.append("; ").append(store.getColumnFamilyName()); perCfExtras.append("; ").append(store.getColumnFamilyName());
perCfExtras.append("=") perCfExtras.append("=")
.append(StringUtils.byteDesc(store.getSizeToFlush().getDataSize())); .append(StringUtils.byteDesc(store.getFlushableSize().getDataSize()));
} }
} }
LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() + LOG.info("Flushing " + + storesToFlush.size() + "/" + stores.size() +
@ -4836,7 +4836,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
private MemstoreSize doDropStoreMemstoreContentsForSeqId(HStore s, long currentSeqId) private MemstoreSize doDropStoreMemstoreContentsForSeqId(HStore s, long currentSeqId)
throws IOException { throws IOException {
MemstoreSize flushableSize = s.getSizeToFlush(); MemstoreSize flushableSize = s.getFlushableSize();
this.decrMemstoreSize(flushableSize); this.decrMemstoreSize(flushableSize);
StoreFlushContext ctx = s.createFlushContext(currentSeqId); StoreFlushContext ctx = s.createFlushContext(currentSeqId);
ctx.prepare(); ctx.prepare();
@ -4933,7 +4933,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
continue; continue;
} }
long storeSeqId = store.getMaxSequenceId(); long storeSeqId = store.getMaxSequenceId().orElse(0L);
List<String> storeFiles = storeDescriptor.getStoreFileList(); List<String> storeFiles = storeDescriptor.getStoreFileList();
try { try {
store.refreshStoreFiles(storeFiles); // replace the files with the new ones store.refreshStoreFiles(storeFiles); // replace the files with the new ones
@ -4943,7 +4943,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
+ " doesn't exist any more. Skip loading the file(s)", ex); + " doesn't exist any more. Skip loading the file(s)", ex);
continue; continue;
} }
if (store.getMaxSequenceId() != storeSeqId) { if (store.getMaxSequenceId().orElse(0L) != storeSeqId) {
// Record latest flush time if we picked up new files // Record latest flush time if we picked up new files
lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime()); lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
} }
@ -4954,7 +4954,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ? StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
null : this.prepareFlushResult.storeFlushCtxs.get(family); null : this.prepareFlushResult.storeFlushCtxs.get(family);
if (ctx != null) { if (ctx != null) {
MemstoreSize snapshotSize = store.getSizeToFlush(); MemstoreSize snapshotSize = store.getFlushableSize();
ctx.abort(); ctx.abort();
this.decrMemstoreSize(snapshotSize); this.decrMemstoreSize(snapshotSize);
this.prepareFlushResult.storeFlushCtxs.remove(family); this.prepareFlushResult.storeFlushCtxs.remove(family);
@ -5085,7 +5085,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (store == null) { if (store == null) {
continue; continue;
} }
if (store.getSizeOfSnapshot().getDataSize() > 0) { if (store.getSnapshotSize().getDataSize() > 0) {
canDrop = false; canDrop = false;
break; break;
} }
@ -5129,12 +5129,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
for (HStore store : stores.values()) { for (HStore store : stores.values()) {
// TODO: some stores might see new data from flush, while others do not which // TODO: some stores might see new data from flush, while others do not which
// MIGHT break atomic edits across column families. // MIGHT break atomic edits across column families.
long maxSeqIdBefore = store.getMaxSequenceId(); long maxSeqIdBefore = store.getMaxSequenceId().orElse(0L);
// refresh the store files. This is similar to observing a region open wal marker. // refresh the store files. This is similar to observing a region open wal marker.
store.refreshStoreFiles(); store.refreshStoreFiles();
long storeSeqId = store.getMaxSequenceId(); long storeSeqId = store.getMaxSequenceId().orElse(0L);
if (storeSeqId < smallestSeqIdInStores) { if (storeSeqId < smallestSeqIdInStores) {
smallestSeqIdInStores = storeSeqId; smallestSeqIdInStores = storeSeqId;
} }
@ -5148,7 +5148,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
null : this.prepareFlushResult.storeFlushCtxs.get( null : this.prepareFlushResult.storeFlushCtxs.get(
store.getColumnFamilyDescriptor().getName()); store.getColumnFamilyDescriptor().getName());
if (ctx != null) { if (ctx != null) {
MemstoreSize snapshotSize = store.getSizeToFlush(); MemstoreSize snapshotSize = store.getFlushableSize();
ctx.abort(); ctx.abort();
this.decrMemstoreSize(snapshotSize); this.decrMemstoreSize(snapshotSize);
this.prepareFlushResult.storeFlushCtxs.remove( this.prepareFlushResult.storeFlushCtxs.remove(
@ -5169,7 +5169,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// advance the mvcc read point so that the new flushed files are visible. // advance the mvcc read point so that the new flushed files are visible.
// either greater than flush seq number or they were already picked up via flush. // either greater than flush seq number or they were already picked up via flush.
for (HStore s : stores.values()) { for (HStore s : stores.values()) {
mvcc.advanceTo(s.getMaxMemstoreTS()); mvcc.advanceTo(s.getMaxMemstoreTS().orElse(0L));
} }
@ -8074,7 +8074,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
for (HStore s : stores.values()) { for (HStore s : stores.values()) {
buf.append(s.getColumnFamilyDescriptor().getNameAsString()); buf.append(s.getColumnFamilyDescriptor().getNameAsString());
buf.append(" size: "); buf.append(" size: ");
buf.append(s.getSizeOfMemStore().getDataSize()); buf.append(s.getMemStoreSize().getDataSize());
buf.append(" "); buf.append(" ");
} }
buf.append("end-of-stores"); buf.append("end-of-stores");

View File

@ -1787,7 +1787,7 @@ public class HRegionServer extends HasThread implements
// Queue a compaction. Will recognize if major is needed. // Queue a compaction. Will recognize if major is needed.
this.instance.compactSplitThread.requestSystemCompaction(hr, s, this.instance.compactSplitThread.requestSystemCompaction(hr, s,
getName() + " requests compaction"); getName() + " requests compaction");
} else if (s.isMajorCompaction()) { } else if (s.shouldPerformMajorCompaction()) {
s.triggerMajorCompaction(); s.triggerMajorCompaction();
if (majorCompactPriority == DEFAULT_PRIORITY || if (majorCompactPriority == DEFAULT_PRIORITY ||
majorCompactPriority > hr.getCompactPriority()) { majorCompactPriority > hr.getCompactPriority()) {

View File

@ -32,6 +32,7 @@ import java.util.Map;
import java.util.NavigableSet; import java.util.NavigableSet;
import java.util.Optional; import java.util.Optional;
import java.util.OptionalDouble; import java.util.OptionalDouble;
import java.util.OptionalLong;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.CompletionService; import java.util.concurrent.CompletionService;
@ -65,6 +66,8 @@ import org.apache.hadoop.hbase.backup.FailedArchiveException;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.conf.ConfigurationManager; import org.apache.hadoop.hbase.conf.ConfigurationManager;
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.crypto.Encryption; import org.apache.hadoop.hbase.io.crypto.Encryption;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@ -120,7 +123,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDes
* not be called directly but by an HRegion manager. * not be called directly but by an HRegion manager.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class HStore implements Store { public class HStore implements Store, HeapSize, StoreConfigInformation, PropagatingConfigurationObserver {
public static final String MEMSTORE_CLASS_NAME = "hbase.regionserver.memstore.class"; public static final String MEMSTORE_CLASS_NAME = "hbase.regionserver.memstore.class";
public static final String COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY = public static final String COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY =
"hbase.server.compactchecker.interval.multiplier"; "hbase.server.compactchecker.interval.multiplier";
@ -396,26 +399,12 @@ public class HStore implements Store {
} }
@Override @Override
@Deprecated public MemstoreSize getFlushableSize() {
public long getFlushableSize() {
MemstoreSize size = getSizeToFlush();
return size.getHeapSize();
}
@Override
public MemstoreSize getSizeToFlush() {
return this.memstore.getFlushableSize(); return this.memstore.getFlushableSize();
} }
@Override @Override
@Deprecated public MemstoreSize getSnapshotSize() {
public long getSnapshotSize() {
MemstoreSize size = getSizeOfSnapshot();
return size.getHeapSize();
}
@Override
public MemstoreSize getSizeOfSnapshot() {
return this.memstore.getSnapshotSize(); return this.memstore.getSnapshotSize();
} }
@ -466,16 +455,13 @@ public class HStore implements Store {
return this.family; return this.family;
} }
/**
* @return The maximum sequence id in all store files. Used for log replay.
*/
@Override @Override
public long getMaxSequenceId() { public OptionalLong getMaxSequenceId() {
return StoreUtils.getMaxSequenceIdInList(this.getStorefiles()); return StoreUtils.getMaxSequenceIdInList(this.getStorefiles());
} }
@Override @Override
public long getMaxMemstoreTS() { public OptionalLong getMaxMemstoreTS() {
return StoreUtils.getMaxMemstoreTSInList(this.getStorefiles()); return StoreUtils.getMaxMemstoreTSInList(this.getStorefiles());
} }
@ -503,7 +489,9 @@ public class HStore implements Store {
return new Path(tabledir, new Path(encodedName, Bytes.toString(family))); return new Path(tabledir, new Path(encodedName, Bytes.toString(family)));
} }
@Override /**
* @return the data block encoder
*/
public HFileDataBlockEncoder getDataBlockEncoder() { public HFileDataBlockEncoder getDataBlockEncoder() {
return dataBlockEncoder; return dataBlockEncoder;
} }
@ -584,20 +572,17 @@ public class HStore implements Store {
return results; return results;
} }
/**
* Checks the underlying store files, and opens the files that have not
* been opened, and removes the store file readers for store files no longer
* available. Mainly used by secondary region replicas to keep up to date with
* the primary region files.
* @throws IOException
*/
@Override @Override
public void refreshStoreFiles() throws IOException { public void refreshStoreFiles() throws IOException {
Collection<StoreFileInfo> newFiles = fs.getStoreFiles(getColumnFamilyName()); Collection<StoreFileInfo> newFiles = fs.getStoreFiles(getColumnFamilyName());
refreshStoreFilesInternal(newFiles); refreshStoreFilesInternal(newFiles);
} }
@Override /**
* Replaces the store files that the store has with the given files. Mainly used by secondary
* region replicas to keep up to date with the primary region files.
* @throws IOException
*/
public void refreshStoreFiles(Collection<String> newFiles) throws IOException { public void refreshStoreFiles(Collection<String> newFiles) throws IOException {
List<StoreFileInfo> storeFiles = new ArrayList<>(newFiles.size()); List<StoreFileInfo> storeFiles = new ArrayList<>(newFiles.size());
for (String file : newFiles) { for (String file : newFiles) {
@ -658,7 +643,8 @@ public class HStore implements Store {
// readers might pick it up. This assumes that the store is not getting any writes (otherwise // readers might pick it up. This assumes that the store is not getting any writes (otherwise
// in-flight transactions might be made visible) // in-flight transactions might be made visible)
if (!toBeAddedFiles.isEmpty()) { if (!toBeAddedFiles.isEmpty()) {
region.getMVCC().advanceTo(this.getMaxSequenceId()); // we must have the max sequence id here as we do have several store files
region.getMVCC().advanceTo(this.getMaxSequenceId().getAsLong());
} }
completeCompaction(toBeRemovedStoreFiles); completeCompaction(toBeRemovedStoreFiles);
@ -878,7 +864,12 @@ public class HStore implements Store {
} }
} }
@Override /**
* Close all the readers We don't need to worry about subsequent requests because the Region holds
* a write lock that will prevent any more reads or writes.
* @return the {@link StoreFile StoreFiles} that were previously being used.
* @throws IOException on failure
*/
public ImmutableCollection<HStoreFile> close() throws IOException { public ImmutableCollection<HStoreFile> close() throws IOException {
this.archiveLock.lock(); this.archiveLock.lock();
this.lock.writeLock().lock(); this.lock.writeLock().lock();
@ -1035,13 +1026,6 @@ public class HStore implements Store {
return sf; return sf;
} }
@Override
public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag) throws IOException {
return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint,
includesTag, false);
}
/** /**
* @param maxKeyCount * @param maxKeyCount
* @param compression Compression algorithm to use * @param compression Compression algorithm to use
@ -1050,7 +1034,6 @@ public class HStore implements Store {
* @param includesTag - includesTag or not * @param includesTag - includesTag or not
* @return Writer for a new StoreFile in the tmp dir. * @return Writer for a new StoreFile in the tmp dir.
*/ */
@Override
public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag, boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
boolean shouldDropBehind) throws IOException { boolean shouldDropBehind) throws IOException {
@ -1068,10 +1051,9 @@ public class HStore implements Store {
*/ */
// TODO : allow the Writer factory to create Writers of ShipperListener type only in case of // TODO : allow the Writer factory to create Writers of ShipperListener type only in case of
// compaction // compaction
@Override
public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag, boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
boolean shouldDropBehind, final TimeRangeTracker trt) throws IOException { boolean shouldDropBehind, TimeRangeTracker trt) throws IOException {
final CacheConfig writerCacheConf; final CacheConfig writerCacheConf;
if (isCompaction) { if (isCompaction) {
// Don't cache data on write on compactions. // Don't cache data on write on compactions.
@ -1301,12 +1283,16 @@ public class HStore implements Store {
return scanners; return scanners;
} }
@Override /**
* @param o Observer who wants to know about changes in set of Readers
*/
public void addChangedReaderObserver(ChangedReadersObserver o) { public void addChangedReaderObserver(ChangedReadersObserver o) {
this.changedReaderObservers.add(o); this.changedReaderObservers.add(o);
} }
@Override /**
* @param o Observer no longer interested in changes in set of Readers.
*/
public void deleteChangedReaderObserver(ChangedReadersObserver o) { public void deleteChangedReaderObserver(ChangedReadersObserver o) {
// We don't check if observer present; it may not be (legitimately) // We don't check if observer present; it may not be (legitimately)
this.changedReaderObservers.remove(o); this.changedReaderObservers.remove(o);
@ -1359,13 +1345,6 @@ public class HStore implements Store {
* @throws IOException * @throws IOException
* @return Storefile we compacted into or null if we failed or opted out early. * @return Storefile we compacted into or null if we failed or opted out early.
*/ */
@Override
public List<HStoreFile> compact(CompactionContext compaction,
ThroughputController throughputController) throws IOException {
return compact(compaction, throughputController, null);
}
@Override
public List<HStoreFile> compact(CompactionContext compaction, public List<HStoreFile> compact(CompactionContext compaction,
ThroughputController throughputController, User user) throws IOException { ThroughputController throughputController, User user) throws IOException {
assert compaction != null; assert compaction != null;
@ -1669,7 +1648,7 @@ public class HStore implements Store {
} }
@Override @Override
public boolean isMajorCompaction() throws IOException { public boolean shouldPerformMajorCompaction() throws IOException {
for (HStoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) { for (HStoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) {
// TODO: what are these reader checks all over the place? // TODO: what are these reader checks all over the place?
if (sf.getReader() == null) { if (sf.getReader() == null) {
@ -1681,7 +1660,10 @@ public class HStore implements Store {
this.storeEngine.getStoreFileManager().getStorefiles()); this.storeEngine.getStoreFileManager().getStorefiles());
} }
@Override public Optional<CompactionContext> requestCompaction() throws IOException {
return requestCompaction(NO_PRIORITY, CompactionLifeCycleTracker.DUMMY, null);
}
public Optional<CompactionContext> requestCompaction(int priority, public Optional<CompactionContext> requestCompaction(int priority,
CompactionLifeCycleTracker tracker, User user) throws IOException { CompactionLifeCycleTracker tracker, User user) throws IOException {
// don't even select for compaction if writes are disabled // don't even select for compaction if writes are disabled
@ -1804,7 +1786,6 @@ public class HStore implements Store {
+ "; total size for store is " + TraditionalBinaryPrefix.long2String(storeSize, "", 1)); + "; total size for store is " + TraditionalBinaryPrefix.long2String(storeSize, "", 1));
} }
@Override
public void cancelRequestedCompaction(CompactionContext compaction) { public void cancelRequestedCompaction(CompactionContext compaction) {
finishCompactionRequest(compaction.getRequest()); finishCompactionRequest(compaction.getRequest());
} }
@ -1899,7 +1880,9 @@ public class HStore implements Store {
} }
} }
@Override /**
* Determines if Store should be split.
*/
public Optional<byte[]> getSplitPoint() { public Optional<byte[]> getSplitPoint() {
this.lock.readLock().lock(); this.lock.readLock().lock();
try { try {
@ -1931,7 +1914,6 @@ public class HStore implements Store {
return storeSize; return storeSize;
} }
@Override
public void triggerMajorCompaction() { public void triggerMajorCompaction() {
this.forceMajor = true; this.forceMajor = true;
} }
@ -1941,7 +1923,14 @@ public class HStore implements Store {
// File administration // File administration
////////////////////////////////////////////////////////////////////////////// //////////////////////////////////////////////////////////////////////////////
@Override /**
* Return a scanner for both the memstore and the HStore files. Assumes we are not in a
* compaction.
* @param scan Scan to apply when scanning the stores
* @param targetCols columns to scan
* @return a scanner over the current key values
* @throws IOException on failure
*/
public KeyValueScanner getScanner(Scan scan, public KeyValueScanner getScanner(Scan scan,
final NavigableSet<byte []> targetCols, long readPt) throws IOException { final NavigableSet<byte []> targetCols, long readPt) throws IOException {
lock.readLock().lock(); lock.readLock().lock();
@ -2032,7 +2021,7 @@ public class HStore implements Store {
return this.storeEngine.getStoreFileManager().getCompactedFilesCount(); return this.storeEngine.getStoreFileManager().getCompactedFilesCount();
} }
private LongStream getStoreFileCreatedTimestampStream() { private LongStream getStoreFileAgeStream() {
return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> { return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> {
if (sf.getReader() == null) { if (sf.getReader() == null) {
LOG.warn("StoreFile " + sf + " has a null Reader"); LOG.warn("StoreFile " + sf + " has a null Reader");
@ -2040,25 +2029,23 @@ public class HStore implements Store {
} else { } else {
return true; return true;
} }
}).filter(HStoreFile::isHFile).mapToLong(sf -> sf.getFileInfo().getCreatedTimestamp()); }).filter(HStoreFile::isHFile).mapToLong(sf -> sf.getFileInfo().getCreatedTimestamp())
.map(t -> EnvironmentEdgeManager.currentTime() - t);
} }
@Override @Override
public long getMaxStoreFileAge() { public OptionalLong getMaxStoreFileAge() {
return EnvironmentEdgeManager.currentTime() - return getStoreFileAgeStream().max();
getStoreFileCreatedTimestampStream().min().orElse(Long.MAX_VALUE);
} }
@Override @Override
public long getMinStoreFileAge() { public OptionalLong getMinStoreFileAge() {
return EnvironmentEdgeManager.currentTime() - return getStoreFileAgeStream().min();
getStoreFileCreatedTimestampStream().max().orElse(0L);
} }
@Override @Override
public long getAvgStoreFileAge() { public OptionalDouble getAvgStoreFileAge() {
OptionalDouble avg = getStoreFileCreatedTimestampStream().average(); return getStoreFileAgeStream().average();
return avg.isPresent() ? EnvironmentEdgeManager.currentTime() - (long) avg.getAsDouble() : 0L;
} }
@Override @Override
@ -2128,14 +2115,7 @@ public class HStore implements Store {
} }
@Override @Override
@Deprecated public MemstoreSize getMemStoreSize() {
public long getMemStoreSize() {
MemstoreSize size = getSizeOfMemStore();
return size.getHeapSize();
}
@Override
public MemstoreSize getSizeOfMemStore() {
return this.memstore.size(); return this.memstore.size();
} }
@ -2148,7 +2128,6 @@ public class HStore implements Store {
return priority; return priority;
} }
@Override
public boolean throttleCompaction(long compactionSize) { public boolean throttleCompaction(long compactionSize) {
return storeEngine.getCompactionPolicy().throttleCompaction(compactionSize); return storeEngine.getCompactionPolicy().throttleCompaction(compactionSize);
} }
@ -2200,7 +2179,6 @@ public class HStore implements Store {
} }
} }
@Override
public StoreFlushContext createFlushContext(long cacheFlushId) { public StoreFlushContext createFlushContext(long cacheFlushId) {
return new StoreFlusherImpl(cacheFlushId); return new StoreFlusherImpl(cacheFlushId);
} }
@ -2344,7 +2322,11 @@ public class HStore implements Store {
return this.storeEngine.needsCompaction(this.filesCompacting); return this.storeEngine.needsCompaction(this.filesCompacting);
} }
@Override /**
* Used for tests.
* @return cache configuration for this Store.
*/
@VisibleForTesting
public CacheConfig getCacheConfig() { public CacheConfig getCacheConfig() {
return this.cacheConf; return this.cacheConf;
} }
@ -2370,7 +2352,6 @@ public class HStore implements Store {
return comparator; return comparator;
} }
@Override
public ScanInfo getScanInfo() { public ScanInfo getScanInfo() {
return scanInfo; return scanInfo;
} }
@ -2490,7 +2471,9 @@ public class HStore implements Store {
archiveLock.unlock(); archiveLock.unlock();
} }
@Override /**
* Closes and archives the compacted files under this store
*/
public synchronized void closeAndArchiveCompactedFiles() throws IOException { public synchronized void closeAndArchiveCompactedFiles() throws IOException {
// ensure other threads do not attempt to archive the same files on close() // ensure other threads do not attempt to archive the same files on close()
archiveLock.lock(); archiveLock.lock();

View File

@ -76,7 +76,7 @@ public class IncreasingToUpperBoundRegionSplitPolicy extends ConstantSizeRegionS
// Get size to check // Get size to check
long sizeToCheck = getSizeToCheck(tableRegionsCount); long sizeToCheck = getSizeToCheck(tableRegionsCount);
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
// If any of the stores is unable to split (eg they contain reference files) // If any of the stores is unable to split (eg they contain reference files)
// then don't split // then don't split
if (!store.canSplit()) { if (!store.canSplit()) {

View File

@ -95,7 +95,7 @@ public interface KeyValueScanner extends Shipper, Closeable {
* this query, based on TTL * this query, based on TTL
* @return true if the scanner should be included in the query * @return true if the scanner should be included in the query
*/ */
boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS); boolean shouldUseScanner(Scan scan, HStore store, long oldestUnexpiredTS);
// "Lazy scanner" optimizations // "Lazy scanner" optimizations

View File

@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException; import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.OptionalDouble;
import java.util.OptionalLong;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -31,7 +33,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CacheStats; import org.apache.hadoop.hbase.io.hfile.CacheStats;
@ -44,6 +45,7 @@ import org.apache.hadoop.hbase.wal.WALProvider;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.hdfs.DFSHedgedReadMetrics; import org.apache.hadoop.hdfs.DFSHedgedReadMetrics;
import org.apache.hadoop.metrics2.MetricsExecutor; import org.apache.hadoop.metrics2.MetricsExecutor;
import org.apache.yetus.audience.InterfaceAudience;
/** /**
* Impl for exposing HRegionServer Information through Hadoop's metrics 2 system. * Impl for exposing HRegionServer Information through Hadoop's metrics 2 system.
@ -765,22 +767,30 @@ class MetricsRegionServerWrapperImpl
tempNumStores += storeList.size(); tempNumStores += storeList.size();
for (Store store : storeList) { for (Store store : storeList) {
tempNumStoreFiles += store.getStorefilesCount(); tempNumStoreFiles += store.getStorefilesCount();
tempMemstoreSize += store.getSizeOfMemStore().getDataSize(); tempMemstoreSize += store.getMemStoreSize().getDataSize();
tempStoreFileSize += store.getStorefilesSize(); tempStoreFileSize += store.getStorefilesSize();
long storeMaxStoreFileAge = store.getMaxStoreFileAge(); OptionalLong storeMaxStoreFileAge = store.getMaxStoreFileAge();
tempMaxStoreFileAge = (storeMaxStoreFileAge > tempMaxStoreFileAge) ? if (storeMaxStoreFileAge.isPresent() &&
storeMaxStoreFileAge : tempMaxStoreFileAge; storeMaxStoreFileAge.getAsLong() > tempMaxStoreFileAge) {
tempMaxStoreFileAge = storeMaxStoreFileAge.getAsLong();
}
long storeMinStoreFileAge = store.getMinStoreFileAge(); OptionalLong storeMinStoreFileAge = store.getMinStoreFileAge();
tempMinStoreFileAge = (storeMinStoreFileAge < tempMinStoreFileAge) ? if (storeMinStoreFileAge.isPresent() &&
storeMinStoreFileAge : tempMinStoreFileAge; storeMinStoreFileAge.getAsLong() < tempMinStoreFileAge) {
tempMinStoreFileAge = storeMinStoreFileAge.getAsLong();
}
long storeHFiles = store.getNumHFiles(); long storeHFiles = store.getNumHFiles();
avgAgeNumerator += store.getAvgStoreFileAge() * storeHFiles;
numHFiles += storeHFiles; numHFiles += storeHFiles;
tempNumReferenceFiles += store.getNumReferenceFiles(); tempNumReferenceFiles += store.getNumReferenceFiles();
OptionalDouble storeAvgStoreFileAge = store.getAvgStoreFileAge();
if (storeAvgStoreFileAge.isPresent()) {
avgAgeNumerator += storeAvgStoreFileAge.getAsDouble() * storeHFiles;
}
tempStorefileIndexSize += store.getStorefilesIndexSize(); tempStorefileIndexSize += store.getStorefilesIndexSize();
tempTotalStaticBloomSize += store.getTotalStaticBloomSize(); tempTotalStaticBloomSize += store.getTotalStaticBloomSize();
tempTotalStaticIndexSize += store.getTotalStaticIndexSize(); tempTotalStaticIndexSize += store.getTotalStaticIndexSize();

View File

@ -21,18 +21,20 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Map; import java.util.Map;
import java.util.OptionalDouble;
import java.util.OptionalLong;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory; import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.metrics2.MetricsExecutor; import org.apache.hadoop.metrics2.MetricsExecutor;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private @InterfaceAudience.Private
public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable { public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable {
@ -227,21 +229,28 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
if (region.stores != null) { if (region.stores != null) {
for (Store store : region.stores.values()) { for (Store store : region.stores.values()) {
tempNumStoreFiles += store.getStorefilesCount(); tempNumStoreFiles += store.getStorefilesCount();
tempMemstoreSize += store.getSizeOfMemStore().getDataSize(); tempMemstoreSize += store.getMemStoreSize().getDataSize();
tempStoreFileSize += store.getStorefilesSize(); tempStoreFileSize += store.getStorefilesSize();
OptionalLong storeMaxStoreFileAge = store.getMaxStoreFileAge();
if (storeMaxStoreFileAge.isPresent() &&
storeMaxStoreFileAge.getAsLong() > tempMaxStoreFileAge) {
tempMaxStoreFileAge = storeMaxStoreFileAge.getAsLong();
}
long storeMaxStoreFileAge = store.getMaxStoreFileAge(); OptionalLong storeMinStoreFileAge = store.getMinStoreFileAge();
tempMaxStoreFileAge = (storeMaxStoreFileAge > tempMaxStoreFileAge) ? if (storeMinStoreFileAge.isPresent() &&
storeMaxStoreFileAge : tempMaxStoreFileAge; storeMinStoreFileAge.getAsLong() < tempMinStoreFileAge) {
tempMinStoreFileAge = storeMinStoreFileAge.getAsLong();
long storeMinStoreFileAge = store.getMinStoreFileAge(); }
tempMinStoreFileAge = (storeMinStoreFileAge < tempMinStoreFileAge) ?
storeMinStoreFileAge : tempMinStoreFileAge;
long storeHFiles = store.getNumHFiles(); long storeHFiles = store.getNumHFiles();
avgAgeNumerator += store.getAvgStoreFileAge() * storeHFiles;
numHFiles += storeHFiles; numHFiles += storeHFiles;
tempNumReferenceFiles += store.getNumReferenceFiles(); tempNumReferenceFiles += store.getNumReferenceFiles();
OptionalDouble storeAvgStoreFileAge = store.getAvgStoreFileAge();
if (storeAvgStoreFileAge.isPresent()) {
avgAgeNumerator += storeAvgStoreFileAge.getAsDouble() * storeHFiles;
}
} }
} }

View File

@ -56,7 +56,7 @@ public abstract class NonLazyKeyValueScanner implements KeyValueScanner {
} }
@Override @Override
public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) { public boolean shouldUseScanner(Scan scan, HStore store, long oldestUnexpiredTS) {
// No optimizations implemented by default. // No optimizations implemented by default.
return true; return true;
} }

View File

@ -749,7 +749,7 @@ public interface Region extends ConfigurationObserver {
* Trigger major compaction on all stores in the region. * Trigger major compaction on all stores in the region.
* <p> * <p>
* Compaction will be performed asynchronously to this call by the RegionServer's * Compaction will be performed asynchronously to this call by the RegionServer's
* CompactSplitThread. See also {@link Store#triggerMajorCompaction()} * CompactSplitThread.
* @throws IOException * @throws IOException
*/ */
void triggerMajorCompaction() throws IOException; void triggerMajorCompaction() throws IOException;

View File

@ -279,7 +279,7 @@ public class SegmentScanner implements KeyValueScanner {
* overridden method * overridden method
*/ */
@Override @Override
public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) { public boolean shouldUseScanner(Scan scan, HStore store, long oldestUnexpiredTS) {
return getSegment().shouldSeek(scan.getColumnFamilyTimeRange() return getSegment().shouldSeek(scan.getColumnFamilyTimeRange()
.getOrDefault(store.getColumnFamilyDescriptor().getName(), scan.getTimeRange()), oldestUnexpiredTS); .getOrDefault(store.getColumnFamilyDescriptor().getName(), scan.getTimeRange()), oldestUnexpiredTS);
} }

View File

@ -19,27 +19,17 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException; import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.Comparator;
import java.util.NavigableSet; import java.util.OptionalDouble;
import java.util.Optional; import java.util.OptionalLong;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
import org.apache.hadoop.hbase.io.HeapSize;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability; import org.apache.yetus.audience.InterfaceStability;
@ -49,7 +39,7 @@ import org.apache.yetus.audience.InterfaceStability;
*/ */
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving @InterfaceStability.Evolving
public interface Store extends HeapSize, StoreConfigInformation, PropagatingConfigurationObserver { public interface Store {
/** /**
* The default priority for user-specified compaction requests. * The default priority for user-specified compaction requests.
@ -59,33 +49,12 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
int NO_PRIORITY = Integer.MIN_VALUE; int NO_PRIORITY = Integer.MIN_VALUE;
// General Accessors // General Accessors
CellComparator getComparator(); Comparator<Cell> getComparator();
Collection<? extends StoreFile> getStorefiles(); Collection<? extends StoreFile> getStorefiles();
Collection<? extends StoreFile> getCompactedFiles(); Collection<? extends StoreFile> getCompactedFiles();
/**
* Close all the readers We don't need to worry about subsequent requests because the Region
* holds a write lock that will prevent any more reads or writes.
* @return the {@link StoreFile StoreFiles} that were previously being used.
* @throws IOException on failure
*/
Collection<? extends StoreFile> close() throws IOException;
/**
* Return a scanner for both the memstore and the HStore files. Assumes we are not in a
* compaction.
* @param scan Scan to apply when scanning the stores
* @param targetCols columns to scan
* @return a scanner over the current key values
* @throws IOException on failure
*/
KeyValueScanner getScanner(Scan scan, final NavigableSet<byte[]> targetCols, long readPt)
throws IOException;
ScanInfo getScanInfo();
/** /**
* When was the last edit done in the memstore * When was the last edit done in the memstore
*/ */
@ -93,186 +62,66 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
FileSystem getFileSystem(); FileSystem getFileSystem();
/**
* @param maxKeyCount
* @param compression Compression algorithm to use
* @param isCompaction whether we are creating a new file in a compaction
* @param includeMVCCReadpoint whether we should out the MVCC readpoint
* @return Writer for a new StoreFile in the tmp dir.
*/
StoreFileWriter createWriterInTmp(
long maxKeyCount,
Compression.Algorithm compression,
boolean isCompaction,
boolean includeMVCCReadpoint,
boolean includesTags
) throws IOException;
/**
* @param maxKeyCount
* @param compression Compression algorithm to use
* @param isCompaction whether we are creating a new file in a compaction
* @param includeMVCCReadpoint whether we should out the MVCC readpoint
* @param shouldDropBehind should the writer drop caches behind writes
* @return Writer for a new StoreFile in the tmp dir.
*/
StoreFileWriter createWriterInTmp(
long maxKeyCount,
Compression.Algorithm compression,
boolean isCompaction,
boolean includeMVCCReadpoint,
boolean includesTags,
boolean shouldDropBehind
) throws IOException;
/**
* @param maxKeyCount
* @param compression Compression algorithm to use
* @param isCompaction whether we are creating a new file in a compaction
* @param includeMVCCReadpoint whether we should out the MVCC readpoint
* @param shouldDropBehind should the writer drop caches behind writes
* @param trt Ready-made timetracker to use.
* @return Writer for a new StoreFile in the tmp dir.
*/
StoreFileWriter createWriterInTmp(
long maxKeyCount,
Compression.Algorithm compression,
boolean isCompaction,
boolean includeMVCCReadpoint,
boolean includesTags,
boolean shouldDropBehind,
final TimeRangeTracker trt
) throws IOException;
// Compaction oriented methods
boolean throttleCompaction(long compactionSize);
/** /**
* getter for CompactionProgress object * getter for CompactionProgress object
* @return CompactionProgress object; can be null * @return CompactionProgress object; can be null
*/ */
CompactionProgress getCompactionProgress(); CompactionProgress getCompactionProgress();
default Optional<CompactionContext> requestCompaction() throws IOException {
return requestCompaction(NO_PRIORITY, CompactionLifeCycleTracker.DUMMY, null);
}
Optional<CompactionContext> requestCompaction(int priority, CompactionLifeCycleTracker tracker,
User user) throws IOException;
void cancelRequestedCompaction(CompactionContext compaction);
/**
* @deprecated see compact(CompactionContext, ThroughputController, User)
*/
@Deprecated
List<? extends StoreFile> compact(CompactionContext compaction,
ThroughputController throughputController) throws IOException;
List<? extends StoreFile> compact(CompactionContext compaction,
ThroughputController throughputController, User user) throws IOException;
/** /**
* Tests whether we should run a major compaction. For example, if the configured major compaction
* interval is reached.
* @return true if we should run a major compaction. * @return true if we should run a major compaction.
*/ */
boolean isMajorCompaction() throws IOException; boolean shouldPerformMajorCompaction() throws IOException;
void triggerMajorCompaction();
/** /**
* See if there's too much store files in this store * See if there's too much store files in this store
* @return true if number of store files is greater than the number defined in minFilesToCompact * @return <code>true</code> if number of store files is greater than the number defined in
* minFilesToCompact
*/ */
boolean needsCompaction(); boolean needsCompaction();
int getCompactPriority(); int getCompactPriority();
StoreFlushContext createFlushContext(long cacheFlushId); /**
* Returns whether this store is splittable, i.e., no reference file in this store.
// Split oriented methods */
boolean canSplit(); boolean canSplit();
/** /**
* Determines if Store should be split. * @return <code>true</code> if the store has any underlying reference files to older HFiles
*/
Optional<byte[]> getSplitPoint();
// General accessors into the state of the store
// TODO abstract some of this out into a metrics class
/**
* @return <tt>true</tt> if the store has any underlying reference files to older HFiles
*/ */
boolean hasReferences(); boolean hasReferences();
/**
* @return The size of this store's memstore, in bytes
* @deprecated Since 2.0 and will be removed in 3.0. Use {@link #getSizeOfMemStore()} instead.
* <p>
* Note: When using off heap MSLAB feature, this will not account the cell data bytes size which
* is in off heap MSLAB area.
*/
@Deprecated
long getMemStoreSize();
/** /**
* @return The size of this store's memstore. * @return The size of this store's memstore.
*/ */
MemstoreSize getSizeOfMemStore(); MemstoreSize getMemStoreSize();
/** /**
* @return The amount of memory we could flush from this memstore; usually this is equal to * @return The amount of memory we could flush from this memstore; usually this is equal to
* {@link #getMemStoreSize()} unless we are carrying snapshots and then it will be the size of * {@link #getMemStoreSize()} unless we are carrying snapshots and then it will be the size of
* outstanding snapshots. * outstanding snapshots.
* @deprecated Since 2.0 and will be removed in 3.0. Use {@link #getSizeToFlush()} instead.
* <p>
* Note: When using off heap MSLAB feature, this will not account the cell data bytes size which
* is in off heap MSLAB area.
*/ */
@Deprecated MemstoreSize getFlushableSize();
long getFlushableSize();
/**
* @return The amount of memory we could flush from this memstore; usually this is equal to
* {@link #getSizeOfMemStore()} unless we are carrying snapshots and then it will be the size of
* outstanding snapshots.
*/
MemstoreSize getSizeToFlush();
/**
* Returns the memstore snapshot size
* @return size of the memstore snapshot
* @deprecated Since 2.0 and will be removed in 3.0. Use {@link #getSizeOfSnapshot()} instead.
* <p>
* Note: When using off heap MSLAB feature, this will not account the cell data bytes size which
* is in off heap MSLAB area.
*/
@Deprecated
long getSnapshotSize();
/** /**
* @return size of the memstore snapshot * @return size of the memstore snapshot
*/ */
MemstoreSize getSizeOfSnapshot(); MemstoreSize getSnapshotSize();
ColumnFamilyDescriptor getColumnFamilyDescriptor(); ColumnFamilyDescriptor getColumnFamilyDescriptor();
/** /**
* @return The maximum sequence id in all store files. * @return The maximum sequence id in all store files.
*/ */
long getMaxSequenceId(); OptionalLong getMaxSequenceId();
/** /**
* @return The maximum memstoreTS in all store files. * @return The maximum memstoreTS in all store files.
*/ */
long getMaxMemstoreTS(); OptionalLong getMaxMemstoreTS();
/**
* @return the data block encoder
*/
HFileDataBlockEncoder getDataBlockEncoder();
/** @return aggregate size of all HStores used in the last compaction */ /** @return aggregate size of all HStores used in the last compaction */
long getLastCompactSize(); long getLastCompactSize();
@ -293,17 +142,17 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
/** /**
* @return Max age of store files in this store * @return Max age of store files in this store
*/ */
long getMaxStoreFileAge(); OptionalLong getMaxStoreFileAge();
/** /**
* @return Min age of store files in this store * @return Min age of store files in this store
*/ */
long getMinStoreFileAge(); OptionalLong getMinStoreFileAge();
/** /**
* @return Average age of store files in this store, 0 if no store files * @return Average age of store files in this store
*/ */
long getAvgStoreFileAge(); OptionalDouble getAvgStoreFileAge();
/** /**
* @return Number of reference files in this store * @return Number of reference files in this store
@ -350,18 +199,10 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
*/ */
long getTotalStaticBloomSize(); long getTotalStaticBloomSize();
// Test-helper methods
/**
* Used for tests.
* @return cache configuration for this Store.
*/
CacheConfig getCacheConfig();
/** /**
* @return the parent region info hosting this store * @return the parent region info hosting this store
*/ */
HRegionInfo getRegionInfo(); RegionInfo getRegionInfo();
RegionCoprocessorHost getCoprocessorHost(); RegionCoprocessorHost getCoprocessorHost();
@ -413,26 +254,15 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
*/ */
long getMajorCompactedCellsSize(); long getMajorCompactedCellsSize();
/*
* @param o Observer who wants to know about changes in set of Readers
*/
void addChangedReaderObserver(ChangedReadersObserver o);
/*
* @param o Observer no longer interested in changes in set of Readers.
*/
void deleteChangedReaderObserver(ChangedReadersObserver o);
/** /**
* @return Whether this store has too many store files. * @return Whether this store has too many store files.
*/ */
boolean hasTooManyStoreFiles(); boolean hasTooManyStoreFiles();
/** /**
* Checks the underlying store files, and opens the files that have not * Checks the underlying store files, and opens the files that have not been opened, and removes
* been opened, and removes the store file readers for store files no longer * the store file readers for store files no longer available. Mainly used by secondary region
* available. Mainly used by secondary region replicas to keep up to date with * replicas to keep up to date with the primary region files.
* the primary region files.
* @throws IOException * @throws IOException
*/ */
void refreshStoreFiles() throws IOException; void refreshStoreFiles() throws IOException;
@ -454,21 +284,8 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
*/ */
double getCompactionPressure(); double getCompactionPressure();
/**
* Replaces the store files that the store has with the given files. Mainly used by
* secondary region replicas to keep up to date with
* the primary region files.
* @throws IOException
*/
void refreshStoreFiles(Collection<String> newFiles) throws IOException;
boolean isPrimaryReplicaStore(); boolean isPrimaryReplicaStore();
/**
* Closes and archives the compacted files under this store
*/
void closeAndArchiveCompactedFiles() throws IOException;
/** /**
* @return true if the memstore may need some extra memory space * @return true if the memstore may need some extra memory space
*/ */

View File

@ -163,7 +163,7 @@ public interface StoreFileManager {
/** /**
* @return the compaction pressure used for compaction throughput tuning. * @return the compaction pressure used for compaction throughput tuning.
* @see Store#getCompactionPressure() * @see HStore#getCompactionPressure()
*/ */
double getCompactionPressure(); double getCompactionPressure();

View File

@ -474,7 +474,7 @@ public class StoreFileScanner implements KeyValueScanner {
} }
@Override @Override
public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) { public boolean shouldUseScanner(Scan scan, HStore store, long oldestUnexpiredTS) {
// if the file has no entries, no need to validate or create a scanner. // if the file has no entries, no need to validate or create a scanner.
byte[] cf = store.getColumnFamilyDescriptor().getName(); byte[] cf = store.getColumnFamilyDescriptor().getName();
TimeRange timeRange = scan.getColumnFamilyTimeRange().get(cf); TimeRange timeRange = scan.getColumnFamilyTimeRange().get(cf);

View File

@ -418,7 +418,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* Will be overridden by testcase so declared as protected. * Will be overridden by testcase so declared as protected.
*/ */
@VisibleForTesting @VisibleForTesting
protected List<KeyValueScanner> selectScannersFrom(Store store, protected List<KeyValueScanner> selectScannersFrom(HStore store,
List<? extends KeyValueScanner> allScanners) { List<? extends KeyValueScanner> allScanners) {
boolean memOnly; boolean memOnly;
boolean filesOnly; boolean filesOnly;

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.Optional; import java.util.Optional;
import java.util.OptionalInt; import java.util.OptionalInt;
import java.util.OptionalLong;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -79,22 +80,17 @@ public class StoreUtils {
* Return the largest memstoreTS found across all storefiles in the given list. Store files that * Return the largest memstoreTS found across all storefiles in the given list. Store files that
* were created by a mapreduce bulk load are ignored, as they do not correspond to any specific * were created by a mapreduce bulk load are ignored, as they do not correspond to any specific
* put operation, and thus do not have a memstoreTS associated with them. * put operation, and thus do not have a memstoreTS associated with them.
* @return 0 if no non-bulk-load files are provided or, this is Store that does not yet have any
* store files.
*/ */
public static long getMaxMemstoreTSInList(Collection<HStoreFile> sfs) { public static OptionalLong getMaxMemstoreTSInList(Collection<HStoreFile> sfs) {
return sfs.stream().filter(sf -> !sf.isBulkLoadResult()).mapToLong(HStoreFile::getMaxMemstoreTS) return sfs.stream().filter(sf -> !sf.isBulkLoadResult()).mapToLong(HStoreFile::getMaxMemstoreTS)
.max().orElse(0L); .max();
} }
/** /**
* Return the highest sequence ID found across all storefiles in the given list. * Return the highest sequence ID found across all storefiles in the given list.
* @param sfs
* @return 0 if no non-bulk-load files are provided or, this is Store that does not yet have any
* store files.
*/ */
public static long getMaxSequenceIdInList(Collection<HStoreFile> sfs) { public static OptionalLong getMaxSequenceIdInList(Collection<HStoreFile> sfs) {
return sfs.stream().mapToLong(HStoreFile::getMaxSequenceId).max().orElse(0L); return sfs.stream().mapToLong(HStoreFile::getMaxSequenceId).max();
} }
/** /**

View File

@ -18,11 +18,12 @@
*/ */
package org.apache.hadoop.hbase.regionserver.compactions; package org.apache.hadoop.hbase.regionserver.compactions;
import static org.apache.hadoop.hbase.regionserver.Store.NO_PRIORITY;
import java.util.Collection; import java.util.Collection;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
@ -39,7 +40,7 @@ public class CompactionRequest {
private boolean isOffPeak = false; private boolean isOffPeak = false;
private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR } private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR }
private DisplayCompactionType isMajor = DisplayCompactionType.MINOR; private DisplayCompactionType isMajor = DisplayCompactionType.MINOR;
private int priority = Store.NO_PRIORITY; private int priority = NO_PRIORITY;
private Collection<HStoreFile> filesToCompact; private Collection<HStoreFile> filesToCompact;
// CompactRequest object creation time. // CompactRequest object creation time.

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver.compactions;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.OptionalLong;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -47,7 +48,10 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTiered
private boolean needEmptyFile(CompactionRequest request) { private boolean needEmptyFile(CompactionRequest request) {
// if we are going to compact the last N files, then we need to emit an empty file to retain the // if we are going to compact the last N files, then we need to emit an empty file to retain the
// maxSeqId if we haven't written out anything. // maxSeqId if we haven't written out anything.
return StoreUtils.getMaxSequenceIdInList(request.getFiles()) == store.getMaxSequenceId(); OptionalLong maxSeqId = StoreUtils.getMaxSequenceIdInList(request.getFiles());
OptionalLong storeMaxSeqId = store.getMaxSequenceId();
return maxSeqId.isPresent() && storeMaxSeqId.isPresent() &&
maxSeqId.getAsLong() == storeMaxSeqId.getAsLong();
} }
public List<Path> compact(final CompactionRequest request, final List<Long> lowerBoundaries, public List<Path> compact(final CompactionRequest request, final List<Long> lowerBoundaries,

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
* {@value #HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND}, using the formula &quot;lower + * {@value #HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND}, using the formula &quot;lower +
* (higer - lower) * compactionPressure&quot;, where compactionPressure is in range [0.0, 1.0]</li> * (higer - lower) * compactionPressure&quot;, where compactionPressure is in range [0.0, 1.0]</li>
* </ul> * </ul>
* @see org.apache.hadoop.hbase.regionserver.Store#getCompactionPressure() * @see org.apache.hadoop.hbase.regionserver.HStore#getCompactionPressure()
*/ */
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class PressureAwareCompactionThroughputController extends PressureAwareThroughputController { public class PressureAwareCompactionThroughputController extends PressureAwareThroughputController {

View File

@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.regionserver.throttle;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.Store;
/** /**
* Helper methods for throttling * Helper methods for throttling
@ -40,7 +40,7 @@ public final class ThroughputControlUtil {
* @param opName Name of the IO operation, e.g. "flush", "compaction", etc. * @param opName Name of the IO operation, e.g. "flush", "compaction", etc.
* @return The name for throttling * @return The name for throttling
*/ */
public static String getNameForThrottling(final Store store, final String opName) { public static String getNameForThrottling(HStore store, String opName) {
int counter; int counter;
for (;;) { for (;;) {
counter = NAME_COUNTER.get(); counter = NAME_COUNTER.get();
@ -49,8 +49,8 @@ public final class ThroughputControlUtil {
break; break;
} }
} }
return store.getRegionInfo().getRegionNameAsString() + NAME_DELIMITER return store.getRegionInfo().getRegionNameAsString() + NAME_DELIMITER +
+ store.getColumnFamilyDescriptor().getNameAsString() store.getColumnFamilyDescriptor().getNameAsString() + NAME_DELIMITER + opName +
+ NAME_DELIMITER + opName + NAME_DELIMITER + counter; NAME_DELIMITER + counter;
} }
} }

View File

@ -99,7 +99,7 @@
<h3>Column Family: <%= cf %></h2> <h3>Column Family: <%= cf %></h2>
<h4>Memstore size (MB): <%= (int) (store.getMemStoreSize() / 1024 / 1024) %></h3> <h4>Memstore size (MB): <%= (int) (store.getMemStoreSize().getHeapSize() / 1024 / 1024) %></h3>
<h4>Store Files</h3> <h4>Store Files</h3>

View File

@ -145,7 +145,7 @@ public class TestIOFencing {
public int countStoreFiles() { public int countStoreFiles() {
int count = 0; int count = 0;
for (Store store : stores.values()) { for (HStore store : stores.values()) {
count += store.getStorefilesCount(); count += store.getStorefilesCount();
} }
return count; return count;

View File

@ -36,19 +36,20 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ChoreService; import org.apache.hadoop.hbase.ChoreService;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate; import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner; import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger; import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -176,7 +177,7 @@ public class TestZooKeeperTableArchiveClient {
final LongTermArchivingHFileCleaner delegate = (LongTermArchivingHFileCleaner) cleaners.get(0); final LongTermArchivingHFileCleaner delegate = (LongTermArchivingHFileCleaner) cleaners.get(0);
// create the region // create the region
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM); ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM);
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
List<Region> regions = new ArrayList<>(); List<Region> regions = new ArrayList<>();
regions.add(region); regions.add(region);
@ -229,7 +230,7 @@ public class TestZooKeeperTableArchiveClient {
List<BaseHFileCleanerDelegate> cleaners = turnOnArchiving(STRING_TABLE_NAME, cleaner); List<BaseHFileCleanerDelegate> cleaners = turnOnArchiving(STRING_TABLE_NAME, cleaner);
final LongTermArchivingHFileCleaner delegate = (LongTermArchivingHFileCleaner) cleaners.get(0); final LongTermArchivingHFileCleaner delegate = (LongTermArchivingHFileCleaner) cleaners.get(0);
// create the region // create the region
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM); ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM);
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd); HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
List<Region> regions = new ArrayList<>(); List<Region> regions = new ArrayList<>();
regions.add(region); regions.add(region);
@ -239,7 +240,7 @@ public class TestZooKeeperTableArchiveClient {
loadFlushAndCompact(region, TEST_FAM); loadFlushAndCompact(region, TEST_FAM);
compactionCleaner.chore(); compactionCleaner.chore();
// create the another table that we don't archive // create the another table that we don't archive
hcd = new HColumnDescriptor(TEST_FAM); hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM);
HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd); HRegion otherRegion = UTIL.createTestRegion(otherTable, hcd);
regions = new ArrayList<>(); regions = new ArrayList<>();
regions.add(otherRegion); regions.add(otherRegion);
@ -400,12 +401,12 @@ public class TestZooKeeperTableArchiveClient {
return allFiles; return allFiles;
} }
private void loadFlushAndCompact(Region region, byte[] family) throws IOException { private void loadFlushAndCompact(HRegion region, byte[] family) throws IOException {
// create two hfiles in the region // create two hfiles in the region
createHFileInRegion(region, family); createHFileInRegion(region, family);
createHFileInRegion(region, family); createHFileInRegion(region, family);
Store s = region.getStore(family); HStore s = region.getStore(family);
int count = s.getStorefilesCount(); int count = s.getStorefilesCount();
assertTrue("Don't have the expected store files, wanted >= 2 store files, but was:" + count, assertTrue("Don't have the expected store files, wanted >= 2 store files, but was:" + count,
count >= 2); count >= 2);

View File

@ -46,10 +46,10 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CachedBlock; import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.regionserver.ScanInfo;
import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.ScannerContext;
@ -134,9 +134,9 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = HRegion region =
TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
Store store = region.getStores().iterator().next(); HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -313,9 +313,9 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = HRegion region = (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName)
TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); .getRegion(regionName);
Store store = region.getStores().iterator().next(); HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);

View File

@ -53,11 +53,11 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CachedBlock; import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache; import org.apache.hadoop.hbase.io.hfile.CombinedBlockCache;
import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.ScannerContext;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -65,9 +65,9 @@ import org.junit.After;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.Ignore;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName; import org.junit.rules.TestName;
@ -184,8 +184,9 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region = (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName)
Store store = region.getStores().iterator().next(); .getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -274,8 +275,9 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
Store store = region.getStores().iterator().next(); (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -332,8 +334,9 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
Store store = region.getStores().iterator().next(); (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -393,7 +396,8 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
BlockCache cache = setCacheProperties(region); BlockCache cache = setCacheProperties(region);
Put put = new Put(ROW); Put put = new Put(ROW);
put.addColumn(FAMILY, QUALIFIER, data); put.addColumn(FAMILY, QUALIFIER, data);
@ -485,7 +489,8 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
BlockCache cache = setCacheProperties(region); BlockCache cache = setCacheProperties(region);
Put put = new Put(ROW); Put put = new Put(ROW);
@ -568,8 +573,9 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
Store store = region.getStores().iterator().next(); (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
BlockCache cache = cacheConf.getBlockCache(); BlockCache cache = cacheConf.getBlockCache();
@ -626,8 +632,9 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
Store store = region.getStores().iterator().next(); (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -708,7 +715,8 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
BlockCache cache = setCacheProperties(region); BlockCache cache = setCacheProperties(region);
Put put = new Put(ROW); Put put = new Put(ROW);
@ -777,11 +785,11 @@ public class TestBlockEvictionFromClient {
} }
} }
private BlockCache setCacheProperties(Region region) { private BlockCache setCacheProperties(HRegion region) {
Iterator<? extends Store> strItr = region.getStores().iterator(); Iterator<HStore> strItr = region.getStores().iterator();
BlockCache cache = null; BlockCache cache = null;
while (strItr.hasNext()) { while (strItr.hasNext()) {
Store store = strItr.next(); HStore store = strItr.next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -807,8 +815,9 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
Store store = region.getStores().iterator().next(); (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -872,8 +881,9 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
Store store = region.getStores().iterator().next(); (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -989,8 +999,9 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
Store store = region.getStores().iterator().next(); (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -1118,8 +1129,9 @@ public class TestBlockEvictionFromClient {
// get the block cache and region // get the block cache and region
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName); RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region =
Store store = region.getStores().iterator().next(); (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -1446,14 +1458,14 @@ public class TestBlockEvictionFromClient {
} }
} }
private void waitForStoreFileCount(Store store, int count, int timeout) private void waitForStoreFileCount(HStore store, int count, int timeout)
throws InterruptedException { throws InterruptedException {
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
while (start + timeout > System.currentTimeMillis() && store.getStorefilesCount() != count) { while (start + timeout > System.currentTimeMillis() && store.getStorefilesCount() != count) {
Thread.sleep(100); Thread.sleep(100);
} }
System.out.println("start=" + start + ", now=" + System.currentTimeMillis() + ", cur=" System.out.println("start=" + start + ", now=" + System.currentTimeMillis() + ", cur=" +
+ store.getStorefilesCount()); store.getStorefilesCount());
assertEquals(count, store.getStorefilesCount()); assertEquals(count, store.getStorefilesCount());
} }

View File

@ -199,19 +199,19 @@ public class TestFromClientSide {
try { try {
Append append = new Append(ROW); Append append = new Append(ROW);
append.addColumn(TEST_UTIL.fam1, QUALIFIER, VALUE); append.addColumn(HBaseTestingUtility.fam1, QUALIFIER, VALUE);
Result result = table.append(append); Result result = table.append(append);
// Verify expected result // Verify expected result
Cell[] cells = result.rawCells(); Cell[] cells = result.rawCells();
assertEquals(1, cells.length); assertEquals(1, cells.length);
assertKey(cells[0], ROW, TEST_UTIL.fam1, QUALIFIER, VALUE); assertKey(cells[0], ROW, HBaseTestingUtility.fam1, QUALIFIER, VALUE);
// Verify expected result again // Verify expected result again
Result readResult = table.get(new Get(ROW)); Result readResult = table.get(new Get(ROW));
cells = readResult.rawCells(); cells = readResult.rawCells();
assertEquals(1, cells.length); assertEquals(1, cells.length);
assertKey(cells[0], ROW, TEST_UTIL.fam1, QUALIFIER, VALUE); assertKey(cells[0], ROW, HBaseTestingUtility.fam1, QUALIFIER, VALUE);
} finally { } finally {
table.close(); table.close();
connection.close(); connection.close();
@ -568,7 +568,7 @@ public class TestFromClientSide {
} }
@Override @Override
protected List<KeyValueScanner> selectScannersFrom(Store store, protected List<KeyValueScanner> selectScannersFrom(HStore store,
List<? extends KeyValueScanner> allScanners) { List<? extends KeyValueScanner> allScanners) {
List<KeyValueScanner> scanners = super.selectScannersFrom(store, allScanners); List<KeyValueScanner> scanners = super.selectScannersFrom(store, allScanners);
List<KeyValueScanner> newScanners = new ArrayList<>(scanners.size()); List<KeyValueScanner> newScanners = new ArrayList<>(scanners.size());
@ -596,7 +596,8 @@ public class TestFromClientSide {
public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s,
final long readPt) throws IOException { final long readPt) throws IOException {
return new MyStoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, readPt); HStore hs = (HStore) store;
return new MyStoreScanner(hs, hs.getScanInfo(), scan, targetCols, readPt);
} }
} }
@ -2043,7 +2044,6 @@ public class TestFromClientSide {
public void testDeleteWithFailed() throws Exception { public void testDeleteWithFailed() throws Exception {
final TableName tableName = TableName.valueOf(name.getMethodName()); final TableName tableName = TableName.valueOf(name.getMethodName());
byte [][] ROWS = makeNAscii(ROW, 6);
byte [][] FAMILIES = makeNAscii(FAMILY, 3); byte [][] FAMILIES = makeNAscii(FAMILY, 3);
byte [][] VALUES = makeN(VALUE, 5); byte [][] VALUES = makeN(VALUE, 5);
long [] ts = {1000, 2000, 3000, 4000, 5000}; long [] ts = {1000, 2000, 3000, 4000, 5000};
@ -2061,7 +2061,7 @@ public class TestFromClientSide {
Get get = new Get(ROW); Get get = new Get(ROW);
get.addFamily(FAMILIES[0]); get.addFamily(FAMILIES[0]);
get.setMaxVersions(Integer.MAX_VALUE); get.readAllVersions();
Result result = ht.get(get); Result result = ht.get(get);
assertTrue(Bytes.equals(result.getValue(FAMILIES[0], QUALIFIER), VALUES[0])); assertTrue(Bytes.equals(result.getValue(FAMILIES[0], QUALIFIER), VALUES[0]));
} }
@ -5301,8 +5301,9 @@ public class TestFromClientSide {
// get the block cache and region // get the block cache and region
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName(); String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName); HRegion region = (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName)
Store store = region.getStores().iterator().next(); .getRegion(regionName);
HStore store = region.getStores().iterator().next();
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
cacheConf.setCacheDataOnWrite(true); cacheConf.setCacheDataOnWrite(true);
cacheConf.setEvictOnClose(true); cacheConf.setEvictOnClose(true);
@ -5394,15 +5395,14 @@ public class TestFromClientSide {
} }
} }
private void waitForStoreFileCount(Store store, int count, int timeout) private void waitForStoreFileCount(HStore store, int count, int timeout)
throws InterruptedException { throws InterruptedException {
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
while (start + timeout > System.currentTimeMillis() && while (start + timeout > System.currentTimeMillis() && store.getStorefilesCount() != count) {
store.getStorefilesCount() != count) {
Thread.sleep(100); Thread.sleep(100);
} }
System.out.println("start=" + start + ", now=" + System.out.println("start=" + start + ", now=" + System.currentTimeMillis() + ", cur=" +
System.currentTimeMillis() + ", cur=" + store.getStorefilesCount()); store.getStorefilesCount());
assertEquals(count, store.getStorefilesCount()); assertEquals(count, store.getStorefilesCount());
} }
@ -5462,8 +5462,8 @@ public class TestFromClientSide {
// Test Initialization. // Test Initialization.
byte [] startKey = Bytes.toBytes("ddc"); byte [] startKey = Bytes.toBytes("ddc");
byte [] endKey = Bytes.toBytes("mmm"); byte [] endKey = Bytes.toBytes("mmm");
final TableName tableName = TableName.valueOf(name.getMethodName()); TableName tableName = TableName.valueOf(name.getMethodName());
Table t = TEST_UTIL.createMultiRegionTable(tableName, new byte[][] { FAMILY }, 10); TEST_UTIL.createMultiRegionTable(tableName, new byte[][] { FAMILY }, 10);
int numOfRegions = -1; int numOfRegions = -1;
try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName)) { try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName)) {

View File

@ -125,7 +125,8 @@ public class TestRegionObserverScannerOpenHook {
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt) Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt)
throws IOException { throws IOException {
scan.setFilter(new NoDataFilter()); scan.setFilter(new NoDataFilter());
return new StoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, readPt); HStore hs = (HStore) store;
return new StoreScanner(hs, hs.getScanInfo(), scan, targetCols, readPt);
} }
} }

View File

@ -25,14 +25,15 @@ import java.util.Collection;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.testclassification.IOTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -109,7 +110,7 @@ public class TestForceCacheImportantBlocks {
setBloomFilterType(BLOOM_TYPE); setBloomFilterType(BLOOM_TYPE);
hcd.setBlocksize(BLOCK_SIZE); hcd.setBlocksize(BLOCK_SIZE);
hcd.setBlockCacheEnabled(cfCacheEnabled); hcd.setBlockCacheEnabled(cfCacheEnabled);
Region region = TEST_UTIL.createTestRegion(TABLE, hcd); HRegion region = TEST_UTIL.createTestRegion(TABLE, hcd);
BlockCache cache = region.getStore(hcd.getName()).getCacheConfig().getBlockCache(); BlockCache cache = region.getStore(hcd.getName()).getCacheConfig().getBlockCache();
CacheStats stats = cache.getStats(); CacheStats stats = cache.getStats();
writeTestData(region); writeTestData(region);

View File

@ -30,21 +30,21 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ByteBufferKeyValue;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.ChunkCreator; import org.apache.hadoop.hbase.regionserver.ChunkCreator;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl; import org.apache.hadoop.hbase.regionserver.MemStoreLABImpl;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -307,7 +307,7 @@ public class TestScannerFromBucketCache {
put.add(kv21); put.add(kv21);
region.put(put); region.put(put);
region.flush(true); region.flush(true);
Store store = region.getStore(fam1); HStore store = region.getStore(fam1);
while (store.getStorefilesCount() <= 0) { while (store.getStorefilesCount() <= 0) {
try { try {
Thread.sleep(20); Thread.sleep(20);

View File

@ -69,7 +69,7 @@ public class DelegatingKeyValueScanner implements KeyValueScanner {
} }
@Override @Override
public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) { public boolean shouldUseScanner(Scan scan, HStore store, long oldestUnexpiredTS) {
return delegate.shouldUseScanner(scan, store, oldestUnexpiredTS); return delegate.shouldUseScanner(scan, store, oldestUnexpiredTS);
} }

View File

@ -54,10 +54,11 @@ public class NoOpScanPolicyObserver implements RegionCoprocessor, RegionObserver
public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
Store store, List<KeyValueScanner> scanners, InternalScanner s, long readPoint) Store store, List<KeyValueScanner> scanners, InternalScanner s, long readPoint)
throws IOException { throws IOException {
ScanInfo oldSI = store.getScanInfo(); HStore hs = (HStore) store;
ScanInfo oldSI = hs.getScanInfo();
ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),
oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, return new StoreScanner(hs, scanInfo, OptionalInt.empty(), scanners,
ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP); ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP);
} }
@ -69,11 +70,12 @@ public class NoOpScanPolicyObserver implements RegionCoprocessor, RegionObserver
final ObserverContext<RegionCoprocessorEnvironment> c, Store store, final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
InternalScanner s, CompactionLifeCycleTracker tracker, long readPoint) throws IOException { InternalScanner s, CompactionLifeCycleTracker tracker, long readPoint) throws IOException {
HStore hs = (HStore) store;
// this demonstrates how to override the scanners default behavior // this demonstrates how to override the scanners default behavior
ScanInfo oldSI = store.getScanInfo(); ScanInfo oldSI = hs.getScanInfo();
ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),
oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, scanType, return new StoreScanner(hs, scanInfo, OptionalInt.empty(), scanners, scanType,
store.getSmallestReadPoint(), earliestPutTs); store.getSmallestReadPoint(), earliestPutTs);
} }
@ -81,11 +83,12 @@ public class NoOpScanPolicyObserver implements RegionCoprocessor, RegionObserver
public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPoint) Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPoint)
throws IOException { throws IOException {
HStore hs = (HStore) store;
Region r = c.getEnvironment().getRegion(); Region r = c.getEnvironment().getRegion();
return scan.isReversed() return scan.isReversed()
? new ReversedStoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, ? new ReversedStoreScanner(hs, hs.getScanInfo(), scan, targetCols,
r.getReadPoint(scan.getIsolationLevel())) r.getReadPoint(scan.getIsolationLevel()))
: new StoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, : new StoreScanner(hs, hs.getScanInfo(), scan, targetCols,
r.getReadPoint(scan.getIsolationLevel())); r.getReadPoint(scan.getIsolationLevel()));
} }
} }

View File

@ -17,6 +17,21 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -60,21 +75,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName; import org.junit.rules.TestName;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
/** /**
* Testing of HRegion.incrementColumnValue, HRegion.increment, * Testing of HRegion.incrementColumnValue, HRegion.increment,
* and HRegion.append * and HRegion.append
@ -84,7 +84,7 @@ public class TestAtomicOperation {
private static final Log LOG = LogFactory.getLog(TestAtomicOperation.class); private static final Log LOG = LogFactory.getLog(TestAtomicOperation.class);
@Rule public TestName name = new TestName(); @Rule public TestName name = new TestName();
Region region = null; HRegion region = null;
private HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU(); private HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
// Test names // Test names

View File

@ -208,7 +208,7 @@ public class TestCacheOnWriteInSchema {
public void testCacheOnWriteInSchema() throws IOException { public void testCacheOnWriteInSchema() throws IOException {
// Write some random data into the store // Write some random data into the store
StoreFileWriter writer = store.createWriterInTmp(Integer.MAX_VALUE, StoreFileWriter writer = store.createWriterInTmp(Integer.MAX_VALUE,
HFile.DEFAULT_COMPRESSION_ALGORITHM, false, true, false); HFile.DEFAULT_COMPRESSION_ALGORITHM, false, true, false, false);
writeStoreFile(writer); writeStoreFile(writer);
writer.close(); writer.close();
// Verify the block types of interest were cached on write // Verify the block types of interest were cached on write

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY; import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES; import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES;
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
@ -182,7 +183,7 @@ public class TestCompaction {
spyR.compactStores(); spyR.compactStores();
// ensure that the compaction stopped, all old files are intact, // ensure that the compaction stopped, all old files are intact,
Store s = r.stores.get(COLUMN_FAMILY); HStore s = r.getStore(COLUMN_FAMILY);
assertEquals(compactionThreshold, s.getStorefilesCount()); assertEquals(compactionThreshold, s.getStorefilesCount());
assertTrue(s.getStorefilesSize() > 15*1000); assertTrue(s.getStorefilesSize() > 15*1000);
// and no new store files persisted past compactStores() // and no new store files persisted past compactStores()
@ -210,8 +211,7 @@ public class TestCompaction {
// Multiple versions allowed for an entry, so the delete isn't enough // 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 // Lower TTL and expire to ensure that all our entries have been wiped
final int ttl = 1000; final int ttl = 1000;
for (Store hstore: this.r.stores.values()) { for (HStore store: this.r.stores.values()) {
HStore store = (HStore)hstore;
ScanInfo old = store.getScanInfo(); ScanInfo old = store.getScanInfo();
ScanInfo si = new ScanInfo(old.getConfiguration(), old.getFamily(), old.getMinVersions(), ScanInfo si = new ScanInfo(old.getConfiguration(), old.getFamily(), old.getMinVersions(),
old.getMaxVersions(), ttl, old.getKeepDeletedCells(), HConstants.DEFAULT_BLOCKSIZE, 0, old.getMaxVersions(), ttl, old.getKeepDeletedCells(), HConstants.DEFAULT_BLOCKSIZE, 0,
@ -307,7 +307,7 @@ public class TestCompaction {
CountDownLatch latch = new CountDownLatch(1); CountDownLatch latch = new CountDownLatch(1);
Tracker tracker = new Tracker(latch); Tracker tracker = new Tracker(latch);
thread.requestCompaction(r, store, "test custom comapction", Store.PRIORITY_USER, tracker, thread.requestCompaction(r, store, "test custom comapction", PRIORITY_USER, tracker,
null); null);
// wait for the latch to complete. // wait for the latch to complete.
latch.await(); latch.await();
@ -340,7 +340,7 @@ public class TestCompaction {
CountDownLatch latch = new CountDownLatch(1); CountDownLatch latch = new CountDownLatch(1);
Tracker tracker = new Tracker(latch); Tracker tracker = new Tracker(latch);
thread.requestCompaction(mockRegion, store, "test custom comapction", Store.PRIORITY_USER, thread.requestCompaction(mockRegion, store, "test custom comapction", PRIORITY_USER,
tracker, null); tracker, null);
// wait for the latch to complete. // wait for the latch to complete.
latch.await(120, TimeUnit.SECONDS); latch.await(120, TimeUnit.SECONDS);
@ -380,7 +380,7 @@ public class TestCompaction {
createStoreFile(r, store.getColumnFamilyName()); createStoreFile(r, store.getColumnFamilyName());
createStoreFile(r, store.getColumnFamilyName()); createStoreFile(r, store.getColumnFamilyName());
createStoreFile(r, store.getColumnFamilyName()); createStoreFile(r, store.getColumnFamilyName());
thread.requestCompaction(r, store, "test mulitple custom comapctions", Store.PRIORITY_USER, thread.requestCompaction(r, store, "test mulitple custom comapctions", PRIORITY_USER,
tracker, null); tracker, null);
} }
// wait for the latch to complete. // wait for the latch to complete.

View File

@ -180,7 +180,7 @@ public class TestCompactionFileNotFound {
// Refresh store files post compaction, this should not open already compacted files // Refresh store files post compaction, this should not open already compacted files
hr1.refreshStoreFiles(true); hr1.refreshStoreFiles(true);
// Archive the store files and try another compaction to see if all is good // Archive the store files and try another compaction to see if all is good
for (Store store : hr1.getStores()) { for (HStore store : hr1.getStores()) {
store.closeAndArchiveCompactedFiles(); store.closeAndArchiveCompactedFiles();
} }
try { try {

View File

@ -40,9 +40,9 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@ -288,10 +288,9 @@ public class TestCompoundBloomFilter {
byte[] qualifier) { byte[] qualifier) {
Scan scan = new Scan().withStartRow(row).withStopRow(row, true); Scan scan = new Scan().withStartRow(row).withStopRow(row, true);
scan.addColumn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME), qualifier); scan.addColumn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME), qualifier);
Store store = mock(Store.class); HStore store = mock(HStore.class);
HColumnDescriptor hcd = mock(HColumnDescriptor.class); when(store.getColumnFamilyDescriptor())
when(hcd.getName()).thenReturn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME)); .thenReturn(ColumnFamilyDescriptorBuilder.of(RandomKeyValueUtil.COLUMN_FAMILY_NAME));
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
return scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); return scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
} }

View File

@ -18,6 +18,8 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER;
import java.io.IOException; import java.io.IOException;
import java.security.Key; import java.security.Key;
import java.security.SecureRandom; import java.security.SecureRandom;
@ -539,7 +541,7 @@ public class TestHMobStore {
// Trigger major compaction // Trigger major compaction
this.store.triggerMajorCompaction(); this.store.triggerMajorCompaction();
Optional<CompactionContext> requestCompaction = Optional<CompactionContext> requestCompaction =
this.store.requestCompaction(Store.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null); this.store.requestCompaction(PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null);
this.store.compact(requestCompaction.get(), NoLimitThroughputController.INSTANCE, null); this.store.compact(requestCompaction.get(), NoLimitThroughputController.INSTANCE, null);
Assert.assertEquals(1, this.store.getStorefiles().size()); Assert.assertEquals(1, this.store.getStorefiles().size());

View File

@ -284,7 +284,7 @@ public class TestHRegion {
@Test @Test
public void testCloseCarryingSnapshot() throws IOException { public void testCloseCarryingSnapshot() throws IOException {
HRegion region = initHRegion(tableName, method, CONF, COLUMN_FAMILY_BYTES); HRegion region = initHRegion(tableName, method, CONF, COLUMN_FAMILY_BYTES);
Store store = region.getStore(COLUMN_FAMILY_BYTES); HStore store = region.getStore(COLUMN_FAMILY_BYTES);
// Get some random bytes. // Get some random bytes.
byte [] value = Bytes.toBytes(method); byte [] value = Bytes.toBytes(method);
// Make a random put against our cf. // Make a random put against our cf.
@ -333,7 +333,7 @@ public class TestHRegion {
HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, faultyLog, HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, faultyLog,
COLUMN_FAMILY_BYTES); COLUMN_FAMILY_BYTES);
Store store = region.getStore(COLUMN_FAMILY_BYTES); HStore store = region.getStore(COLUMN_FAMILY_BYTES);
// Get some random bytes. // Get some random bytes.
byte [] value = Bytes.toBytes(method); byte [] value = Bytes.toBytes(method);
faultyLog.setStoreFlushCtx(store.createFlushContext(12345)); faultyLog.setStoreFlushCtx(store.createFlushContext(12345));
@ -350,7 +350,7 @@ public class TestHRegion {
} finally { } finally {
assertTrue("The regionserver should have thrown an exception", threwIOE); assertTrue("The regionserver should have thrown an exception", threwIOE);
} }
long sz = store.getSizeToFlush().getDataSize(); long sz = store.getFlushableSize().getDataSize();
assertTrue("flushable size should be zero, but it is " + sz, sz == 0); assertTrue("flushable size should be zero, but it is " + sz, sz == 0);
HBaseTestingUtility.closeRegionAndWAL(region); HBaseTestingUtility.closeRegionAndWAL(region);
} }
@ -382,7 +382,7 @@ public class TestHRegion {
FSHLog hLog = new FSHLog(fs, rootDir, "testMemstoreSizeWithFlushCanceling", CONF); FSHLog hLog = new FSHLog(fs, rootDir, "testMemstoreSizeWithFlushCanceling", CONF);
HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog, HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog,
COLUMN_FAMILY_BYTES); COLUMN_FAMILY_BYTES);
Store store = region.getStore(COLUMN_FAMILY_BYTES); HStore store = region.getStore(COLUMN_FAMILY_BYTES);
assertEquals(0, region.getMemstoreSize()); assertEquals(0, region.getMemstoreSize());
// Put some value and make sure flush could be completed normally // Put some value and make sure flush could be completed normally
@ -394,7 +394,7 @@ public class TestHRegion {
assertTrue(onePutSize > 0); assertTrue(onePutSize > 0);
region.flush(true); region.flush(true);
assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize()); assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize());
assertEquals("flushable size should be zero", 0, store.getSizeToFlush().getDataSize()); assertEquals("flushable size should be zero", 0, store.getFlushableSize().getDataSize());
// save normalCPHost and replaced by mockedCPHost, which will cancel flush requests // save normalCPHost and replaced by mockedCPHost, which will cancel flush requests
RegionCoprocessorHost normalCPHost = region.getCoprocessorHost(); RegionCoprocessorHost normalCPHost = region.getCoprocessorHost();
@ -406,13 +406,13 @@ public class TestHRegion {
region.flush(true); region.flush(true);
assertEquals("memstoreSize should NOT be zero", onePutSize, region.getMemstoreSize()); assertEquals("memstoreSize should NOT be zero", onePutSize, region.getMemstoreSize());
assertEquals("flushable size should NOT be zero", onePutSize, assertEquals("flushable size should NOT be zero", onePutSize,
store.getSizeToFlush().getDataSize()); store.getFlushableSize().getDataSize());
// set normalCPHost and flush again, the snapshot will be flushed // set normalCPHost and flush again, the snapshot will be flushed
region.setCoprocessorHost(normalCPHost); region.setCoprocessorHost(normalCPHost);
region.flush(true); region.flush(true);
assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize()); assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize());
assertEquals("flushable size should be zero", 0, store.getSizeToFlush().getDataSize()); assertEquals("flushable size should be zero", 0, store.getFlushableSize().getDataSize());
HBaseTestingUtility.closeRegionAndWAL(region); HBaseTestingUtility.closeRegionAndWAL(region);
} }
@ -424,7 +424,7 @@ public class TestHRegion {
FSHLog hLog = new FSHLog(fs, rootDir, testName, CONF); FSHLog hLog = new FSHLog(fs, rootDir, testName, CONF);
HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog, HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog,
COLUMN_FAMILY_BYTES); COLUMN_FAMILY_BYTES);
Store store = region.getStore(COLUMN_FAMILY_BYTES); HStore store = region.getStore(COLUMN_FAMILY_BYTES);
assertEquals(0, region.getMemstoreSize()); assertEquals(0, region.getMemstoreSize());
// Put one value // Put one value
@ -450,7 +450,7 @@ public class TestHRegion {
long expectedSize = onePutSize * 2; long expectedSize = onePutSize * 2;
assertEquals("memstoreSize should be incremented", expectedSize, region.getMemstoreSize()); assertEquals("memstoreSize should be incremented", expectedSize, region.getMemstoreSize());
assertEquals("flushable size should be incremented", expectedSize, assertEquals("flushable size should be incremented", expectedSize,
store.getSizeToFlush().getDataSize()); store.getFlushableSize().getDataSize());
region.setCoprocessorHost(null); region.setCoprocessorHost(null);
HBaseTestingUtility.closeRegionAndWAL(region); HBaseTestingUtility.closeRegionAndWAL(region);
@ -565,7 +565,7 @@ public class TestHRegion {
p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[])null)); p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[])null));
region.put(p1); region.put(p1);
// Manufacture an outstanding snapshot -- fake a failed flush by doing prepare step only. // Manufacture an outstanding snapshot -- fake a failed flush by doing prepare step only.
Store store = region.getStore(COLUMN_FAMILY_BYTES); HStore store = region.getStore(COLUMN_FAMILY_BYTES);
StoreFlushContext storeFlushCtx = store.createFlushContext(12345); StoreFlushContext storeFlushCtx = store.createFlushContext(12345);
storeFlushCtx.prepare(); storeFlushCtx.prepare();
// Now add two entries to the foreground memstore. // Now add two entries to the foreground memstore.
@ -699,7 +699,7 @@ public class TestHRegion {
} }
MonitoredTask status = TaskMonitor.get().createStatus(method); MonitoredTask status = TaskMonitor.get().createStatus(method);
Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR); Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId - 1); maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId - 1);
} }
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status); long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
@ -751,7 +751,7 @@ public class TestHRegion {
long recoverSeqId = 1030; long recoverSeqId = 1030;
MonitoredTask status = TaskMonitor.get().createStatus(method); MonitoredTask status = TaskMonitor.get().createStatus(method);
Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR); Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1); maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
} }
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status); long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
@ -796,7 +796,7 @@ public class TestHRegion {
dos.close(); dos.close();
Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR); Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId); maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId);
} }
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, null); long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, null);
@ -854,7 +854,7 @@ public class TestHRegion {
long recoverSeqId = 1030; long recoverSeqId = 1030;
Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR); Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
MonitoredTask status = TaskMonitor.get().createStatus(method); MonitoredTask status = TaskMonitor.get().createStatus(method);
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1); maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
} }
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status); long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
@ -3713,7 +3713,7 @@ public class TestHRegion {
if (i != 0 && i % compactInterval == 0) { if (i != 0 && i % compactInterval == 0) {
region.compact(true); region.compact(true);
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
store.closeAndArchiveCompactedFiles(); store.closeAndArchiveCompactedFiles();
} }
} }
@ -3893,7 +3893,7 @@ public class TestHRegion {
// Compact regularly to avoid creating too many files and exceeding // Compact regularly to avoid creating too many files and exceeding
// the ulimit. // the ulimit.
region.compact(false); region.compact(false);
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
store.closeAndArchiveCompactedFiles(); store.closeAndArchiveCompactedFiles();
} }
} }

View File

@ -35,8 +35,6 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -53,23 +51,44 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl; import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl;
import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult; import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
import org.apache.hadoop.util.StringUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
@ -81,24 +100,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescript
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
import org.apache.hadoop.util.StringUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
/** /**
* Tests of HRegion methods for replaying flush, compaction, region open, etc events for secondary * Tests of HRegion methods for replaying flush, compaction, region open, etc events for secondary
@ -127,7 +128,7 @@ public class TestHRegionReplayEvents {
// per test fields // per test fields
private Path rootDir; private Path rootDir;
private HTableDescriptor htd; private TableDescriptor htd;
private long time; private long time;
private RegionServerServices rss; private RegionServerServices rss;
private HRegionInfo primaryHri, secondaryHri; private HRegionInfo primaryHri, secondaryHri;
@ -146,11 +147,11 @@ public class TestHRegionReplayEvents {
rootDir = new Path(dir + method); rootDir = new Path(dir + method);
TEST_UTIL.getConfiguration().set(HConstants.HBASE_DIR, rootDir.toString()); TEST_UTIL.getConfiguration().set(HConstants.HBASE_DIR, rootDir.toString());
method = name.getMethodName(); method = name.getMethodName();
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf(method));
htd = new HTableDescriptor(TableName.valueOf(method));
for (byte[] family : families) { for (byte[] family : families) {
htd.addFamily(new HColumnDescriptor(family)); builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
} }
htd = builder.build();
time = System.currentTimeMillis(); time = System.currentTimeMillis();
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null); ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
@ -338,10 +339,10 @@ public class TestHRegionReplayEvents {
if (flushDesc != null) { if (flushDesc != null) {
// first verify that everything is replayed and visible before flush event replay // first verify that everything is replayed and visible before flush event replay
verifyData(secondaryRegion, 0, lastReplayed, cq, families); verifyData(secondaryRegion, 0, lastReplayed, cq, families);
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
long storeMemstoreSize = store.getMemStoreSize(); long storeMemstoreSize = store.getMemStoreSize().getHeapSize();
long regionMemstoreSize = secondaryRegion.getMemstoreSize(); long regionMemstoreSize = secondaryRegion.getMemstoreSize();
long storeFlushableSize = store.getFlushableSize(); long storeFlushableSize = store.getFlushableSize().getHeapSize();
long storeSize = store.getSize(); long storeSize = store.getSize();
long storeSizeUncompressed = store.getStoreSizeUncompressed(); long storeSizeUncompressed = store.getStoreSizeUncompressed();
if (flushDesc.getAction() == FlushAction.START_FLUSH) { if (flushDesc.getAction() == FlushAction.START_FLUSH) {
@ -351,7 +352,7 @@ public class TestHRegionReplayEvents {
assertEquals(result.flushOpSeqId, flushDesc.getFlushSequenceNumber()); assertEquals(result.flushOpSeqId, flushDesc.getFlushSequenceNumber());
// assert that the store memstore is smaller now // assert that the store memstore is smaller now
long newStoreMemstoreSize = store.getMemStoreSize(); long newStoreMemstoreSize = store.getMemStoreSize().getHeapSize();
LOG.info("Memstore size reduced by:" LOG.info("Memstore size reduced by:"
+ StringUtils.humanReadableInt(newStoreMemstoreSize - storeMemstoreSize)); + StringUtils.humanReadableInt(newStoreMemstoreSize - storeMemstoreSize));
assertTrue(storeMemstoreSize > newStoreMemstoreSize); assertTrue(storeMemstoreSize > newStoreMemstoreSize);
@ -362,10 +363,10 @@ public class TestHRegionReplayEvents {
// assert that the flush files are picked // assert that the flush files are picked
expectedStoreFileCount++; expectedStoreFileCount++;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
long newFlushableSize = store.getFlushableSize(); long newFlushableSize = store.getFlushableSize().getHeapSize();
assertTrue(storeFlushableSize > newFlushableSize); assertTrue(storeFlushableSize > newFlushableSize);
// assert that the region memstore is smaller now // assert that the region memstore is smaller now
@ -383,7 +384,7 @@ public class TestHRegionReplayEvents {
secondaryRegion.replayWALCompactionMarker(compactionDesc, true, false, Long.MAX_VALUE); secondaryRegion.replayWALCompactionMarker(compactionDesc, true, false, Long.MAX_VALUE);
// assert that the compaction is applied // assert that the compaction is applied
for (Store store : secondaryRegion.getStores()) { for (HStore store : secondaryRegion.getStores()) {
if (store.getColumnFamilyName().equals("cf1")) { if (store.getColumnFamilyName().equals("cf1")) {
assertEquals(1, store.getStorefilesCount()); assertEquals(1, store.getStorefilesCount());
} else { } else {
@ -401,7 +402,7 @@ public class TestHRegionReplayEvents {
LOG.info("-- Verifying edits from primary. Ensuring that files are not deleted"); LOG.info("-- Verifying edits from primary. Ensuring that files are not deleted");
verifyData(primaryRegion, 0, lastReplayed, cq, families); verifyData(primaryRegion, 0, lastReplayed, cq, families);
for (Store store : primaryRegion.getStores()) { for (HStore store : primaryRegion.getStores()) {
if (store.getColumnFamilyName().equals("cf1")) { if (store.getColumnFamilyName().equals("cf1")) {
assertEquals(1, store.getStorefilesCount()); assertEquals(1, store.getStorefilesCount());
} else { } else {
@ -437,10 +438,10 @@ public class TestHRegionReplayEvents {
= WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0)); = WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
if (flushDesc != null) { if (flushDesc != null) {
// first verify that everything is replayed and visible before flush event replay // first verify that everything is replayed and visible before flush event replay
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
long storeMemstoreSize = store.getMemStoreSize(); long storeMemstoreSize = store.getMemStoreSize().getHeapSize();
long regionMemstoreSize = secondaryRegion.getMemstoreSize(); long regionMemstoreSize = secondaryRegion.getMemstoreSize();
long storeFlushableSize = store.getFlushableSize(); long storeFlushableSize = store.getFlushableSize().getHeapSize();
if (flushDesc.getAction() == FlushAction.START_FLUSH) { if (flushDesc.getAction() == FlushAction.START_FLUSH) {
startFlushDesc = flushDesc; startFlushDesc = flushDesc;
@ -452,7 +453,7 @@ public class TestHRegionReplayEvents {
assertTrue(storeFlushableSize > 0); assertTrue(storeFlushableSize > 0);
// assert that the store memstore is smaller now // assert that the store memstore is smaller now
long newStoreMemstoreSize = store.getMemStoreSize(); long newStoreMemstoreSize = store.getMemStoreSize().getHeapSize();
LOG.info("Memstore size reduced by:" LOG.info("Memstore size reduced by:"
+ StringUtils.humanReadableInt(newStoreMemstoreSize - storeMemstoreSize)); + StringUtils.humanReadableInt(newStoreMemstoreSize - storeMemstoreSize));
assertTrue(storeMemstoreSize > newStoreMemstoreSize); assertTrue(storeMemstoreSize > newStoreMemstoreSize);
@ -571,7 +572,7 @@ public class TestHRegionReplayEvents {
// no store files in the region // no store files in the region
int expectedStoreFileCount = 0; int expectedStoreFileCount = 0;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
long regionMemstoreSize = secondaryRegion.getMemstoreSize(); long regionMemstoreSize = secondaryRegion.getMemstoreSize();
@ -586,11 +587,11 @@ public class TestHRegionReplayEvents {
// assert that the flush files are picked // assert that the flush files are picked
expectedStoreFileCount++; expectedStoreFileCount++;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
long newFlushableSize = store.getFlushableSize(); long newFlushableSize = store.getFlushableSize().getHeapSize();
assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped
// assert that the region memstore is same as before // assert that the region memstore is same as before
@ -661,7 +662,7 @@ public class TestHRegionReplayEvents {
// no store files in the region // no store files in the region
int expectedStoreFileCount = 0; int expectedStoreFileCount = 0;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
long regionMemstoreSize = secondaryRegion.getMemstoreSize(); long regionMemstoreSize = secondaryRegion.getMemstoreSize();
@ -676,11 +677,11 @@ public class TestHRegionReplayEvents {
// assert that the flush files are picked // assert that the flush files are picked
expectedStoreFileCount++; expectedStoreFileCount++;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
long newFlushableSize = store.getFlushableSize(); long newFlushableSize = store.getFlushableSize().getHeapSize();
assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped
// assert that the region memstore is smaller than before, but not empty // assert that the region memstore is smaller than before, but not empty
@ -762,7 +763,7 @@ public class TestHRegionReplayEvents {
// no store files in the region // no store files in the region
int expectedStoreFileCount = 0; int expectedStoreFileCount = 0;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
long regionMemstoreSize = secondaryRegion.getMemstoreSize(); long regionMemstoreSize = secondaryRegion.getMemstoreSize();
@ -772,8 +773,8 @@ public class TestHRegionReplayEvents {
assertTrue(commitFlushDesc.getFlushSequenceNumber() > 0); assertTrue(commitFlushDesc.getFlushSequenceNumber() > 0);
// ensure all files are visible in secondary // ensure all files are visible in secondary
for (Store store : secondaryRegion.getStores()) { for (HStore store : secondaryRegion.getStores()) {
assertTrue(store.getMaxSequenceId() <= secondaryRegion.getReadPoint(null)); assertTrue(store.getMaxSequenceId().orElse(0L) <= secondaryRegion.getReadPoint(null));
} }
LOG.info("-- Replaying flush commit in secondary" + commitFlushDesc); LOG.info("-- Replaying flush commit in secondary" + commitFlushDesc);
@ -781,11 +782,11 @@ public class TestHRegionReplayEvents {
// assert that the flush files are picked // assert that the flush files are picked
expectedStoreFileCount++; expectedStoreFileCount++;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
long newFlushableSize = store.getFlushableSize(); long newFlushableSize = store.getFlushableSize().getHeapSize();
if (droppableMemstore) { if (droppableMemstore) {
// assert that the memstore is dropped // assert that the memstore is dropped
assertTrue(newFlushableSize == MutableSegment.DEEP_OVERHEAD); assertTrue(newFlushableSize == MutableSegment.DEEP_OVERHEAD);
@ -861,7 +862,7 @@ public class TestHRegionReplayEvents {
// no store files in the region // no store files in the region
int expectedStoreFileCount = 0; int expectedStoreFileCount = 0;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
long regionMemstoreSize = secondaryRegion.getMemstoreSize(); long regionMemstoreSize = secondaryRegion.getMemstoreSize();
@ -873,11 +874,11 @@ public class TestHRegionReplayEvents {
// assert that the flush files are picked // assert that the flush files are picked
expectedStoreFileCount++; expectedStoreFileCount++;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
long newFlushableSize = store.getFlushableSize(); long newFlushableSize = store.getFlushableSize().getHeapSize();
assertTrue(newFlushableSize == MutableSegment.DEEP_OVERHEAD); assertTrue(newFlushableSize == MutableSegment.DEEP_OVERHEAD);
// assert that the region memstore is empty // assert that the region memstore is empty
@ -942,7 +943,7 @@ public class TestHRegionReplayEvents {
// no store files in the region // no store files in the region
int expectedStoreFileCount = 0; int expectedStoreFileCount = 0;
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
@ -952,11 +953,11 @@ public class TestHRegionReplayEvents {
// assert that the flush files are picked // assert that the flush files are picked
expectedStoreFileCount = 2; // two flushes happened expectedStoreFileCount = 2; // two flushes happened
for (Store s : secondaryRegion.getStores()) { for (HStore s : secondaryRegion.getStores()) {
assertEquals(expectedStoreFileCount, s.getStorefilesCount()); assertEquals(expectedStoreFileCount, s.getStorefilesCount());
} }
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1")); HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
MemstoreSize newSnapshotSize = store.getSizeOfSnapshot(); MemstoreSize newSnapshotSize = store.getSnapshotSize();
assertTrue(newSnapshotSize.getDataSize() == 0); assertTrue(newSnapshotSize.getDataSize() == 0);
// assert that the region memstore is empty // assert that the region memstore is empty

View File

@ -1,5 +1,4 @@
/* /**
*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -16,18 +15,18 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import static org.mockito.Matchers.any; import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times; import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.io.IOException; import java.io.IOException;
import java.lang.ref.SoftReference; import java.lang.ref.SoftReference;
@ -60,19 +59,22 @@ import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MemoryCompactionPolicy; import org.apache.hadoop.hbase.MemoryCompactionPolicy;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.filter.FilterBase;
@ -95,10 +97,12 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper; import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge; import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
import org.junit.After; import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
@ -109,7 +113,7 @@ import org.mockito.Mockito;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
/** /**
* Test class for the Store * Test class for the HStore
*/ */
@Category({ RegionServerTests.class, MediumTests.class }) @Category({ RegionServerTests.class, MediumTests.class })
public class TestHStore { public class TestHStore {
@ -117,6 +121,7 @@ public class TestHStore {
@Rule @Rule
public TestName name = new TestName(); public TestName name = new TestName();
HRegion region;
HStore store; HStore store;
byte [] table = Bytes.toBytes("table"); byte [] table = Bytes.toBytes("table");
byte [] family = Bytes.toBytes("family"); byte [] family = Bytes.toBytes("family");
@ -138,8 +143,8 @@ public class TestHStore {
long id = System.currentTimeMillis(); long id = System.currentTimeMillis();
Get get = new Get(row); Get get = new Get(row);
private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString(); private static final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString();
/** /**
@ -164,55 +169,51 @@ public class TestHStore {
init(methodName, TEST_UTIL.getConfiguration()); init(methodName, TEST_UTIL.getConfiguration());
} }
private Store init(String methodName, Configuration conf) throws IOException { private HStore init(String methodName, Configuration conf) throws IOException {
HColumnDescriptor hcd = new HColumnDescriptor(family);
// some of the tests write 4 versions and then flush // some of the tests write 4 versions and then flush
// (with HBASE-4241, lower versions are collected on flush) // (with HBASE-4241, lower versions are collected on flush)
hcd.setMaxVersions(4); return init(methodName, conf,
return init(methodName, conf, hcd); ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(4).build());
} }
private HStore init(String methodName, Configuration conf, HColumnDescriptor hcd) private HStore init(String methodName, Configuration conf, ColumnFamilyDescriptor hcd)
throws IOException { throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); return init(methodName, conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)), hcd);
return init(methodName, conf, htd, hcd);
} }
private HStore init(String methodName, Configuration conf, HTableDescriptor htd, private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,
HColumnDescriptor hcd) throws IOException { ColumnFamilyDescriptor hcd) throws IOException {
return init(methodName, conf, htd, hcd, null); return init(methodName, conf, builder, hcd, null);
} }
@SuppressWarnings("deprecation") private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,
private HStore init(String methodName, Configuration conf, HTableDescriptor htd, ColumnFamilyDescriptor hcd, MyStoreHook hook) throws IOException {
HColumnDescriptor hcd, MyStoreHook hook) throws IOException { return init(methodName, conf, builder, hcd, hook, false);
return init(methodName, conf, htd, hcd, hook, false);
} }
@SuppressWarnings("deprecation")
private HStore init(String methodName, Configuration conf, HTableDescriptor htd, private void initHRegion(String methodName, Configuration conf, TableDescriptorBuilder builder,
HColumnDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException { ColumnFamilyDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {
//Setting up a Store TableDescriptor htd = builder.addColumnFamily(hcd).build();
Path basedir = new Path(DIR+methodName); Path basedir = new Path(DIR + methodName);
Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName()); Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
final Path logdir = new Path(basedir, AbstractFSWALProvider.getWALDirectoryName(methodName)); final Path logdir = new Path(basedir, AbstractFSWALProvider.getWALDirectoryName(methodName));
FileSystem fs = FileSystem.get(conf); FileSystem fs = FileSystem.get(conf);
fs.delete(logdir, true); fs.delete(logdir, true);
if (htd.hasFamily(hcd.getName())) {
htd.modifyFamily(hcd);
} else {
htd.addFamily(hcd);
}
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null); MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null);
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false); HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
final Configuration walConf = new Configuration(conf); Configuration walConf = new Configuration(conf);
FSUtils.setRootDir(walConf, basedir); FSUtils.setRootDir(walConf, basedir);
final WALFactory wals = new WALFactory(walConf, null, methodName); WALFactory wals = new WALFactory(walConf, null, methodName);
HRegion region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes(), region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info),
info.getTable().getNamespace()), fs, conf, info, htd, null); wals.getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace()), conf, htd, null);
}
private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,
ColumnFamilyDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {
initHRegion(methodName, conf, builder, hcd, hook, switchToPread);
if (hook == null) { if (hook == null) {
store = new HStore(region, hcd, conf); store = new HStore(region, hcd, conf);
} else { } else {
@ -299,13 +300,14 @@ public class TestHStore {
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
FileSystem fs = FileSystem.get(conf); FileSystem fs = FileSystem.get(conf);
HColumnDescriptor hcd = new HColumnDescriptor(family); ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(family)
hcd.setCompressionType(Compression.Algorithm.GZ); .setCompressionType(Compression.Algorithm.GZ).setDataBlockEncoding(DataBlockEncoding.DIFF)
hcd.setDataBlockEncoding(DataBlockEncoding.DIFF); .build();
init(name.getMethodName(), conf, hcd); init(name.getMethodName(), conf, hcd);
// Test createWriterInTmp() // Test createWriterInTmp()
StoreFileWriter writer = store.createWriterInTmp(4, hcd.getCompressionType(), false, true, false); StoreFileWriter writer =
store.createWriterInTmp(4, hcd.getCompressionType(), false, true, false, false);
Path path = writer.getPath(); Path path = writer.getPath();
writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1))); writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1)));
writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2))); writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2)));
@ -341,10 +343,8 @@ public class TestHStore {
// Set the compaction threshold higher to avoid normal compactions. // Set the compaction threshold higher to avoid normal compactions.
conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 5); conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 5);
HColumnDescriptor hcd = new HColumnDescriptor(family); init(name.getMethodName() + "-" + minVersions, conf, ColumnFamilyDescriptorBuilder
hcd.setMinVersions(minVersions); .newBuilder(family).setMinVersions(minVersions).setTimeToLive(ttl).build());
hcd.setTimeToLive(ttl);
init(name.getMethodName() + "-" + minVersions, conf, hcd);
long storeTtl = this.store.getScanInfo().getTtl(); long storeTtl = this.store.getScanInfo().getTtl();
long sleepTime = storeTtl / storeFileNum; long sleepTime = storeTtl / storeFileNum;
@ -605,6 +605,22 @@ public class TestHStore {
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
EnvironmentEdgeManagerTestHelper.reset(); EnvironmentEdgeManagerTestHelper.reset();
if (store != null) {
try {
store.close();
} catch (IOException e) {
}
store = null;
}
if (region != null) {
region.close();
region = null;
}
}
@AfterClass
public static void tearDownAfterClass() throws IOException {
TEST_UTIL.cleanupTestDir();
} }
@Test @Test
@ -830,17 +846,19 @@ public class TestHStore {
// HTD overrides XML. // HTD overrides XML.
--anyValue; --anyValue;
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); init(name.getMethodName() + "-htd", conf, TableDescriptorBuilder
HColumnDescriptor hcd = new HColumnDescriptor(family); .newBuilder(TableName.valueOf(table)).setValue(CONFIG_KEY, Long.toString(anyValue)),
htd.setConfiguration(CONFIG_KEY, Long.toString(anyValue)); ColumnFamilyDescriptorBuilder.of(family));
init(name.getMethodName() + "-htd", conf, htd, hcd);
assertTrue(store.throttleCompaction(anyValue + 1)); assertTrue(store.throttleCompaction(anyValue + 1));
assertFalse(store.throttleCompaction(anyValue)); assertFalse(store.throttleCompaction(anyValue));
// HCD overrides them both. // HCD overrides them both.
--anyValue; --anyValue;
hcd.setConfiguration(CONFIG_KEY, Long.toString(anyValue)); init(name.getMethodName() + "-hcd", conf,
init(name.getMethodName() + "-hcd", conf, htd, hcd); TableDescriptorBuilder.newBuilder(TableName.valueOf(table)).setValue(CONFIG_KEY,
Long.toString(anyValue)),
ColumnFamilyDescriptorBuilder.newBuilder(family).setValue(CONFIG_KEY, Long.toString(anyValue))
.build());
assertTrue(store.throttleCompaction(anyValue + 1)); assertTrue(store.throttleCompaction(anyValue + 1));
assertFalse(store.throttleCompaction(anyValue)); assertFalse(store.throttleCompaction(anyValue));
} }
@ -868,7 +886,7 @@ public class TestHStore {
private void addStoreFile() throws IOException { private void addStoreFile() throws IOException {
HStoreFile f = this.store.getStorefiles().iterator().next(); HStoreFile f = this.store.getStorefiles().iterator().next();
Path storedir = f.getPath().getParent(); Path storedir = f.getPath().getParent();
long seqid = this.store.getMaxSequenceId(); long seqid = this.store.getMaxSequenceId().orElse(0L);
Configuration c = TEST_UTIL.getConfiguration(); Configuration c = TEST_UTIL.getConfiguration();
FileSystem fs = FileSystem.get(c); FileSystem fs = FileSystem.get(c);
HFileContext fileContext = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build(); HFileContext fileContext = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();
@ -995,20 +1013,23 @@ public class TestHStore {
public void testNumberOfMemStoreScannersAfterFlush() throws IOException { public void testNumberOfMemStoreScannersAfterFlush() throws IOException {
long seqId = 100; long seqId = 100;
long timestamp = System.currentTimeMillis(); long timestamp = System.currentTimeMillis();
Cell cell0 = CellUtil.createCell(row, family, qf1, timestamp, Cell cell0 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
KeyValue.Type.Put.getCode(), qf1); .setQualifier(qf1).setTimestamp(timestamp).setType(KeyValue.Type.Put.getCode())
.setValue(qf1).build();
CellUtil.setSequenceId(cell0, seqId); CellUtil.setSequenceId(cell0, seqId);
testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Collections.EMPTY_LIST); testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Collections.emptyList());
Cell cell1 = CellUtil.createCell(row, family, qf2, timestamp, Cell cell1 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
KeyValue.Type.Put.getCode(), qf1); .setQualifier(qf2).setTimestamp(timestamp).setType(KeyValue.Type.Put.getCode())
.setValue(qf1).build();
CellUtil.setSequenceId(cell1, seqId); CellUtil.setSequenceId(cell1, seqId);
testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1)); testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1));
seqId = 101; seqId = 101;
timestamp = System.currentTimeMillis(); timestamp = System.currentTimeMillis();
Cell cell2 = CellUtil.createCell(row2, family, qf2, timestamp, Cell cell2 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row2).setFamily(family)
KeyValue.Type.Put.getCode(), qf1); .setQualifier(qf2).setTimestamp(timestamp).setType(KeyValue.Type.Put.getCode())
.setValue(qf1).build();
CellUtil.setSequenceId(cell2, seqId); CellUtil.setSequenceId(cell2, seqId);
testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1, cell2)); testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1, cell2));
} }
@ -1052,15 +1073,16 @@ public class TestHStore {
} }
} }
private Cell createCell(byte[] qualifier, long ts, long sequenceId, byte[] value) throws IOException { private Cell createCell(byte[] qualifier, long ts, long sequenceId, byte[] value)
Cell c = CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put.getCode(), value); throws IOException {
CellUtil.setSequenceId(c, sequenceId); return createCell(row, qualifier, ts, sequenceId, value);
return c;
} }
private Cell createCell(byte[] row, byte[] qualifier, long ts, long sequenceId, byte[] value) private Cell createCell(byte[] row, byte[] qualifier, long ts, long sequenceId, byte[] value)
throws IOException { throws IOException {
Cell c = CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put.getCode(), value); Cell c = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
.setQualifier(qualifier).setTimestamp(ts).setType(KeyValue.Type.Put.getCode())
.setValue(value).build();
CellUtil.setSequenceId(c, sequenceId); CellUtil.setSequenceId(c, sequenceId);
return c; return c;
} }
@ -1154,8 +1176,6 @@ public class TestHStore {
private void testFlushBeforeCompletingScan(MyListHook hook, Filter filter, int expectedSize) private void testFlushBeforeCompletingScan(MyListHook hook, Filter filter, int expectedSize)
throws IOException, InterruptedException { throws IOException, InterruptedException {
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
HColumnDescriptor hcd = new HColumnDescriptor(family);
hcd.setMaxVersions(1);
byte[] r0 = Bytes.toBytes("row0"); byte[] r0 = Bytes.toBytes("row0");
byte[] r1 = Bytes.toBytes("row1"); byte[] r1 = Bytes.toBytes("row1");
byte[] r2 = Bytes.toBytes("row2"); byte[] r2 = Bytes.toBytes("row2");
@ -1165,12 +1185,14 @@ public class TestHStore {
MemstoreSize memStoreSize = new MemstoreSize(); MemstoreSize memStoreSize = new MemstoreSize();
long ts = EnvironmentEdgeManager.currentTime(); long ts = EnvironmentEdgeManager.currentTime();
long seqId = 100; long seqId = 100;
init(name.getMethodName(), conf, new HTableDescriptor(TableName.valueOf(table)), hcd, new MyStoreHook() { init(name.getMethodName(), conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)),
@Override ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(1).build(),
public long getSmallestReadPoint(HStore store) { new MyStoreHook() {
return seqId + 3; @Override
} public long getSmallestReadPoint(HStore store) {
}); return seqId + 3;
}
});
// The cells having the value0 won't be flushed to disk because the value of max version is 1 // The cells having the value0 won't be flushed to disk because the value of max version is 1
store.add(createCell(r0, qf1, ts, seqId, value0), memStoreSize); store.add(createCell(r0, qf1, ts, seqId, value0), memStoreSize);
store.add(createCell(r0, qf2, ts, seqId, value0), memStoreSize); store.add(createCell(r0, qf2, ts, seqId, value0), memStoreSize);
@ -1216,9 +1238,8 @@ public class TestHStore {
public void testCreateScannerAndSnapshotConcurrently() throws IOException, InterruptedException { public void testCreateScannerAndSnapshotConcurrently() throws IOException, InterruptedException {
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore.class.getName()); conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore.class.getName());
HColumnDescriptor hcd = new HColumnDescriptor(family); init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC); .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
init(name.getMethodName(), conf, hcd);
byte[] value = Bytes.toBytes("value"); byte[] value = Bytes.toBytes("value");
MemstoreSize memStoreSize = new MemstoreSize(); MemstoreSize memStoreSize = new MemstoreSize();
long ts = EnvironmentEdgeManager.currentTime(); long ts = EnvironmentEdgeManager.currentTime();
@ -1408,9 +1429,8 @@ public class TestHStore {
conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushSize)); conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushSize));
// Set the lower threshold to invoke the "MERGE" policy // Set the lower threshold to invoke the "MERGE" policy
conf.set(MemStoreCompactor.COMPACTING_MEMSTORE_THRESHOLD_KEY, String.valueOf(0)); conf.set(MemStoreCompactor.COMPACTING_MEMSTORE_THRESHOLD_KEY, String.valueOf(0));
HColumnDescriptor hcd = new HColumnDescriptor(family); init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC); .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
init(name.getMethodName(), conf, hcd);
byte[] value = Bytes.toBytes("thisisavarylargevalue"); byte[] value = Bytes.toBytes("thisisavarylargevalue");
MemstoreSize memStoreSize = new MemstoreSize(); MemstoreSize memStoreSize = new MemstoreSize();
long ts = EnvironmentEdgeManager.currentTime(); long ts = EnvironmentEdgeManager.currentTime();
@ -1445,18 +1465,57 @@ public class TestHStore {
storeFlushCtx.commit(Mockito.mock(MonitoredTask.class)); storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
} }
private MyStore initMyStore(String methodName, Configuration conf, MyStoreHook hook) @Test
throws IOException { public void testAge() throws IOException {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); long currentTime = System.currentTimeMillis();
HColumnDescriptor hcd = new HColumnDescriptor(family); ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
hcd.setMaxVersions(5); edge.setValue(currentTime);
return (MyStore) init(methodName, conf, htd, hcd, hook); EnvironmentEdgeManager.injectEdge(edge);
Configuration conf = TEST_UTIL.getConfiguration();
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(family);
initHRegion(name.getMethodName(), conf,
TableDescriptorBuilder.newBuilder(TableName.valueOf(table)), hcd, null, false);
HStore store = new HStore(region, hcd, conf) {
@Override
protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf,
CellComparator kvComparator) throws IOException {
List<HStoreFile> storefiles =
Arrays.asList(mockStoreFile(currentTime - 10), mockStoreFile(currentTime - 100),
mockStoreFile(currentTime - 1000), mockStoreFile(currentTime - 10000));
StoreFileManager sfm = mock(StoreFileManager.class);
when(sfm.getStorefiles()).thenReturn(storefiles);
StoreEngine<?, ?, ?, ?> storeEngine = mock(StoreEngine.class);
when(storeEngine.getStoreFileManager()).thenReturn(sfm);
return storeEngine;
}
};
assertEquals(10L, store.getMinStoreFileAge().getAsLong());
assertEquals(10000L, store.getMaxStoreFileAge().getAsLong());
assertEquals((10 + 100 + 1000 + 10000) / 4.0, store.getAvgStoreFileAge().getAsDouble(), 1E-4);
} }
class MyStore extends HStore { private HStoreFile mockStoreFile(long createdTime) {
StoreFileInfo info = mock(StoreFileInfo.class);
when(info.getCreatedTimestamp()).thenReturn(createdTime);
HStoreFile sf = mock(HStoreFile.class);
when(sf.getReader()).thenReturn(mock(StoreFileReader.class));
when(sf.isHFile()).thenReturn(true);
when(sf.getFileInfo()).thenReturn(info);
return sf;
}
private MyStore initMyStore(String methodName, Configuration conf, MyStoreHook hook)
throws IOException {
return (MyStore) init(methodName, conf,
TableDescriptorBuilder.newBuilder(TableName.valueOf(table)),
ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(5).build(), hook);
}
private class MyStore extends HStore {
private final MyStoreHook hook; private final MyStoreHook hook;
MyStore(final HRegion region, final HColumnDescriptor family, final Configuration confParam, MyStore(final HRegion region, final ColumnFamilyDescriptor family, final Configuration confParam,
MyStoreHook hook, boolean switchToPread) throws IOException { MyStoreHook hook, boolean switchToPread) throws IOException {
super(region, family, confParam); super(region, family, confParam);
this.hook = hook; this.hook = hook;
@ -1479,8 +1538,10 @@ public class TestHStore {
} }
private abstract class MyStoreHook { private abstract class MyStoreHook {
void getScanners(MyStore store) throws IOException { void getScanners(MyStore store) throws IOException {
} }
long getSmallestReadPoint(HStore store) { long getSmallestReadPoint(HStore store) {
return store.getHRegion().getSmallestReadPoint(); return store.getHRegion().getSmallestReadPoint();
} }
@ -1488,13 +1549,10 @@ public class TestHStore {
@Test @Test
public void testSwitchingPreadtoStreamParallelyWithCompactionDischarger() throws Exception { public void testSwitchingPreadtoStreamParallelyWithCompactionDischarger() throws Exception {
int flushSize = 500;
Configuration conf = HBaseConfiguration.create(); Configuration conf = HBaseConfiguration.create();
conf.set("hbase.hstore.engine.class", DummyStoreEngine.class.getName()); conf.set("hbase.hstore.engine.class", DummyStoreEngine.class.getName());
conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 0); conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 0);
// Set the lower threshold to invoke the "MERGE" policy // Set the lower threshold to invoke the "MERGE" policy
HColumnDescriptor hcd = new HColumnDescriptor(family);
hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
MyStore store = initMyStore(name.getMethodName(), conf, new MyStoreHook() {}); MyStore store = initMyStore(name.getMethodName(), conf, new MyStoreHook() {});
MemstoreSize memStoreSize = new MemstoreSize(); MemstoreSize memStoreSize = new MemstoreSize();
long ts = System.currentTimeMillis(); long ts = System.currentTimeMillis();
@ -1520,7 +1578,6 @@ public class TestHStore {
flushStore(store, seqID); flushStore(store, seqID);
assertEquals(3, store.getStorefilesCount()); assertEquals(3, store.getStorefilesCount());
ScanInfo scanInfo = store.getScanInfo();
Scan scan = new Scan(); Scan scan = new Scan();
scan.addFamily(family); scan.addFamily(family);
Collection<HStoreFile> storefiles2 = store.getStorefiles(); Collection<HStoreFile> storefiles2 = store.getStorefiles();
@ -1547,7 +1604,6 @@ public class TestHStore {
ArrayList<HStoreFile> actualStorefiles1 = Lists.newArrayList(storefiles2); ArrayList<HStoreFile> actualStorefiles1 = Lists.newArrayList(storefiles2);
actualStorefiles1.removeAll(actualStorefiles); actualStorefiles1.removeAll(actualStorefiles);
// Do compaction // Do compaction
List<Exception> exceptions = new ArrayList<Exception>();
MyThread thread = new MyThread(storeScanner); MyThread thread = new MyThread(storeScanner);
thread.start(); thread.start();
store.replaceStoreFiles(actualStorefiles, actualStorefiles1); store.replaceStoreFiles(actualStorefiles, actualStorefiles1);
@ -1684,7 +1740,7 @@ public class TestHStore {
public Object[] toArray() {return delegatee.toArray();} public Object[] toArray() {return delegatee.toArray();}
@Override @Override
public <T> T[] toArray(T[] a) {return delegatee.toArray(a);} public <R> R[] toArray(R[] a) {return delegatee.toArray(a);}
@Override @Override
public boolean add(T e) { public boolean add(T e) {

View File

@ -45,6 +45,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@ -205,11 +207,10 @@ public class TestHStoreFile extends HBaseTestCase {
@Test @Test
public void testEmptyStoreFileRestrictKeyRanges() throws Exception { public void testEmptyStoreFileRestrictKeyRanges() throws Exception {
StoreFileReader reader = mock(StoreFileReader.class); StoreFileReader reader = mock(StoreFileReader.class);
Store store = mock(Store.class); HStore store = mock(HStore.class);
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
byte[] cf = Bytes.toBytes("ty"); byte[] cf = Bytes.toBytes("ty");
when(hcd.getName()).thenReturn(cf); ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.of(cf);
when(store.getColumnFamilyDescriptor()).thenReturn(hcd); when(store.getColumnFamilyDescriptor()).thenReturn(cfd);
StoreFileScanner scanner = StoreFileScanner scanner =
new StoreFileScanner(reader, mock(HFileScanner.class), false, false, 0, 0, true); new StoreFileScanner(reader, mock(HFileScanner.class), false, false, 0, 0, true);
Scan scan = new Scan(); Scan scan = new Scan();
@ -526,10 +527,8 @@ public class TestHStoreFile extends HBaseTestCase {
Scan scan = new Scan(row.getBytes(),row.getBytes()); Scan scan = new Scan(row.getBytes(),row.getBytes());
scan.addColumn("family".getBytes(), "family:col".getBytes()); scan.addColumn("family".getBytes(), "family:col".getBytes());
Store store = mock(Store.class); HStore store = mock(HStore.class);
HColumnDescriptor hcd = mock(HColumnDescriptor.class); when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
when(hcd.getName()).thenReturn(Bytes.toBytes("family"));
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE); boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
if (i % 2 == 0) { if (i % 2 == 0) {
if (!exists) falseNeg++; if (!exists) falseNeg++;
@ -713,10 +712,8 @@ public class TestHStoreFile extends HBaseTestCase {
StoreFileScanner scanner = getStoreFileScanner(reader, false, false); StoreFileScanner scanner = getStoreFileScanner(reader, false, false);
assertEquals(expKeys[x], reader.generalBloomFilter.getKeyCount()); assertEquals(expKeys[x], reader.generalBloomFilter.getKeyCount());
Store store = mock(Store.class); HStore store = mock(HStore.class);
HColumnDescriptor hcd = mock(HColumnDescriptor.class); when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
when(hcd.getName()).thenReturn(Bytes.toBytes("family"));
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
// check false positives rate // check false positives rate
int falsePos = 0; int falsePos = 0;
int falseNeg = 0; int falseNeg = 0;
@ -857,10 +854,8 @@ public class TestHStoreFile extends HBaseTestCase {
HStoreFile hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf, HStoreFile hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
Store store = mock(Store.class); HStore store = mock(HStore.class);
HColumnDescriptor hcd = mock(HColumnDescriptor.class); when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of(family));
when(hcd.getName()).thenReturn(family);
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
hsf.initReader(); hsf.initReader();
StoreFileReader reader = hsf.getReader(); StoreFileReader reader = hsf.getReader();
StoreFileScanner scanner = getStoreFileScanner(reader, false, false); StoreFileScanner scanner = getStoreFileScanner(reader, false, false);

View File

@ -99,7 +99,7 @@ public class TestKeepDeletes {
// keep 3 versions, rows do not expire // keep 3 versions, rows do not expire
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3,
HConstants.FOREVER, KeepDeletedCells.TRUE); HConstants.FOREVER, KeepDeletedCells.TRUE);
Region region = hbu.createLocalHRegion(htd, null, null); HRegion region = hbu.createLocalHRegion(htd, null, null);
long ts = EnvironmentEdgeManager.currentTime(); long ts = EnvironmentEdgeManager.currentTime();
Put p = new Put(T1, ts); Put p = new Put(T1, ts);
@ -241,7 +241,7 @@ public class TestKeepDeletes {
// KEEP_DELETED_CELLS is NOT enabled // KEEP_DELETED_CELLS is NOT enabled
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3, HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3,
HConstants.FOREVER, KeepDeletedCells.FALSE); HConstants.FOREVER, KeepDeletedCells.FALSE);
Region region = hbu.createLocalHRegion(htd, null, null); HRegion region = hbu.createLocalHRegion(htd, null, null);
long ts = EnvironmentEdgeManager.currentTime(); long ts = EnvironmentEdgeManager.currentTime();
Put p = new Put(T1, ts); Put p = new Put(T1, ts);
@ -408,7 +408,7 @@ public class TestKeepDeletes {
public void testDeleteMarkerExpirationEmptyStore() throws Exception { public void testDeleteMarkerExpirationEmptyStore() throws Exception {
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1,
HConstants.FOREVER, KeepDeletedCells.TRUE); HConstants.FOREVER, KeepDeletedCells.TRUE);
Region region = hbu.createLocalHRegion(htd, null, null); HRegion region = hbu.createLocalHRegion(htd, null, null);
long ts = EnvironmentEdgeManager.currentTime(); long ts = EnvironmentEdgeManager.currentTime();
@ -451,7 +451,7 @@ public class TestKeepDeletes {
public void testDeleteMarkerExpiration() throws Exception { public void testDeleteMarkerExpiration() throws Exception {
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1,
HConstants.FOREVER, KeepDeletedCells.TRUE); HConstants.FOREVER, KeepDeletedCells.TRUE);
Region region = hbu.createLocalHRegion(htd, null, null); HRegion region = hbu.createLocalHRegion(htd, null, null);
long ts = EnvironmentEdgeManager.currentTime(); long ts = EnvironmentEdgeManager.currentTime();
@ -514,7 +514,7 @@ public class TestKeepDeletes {
public void testWithOldRow() throws Exception { public void testWithOldRow() throws Exception {
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1,
HConstants.FOREVER, KeepDeletedCells.TRUE); HConstants.FOREVER, KeepDeletedCells.TRUE);
Region region = hbu.createLocalHRegion(htd, null, null); HRegion region = hbu.createLocalHRegion(htd, null, null);
long ts = EnvironmentEdgeManager.currentTime(); long ts = EnvironmentEdgeManager.currentTime();
@ -674,7 +674,7 @@ public class TestKeepDeletes {
public void testDeleteMarkerVersioning() throws Exception { public void testDeleteMarkerVersioning() throws Exception {
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1, HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1,
HConstants.FOREVER, KeepDeletedCells.TRUE); HConstants.FOREVER, KeepDeletedCells.TRUE);
Region region = hbu.createLocalHRegion(htd, null, null); HRegion region = hbu.createLocalHRegion(htd, null, null);
long ts = EnvironmentEdgeManager.currentTime(); long ts = EnvironmentEdgeManager.currentTime();
Put p = new Put(T1, ts); Put p = new Put(T1, ts);
@ -818,7 +818,7 @@ public class TestKeepDeletes {
public void testWithMinVersions() throws Exception { public void testWithMinVersions() throws Exception {
HTableDescriptor htd = HTableDescriptor htd =
hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.TRUE); hbu.createTableDescriptor(name.getMethodName(), 3, 1000, 1, KeepDeletedCells.TRUE);
Region region = hbu.createLocalHRegion(htd, null, null); HRegion region = hbu.createLocalHRegion(htd, null, null);
long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past
@ -897,7 +897,7 @@ public class TestKeepDeletes {
public void testWithTTL() throws Exception { public void testWithTTL() throws Exception {
HTableDescriptor htd = HTableDescriptor htd =
hbu.createTableDescriptor(name.getMethodName(), 1, 1000, 1, KeepDeletedCells.TTL); hbu.createTableDescriptor(name.getMethodName(), 1, 1000, 1, KeepDeletedCells.TTL);
Region region = hbu.createLocalHRegion(htd, null, null); HRegion region = hbu.createLocalHRegion(htd, null, null);
long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past long ts = EnvironmentEdgeManager.currentTime() - 2000; // 2s in the past
@ -945,7 +945,7 @@ public class TestKeepDeletes {
} }
private int countDeleteMarkers(Region region) throws IOException { private int countDeleteMarkers(HRegion region) throws IOException {
Scan s = new Scan(); Scan s = new Scan();
s.setRaw(true); s.setRaw(true);
// use max versions from the store(s) // use max versions from the store(s)

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY; import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES; import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES;
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
@ -181,8 +182,8 @@ public class TestMajorCompaction {
public void majorCompactionWithDataBlockEncoding(boolean inCacheOnly) public void majorCompactionWithDataBlockEncoding(boolean inCacheOnly)
throws Exception { throws Exception {
Map<Store, HFileDataBlockEncoder> replaceBlockCache = new HashMap<>(); Map<HStore, HFileDataBlockEncoder> replaceBlockCache = new HashMap<>();
for (Store store : r.getStores()) { for (HStore store : r.getStores()) {
HFileDataBlockEncoder blockEncoder = store.getDataBlockEncoder(); HFileDataBlockEncoder blockEncoder = store.getDataBlockEncoder();
replaceBlockCache.put(store, blockEncoder); replaceBlockCache.put(store, blockEncoder);
final DataBlockEncoding inCache = DataBlockEncoding.PREFIX; final DataBlockEncoding inCache = DataBlockEncoding.PREFIX;
@ -194,7 +195,7 @@ public class TestMajorCompaction {
majorCompaction(); majorCompaction();
// restore settings // restore settings
for (Entry<Store, HFileDataBlockEncoder> entry : replaceBlockCache.entrySet()) { for (Entry<HStore, HFileDataBlockEncoder> entry : replaceBlockCache.entrySet()) {
((HStore)entry.getKey()).setDataBlockEncoderInTest(entry.getValue()); ((HStore)entry.getKey()).setDataBlockEncoderInTest(entry.getValue());
} }
} }
@ -211,11 +212,11 @@ public class TestMajorCompaction {
// Default is that there only 3 (MAXVERSIONS) versions allowed per column. // Default is that there only 3 (MAXVERSIONS) versions allowed per column.
// //
// Assert == 3 when we ask for versions. // Assert == 3 when we ask for versions.
Result result = r.get(new Get(STARTROW).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100)); Result result = r.get(new Get(STARTROW).addFamily(COLUMN_FAMILY_TEXT).readVersions(100));
assertEquals(compactionThreshold, result.size()); assertEquals(compactionThreshold, result.size());
// see if CompactionProgress is in place but null // see if CompactionProgress is in place but null
for (Store store : r.getStores()) { for (HStore store : r.getStores()) {
assertNull(store.getCompactionProgress()); assertNull(store.getCompactionProgress());
} }
@ -224,7 +225,7 @@ public class TestMajorCompaction {
// see if CompactionProgress has done its thing on at least one store // see if CompactionProgress has done its thing on at least one store
int storeCount = 0; int storeCount = 0;
for (Store store : r.getStores()) { for (HStore store : r.getStores()) {
CompactionProgress progress = store.getCompactionProgress(); CompactionProgress progress = store.getCompactionProgress();
if( progress != null ) { if( progress != null ) {
++storeCount; ++storeCount;
@ -240,8 +241,7 @@ public class TestMajorCompaction {
secondRowBytes[START_KEY_BYTES.length - 1]++; secondRowBytes[START_KEY_BYTES.length - 1]++;
// Always 3 versions if that is what max versions is. // Always 3 versions if that is what max versions is.
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT). result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).readVersions(100));
setMaxVersions(100));
LOG.debug("Row " + Bytes.toStringBinary(secondRowBytes) + " after " + LOG.debug("Row " + Bytes.toStringBinary(secondRowBytes) + " after " +
"initial compaction: " + result); "initial compaction: " + result);
assertEquals("Invalid number of versions of row " assertEquals("Invalid number of versions of row "
@ -260,26 +260,26 @@ public class TestMajorCompaction {
r.delete(delete); r.delete(delete);
// Assert deleted. // Assert deleted.
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100)); result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).readVersions(100));
assertTrue("Second row should have been deleted", result.isEmpty()); assertTrue("Second row should have been deleted", result.isEmpty());
r.flush(true); r.flush(true);
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100)); result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).readVersions(100));
assertTrue("Second row should have been deleted", result.isEmpty()); assertTrue("Second row should have been deleted", result.isEmpty());
// Add a bit of data and flush. Start adding at 'bbb'. // Add a bit of data and flush. Start adding at 'bbb'.
createSmallerStoreFile(this.r); createSmallerStoreFile(this.r);
r.flush(true); r.flush(true);
// Assert that the second row is still deleted. // Assert that the second row is still deleted.
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100)); result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).readVersions(100));
assertTrue("Second row should still be deleted", result.isEmpty()); assertTrue("Second row should still be deleted", result.isEmpty());
// Force major compaction. // Force major compaction.
r.compact(true); r.compact(true);
assertEquals(r.getStore(COLUMN_FAMILY_TEXT).getStorefiles().size(), 1); assertEquals(r.getStore(COLUMN_FAMILY_TEXT).getStorefiles().size(), 1);
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).setMaxVersions(100)); result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).readVersions(100));
assertTrue("Second row should still be deleted", result.isEmpty()); assertTrue("Second row should still be deleted", result.isEmpty());
// Make sure the store files do have some 'aaa' keys in them -- exactly 3. // Make sure the store files do have some 'aaa' keys in them -- exactly 3.
@ -290,8 +290,7 @@ public class TestMajorCompaction {
// Multiple versions allowed for an entry, so the delete isn't enough // 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 // Lower TTL and expire to ensure that all our entries have been wiped
final int ttl = 1000; final int ttl = 1000;
for (Store hstore : r.getStores()) { for (HStore store : r.getStores()) {
HStore store = ((HStore) hstore);
ScanInfo old = store.getScanInfo(); ScanInfo old = store.getScanInfo();
ScanInfo si = new ScanInfo(old.getConfiguration(), old.getFamily(), old.getMinVersions(), ScanInfo si = new ScanInfo(old.getConfiguration(), old.getFamily(), old.getMinVersions(),
old.getMaxVersions(), ttl, old.getKeepDeletedCells(), old.getPreadMaxBytes(), 0, old.getMaxVersions(), ttl, old.getKeepDeletedCells(), old.getPreadMaxBytes(), 0,
@ -411,7 +410,7 @@ public class TestMajorCompaction {
*/ */
@Test @Test
public void testNonUserMajorCompactionRequest() throws Exception { public void testNonUserMajorCompactionRequest() throws Exception {
Store store = r.getStore(COLUMN_FAMILY); HStore store = r.getStore(COLUMN_FAMILY);
createStoreFile(r); createStoreFile(r);
for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) { for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
createStoreFile(r); createStoreFile(r);
@ -431,14 +430,14 @@ public class TestMajorCompaction {
*/ */
@Test @Test
public void testUserMajorCompactionRequest() throws IOException{ public void testUserMajorCompactionRequest() throws IOException{
Store store = r.getStore(COLUMN_FAMILY); HStore store = r.getStore(COLUMN_FAMILY);
createStoreFile(r); createStoreFile(r);
for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) { for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
createStoreFile(r); createStoreFile(r);
} }
store.triggerMajorCompaction(); store.triggerMajorCompaction();
CompactionRequest request = CompactionRequest request =
store.requestCompaction(Store.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null).get() store.requestCompaction(PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null).get()
.getRequest(); .getRequest();
assertNotNull("Expected to receive a compaction request", request); assertNotNull("Expected to receive a compaction request", request);
assertEquals( assertEquals(

View File

@ -31,14 +31,14 @@ import org.apache.hadoop.hbase.HBaseTestCase;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.wal.WAL; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.wal.WAL;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule; import org.junit.Rule;
@ -46,7 +46,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName; import org.junit.rules.TestName;
/** /**
* Test minor compactions * Test minor compactions
*/ */
@ -57,7 +56,7 @@ public class TestMinorCompaction {
private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU(); private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU();
protected Configuration conf = UTIL.getConfiguration(); protected Configuration conf = UTIL.getConfiguration();
private Region r = null; private HRegion r = null;
private HTableDescriptor htd = null; private HTableDescriptor htd = null;
private int compactionThreshold; private int compactionThreshold;
private byte[] firstRowBytes, secondRowBytes, thirdRowBytes; private byte[] firstRowBytes, secondRowBytes, thirdRowBytes;
@ -205,7 +204,7 @@ public class TestMinorCompaction {
assertEquals(compactionThreshold, result.size()); assertEquals(compactionThreshold, result.size());
// do a compaction // do a compaction
Store store2 = r.getStore(fam2); HStore store2 = r.getStore(fam2);
int numFiles1 = store2.getStorefiles().size(); int numFiles1 = store2.getStorefiles().size();
assertTrue("Was expecting to see 4 store files", numFiles1 > compactionThreshold); // > 3 assertTrue("Was expecting to see 4 store files", numFiles1 > compactionThreshold); // > 3
((HStore)store2).compactRecentForTestingAssumingDefaultPolicy(compactionThreshold); // = 3 ((HStore)store2).compactRecentForTestingAssumingDefaultPolicy(compactionThreshold); // = 3

View File

@ -298,7 +298,7 @@ public class TestMobStoreCompaction {
} }
private int countStoreFiles() throws IOException { private int countStoreFiles() throws IOException {
Store store = region.getStore(COLUMN_FAMILY); HStore store = region.getStore(COLUMN_FAMILY);
return store.getStorefilesCount(); return store.getStorefilesCount();
} }

View File

@ -152,9 +152,9 @@ public class TestPerColumnFamilyFlush {
long smallestSeqCF3 = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3 = region.getOldestSeqIdOfStore(FAMILY3);
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
// Get the overall smallest LSN in the region's memstores. // Get the overall smallest LSN in the region's memstores.
long smallestSeqInRegionCurrentMemstore = getWAL(region) long smallestSeqInRegionCurrentMemstore = getWAL(region)
@ -184,9 +184,9 @@ public class TestPerColumnFamilyFlush {
MemstoreSize oldCF3MemstoreSize = cf3MemstoreSize; MemstoreSize oldCF3MemstoreSize = cf3MemstoreSize;
// Recalculate everything // Recalculate everything
cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore(); cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore(); cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore(); cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
totalMemstoreSize = region.getMemstoreSize(); totalMemstoreSize = region.getMemstoreSize();
smallestSeqInRegionCurrentMemstore = getWAL(region) smallestSeqInRegionCurrentMemstore = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -216,15 +216,15 @@ public class TestPerColumnFamilyFlush {
} }
// How much does the CF3 memstore occupy? Will be used later. // How much does the CF3 memstore occupy? Will be used later.
oldCF3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore(); oldCF3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
// Flush again // Flush again
region.flush(false); region.flush(false);
// Recalculate everything // Recalculate everything
cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore(); cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore(); cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore(); cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
totalMemstoreSize = region.getMemstoreSize(); totalMemstoreSize = region.getMemstoreSize();
smallestSeqInRegionCurrentMemstore = getWAL(region) smallestSeqInRegionCurrentMemstore = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -288,9 +288,9 @@ public class TestPerColumnFamilyFlush {
long totalMemstoreSize = region.getMemstoreSize(); long totalMemstoreSize = region.getMemstoreSize();
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
// Some other sanity checks. // Some other sanity checks.
assertTrue(cf1MemstoreSize.getDataSize() > 0); assertTrue(cf1MemstoreSize.getDataSize() > 0);
@ -305,9 +305,9 @@ public class TestPerColumnFamilyFlush {
// Flush! // Flush!
region.flush(false); region.flush(false);
cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore(); cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore(); cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore(); cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
totalMemstoreSize = region.getMemstoreSize(); totalMemstoreSize = region.getMemstoreSize();
long smallestSeqInRegionCurrentMemstore = long smallestSeqInRegionCurrentMemstore =
region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -380,9 +380,9 @@ public class TestPerColumnFamilyFlush {
totalMemstoreSize = desiredRegion.getMemstoreSize(); totalMemstoreSize = desiredRegion.getMemstoreSize();
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
cf1MemstoreSize = desiredRegion.getStore(FAMILY1).getSizeOfMemStore().getDataSize(); cf1MemstoreSize = desiredRegion.getStore(FAMILY1).getMemStoreSize().getDataSize();
cf2MemstoreSize = desiredRegion.getStore(FAMILY2).getSizeOfMemStore().getDataSize(); cf2MemstoreSize = desiredRegion.getStore(FAMILY2).getMemStoreSize().getDataSize();
cf3MemstoreSize = desiredRegion.getStore(FAMILY3).getSizeOfMemStore().getDataSize(); cf3MemstoreSize = desiredRegion.getStore(FAMILY3).getMemStoreSize().getDataSize();
// CF1 Should have been flushed // CF1 Should have been flushed
assertEquals(0, cf1MemstoreSize); assertEquals(0, cf1MemstoreSize);
@ -500,9 +500,9 @@ public class TestPerColumnFamilyFlush {
} }
} }
assertEquals(maxLogs, getNumRolledLogFiles(desiredRegion)); assertEquals(maxLogs, getNumRolledLogFiles(desiredRegion));
assertTrue(desiredRegion.getStore(FAMILY1).getMemStoreSize() > cfFlushSizeLowerBound); assertTrue(desiredRegion.getStore(FAMILY1).getMemStoreSize().getHeapSize() > cfFlushSizeLowerBound);
assertTrue(desiredRegion.getStore(FAMILY2).getMemStoreSize() < cfFlushSizeLowerBound); assertTrue(desiredRegion.getStore(FAMILY2).getMemStoreSize().getHeapSize() < cfFlushSizeLowerBound);
assertTrue(desiredRegion.getStore(FAMILY3).getMemStoreSize() < cfFlushSizeLowerBound); assertTrue(desiredRegion.getStore(FAMILY3).getMemStoreSize().getHeapSize() < cfFlushSizeLowerBound);
table.put(createPut(1, 12345678)); table.put(createPut(1, 12345678));
// Make numRolledLogFiles greater than maxLogs // Make numRolledLogFiles greater than maxLogs
desiredRegionAndServer.getSecond().walRoller.requestRollAll(); desiredRegionAndServer.getSecond().walRoller.requestRollAll();
@ -525,9 +525,12 @@ public class TestPerColumnFamilyFlush {
}); });
LOG.info("Finished waiting on flush after too many WALs..."); LOG.info("Finished waiting on flush after too many WALs...");
// Individual families should have been flushed. // Individual families should have been flushed.
assertEquals(MutableSegment.DEEP_OVERHEAD, desiredRegion.getStore(FAMILY1).getMemStoreSize()); assertEquals(MutableSegment.DEEP_OVERHEAD,
assertEquals(MutableSegment.DEEP_OVERHEAD, desiredRegion.getStore(FAMILY2).getMemStoreSize()); desiredRegion.getStore(FAMILY1).getMemStoreSize().getHeapSize());
assertEquals(MutableSegment.DEEP_OVERHEAD, desiredRegion.getStore(FAMILY3).getMemStoreSize()); assertEquals(MutableSegment.DEEP_OVERHEAD,
desiredRegion.getStore(FAMILY2).getMemStoreSize().getHeapSize());
assertEquals(MutableSegment.DEEP_OVERHEAD,
desiredRegion.getStore(FAMILY3).getMemStoreSize().getHeapSize());
// let WAL cleanOldLogs // let WAL cleanOldLogs
assertNull(getWAL(desiredRegion).rollWriter(true)); assertNull(getWAL(desiredRegion).rollWriter(true));
assertTrue(getNumRolledLogFiles(desiredRegion) < maxLogs); assertTrue(getNumRolledLogFiles(desiredRegion) < maxLogs);

View File

@ -50,7 +50,7 @@ public class TestRegionSplitPolicy {
private Configuration conf; private Configuration conf;
private HTableDescriptor htd; private HTableDescriptor htd;
private HRegion mockRegion; private HRegion mockRegion;
private List<Store> stores; private List<HStore> stores;
private static final TableName TABLENAME = TableName.valueOf("t"); private static final TableName TABLENAME = TableName.valueOf("t");
@Rule @Rule

View File

@ -169,9 +169,9 @@ public class TestWalAndCompactingMemStoreFlush {
long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3);
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize();
// Get the overall smallest LSN in the region's memstores. // Get the overall smallest LSN in the region's memstores.
long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region)
@ -220,9 +220,9 @@ public class TestWalAndCompactingMemStoreFlush {
region.flush(false); region.flush(false);
// Recalculate everything // Recalculate everything
MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize();
long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -264,7 +264,7 @@ public class TestWalAndCompactingMemStoreFlush {
+ smallestSeqCF2PhaseII +", the smallest sequence in CF3:" + smallestSeqCF3PhaseII + "\n"; + smallestSeqCF2PhaseII +", the smallest sequence in CF3:" + smallestSeqCF3PhaseII + "\n";
// How much does the CF1 memstore occupy? Will be used later. // How much does the CF1 memstore occupy? Will be used later.
MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize();
long smallestSeqCF1PhaseIII = region.getOldestSeqIdOfStore(FAMILY1); long smallestSeqCF1PhaseIII = region.getOldestSeqIdOfStore(FAMILY1);
s = s + "----After more puts into CF1 its size is:" + cf1MemstoreSizePhaseIII s = s + "----After more puts into CF1 its size is:" + cf1MemstoreSizePhaseIII
@ -278,9 +278,9 @@ public class TestWalAndCompactingMemStoreFlush {
region.flush(false); region.flush(false);
// Recalculate everything // Recalculate everything
MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize();
long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -319,9 +319,9 @@ public class TestWalAndCompactingMemStoreFlush {
region.flush(true); region.flush(true);
// Recalculate everything // Recalculate everything
MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize();
long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -410,9 +410,9 @@ public class TestWalAndCompactingMemStoreFlush {
long smallestSeqCF2PhaseI = region.getOldestSeqIdOfStore(FAMILY2); long smallestSeqCF2PhaseI = region.getOldestSeqIdOfStore(FAMILY2);
long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3);
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize();
// Get the overall smallest LSN in the region's memstores. // Get the overall smallest LSN in the region's memstores.
long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -460,9 +460,9 @@ public class TestWalAndCompactingMemStoreFlush {
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE II - collect sizes */ /* PHASE II - collect sizes */
// Recalculate everything // Recalculate everything
MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize();
long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
// Find the smallest LSNs for edits wrt to each CF. // Find the smallest LSNs for edits wrt to each CF.
@ -509,7 +509,7 @@ public class TestWalAndCompactingMemStoreFlush {
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE III - collect sizes */ /* PHASE III - collect sizes */
// How much does the CF1 memstore occupy now? Will be used later. // How much does the CF1 memstore occupy now? Will be used later.
MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize();
long totalMemstoreSizePhaseIII = region.getMemstoreSize(); long totalMemstoreSizePhaseIII = region.getMemstoreSize();
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
@ -531,9 +531,9 @@ public class TestWalAndCompactingMemStoreFlush {
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE IV - collect sizes */ /* PHASE IV - collect sizes */
// Recalculate everything // Recalculate everything
MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize();
long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3); long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3);
@ -563,9 +563,9 @@ public class TestWalAndCompactingMemStoreFlush {
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE V - collect sizes */ /* PHASE V - collect sizes */
// Recalculate everything // Recalculate everything
MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize();
long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region) long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region)
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
long totalMemstoreSizePhaseV = region.getMemstoreSize(); long totalMemstoreSizePhaseV = region.getMemstoreSize();
@ -601,9 +601,9 @@ public class TestWalAndCompactingMemStoreFlush {
region.put(createPut(5, i)); region.put(createPut(5, i));
} }
MemstoreSize cf1ActiveSizePhaseVI = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1ActiveSizePhaseVI = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf3ActiveSizePhaseVI = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3ActiveSizePhaseVI = region.getStore(FAMILY3).getMemStoreSize();
MemstoreSize cf5ActiveSizePhaseVI = region.getStore(FAMILIES[4]).getSizeOfMemStore(); MemstoreSize cf5ActiveSizePhaseVI = region.getStore(FAMILIES[4]).getMemStoreSize();
/*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/
/* PHASE VI - Flush */ /* PHASE VI - Flush */
@ -614,9 +614,9 @@ public class TestWalAndCompactingMemStoreFlush {
// Since we won't find any CF above the threshold, and hence no specific // Since we won't find any CF above the threshold, and hence no specific
// store to flush, we should flush all the memstores // store to flush, we should flush all the memstores
// Also compacted memstores are flushed to disk, but not entirely emptied // Also compacted memstores are flushed to disk, but not entirely emptied
MemstoreSize cf1ActiveSizePhaseVII = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1ActiveSizePhaseVII = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf3ActiveSizePhaseVII = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3ActiveSizePhaseVII = region.getStore(FAMILY3).getMemStoreSize();
MemstoreSize cf5ActiveSizePhaseVII = region.getStore(FAMILIES[4]).getSizeOfMemStore(); MemstoreSize cf5ActiveSizePhaseVII = region.getStore(FAMILIES[4]).getMemStoreSize();
assertTrue(cf1ActiveSizePhaseVII.getDataSize() < cf1ActiveSizePhaseVI.getDataSize()); assertTrue(cf1ActiveSizePhaseVII.getDataSize() < cf1ActiveSizePhaseVI.getDataSize());
assertTrue(cf3ActiveSizePhaseVII.getDataSize() < cf3ActiveSizePhaseVI.getDataSize()); assertTrue(cf3ActiveSizePhaseVII.getDataSize() < cf3ActiveSizePhaseVI.getDataSize());
@ -663,9 +663,9 @@ public class TestWalAndCompactingMemStoreFlush {
long totalMemstoreSize = region.getMemstoreSize(); long totalMemstoreSize = region.getMemstoreSize();
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize();
// Some other sanity checks. // Some other sanity checks.
assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0); assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0);
@ -689,7 +689,7 @@ public class TestWalAndCompactingMemStoreFlush {
cms3.flushInMemory(); cms3.flushInMemory();
region.flush(false); region.flush(false);
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize();
long smallestSeqInRegionCurrentMemstorePhaseII = long smallestSeqInRegionCurrentMemstorePhaseII =
region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
@ -798,9 +798,9 @@ public class TestWalAndCompactingMemStoreFlush {
long totalMemstoreSize = region.getMemstoreSize(); long totalMemstoreSize = region.getMemstoreSize();
// Find the sizes of the memstores of each CF. // Find the sizes of the memstores of each CF.
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize();
// Some other sanity checks. // Some other sanity checks.
assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0); assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0);
@ -829,9 +829,9 @@ public class TestWalAndCompactingMemStoreFlush {
// Flush-to-disk! CF2 only should be flushed // Flush-to-disk! CF2 only should be flushed
region.flush(false); region.flush(false);
MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize();
MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getSizeOfMemStore(); MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize();
// CF1 should be flushed in memory and just flattened, so CF1 heap overhead should be smaller // CF1 should be flushed in memory and just flattened, so CF1 heap overhead should be smaller
assertTrue(cf1MemstoreSizePhaseI.getHeapSize() > cf1MemstoreSizePhaseII.getHeapSize()); assertTrue(cf1MemstoreSizePhaseI.getHeapSize() > cf1MemstoreSizePhaseII.getHeapSize());
@ -855,7 +855,7 @@ public class TestWalAndCompactingMemStoreFlush {
region.put(createPut(2, i)); region.put(createPut(2, i));
} }
MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize();
// Flush in memory! // Flush in memory!
((CompactingMemStore) ((HStore)region.getStore(FAMILY1)).memstore).flushInMemory(); ((CompactingMemStore) ((HStore)region.getStore(FAMILY1)).memstore).flushInMemory();
@ -871,8 +871,8 @@ public class TestWalAndCompactingMemStoreFlush {
} }
region.flush(false); region.flush(false);
MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getSizeOfMemStore(); MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize();
MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getSizeOfMemStore(); MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize();
assertEquals(2*cf1MemstoreSizePhaseI.getDataSize(), cf1MemstoreSizePhaseIV.getDataSize()); assertEquals(2*cf1MemstoreSizePhaseI.getDataSize(), cf1MemstoreSizePhaseIV.getDataSize());
// the decrease in the heap size due to usage of CellArrayMap instead of CSLM // the decrease in the heap size due to usage of CellArrayMap instead of CSLM

View File

@ -31,6 +31,7 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.OptionalLong;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@ -104,7 +105,7 @@ public class TestDateTieredCompactor {
when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(), when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers); anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
when(store.getComparator()).thenReturn(CellComparator.COMPARATOR); when(store.getComparator()).thenReturn(CellComparator.COMPARATOR);
long maxSequenceId = StoreUtils.getMaxSequenceIdInList(storefiles); OptionalLong maxSequenceId = StoreUtils.getMaxSequenceIdInList(storefiles);
when(store.getMaxSequenceId()).thenReturn(maxSequenceId); when(store.getMaxSequenceId()).thenReturn(maxSequenceId);
return new DateTieredCompactor(conf, store) { return new DateTieredCompactor(conf, store) {

View File

@ -26,20 +26,21 @@ import java.util.Random;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -65,32 +66,31 @@ public class TestFIFOCompactionPolicy {
private final byte[] qualifier = Bytes.toBytes("q"); private final byte[] qualifier = Bytes.toBytes("q");
private Store getStoreWithName(TableName tableName) { private HStore getStoreWithName(TableName tableName) {
MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster(); MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads(); List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
HRegionServer hrs = rsts.get(i).getRegionServer(); HRegionServer hrs = rsts.get(i).getRegionServer();
for (Region region : hrs.getRegions(tableName)) { for (Region region : hrs.getRegions(tableName)) {
return region.getStores().iterator().next(); return ((HRegion) region).getStores().iterator().next();
} }
} }
return null; return null;
} }
private Store prepareData() throws IOException { private HStore prepareData() throws IOException {
Admin admin = TEST_UTIL.getAdmin(); Admin admin = TEST_UTIL.getAdmin();
if (admin.tableExists(tableName)) { if (admin.tableExists(tableName)) {
admin.disableTable(tableName); admin.disableTable(tableName);
admin.deleteTable(tableName); admin.deleteTable(tableName);
} }
HTableDescriptor desc = new HTableDescriptor(tableName); TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, .setValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
FIFOCompactionPolicy.class.getName()); FIFOCompactionPolicy.class.getName())
desc.setConfiguration(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, .setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
DisabledRegionSplitPolicy.class.getName()); DisabledRegionSplitPolicy.class.getName())
HColumnDescriptor colDesc = new HColumnDescriptor(family); .addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setTimeToLive(1).build())
colDesc.setTimeToLive(1); // 1 sec .build();
desc.addFamily(colDesc);
admin.createTable(desc); admin.createTable(desc);
Table table = TEST_UTIL.getConnection().getTable(tableName); Table table = TEST_UTIL.getConnection().getTable(tableName);
@ -129,7 +129,7 @@ public class TestFIFOCompactionPolicy {
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
try { try {
Store store = prepareData(); HStore store = prepareData();
assertEquals(10, store.getStorefilesCount()); assertEquals(10, store.getStorefilesCount());
TEST_UTIL.getAdmin().majorCompact(tableName); TEST_UTIL.getAdmin().majorCompact(tableName);
while (store.getStorefilesCount() > 1) { while (store.getStorefilesCount() > 1) {
@ -141,9 +141,8 @@ public class TestFIFOCompactionPolicy {
} }
} }
@Test @Test
public void testSanityCheckTTL() throws Exception public void testSanityCheckTTL() throws Exception {
{
Configuration conf = TEST_UTIL.getConfiguration(); Configuration conf = TEST_UTIL.getConfiguration();
conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10000); conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10000);
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
@ -154,25 +153,23 @@ public class TestFIFOCompactionPolicy {
admin.disableTable(tableName); admin.disableTable(tableName);
admin.deleteTable(tableName); admin.deleteTable(tableName);
} }
HTableDescriptor desc = new HTableDescriptor(tableName); TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, .setValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
FIFOCompactionPolicy.class.getName()); FIFOCompactionPolicy.class.getName())
desc.setConfiguration(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, .setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
DisabledRegionSplitPolicy.class.getName()); DisabledRegionSplitPolicy.class.getName())
HColumnDescriptor colDesc = new HColumnDescriptor(family); .addColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build();
desc.addFamily(colDesc); try {
try{
admin.createTable(desc); admin.createTable(desc);
Assert.fail(); Assert.fail();
}catch(Exception e){ } catch (Exception e) {
}finally{ } finally {
TEST_UTIL.shutdownMiniCluster(); TEST_UTIL.shutdownMiniCluster();
} }
} }
@Test @Test
public void testSanityCheckMinVersion() throws Exception public void testSanityCheckMinVersion() throws Exception {
{
Configuration conf = TEST_UTIL.getConfiguration(); Configuration conf = TEST_UTIL.getConfiguration();
conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10000); conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10000);
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
@ -183,27 +180,25 @@ public class TestFIFOCompactionPolicy {
admin.disableTable(tableName); admin.disableTable(tableName);
admin.deleteTable(tableName); admin.deleteTable(tableName);
} }
HTableDescriptor desc = new HTableDescriptor(tableName); TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, .setValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
FIFOCompactionPolicy.class.getName()); FIFOCompactionPolicy.class.getName())
desc.setConfiguration(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, .setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
DisabledRegionSplitPolicy.class.getName()); DisabledRegionSplitPolicy.class.getName())
HColumnDescriptor colDesc = new HColumnDescriptor(family); .addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setTimeToLive(1)
colDesc.setTimeToLive(1); // 1 sec .setMinVersions(1).build())
colDesc.setMinVersions(1); .build();
desc.addFamily(colDesc); try {
try{
admin.createTable(desc); admin.createTable(desc);
Assert.fail(); Assert.fail();
}catch(Exception e){ } catch (Exception e) {
}finally{ } finally {
TEST_UTIL.shutdownMiniCluster(); TEST_UTIL.shutdownMiniCluster();
} }
} }
@Test @Test
public void testSanityCheckBlockingStoreFiles() throws Exception public void testSanityCheckBlockingStoreFiles() throws Exception {
{
Configuration conf = TEST_UTIL.getConfiguration(); Configuration conf = TEST_UTIL.getConfiguration();
conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10); conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10);
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
@ -214,19 +209,18 @@ public class TestFIFOCompactionPolicy {
admin.disableTable(tableName); admin.disableTable(tableName);
admin.deleteTable(tableName); admin.deleteTable(tableName);
} }
HTableDescriptor desc = new HTableDescriptor(tableName); TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY, .setValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
FIFOCompactionPolicy.class.getName()); FIFOCompactionPolicy.class.getName())
desc.setConfiguration(HConstants.HBASE_REGION_SPLIT_POLICY_KEY, .setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
DisabledRegionSplitPolicy.class.getName()); DisabledRegionSplitPolicy.class.getName())
HColumnDescriptor colDesc = new HColumnDescriptor(family); .addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setTimeToLive(1).build())
colDesc.setTimeToLive(1); // 1 sec .build();
desc.addFamily(colDesc); try {
try{
admin.createTable(desc); admin.createTable(desc);
Assert.fail(); Assert.fail();
}catch(Exception e){ } catch (Exception e) {
}finally{ } finally {
TEST_UTIL.shutdownMiniCluster(); TEST_UTIL.shutdownMiniCluster();
} }
} }

View File

@ -22,33 +22,30 @@ import static org.junit.Assert.assertTrue;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.Random; import java.util.concurrent.ThreadLocalRandom;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreEngine; import org.apache.hadoop.hbase.regionserver.StoreEngine;
import org.apache.hadoop.hbase.regionserver.StripeStoreConfig; import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
import org.apache.hadoop.hbase.regionserver.StripeStoreEngine; import org.apache.hadoop.hbase.regionserver.StripeStoreEngine;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.regionserver.throttle.PressureAwareCompactionThroughputController;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -71,30 +68,29 @@ public class TestCompactionWithThroughputController {
private final byte[] qualifier = Bytes.toBytes("q"); private final byte[] qualifier = Bytes.toBytes("q");
private Store getStoreWithName(TableName tableName) { private HStore getStoreWithName(TableName tableName) {
MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster(); MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads(); List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
HRegionServer hrs = rsts.get(i).getRegionServer(); HRegionServer hrs = rsts.get(i).getRegionServer();
for (Region region : hrs.getRegions(tableName)) { for (Region region : hrs.getRegions(tableName)) {
return region.getStores().iterator().next(); return ((HRegion) region).getStores().iterator().next();
} }
} }
return null; return null;
} }
private Store prepareData() throws IOException { private HStore prepareData() throws IOException {
Admin admin = TEST_UTIL.getAdmin(); Admin admin = TEST_UTIL.getAdmin();
if (admin.tableExists(tableName)) { if (admin.tableExists(tableName)) {
admin.disableTable(tableName); admin.disableTable(tableName);
admin.deleteTable(tableName); admin.deleteTable(tableName);
} }
Table table = TEST_UTIL.createTable(tableName, family); Table table = TEST_UTIL.createTable(tableName, family);
Random rand = new Random();
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
for (int j = 0; j < 10; j++) { for (int j = 0; j < 10; j++) {
byte[] value = new byte[128 * 1024]; byte[] value = new byte[128 * 1024];
rand.nextBytes(value); ThreadLocalRandom.current().nextBytes(value);
table.put(new Put(Bytes.toBytes(i * 10 + j)).addColumn(family, qualifier, value)); table.put(new Put(Bytes.toBytes(i * 10 + j)).addColumn(family, qualifier, value));
} }
admin.flush(tableName); admin.flush(tableName);
@ -121,7 +117,7 @@ public class TestCompactionWithThroughputController {
PressureAwareCompactionThroughputController.class.getName()); PressureAwareCompactionThroughputController.class.getName());
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
try { try {
Store store = prepareData(); HStore store = prepareData();
assertEquals(10, store.getStorefilesCount()); assertEquals(10, store.getStorefilesCount());
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
TEST_UTIL.getAdmin().majorCompact(tableName); TEST_UTIL.getAdmin().majorCompact(tableName);
@ -150,7 +146,7 @@ public class TestCompactionWithThroughputController {
NoLimitThroughputController.class.getName()); NoLimitThroughputController.class.getName());
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
try { try {
Store store = prepareData(); HStore store = prepareData();
assertEquals(10, store.getStorefilesCount()); assertEquals(10, store.getStorefilesCount());
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
TEST_UTIL.getAdmin().majorCompact(tableName); TEST_UTIL.getAdmin().majorCompact(tableName);
@ -199,10 +195,10 @@ public class TestCompactionWithThroughputController {
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
Connection conn = ConnectionFactory.createConnection(conf); Connection conn = ConnectionFactory.createConnection(conf);
try { try {
HTableDescriptor htd = new HTableDescriptor(tableName); TEST_UTIL.getAdmin()
htd.addFamily(new HColumnDescriptor(family)); .createTable(TableDescriptorBuilder.newBuilder(tableName)
htd.setCompactionEnabled(false); .addColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCompactionEnabled(false)
TEST_UTIL.getAdmin().createTable(htd); .build());
TEST_UTIL.waitTableAvailable(tableName); TEST_UTIL.waitTableAvailable(tableName);
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName); HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
PressureAwareCompactionThroughputController throughputController = PressureAwareCompactionThroughputController throughputController =
@ -256,12 +252,12 @@ public class TestCompactionWithThroughputController {
TEST_UTIL.startMiniCluster(1); TEST_UTIL.startMiniCluster(1);
Connection conn = ConnectionFactory.createConnection(conf); Connection conn = ConnectionFactory.createConnection(conf);
try { try {
HTableDescriptor htd = new HTableDescriptor(tableName); TEST_UTIL.getAdmin()
htd.addFamily(new HColumnDescriptor(family)); .createTable(TableDescriptorBuilder.newBuilder(tableName)
htd.setCompactionEnabled(false); .addColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCompactionEnabled(false)
TEST_UTIL.getAdmin().createTable(htd); .build());
TEST_UTIL.waitTableAvailable(tableName); TEST_UTIL.waitTableAvailable(tableName);
HStore store = (HStore) getStoreWithName(tableName); HStore store = getStoreWithName(tableName);
assertEquals(0, store.getStorefilesCount()); assertEquals(0, store.getStorefilesCount());
assertEquals(0.0, store.getCompactionPressure(), EPSILON); assertEquals(0.0, store.getCompactionPressure(), EPSILON);
Table table = conn.getTable(tableName); Table table = conn.getTable(tableName);

View File

@ -22,25 +22,26 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.master.LoadBalancer; import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreEngine; import org.apache.hadoop.hbase.regionserver.StoreEngine;
import org.apache.hadoop.hbase.regionserver.StripeStoreEngine; import org.apache.hadoop.hbase.regionserver.StripeStoreEngine;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Rule; import org.junit.Rule;
@ -73,13 +74,13 @@ public class TestFlushWithThroughputController {
hbtu.shutdownMiniCluster(); hbtu.shutdownMiniCluster();
} }
private Store getStoreWithName(TableName tableName) { private HStore getStoreWithName(TableName tableName) {
MiniHBaseCluster cluster = hbtu.getMiniHBaseCluster(); MiniHBaseCluster cluster = hbtu.getMiniHBaseCluster();
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads(); List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) { for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
HRegionServer hrs = rsts.get(i).getRegionServer(); HRegionServer hrs = rsts.get(i).getRegionServer();
for (Region region : hrs.getRegions(tableName)) { for (Region region : hrs.getRegions(tableName)) {
return region.getStores().iterator().next(); return ((HRegion) region).getStores().iterator().next();
} }
} }
return null; return null;
@ -114,7 +115,7 @@ public class TestFlushWithThroughputController {
hbtu.getAdmin().flush(tableName); hbtu.getAdmin().flush(tableName);
duration += System.nanoTime() - startTime; duration += System.nanoTime() - startTime;
} }
Store store = getStoreWithName(tableName); HStore store = getStoreWithName(tableName);
assertEquals(NUM_FLUSHES, store.getStorefilesCount()); assertEquals(NUM_FLUSHES, store.getStorefilesCount());
double throughput = (double)store.getStorefilesSize() double throughput = (double)store.getStorefilesSize()
/ TimeUnit.NANOSECONDS.toSeconds(duration); / TimeUnit.NANOSECONDS.toSeconds(duration);
@ -157,10 +158,9 @@ public class TestFlushWithThroughputController {
3000); 3000);
hbtu.startMiniCluster(1); hbtu.startMiniCluster(1);
Connection conn = ConnectionFactory.createConnection(conf); Connection conn = ConnectionFactory.createConnection(conf);
HTableDescriptor htd = new HTableDescriptor(tableName); hbtu.getAdmin().createTable(TableDescriptorBuilder.newBuilder(tableName)
htd.addFamily(new HColumnDescriptor(family)); .addColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCompactionEnabled(false)
htd.setCompactionEnabled(false); .build());
hbtu.getAdmin().createTable(htd);
hbtu.waitTableAvailable(tableName); hbtu.waitTableAvailable(tableName);
HRegionServer regionServer = hbtu.getRSForFirstRegionInTable(tableName); HRegionServer regionServer = hbtu.getRSForFirstRegionInTable(tableName);
PressureAwareFlushThroughputController throughputController = PressureAwareFlushThroughputController throughputController =

View File

@ -242,10 +242,11 @@ public abstract class AbstractTestWALReplay {
assertEquals(0, count); assertEquals(0, count);
// flush region and make major compaction // flush region and make major compaction
Region region = destServer.getOnlineRegion(destRegion.getRegionInfo().getRegionName()); HRegion region =
(HRegion) destServer.getOnlineRegion(destRegion.getRegionInfo().getRegionName());
region.flush(true); region.flush(true);
// wait to complete major compaction // wait to complete major compaction
for (Store store : region.getStores()) { for (HStore store : region.getStores()) {
store.triggerMajorCompaction(); store.triggerMajorCompaction();
} }
region.compact(true); region.compact(true);

View File

@ -250,18 +250,19 @@ public class TestCoprocessorScanPolicy {
public InternalScanner preFlushScannerOpen( public InternalScanner preFlushScannerOpen(
final ObserverContext<RegionCoprocessorEnvironment> c, Store store, final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
List<KeyValueScanner> scanners, InternalScanner s, long readPoint) throws IOException { List<KeyValueScanner> scanners, InternalScanner s, long readPoint) throws IOException {
HStore hs = (HStore) store;
Long newTtl = ttls.get(store.getTableName()); Long newTtl = ttls.get(store.getTableName());
if (newTtl != null) { if (newTtl != null) {
System.out.println("PreFlush:" + newTtl); System.out.println("PreFlush:" + newTtl);
} }
Integer newVersions = versions.get(store.getTableName()); Integer newVersions = versions.get(store.getTableName());
ScanInfo oldSI = store.getScanInfo(); ScanInfo oldSI = hs.getScanInfo();
ColumnFamilyDescriptor family = store.getColumnFamilyDescriptor(); ColumnFamilyDescriptor family = store.getColumnFamilyDescriptor();
ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(), ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(),
family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions, family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions,
newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(), newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(),
family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), family.isNewVersionBehavior()); family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), family.isNewVersionBehavior());
return new StoreScanner((HStore) store, scanInfo, return new StoreScanner(hs, scanInfo,
newVersions == null ? OptionalInt.empty() : OptionalInt.of(newVersions.intValue()), newVersions == null ? OptionalInt.empty() : OptionalInt.of(newVersions.intValue()),
scanners, ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), scanners, ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(),
HConstants.OLDEST_TIMESTAMP); HConstants.OLDEST_TIMESTAMP);
@ -272,16 +273,17 @@ public class TestCoprocessorScanPolicy {
final ObserverContext<RegionCoprocessorEnvironment> c, Store store, final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
InternalScanner s,CompactionLifeCycleTracker tracker, long readPoint) throws IOException { InternalScanner s,CompactionLifeCycleTracker tracker, long readPoint) throws IOException {
HStore hs = (HStore) store;
Long newTtl = ttls.get(store.getTableName()); Long newTtl = ttls.get(store.getTableName());
Integer newVersions = versions.get(store.getTableName()); Integer newVersions = versions.get(store.getTableName());
ScanInfo oldSI = store.getScanInfo(); ScanInfo oldSI = hs.getScanInfo();
ColumnFamilyDescriptor family = store.getColumnFamilyDescriptor(); ColumnFamilyDescriptor family = store.getColumnFamilyDescriptor();
ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(), ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(),
family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions, family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions,
newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(), newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(),
family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(),
family.isNewVersionBehavior()); family.isNewVersionBehavior());
return new StoreScanner((HStore) store, scanInfo, return new StoreScanner(hs, scanInfo,
newVersions == null ? OptionalInt.empty() : OptionalInt.of(newVersions.intValue()), newVersions == null ? OptionalInt.empty() : OptionalInt.of(newVersions.intValue()),
scanners, scanType, store.getSmallestReadPoint(), earliestPutTs); scanners, scanType, store.getSmallestReadPoint(), earliestPutTs);
} }
@ -292,16 +294,17 @@ public class TestCoprocessorScanPolicy {
final NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt) throws IOException { final NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt) throws IOException {
TableName tn = store.getTableName(); TableName tn = store.getTableName();
if (!tn.isSystemTable()) { if (!tn.isSystemTable()) {
HStore hs = (HStore) store;
Long newTtl = ttls.get(store.getTableName()); Long newTtl = ttls.get(store.getTableName());
Integer newVersions = versions.get(store.getTableName()); Integer newVersions = versions.get(store.getTableName());
ScanInfo oldSI = store.getScanInfo(); ScanInfo oldSI = hs.getScanInfo();
ColumnFamilyDescriptor family = store.getColumnFamilyDescriptor(); ColumnFamilyDescriptor family = store.getColumnFamilyDescriptor();
ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(), ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(),
family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions, family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions,
newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(), newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(),
family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(),
family.isNewVersionBehavior()); family.isNewVersionBehavior());
return new StoreScanner((HStore) store, scanInfo, scan, targetCols, readPt); return new StoreScanner(hs, scanInfo, scan, targetCols, readPt);
} else { } else {
return s; return s;
} }