HBASE-18826 Use HStore instead of Store in our own code base and remove unnecessary methods in Store interface
This commit is contained in:
parent
f73a3a6fb7
commit
d26b8f8ddd
|
@ -583,6 +583,11 @@ public class ColumnFamilyDescriptorBuilder {
|
|||
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
|
||||
* number of versions, compression settings, etc.
|
||||
|
|
|
@ -203,7 +203,7 @@ public class ZooKeeperScanPolicyObserver implements RegionCoprocessor, RegionObs
|
|||
if (data == null) {
|
||||
return null;
|
||||
}
|
||||
ScanInfo oldSI = store.getScanInfo();
|
||||
ScanInfo oldSI = ((HStore) store).getScanInfo();
|
||||
if (oldSI.getTtl() == Long.MAX_VALUE) {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -15,9 +15,10 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
|
@ -161,12 +162,12 @@ public class CompactionTool extends Configured implements Tool {
|
|||
}
|
||||
do {
|
||||
Optional<CompactionContext> compaction =
|
||||
store.requestCompaction(Store.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null);
|
||||
store.requestCompaction(PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null);
|
||||
if (!compaction.isPresent()) {
|
||||
break;
|
||||
}
|
||||
List<HStoreFile> storeFiles =
|
||||
store.compact(compaction.get(), NoLimitThroughputController.INSTANCE);
|
||||
store.compact(compaction.get(), NoLimitThroughputController.INSTANCE, null);
|
||||
if (storeFiles != null && !storeFiles.isEmpty()) {
|
||||
if (keepCompactedFiles && deleteCompacted) {
|
||||
for (HStoreFile storeFile: storeFiles) {
|
||||
|
|
|
@ -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.regionserver.BloomType;
|
||||
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.TimeRangeTracker;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
|
@ -1258,7 +1258,7 @@ public class TestHFileOutputFormat2 {
|
|||
public Boolean call() throws Exception {
|
||||
List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]);
|
||||
for (HRegion region : regions) {
|
||||
for (Store store : region.getStores()) {
|
||||
for (HStore store : region.getStores()) {
|
||||
store.closeAndArchiveCompactedFiles();
|
||||
}
|
||||
}
|
||||
|
@ -1277,7 +1277,7 @@ public class TestHFileOutputFormat2 {
|
|||
public Boolean call() throws Exception {
|
||||
List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]);
|
||||
for (HRegion region : regions) {
|
||||
for (Store store : region.getStores()) {
|
||||
for (HStore store : region.getStores()) {
|
||||
store.closeAndArchiveCompactedFiles();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -105,7 +105,7 @@ public class BusyRegionSplitPolicy extends IncreasingToUpperBoundRegionSplitPoli
|
|||
return false;
|
||||
}
|
||||
|
||||
for (Store store: region.getStores()) {
|
||||
for (HStore store: region.getStores()) {
|
||||
if (!store.canSplit()) {
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -18,6 +18,9 @@
|
|||
*/
|
||||
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.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
|
@ -35,7 +38,6 @@ import java.util.concurrent.TimeUnit;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
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.PropagatingConfigurationObserver;
|
||||
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.ThroughputController;
|
||||
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.StealJobQueue;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
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
|
||||
|
@ -195,7 +199,7 @@ public class CompactSplit implements PropagatingConfigurationObserver {
|
|||
|
||||
public synchronized boolean requestSplit(final Region r) {
|
||||
// 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();
|
||||
if (midKey != null) {
|
||||
requestSplit(r, midKey);
|
||||
|
@ -298,13 +302,13 @@ public class CompactSplit implements PropagatingConfigurationObserver {
|
|||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
public synchronized void requestSystemCompaction(HRegion region, HStore store, String why)
|
||||
throws IOException {
|
||||
requestCompactionInternal(region, store, why, Store.NO_PRIORITY, false,
|
||||
requestCompactionInternal(region, store, why, NO_PRIORITY, false,
|
||||
CompactionLifeCycleTracker.DUMMY, null);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,11 +24,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.ScheduledChore;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
|
@ -95,12 +92,11 @@ public class CompactedHFilesDischarger extends ScheduledChore {
|
|||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Started compacted hfiles cleaner on " + region.getRegionInfo());
|
||||
}
|
||||
for (Store store : region.getStores()) {
|
||||
for (HStore store : ((HRegion) region).getStores()) {
|
||||
try {
|
||||
if (useExecutor && regionServerServices != null) {
|
||||
CompactedHFilesDischargeHandler handler = new CompactedHFilesDischargeHandler(
|
||||
(Server) regionServerServices, EventType.RS_COMPACTED_FILES_DISCHARGER,
|
||||
(HStore) store);
|
||||
(Server) regionServerServices, EventType.RS_COMPACTED_FILES_DISCHARGER, store);
|
||||
regionServerServices.getExecutorService().submit(handler);
|
||||
} else {
|
||||
// call synchronously if the RegionServerServices are not
|
||||
|
|
|
@ -72,7 +72,7 @@ public class ConstantSizeRegionSplitPolicy extends RegionSplitPolicy {
|
|||
boolean force = region.shouldForceSplit();
|
||||
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)
|
||||
// then don't split
|
||||
if ((!store.canSplit())) {
|
||||
|
|
|
@ -78,11 +78,11 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy {
|
|||
}
|
||||
|
||||
protected boolean shouldFlush(HStore store) {
|
||||
if (store.getSizeOfMemStore().getDataSize() > this.flushSizeLowerBound) {
|
||||
if (store.getMemStoreSize().getDataSize() > this.flushSizeLowerBound) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Flush Column Family " + store.getColumnFamilyName() + " of " +
|
||||
region.getRegionInfo().getEncodedName() + " because memstoreSize=" +
|
||||
store.getSizeOfMemStore().getDataSize() + " > lower bound="
|
||||
store.getMemStoreSize().getDataSize() + " > lower bound="
|
||||
+ this.flushSizeLowerBound);
|
||||
}
|
||||
return true;
|
||||
|
|
|
@ -1011,13 +1011,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
hasSloppyStores = true;
|
||||
}
|
||||
|
||||
long storeMaxSequenceId = store.getMaxSequenceId();
|
||||
long storeMaxSequenceId = store.getMaxSequenceId().orElse(0L);
|
||||
maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(),
|
||||
storeMaxSequenceId);
|
||||
if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) {
|
||||
maxSeqId = storeMaxSequenceId;
|
||||
}
|
||||
long maxStoreMemstoreTS = store.getMaxMemstoreTS();
|
||||
long maxStoreMemstoreTS = store.getMaxMemstoreTS().orElse(0L);
|
||||
if (maxStoreMemstoreTS > maxMemstoreTS) {
|
||||
maxMemstoreTS = maxStoreMemstoreTS;
|
||||
}
|
||||
|
@ -1645,7 +1645,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
// close each store in parallel
|
||||
for (HStore store : stores.values()) {
|
||||
MemstoreSize flushableSize = store.getSizeToFlush();
|
||||
MemstoreSize flushableSize = store.getFlushableSize();
|
||||
if (!(abort || flushableSize.getDataSize() == 0 || writestate.readOnly)) {
|
||||
if (getRegionServerServices() != null) {
|
||||
getRegionServerServices().abort("Assertion failed while closing store "
|
||||
|
@ -1717,7 +1717,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
|
||||
private long getMemstoreHeapSize() {
|
||||
return stores.values().stream().mapToLong(s -> s.getSizeOfMemStore().getHeapSize()).sum();
|
||||
return stores.values().stream().mapToLong(s -> s.getMemStoreSize().getHeapSize()).sum();
|
||||
}
|
||||
|
||||
@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
|
||||
//are met. Return true on first such memstore hit.
|
||||
for (Store s : stores.values()) {
|
||||
for (HStore s : stores.values()) {
|
||||
if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
|
||||
// we have an old enough edit in the memstore, flush
|
||||
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) {
|
||||
MemstoreSize flushableSize = s.getSizeToFlush();
|
||||
MemstoreSize flushableSize = s.getFlushableSize();
|
||||
totalSizeOfFlushableStores.incMemstoreSize(flushableSize);
|
||||
storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(), s.createFlushContext(flushOpSeqId));
|
||||
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) {
|
||||
perCfExtras.append("; ").append(store.getColumnFamilyName());
|
||||
perCfExtras.append("=")
|
||||
.append(StringUtils.byteDesc(store.getSizeToFlush().getDataSize()));
|
||||
.append(StringUtils.byteDesc(store.getFlushableSize().getDataSize()));
|
||||
}
|
||||
}
|
||||
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)
|
||||
throws IOException {
|
||||
MemstoreSize flushableSize = s.getSizeToFlush();
|
||||
MemstoreSize flushableSize = s.getFlushableSize();
|
||||
this.decrMemstoreSize(flushableSize);
|
||||
StoreFlushContext ctx = s.createFlushContext(currentSeqId);
|
||||
ctx.prepare();
|
||||
|
@ -4933,7 +4933,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
continue;
|
||||
}
|
||||
|
||||
long storeSeqId = store.getMaxSequenceId();
|
||||
long storeSeqId = store.getMaxSequenceId().orElse(0L);
|
||||
List<String> storeFiles = storeDescriptor.getStoreFileList();
|
||||
try {
|
||||
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);
|
||||
continue;
|
||||
}
|
||||
if (store.getMaxSequenceId() != storeSeqId) {
|
||||
if (store.getMaxSequenceId().orElse(0L) != storeSeqId) {
|
||||
// Record latest flush time if we picked up new files
|
||||
lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
|
||||
}
|
||||
|
@ -4954,7 +4954,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
|
||||
null : this.prepareFlushResult.storeFlushCtxs.get(family);
|
||||
if (ctx != null) {
|
||||
MemstoreSize snapshotSize = store.getSizeToFlush();
|
||||
MemstoreSize snapshotSize = store.getFlushableSize();
|
||||
ctx.abort();
|
||||
this.decrMemstoreSize(snapshotSize);
|
||||
this.prepareFlushResult.storeFlushCtxs.remove(family);
|
||||
|
@ -5085,7 +5085,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
if (store == null) {
|
||||
continue;
|
||||
}
|
||||
if (store.getSizeOfSnapshot().getDataSize() > 0) {
|
||||
if (store.getSnapshotSize().getDataSize() > 0) {
|
||||
canDrop = false;
|
||||
break;
|
||||
}
|
||||
|
@ -5129,12 +5129,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
for (HStore store : stores.values()) {
|
||||
// TODO: some stores might see new data from flush, while others do not which
|
||||
// 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.
|
||||
store.refreshStoreFiles();
|
||||
|
||||
long storeSeqId = store.getMaxSequenceId();
|
||||
long storeSeqId = store.getMaxSequenceId().orElse(0L);
|
||||
if (storeSeqId < smallestSeqIdInStores) {
|
||||
smallestSeqIdInStores = storeSeqId;
|
||||
}
|
||||
|
@ -5148,7 +5148,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
null : this.prepareFlushResult.storeFlushCtxs.get(
|
||||
store.getColumnFamilyDescriptor().getName());
|
||||
if (ctx != null) {
|
||||
MemstoreSize snapshotSize = store.getSizeToFlush();
|
||||
MemstoreSize snapshotSize = store.getFlushableSize();
|
||||
ctx.abort();
|
||||
this.decrMemstoreSize(snapshotSize);
|
||||
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.
|
||||
// either greater than flush seq number or they were already picked up via flush.
|
||||
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()) {
|
||||
buf.append(s.getColumnFamilyDescriptor().getNameAsString());
|
||||
buf.append(" size: ");
|
||||
buf.append(s.getSizeOfMemStore().getDataSize());
|
||||
buf.append(s.getMemStoreSize().getDataSize());
|
||||
buf.append(" ");
|
||||
}
|
||||
buf.append("end-of-stores");
|
||||
|
|
|
@ -1787,7 +1787,7 @@ public class HRegionServer extends HasThread implements
|
|||
// Queue a compaction. Will recognize if major is needed.
|
||||
this.instance.compactSplitThread.requestSystemCompaction(hr, s,
|
||||
getName() + " requests compaction");
|
||||
} else if (s.isMajorCompaction()) {
|
||||
} else if (s.shouldPerformMajorCompaction()) {
|
||||
s.triggerMajorCompaction();
|
||||
if (majorCompactPriority == DEFAULT_PRIORITY ||
|
||||
majorCompactPriority > hr.getCompactPriority()) {
|
||||
|
|
|
@ -32,6 +32,7 @@ import java.util.Map;
|
|||
import java.util.NavigableSet;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalDouble;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
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.Scan;
|
||||
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.crypto.Encryption;
|
||||
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.
|
||||
*/
|
||||
@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 COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY =
|
||||
"hbase.server.compactchecker.interval.multiplier";
|
||||
|
@ -396,26 +399,12 @@ public class HStore implements Store {
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public long getFlushableSize() {
|
||||
MemstoreSize size = getSizeToFlush();
|
||||
return size.getHeapSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public MemstoreSize getSizeToFlush() {
|
||||
public MemstoreSize getFlushableSize() {
|
||||
return this.memstore.getFlushableSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public long getSnapshotSize() {
|
||||
MemstoreSize size = getSizeOfSnapshot();
|
||||
return size.getHeapSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public MemstoreSize getSizeOfSnapshot() {
|
||||
public MemstoreSize getSnapshotSize() {
|
||||
return this.memstore.getSnapshotSize();
|
||||
}
|
||||
|
||||
|
@ -466,16 +455,13 @@ public class HStore implements Store {
|
|||
return this.family;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The maximum sequence id in all store files. Used for log replay.
|
||||
*/
|
||||
@Override
|
||||
public long getMaxSequenceId() {
|
||||
public OptionalLong getMaxSequenceId() {
|
||||
return StoreUtils.getMaxSequenceIdInList(this.getStorefiles());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxMemstoreTS() {
|
||||
public OptionalLong getMaxMemstoreTS() {
|
||||
return StoreUtils.getMaxMemstoreTSInList(this.getStorefiles());
|
||||
}
|
||||
|
||||
|
@ -503,7 +489,9 @@ public class HStore implements Store {
|
|||
return new Path(tabledir, new Path(encodedName, Bytes.toString(family)));
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
* @return the data block encoder
|
||||
*/
|
||||
public HFileDataBlockEncoder getDataBlockEncoder() {
|
||||
return dataBlockEncoder;
|
||||
}
|
||||
|
@ -584,20 +572,17 @@ public class HStore implements Store {
|
|||
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
|
||||
public void refreshStoreFiles() throws IOException {
|
||||
Collection<StoreFileInfo> newFiles = fs.getStoreFiles(getColumnFamilyName());
|
||||
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 {
|
||||
List<StoreFileInfo> storeFiles = new ArrayList<>(newFiles.size());
|
||||
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
|
||||
// in-flight transactions might be made visible)
|
||||
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);
|
||||
|
@ -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 {
|
||||
this.archiveLock.lock();
|
||||
this.lock.writeLock().lock();
|
||||
|
@ -1035,13 +1026,6 @@ public class HStore implements Store {
|
|||
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 compression Compression algorithm to use
|
||||
|
@ -1050,7 +1034,6 @@ public class HStore implements Store {
|
|||
* @param includesTag - includesTag or not
|
||||
* @return Writer for a new StoreFile in the tmp dir.
|
||||
*/
|
||||
@Override
|
||||
public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
|
||||
boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
|
||||
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
|
||||
// compaction
|
||||
@Override
|
||||
public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression,
|
||||
boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag,
|
||||
boolean shouldDropBehind, final TimeRangeTracker trt) throws IOException {
|
||||
boolean shouldDropBehind, TimeRangeTracker trt) throws IOException {
|
||||
final CacheConfig writerCacheConf;
|
||||
if (isCompaction) {
|
||||
// Don't cache data on write on compactions.
|
||||
|
@ -1301,12 +1283,16 @@ public class HStore implements Store {
|
|||
return scanners;
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
* @param o Observer who wants to know about changes in set of Readers
|
||||
*/
|
||||
public void addChangedReaderObserver(ChangedReadersObserver o) {
|
||||
this.changedReaderObservers.add(o);
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
* @param o Observer no longer interested in changes in set of Readers.
|
||||
*/
|
||||
public void deleteChangedReaderObserver(ChangedReadersObserver o) {
|
||||
// We don't check if observer present; it may not be (legitimately)
|
||||
this.changedReaderObservers.remove(o);
|
||||
|
@ -1359,13 +1345,6 @@ public class HStore implements Store {
|
|||
* @throws IOException
|
||||
* @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,
|
||||
ThroughputController throughputController, User user) throws IOException {
|
||||
assert compaction != null;
|
||||
|
@ -1669,7 +1648,7 @@ public class HStore implements Store {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean isMajorCompaction() throws IOException {
|
||||
public boolean shouldPerformMajorCompaction() throws IOException {
|
||||
for (HStoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) {
|
||||
// TODO: what are these reader checks all over the place?
|
||||
if (sf.getReader() == null) {
|
||||
|
@ -1681,7 +1660,10 @@ public class HStore implements Store {
|
|||
this.storeEngine.getStoreFileManager().getStorefiles());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<CompactionContext> requestCompaction() throws IOException {
|
||||
return requestCompaction(NO_PRIORITY, CompactionLifeCycleTracker.DUMMY, null);
|
||||
}
|
||||
|
||||
public Optional<CompactionContext> requestCompaction(int priority,
|
||||
CompactionLifeCycleTracker tracker, User user) throws IOException {
|
||||
// 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));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cancelRequestedCompaction(CompactionContext compaction) {
|
||||
finishCompactionRequest(compaction.getRequest());
|
||||
}
|
||||
|
@ -1899,7 +1880,9 @@ public class HStore implements Store {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
* Determines if Store should be split.
|
||||
*/
|
||||
public Optional<byte[]> getSplitPoint() {
|
||||
this.lock.readLock().lock();
|
||||
try {
|
||||
|
@ -1931,7 +1914,6 @@ public class HStore implements Store {
|
|||
return storeSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void triggerMajorCompaction() {
|
||||
this.forceMajor = true;
|
||||
}
|
||||
|
@ -1941,7 +1923,14 @@ public class HStore implements Store {
|
|||
// 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,
|
||||
final NavigableSet<byte []> targetCols, long readPt) throws IOException {
|
||||
lock.readLock().lock();
|
||||
|
@ -2032,7 +2021,7 @@ public class HStore implements Store {
|
|||
return this.storeEngine.getStoreFileManager().getCompactedFilesCount();
|
||||
}
|
||||
|
||||
private LongStream getStoreFileCreatedTimestampStream() {
|
||||
private LongStream getStoreFileAgeStream() {
|
||||
return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> {
|
||||
if (sf.getReader() == null) {
|
||||
LOG.warn("StoreFile " + sf + " has a null Reader");
|
||||
|
@ -2040,25 +2029,23 @@ public class HStore implements Store {
|
|||
} else {
|
||||
return true;
|
||||
}
|
||||
}).filter(HStoreFile::isHFile).mapToLong(sf -> sf.getFileInfo().getCreatedTimestamp());
|
||||
}).filter(HStoreFile::isHFile).mapToLong(sf -> sf.getFileInfo().getCreatedTimestamp())
|
||||
.map(t -> EnvironmentEdgeManager.currentTime() - t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxStoreFileAge() {
|
||||
return EnvironmentEdgeManager.currentTime() -
|
||||
getStoreFileCreatedTimestampStream().min().orElse(Long.MAX_VALUE);
|
||||
public OptionalLong getMaxStoreFileAge() {
|
||||
return getStoreFileAgeStream().max();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMinStoreFileAge() {
|
||||
return EnvironmentEdgeManager.currentTime() -
|
||||
getStoreFileCreatedTimestampStream().max().orElse(0L);
|
||||
public OptionalLong getMinStoreFileAge() {
|
||||
return getStoreFileAgeStream().min();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getAvgStoreFileAge() {
|
||||
OptionalDouble avg = getStoreFileCreatedTimestampStream().average();
|
||||
return avg.isPresent() ? EnvironmentEdgeManager.currentTime() - (long) avg.getAsDouble() : 0L;
|
||||
public OptionalDouble getAvgStoreFileAge() {
|
||||
return getStoreFileAgeStream().average();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2128,14 +2115,7 @@ public class HStore implements Store {
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public long getMemStoreSize() {
|
||||
MemstoreSize size = getSizeOfMemStore();
|
||||
return size.getHeapSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
public MemstoreSize getSizeOfMemStore() {
|
||||
public MemstoreSize getMemStoreSize() {
|
||||
return this.memstore.size();
|
||||
}
|
||||
|
||||
|
@ -2148,7 +2128,6 @@ public class HStore implements Store {
|
|||
return priority;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean throttleCompaction(long compactionSize) {
|
||||
return storeEngine.getCompactionPolicy().throttleCompaction(compactionSize);
|
||||
}
|
||||
|
@ -2200,7 +2179,6 @@ public class HStore implements Store {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoreFlushContext createFlushContext(long cacheFlushId) {
|
||||
return new StoreFlusherImpl(cacheFlushId);
|
||||
}
|
||||
|
@ -2344,7 +2322,11 @@ public class HStore implements Store {
|
|||
return this.storeEngine.needsCompaction(this.filesCompacting);
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
* Used for tests.
|
||||
* @return cache configuration for this Store.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public CacheConfig getCacheConfig() {
|
||||
return this.cacheConf;
|
||||
}
|
||||
|
@ -2370,7 +2352,6 @@ public class HStore implements Store {
|
|||
return comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScanInfo getScanInfo() {
|
||||
return scanInfo;
|
||||
}
|
||||
|
@ -2490,7 +2471,9 @@ public class HStore implements Store {
|
|||
archiveLock.unlock();
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
* Closes and archives the compacted files under this store
|
||||
*/
|
||||
public synchronized void closeAndArchiveCompactedFiles() throws IOException {
|
||||
// ensure other threads do not attempt to archive the same files on close()
|
||||
archiveLock.lock();
|
||||
|
|
|
@ -76,7 +76,7 @@ public class IncreasingToUpperBoundRegionSplitPolicy extends ConstantSizeRegionS
|
|||
// Get size to check
|
||||
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)
|
||||
// then don't split
|
||||
if (!store.canSplit()) {
|
||||
|
|
|
@ -95,7 +95,7 @@ public interface KeyValueScanner extends Shipper, Closeable {
|
|||
* this query, based on TTL
|
||||
* @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
|
||||
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.OptionalDouble;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
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.HRegionInfo;
|
||||
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.CacheConfig;
|
||||
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.hdfs.DFSHedgedReadMetrics;
|
||||
import org.apache.hadoop.metrics2.MetricsExecutor;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Impl for exposing HRegionServer Information through Hadoop's metrics 2 system.
|
||||
|
@ -765,22 +767,30 @@ class MetricsRegionServerWrapperImpl
|
|||
tempNumStores += storeList.size();
|
||||
for (Store store : storeList) {
|
||||
tempNumStoreFiles += store.getStorefilesCount();
|
||||
tempMemstoreSize += store.getSizeOfMemStore().getDataSize();
|
||||
tempMemstoreSize += store.getMemStoreSize().getDataSize();
|
||||
tempStoreFileSize += store.getStorefilesSize();
|
||||
|
||||
long storeMaxStoreFileAge = store.getMaxStoreFileAge();
|
||||
tempMaxStoreFileAge = (storeMaxStoreFileAge > tempMaxStoreFileAge) ?
|
||||
storeMaxStoreFileAge : tempMaxStoreFileAge;
|
||||
OptionalLong storeMaxStoreFileAge = store.getMaxStoreFileAge();
|
||||
if (storeMaxStoreFileAge.isPresent() &&
|
||||
storeMaxStoreFileAge.getAsLong() > tempMaxStoreFileAge) {
|
||||
tempMaxStoreFileAge = storeMaxStoreFileAge.getAsLong();
|
||||
}
|
||||
|
||||
long storeMinStoreFileAge = store.getMinStoreFileAge();
|
||||
tempMinStoreFileAge = (storeMinStoreFileAge < tempMinStoreFileAge) ?
|
||||
storeMinStoreFileAge : tempMinStoreFileAge;
|
||||
OptionalLong storeMinStoreFileAge = store.getMinStoreFileAge();
|
||||
if (storeMinStoreFileAge.isPresent() &&
|
||||
storeMinStoreFileAge.getAsLong() < tempMinStoreFileAge) {
|
||||
tempMinStoreFileAge = storeMinStoreFileAge.getAsLong();
|
||||
}
|
||||
|
||||
long storeHFiles = store.getNumHFiles();
|
||||
avgAgeNumerator += store.getAvgStoreFileAge() * storeHFiles;
|
||||
numHFiles += storeHFiles;
|
||||
tempNumReferenceFiles += store.getNumReferenceFiles();
|
||||
|
||||
OptionalDouble storeAvgStoreFileAge = store.getAvgStoreFileAge();
|
||||
if (storeAvgStoreFileAge.isPresent()) {
|
||||
avgAgeNumerator += storeAvgStoreFileAge.getAsDouble() * storeHFiles;
|
||||
}
|
||||
|
||||
tempStorefileIndexSize += store.getStorefilesIndexSize();
|
||||
tempTotalStaticBloomSize += store.getTotalStaticBloomSize();
|
||||
tempTotalStaticIndexSize += store.getTotalStaticIndexSize();
|
||||
|
|
|
@ -21,18 +21,20 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.OptionalDouble;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.metrics2.MetricsExecutor;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable {
|
||||
|
@ -227,21 +229,28 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
|||
if (region.stores != null) {
|
||||
for (Store store : region.stores.values()) {
|
||||
tempNumStoreFiles += store.getStorefilesCount();
|
||||
tempMemstoreSize += store.getSizeOfMemStore().getDataSize();
|
||||
tempMemstoreSize += store.getMemStoreSize().getDataSize();
|
||||
tempStoreFileSize += store.getStorefilesSize();
|
||||
OptionalLong storeMaxStoreFileAge = store.getMaxStoreFileAge();
|
||||
if (storeMaxStoreFileAge.isPresent() &&
|
||||
storeMaxStoreFileAge.getAsLong() > tempMaxStoreFileAge) {
|
||||
tempMaxStoreFileAge = storeMaxStoreFileAge.getAsLong();
|
||||
}
|
||||
|
||||
long storeMaxStoreFileAge = store.getMaxStoreFileAge();
|
||||
tempMaxStoreFileAge = (storeMaxStoreFileAge > tempMaxStoreFileAge) ?
|
||||
storeMaxStoreFileAge : tempMaxStoreFileAge;
|
||||
|
||||
long storeMinStoreFileAge = store.getMinStoreFileAge();
|
||||
tempMinStoreFileAge = (storeMinStoreFileAge < tempMinStoreFileAge) ?
|
||||
storeMinStoreFileAge : tempMinStoreFileAge;
|
||||
OptionalLong storeMinStoreFileAge = store.getMinStoreFileAge();
|
||||
if (storeMinStoreFileAge.isPresent() &&
|
||||
storeMinStoreFileAge.getAsLong() < tempMinStoreFileAge) {
|
||||
tempMinStoreFileAge = storeMinStoreFileAge.getAsLong();
|
||||
}
|
||||
|
||||
long storeHFiles = store.getNumHFiles();
|
||||
avgAgeNumerator += store.getAvgStoreFileAge() * storeHFiles;
|
||||
numHFiles += storeHFiles;
|
||||
tempNumReferenceFiles += store.getNumReferenceFiles();
|
||||
|
||||
OptionalDouble storeAvgStoreFileAge = store.getAvgStoreFileAge();
|
||||
if (storeAvgStoreFileAge.isPresent()) {
|
||||
avgAgeNumerator += storeAvgStoreFileAge.getAsDouble() * storeHFiles;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -56,7 +56,7 @@ public abstract class NonLazyKeyValueScanner implements KeyValueScanner {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) {
|
||||
public boolean shouldUseScanner(Scan scan, HStore store, long oldestUnexpiredTS) {
|
||||
// No optimizations implemented by default.
|
||||
return true;
|
||||
}
|
||||
|
|
|
@ -749,7 +749,7 @@ public interface Region extends ConfigurationObserver {
|
|||
* Trigger major compaction on all stores in the region.
|
||||
* <p>
|
||||
* Compaction will be performed asynchronously to this call by the RegionServer's
|
||||
* CompactSplitThread. See also {@link Store#triggerMajorCompaction()}
|
||||
* CompactSplitThread.
|
||||
* @throws IOException
|
||||
*/
|
||||
void triggerMajorCompaction() throws IOException;
|
||||
|
|
|
@ -279,7 +279,7 @@ public class SegmentScanner implements KeyValueScanner {
|
|||
* overridden method
|
||||
*/
|
||||
@Override
|
||||
public boolean shouldUseScanner(Scan scan, Store store, long oldestUnexpiredTS) {
|
||||
public boolean shouldUseScanner(Scan scan, HStore store, long oldestUnexpiredTS) {
|
||||
return getSegment().shouldSeek(scan.getColumnFamilyTimeRange()
|
||||
.getOrDefault(store.getColumnFamilyDescriptor().getName(), scan.getTimeRange()), oldestUnexpiredTS);
|
||||
}
|
||||
|
|
|
@ -19,27 +19,17 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.Optional;
|
||||
import java.util.Comparator;
|
||||
import java.util.OptionalDouble;
|
||||
import java.util.OptionalLong;
|
||||
|
||||
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.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
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.client.RegionInfo;
|
||||
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.InterfaceStability;
|
||||
|
||||
|
@ -49,7 +39,7 @@ import org.apache.yetus.audience.InterfaceStability;
|
|||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public interface Store extends HeapSize, StoreConfigInformation, PropagatingConfigurationObserver {
|
||||
public interface Store {
|
||||
|
||||
/**
|
||||
* 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;
|
||||
|
||||
// General Accessors
|
||||
CellComparator getComparator();
|
||||
Comparator<Cell> getComparator();
|
||||
|
||||
Collection<? extends StoreFile> getStorefiles();
|
||||
|
||||
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
|
||||
*/
|
||||
|
@ -93,186 +62,66 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
|
||||
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
|
||||
* @return CompactionProgress object; can be null
|
||||
*/
|
||||
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.
|
||||
*/
|
||||
boolean isMajorCompaction() throws IOException;
|
||||
|
||||
void triggerMajorCompaction();
|
||||
boolean shouldPerformMajorCompaction() throws IOException;
|
||||
|
||||
/**
|
||||
* 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();
|
||||
|
||||
int getCompactPriority();
|
||||
|
||||
StoreFlushContext createFlushContext(long cacheFlushId);
|
||||
|
||||
// Split oriented methods
|
||||
|
||||
/**
|
||||
* Returns whether this store is splittable, i.e., no reference file in this store.
|
||||
*/
|
||||
boolean canSplit();
|
||||
|
||||
/**
|
||||
* Determines if Store should be split.
|
||||
*/
|
||||
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
|
||||
* @return <code>true</code> if the store has any underlying reference files to older HFiles
|
||||
*/
|
||||
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.
|
||||
*/
|
||||
MemstoreSize getSizeOfMemStore();
|
||||
MemstoreSize getMemStoreSize();
|
||||
|
||||
/**
|
||||
* @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
|
||||
* 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
|
||||
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();
|
||||
MemstoreSize getFlushableSize();
|
||||
|
||||
/**
|
||||
* @return size of the memstore snapshot
|
||||
*/
|
||||
MemstoreSize getSizeOfSnapshot();
|
||||
MemstoreSize getSnapshotSize();
|
||||
|
||||
ColumnFamilyDescriptor getColumnFamilyDescriptor();
|
||||
|
||||
/**
|
||||
* @return The maximum sequence id in all store files.
|
||||
*/
|
||||
long getMaxSequenceId();
|
||||
OptionalLong getMaxSequenceId();
|
||||
|
||||
/**
|
||||
* @return The maximum memstoreTS in all store files.
|
||||
*/
|
||||
long getMaxMemstoreTS();
|
||||
|
||||
/**
|
||||
* @return the data block encoder
|
||||
*/
|
||||
HFileDataBlockEncoder getDataBlockEncoder();
|
||||
OptionalLong getMaxMemstoreTS();
|
||||
|
||||
/** @return aggregate size of all HStores used in the last compaction */
|
||||
long getLastCompactSize();
|
||||
|
@ -293,17 +142,17 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
/**
|
||||
* @return Max age of store files in this store
|
||||
*/
|
||||
long getMaxStoreFileAge();
|
||||
OptionalLong getMaxStoreFileAge();
|
||||
|
||||
/**
|
||||
* @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
|
||||
|
@ -350,18 +199,10 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
*/
|
||||
long getTotalStaticBloomSize();
|
||||
|
||||
// Test-helper methods
|
||||
|
||||
/**
|
||||
* Used for tests.
|
||||
* @return cache configuration for this Store.
|
||||
*/
|
||||
CacheConfig getCacheConfig();
|
||||
|
||||
/**
|
||||
* @return the parent region info hosting this store
|
||||
*/
|
||||
HRegionInfo getRegionInfo();
|
||||
RegionInfo getRegionInfo();
|
||||
|
||||
RegionCoprocessorHost getCoprocessorHost();
|
||||
|
||||
|
@ -413,26 +254,15 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
*/
|
||||
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.
|
||||
*/
|
||||
boolean hasTooManyStoreFiles();
|
||||
|
||||
/**
|
||||
* 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.
|
||||
* 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
|
||||
*/
|
||||
void refreshStoreFiles() throws IOException;
|
||||
|
@ -454,21 +284,8 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
|
|||
*/
|
||||
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();
|
||||
|
||||
/**
|
||||
* Closes and archives the compacted files under this store
|
||||
*/
|
||||
void closeAndArchiveCompactedFiles() throws IOException;
|
||||
|
||||
/**
|
||||
* @return true if the memstore may need some extra memory space
|
||||
*/
|
||||
|
|
|
@ -163,7 +163,7 @@ public interface StoreFileManager {
|
|||
|
||||
/**
|
||||
* @return the compaction pressure used for compaction throughput tuning.
|
||||
* @see Store#getCompactionPressure()
|
||||
* @see HStore#getCompactionPressure()
|
||||
*/
|
||||
double getCompactionPressure();
|
||||
|
||||
|
|
|
@ -474,7 +474,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
}
|
||||
|
||||
@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.
|
||||
byte[] cf = store.getColumnFamilyDescriptor().getName();
|
||||
TimeRange timeRange = scan.getColumnFamilyTimeRange().get(cf);
|
||||
|
|
|
@ -418,7 +418,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
|
|||
* Will be overridden by testcase so declared as protected.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected List<KeyValueScanner> selectScannersFrom(Store store,
|
||||
protected List<KeyValueScanner> selectScannersFrom(HStore store,
|
||||
List<? extends KeyValueScanner> allScanners) {
|
||||
boolean memOnly;
|
||||
boolean filesOnly;
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
|||
import java.util.Collection;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.OptionalLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
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
|
||||
* 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.
|
||||
* @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)
|
||||
.max().orElse(0L);
|
||||
.max();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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) {
|
||||
return sfs.stream().mapToLong(HStoreFile::getMaxSequenceId).max().orElse(0L);
|
||||
public static OptionalLong getMaxSequenceIdInList(Collection<HStoreFile> sfs) {
|
||||
return sfs.stream().mapToLong(HStoreFile::getMaxSequenceId).max();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,11 +18,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.compactions;
|
||||
|
||||
import static org.apache.hadoop.hbase.regionserver.Store.NO_PRIORITY;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
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.util.StringUtils.TraditionalBinaryPrefix;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
@ -39,7 +40,7 @@ public class CompactionRequest {
|
|||
private boolean isOffPeak = false;
|
||||
private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR }
|
||||
private DisplayCompactionType isMajor = DisplayCompactionType.MINOR;
|
||||
private int priority = Store.NO_PRIORITY;
|
||||
private int priority = NO_PRIORITY;
|
||||
private Collection<HStoreFile> filesToCompact;
|
||||
|
||||
// CompactRequest object creation time.
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver.compactions;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.OptionalLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -47,7 +48,10 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTiered
|
|||
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
|
||||
// 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,
|
||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
|
|||
* {@value #HBASE_HSTORE_COMPACTION_MAX_THROUGHPUT_HIGHER_BOUND}, using the formula "lower +
|
||||
* (higer - lower) * compactionPressure", where compactionPressure is in range [0.0, 1.0]</li>
|
||||
* </ul>
|
||||
* @see org.apache.hadoop.hbase.regionserver.Store#getCompactionPressure()
|
||||
* @see org.apache.hadoop.hbase.regionserver.HStore#getCompactionPressure()
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
|
||||
public class PressureAwareCompactionThroughputController extends PressureAwareThroughputController {
|
||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.hadoop.hbase.regionserver.throttle;
|
|||
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
|
||||
/**
|
||||
* Helper methods for throttling
|
||||
|
@ -40,7 +40,7 @@ public final class ThroughputControlUtil {
|
|||
* @param opName Name of the IO operation, e.g. "flush", "compaction", etc.
|
||||
* @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;
|
||||
for (;;) {
|
||||
counter = NAME_COUNTER.get();
|
||||
|
@ -49,8 +49,8 @@ public final class ThroughputControlUtil {
|
|||
break;
|
||||
}
|
||||
}
|
||||
return store.getRegionInfo().getRegionNameAsString() + NAME_DELIMITER
|
||||
+ store.getColumnFamilyDescriptor().getNameAsString()
|
||||
+ NAME_DELIMITER + opName + NAME_DELIMITER + counter;
|
||||
return store.getRegionInfo().getRegionNameAsString() + NAME_DELIMITER +
|
||||
store.getColumnFamilyDescriptor().getNameAsString() + NAME_DELIMITER + opName +
|
||||
NAME_DELIMITER + counter;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -99,7 +99,7 @@
|
|||
|
||||
<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>
|
||||
|
||||
|
|
|
@ -145,7 +145,7 @@ public class TestIOFencing {
|
|||
|
||||
public int countStoreFiles() {
|
||||
int count = 0;
|
||||
for (Store store : stores.values()) {
|
||||
for (HStore store : stores.values()) {
|
||||
count += store.getStorefilesCount();
|
||||
}
|
||||
return count;
|
||||
|
|
|
@ -36,19 +36,20 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ChoreService;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
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.Put;
|
||||
import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
|
||||
import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
|
||||
import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger;
|
||||
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.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MiscTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -176,7 +177,7 @@ public class TestZooKeeperTableArchiveClient {
|
|||
final LongTermArchivingHFileCleaner delegate = (LongTermArchivingHFileCleaner) cleaners.get(0);
|
||||
|
||||
// create the region
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM);
|
||||
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM);
|
||||
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
|
||||
List<Region> regions = new ArrayList<>();
|
||||
regions.add(region);
|
||||
|
@ -229,7 +230,7 @@ public class TestZooKeeperTableArchiveClient {
|
|||
List<BaseHFileCleanerDelegate> cleaners = turnOnArchiving(STRING_TABLE_NAME, cleaner);
|
||||
final LongTermArchivingHFileCleaner delegate = (LongTermArchivingHFileCleaner) cleaners.get(0);
|
||||
// create the region
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAM);
|
||||
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(TEST_FAM);
|
||||
HRegion region = UTIL.createTestRegion(STRING_TABLE_NAME, hcd);
|
||||
List<Region> regions = new ArrayList<>();
|
||||
regions.add(region);
|
||||
|
@ -239,7 +240,7 @@ public class TestZooKeeperTableArchiveClient {
|
|||
loadFlushAndCompact(region, TEST_FAM);
|
||||
compactionCleaner.chore();
|
||||
// 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);
|
||||
regions = new ArrayList<>();
|
||||
regions.add(otherRegion);
|
||||
|
@ -400,12 +401,12 @@ public class TestZooKeeperTableArchiveClient {
|
|||
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
|
||||
createHFileInRegion(region, family);
|
||||
createHFileInRegion(region, family);
|
||||
|
||||
Store s = region.getStore(family);
|
||||
HStore s = region.getStore(family);
|
||||
int count = s.getStorefilesCount();
|
||||
assertTrue("Don't have the expected store files, wanted >= 2 store files, but was:" + count,
|
||||
count >= 2);
|
||||
|
|
|
@ -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.CacheConfig;
|
||||
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.InternalScanner;
|
||||
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.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
|
@ -134,9 +134,9 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region =
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region =
|
||||
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -313,9 +313,9 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region =
|
||||
TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region = (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName)
|
||||
.getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
|
|
@ -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.CombinedBlockCache;
|
||||
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.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
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.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -65,9 +65,9 @@ import org.junit.After;
|
|||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
|
@ -184,8 +184,9 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region = (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName)
|
||||
.getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -274,8 +275,9 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region =
|
||||
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -332,8 +334,9 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region =
|
||||
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -393,7 +396,8 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
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);
|
||||
Put put = new Put(ROW);
|
||||
put.addColumn(FAMILY, QUALIFIER, data);
|
||||
|
@ -485,7 +489,8 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
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);
|
||||
|
||||
Put put = new Put(ROW);
|
||||
|
@ -568,8 +573,9 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region =
|
||||
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setEvictOnClose(true);
|
||||
BlockCache cache = cacheConf.getBlockCache();
|
||||
|
@ -626,8 +632,9 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region =
|
||||
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -708,7 +715,8 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
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);
|
||||
|
||||
Put put = new Put(ROW);
|
||||
|
@ -777,11 +785,11 @@ public class TestBlockEvictionFromClient {
|
|||
}
|
||||
}
|
||||
|
||||
private BlockCache setCacheProperties(Region region) {
|
||||
Iterator<? extends Store> strItr = region.getStores().iterator();
|
||||
private BlockCache setCacheProperties(HRegion region) {
|
||||
Iterator<HStore> strItr = region.getStores().iterator();
|
||||
BlockCache cache = null;
|
||||
while (strItr.hasNext()) {
|
||||
Store store = strItr.next();
|
||||
HStore store = strItr.next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -807,8 +815,9 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region =
|
||||
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -872,8 +881,9 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region =
|
||||
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -989,8 +999,9 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region =
|
||||
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -1118,8 +1129,9 @@ public class TestBlockEvictionFromClient {
|
|||
// get the block cache and region
|
||||
RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region =
|
||||
(HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(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 {
|
||||
long start = System.currentTimeMillis();
|
||||
while (start + timeout > System.currentTimeMillis() && store.getStorefilesCount() != count) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
System.out.println("start=" + start + ", now=" + System.currentTimeMillis() + ", cur="
|
||||
+ store.getStorefilesCount());
|
||||
System.out.println("start=" + start + ", now=" + System.currentTimeMillis() + ", cur=" +
|
||||
store.getStorefilesCount());
|
||||
assertEquals(count, store.getStorefilesCount());
|
||||
}
|
||||
|
||||
|
|
|
@ -199,19 +199,19 @@ public class TestFromClientSide {
|
|||
|
||||
try {
|
||||
Append append = new Append(ROW);
|
||||
append.addColumn(TEST_UTIL.fam1, QUALIFIER, VALUE);
|
||||
append.addColumn(HBaseTestingUtility.fam1, QUALIFIER, VALUE);
|
||||
Result result = table.append(append);
|
||||
|
||||
// Verify expected result
|
||||
Cell[] cells = result.rawCells();
|
||||
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
|
||||
Result readResult = table.get(new Get(ROW));
|
||||
cells = readResult.rawCells();
|
||||
assertEquals(1, cells.length);
|
||||
assertKey(cells[0], ROW, TEST_UTIL.fam1, QUALIFIER, VALUE);
|
||||
assertKey(cells[0], ROW, HBaseTestingUtility.fam1, QUALIFIER, VALUE);
|
||||
} finally {
|
||||
table.close();
|
||||
connection.close();
|
||||
|
@ -568,7 +568,7 @@ public class TestFromClientSide {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<KeyValueScanner> selectScannersFrom(Store store,
|
||||
protected List<KeyValueScanner> selectScannersFrom(HStore store,
|
||||
List<? extends KeyValueScanner> allScanners) {
|
||||
List<KeyValueScanner> scanners = super.selectScannersFrom(store, allScanners);
|
||||
List<KeyValueScanner> newScanners = new ArrayList<>(scanners.size());
|
||||
|
@ -596,7 +596,8 @@ public class TestFromClientSide {
|
|||
public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s,
|
||||
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 {
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
|
||||
byte [][] ROWS = makeNAscii(ROW, 6);
|
||||
byte [][] FAMILIES = makeNAscii(FAMILY, 3);
|
||||
byte [][] VALUES = makeN(VALUE, 5);
|
||||
long [] ts = {1000, 2000, 3000, 4000, 5000};
|
||||
|
@ -2061,7 +2061,7 @@ public class TestFromClientSide {
|
|||
|
||||
Get get = new Get(ROW);
|
||||
get.addFamily(FAMILIES[0]);
|
||||
get.setMaxVersions(Integer.MAX_VALUE);
|
||||
get.readAllVersions();
|
||||
Result result = ht.get(get);
|
||||
assertTrue(Bytes.equals(result.getValue(FAMILIES[0], QUALIFIER), VALUES[0]));
|
||||
}
|
||||
|
@ -5301,8 +5301,9 @@ public class TestFromClientSide {
|
|||
// get the block cache and region
|
||||
String regionName = locator.getAllRegionLocations().get(0).getRegionInfo().getEncodedName();
|
||||
|
||||
Region region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
|
||||
Store store = region.getStores().iterator().next();
|
||||
HRegion region = (HRegion) TEST_UTIL.getRSForFirstRegionInTable(tableName)
|
||||
.getRegion(regionName);
|
||||
HStore store = region.getStores().iterator().next();
|
||||
CacheConfig cacheConf = store.getCacheConfig();
|
||||
cacheConf.setCacheDataOnWrite(true);
|
||||
cacheConf.setEvictOnClose(true);
|
||||
|
@ -5394,15 +5395,14 @@ public class TestFromClientSide {
|
|||
}
|
||||
}
|
||||
|
||||
private void waitForStoreFileCount(Store store, int count, int timeout)
|
||||
throws InterruptedException {
|
||||
private void waitForStoreFileCount(HStore store, int count, int timeout)
|
||||
throws InterruptedException {
|
||||
long start = System.currentTimeMillis();
|
||||
while (start + timeout > System.currentTimeMillis() &&
|
||||
store.getStorefilesCount() != count) {
|
||||
while (start + timeout > System.currentTimeMillis() && store.getStorefilesCount() != count) {
|
||||
Thread.sleep(100);
|
||||
}
|
||||
System.out.println("start=" + start + ", now=" +
|
||||
System.currentTimeMillis() + ", cur=" + store.getStorefilesCount());
|
||||
System.out.println("start=" + start + ", now=" + System.currentTimeMillis() + ", cur=" +
|
||||
store.getStorefilesCount());
|
||||
assertEquals(count, store.getStorefilesCount());
|
||||
}
|
||||
|
||||
|
@ -5462,8 +5462,8 @@ public class TestFromClientSide {
|
|||
// Test Initialization.
|
||||
byte [] startKey = Bytes.toBytes("ddc");
|
||||
byte [] endKey = Bytes.toBytes("mmm");
|
||||
final TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
Table t = TEST_UTIL.createMultiRegionTable(tableName, new byte[][] { FAMILY }, 10);
|
||||
TableName tableName = TableName.valueOf(name.getMethodName());
|
||||
TEST_UTIL.createMultiRegionTable(tableName, new byte[][] { FAMILY }, 10);
|
||||
|
||||
int numOfRegions = -1;
|
||||
try (RegionLocator r = TEST_UTIL.getConnection().getRegionLocator(tableName)) {
|
||||
|
|
|
@ -125,7 +125,8 @@ public class TestRegionObserverScannerOpenHook {
|
|||
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt)
|
||||
throws IOException {
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -25,14 +25,15 @@ import java.util.Collection;
|
|||
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
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.Put;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression;
|
||||
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
|
||||
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.testclassification.IOTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -109,7 +110,7 @@ public class TestForceCacheImportantBlocks {
|
|||
setBloomFilterType(BLOOM_TYPE);
|
||||
hcd.setBlocksize(BLOCK_SIZE);
|
||||
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();
|
||||
CacheStats stats = cache.getStats();
|
||||
writeTestData(region);
|
||||
|
|
|
@ -30,21 +30,21 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.ByteBufferKeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.regionserver.ChunkCreator;
|
||||
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.MemStoreLABImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -307,7 +307,7 @@ public class TestScannerFromBucketCache {
|
|||
put.add(kv21);
|
||||
region.put(put);
|
||||
region.flush(true);
|
||||
Store store = region.getStore(fam1);
|
||||
HStore store = region.getStore(fam1);
|
||||
while (store.getStorefilesCount() <= 0) {
|
||||
try {
|
||||
Thread.sleep(20);
|
||||
|
|
|
@ -69,7 +69,7 @@ public class DelegatingKeyValueScanner implements KeyValueScanner {
|
|||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
|
|
|
@ -54,10 +54,11 @@ public class NoOpScanPolicyObserver implements RegionCoprocessor, RegionObserver
|
|||
public InternalScanner preFlushScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
Store store, List<KeyValueScanner> scanners, InternalScanner s, long readPoint)
|
||||
throws IOException {
|
||||
ScanInfo oldSI = store.getScanInfo();
|
||||
HStore hs = (HStore) store;
|
||||
ScanInfo oldSI = hs.getScanInfo();
|
||||
ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -69,11 +70,12 @@ public class NoOpScanPolicyObserver implements RegionCoprocessor, RegionObserver
|
|||
final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
|
||||
List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
|
||||
InternalScanner s, CompactionLifeCycleTracker tracker, long readPoint) throws IOException {
|
||||
HStore hs = (HStore) store;
|
||||
// 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(),
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -81,11 +83,12 @@ public class NoOpScanPolicyObserver implements RegionCoprocessor, RegionObserver
|
|||
public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
|
||||
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPoint)
|
||||
throws IOException {
|
||||
HStore hs = (HStore) store;
|
||||
Region r = c.getEnvironment().getRegion();
|
||||
return scan.isReversed()
|
||||
? new ReversedStoreScanner((HStore) store, store.getScanInfo(), scan, targetCols,
|
||||
? new ReversedStoreScanner(hs, hs.getScanInfo(), scan, targetCols,
|
||||
r.getReadPoint(scan.getIsolationLevel()))
|
||||
: new StoreScanner((HStore) store, store.getScanInfo(), scan, targetCols,
|
||||
: new StoreScanner(hs, hs.getScanInfo(), scan, targetCols,
|
||||
r.getReadPoint(scan.getIsolationLevel()));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,6 +17,21 @@
|
|||
*/
|
||||
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.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -60,21 +75,6 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
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,
|
||||
* and HRegion.append
|
||||
|
@ -84,7 +84,7 @@ public class TestAtomicOperation {
|
|||
private static final Log LOG = LogFactory.getLog(TestAtomicOperation.class);
|
||||
@Rule public TestName name = new TestName();
|
||||
|
||||
Region region = null;
|
||||
HRegion region = null;
|
||||
private HBaseTestingUtility TEST_UTIL = HBaseTestingUtility.createLocalHTU();
|
||||
|
||||
// Test names
|
||||
|
|
|
@ -208,7 +208,7 @@ public class TestCacheOnWriteInSchema {
|
|||
public void testCacheOnWriteInSchema() throws IOException {
|
||||
// Write some random data into the store
|
||||
StoreFileWriter writer = store.createWriterInTmp(Integer.MAX_VALUE,
|
||||
HFile.DEFAULT_COMPRESSION_ALGORITHM, false, true, false);
|
||||
HFile.DEFAULT_COMPRESSION_ALGORITHM, false, true, false, false);
|
||||
writeStoreFile(writer);
|
||||
writer.close();
|
||||
// Verify the block types of interest were cached on write
|
||||
|
|
|
@ -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_BYTES;
|
||||
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.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
@ -182,7 +183,7 @@ public class TestCompaction {
|
|||
spyR.compactStores();
|
||||
|
||||
// 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());
|
||||
assertTrue(s.getStorefilesSize() > 15*1000);
|
||||
// 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
|
||||
// Lower TTL and expire to ensure that all our entries have been wiped
|
||||
final int ttl = 1000;
|
||||
for (Store hstore: this.r.stores.values()) {
|
||||
HStore store = (HStore)hstore;
|
||||
for (HStore store: this.r.stores.values()) {
|
||||
ScanInfo old = store.getScanInfo();
|
||||
ScanInfo si = new ScanInfo(old.getConfiguration(), old.getFamily(), old.getMinVersions(),
|
||||
old.getMaxVersions(), ttl, old.getKeepDeletedCells(), HConstants.DEFAULT_BLOCKSIZE, 0,
|
||||
|
@ -307,7 +307,7 @@ public class TestCompaction {
|
|||
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
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);
|
||||
// wait for the latch to complete.
|
||||
latch.await();
|
||||
|
@ -340,7 +340,7 @@ public class TestCompaction {
|
|||
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
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);
|
||||
// wait for the latch to complete.
|
||||
latch.await(120, TimeUnit.SECONDS);
|
||||
|
@ -380,7 +380,7 @@ public class TestCompaction {
|
|||
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);
|
||||
}
|
||||
// wait for the latch to complete.
|
||||
|
|
|
@ -180,7 +180,7 @@ public class TestCompactionFileNotFound {
|
|||
// Refresh store files post compaction, this should not open already compacted files
|
||||
hr1.refreshStoreFiles(true);
|
||||
// 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();
|
||||
}
|
||||
try {
|
||||
|
|
|
@ -40,9 +40,9 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
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.io.hfile.BlockCache;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
@ -288,10 +288,9 @@ public class TestCompoundBloomFilter {
|
|||
byte[] qualifier) {
|
||||
Scan scan = new Scan().withStartRow(row).withStopRow(row, true);
|
||||
scan.addColumn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME), qualifier);
|
||||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
when(hcd.getName()).thenReturn(Bytes.toBytes(RandomKeyValueUtil.COLUMN_FAMILY_NAME));
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor())
|
||||
.thenReturn(ColumnFamilyDescriptorBuilder.of(RandomKeyValueUtil.COLUMN_FAMILY_NAME));
|
||||
return scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
}
|
||||
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.security.Key;
|
||||
import java.security.SecureRandom;
|
||||
|
@ -539,7 +541,7 @@ public class TestHMobStore {
|
|||
// Trigger major compaction
|
||||
this.store.triggerMajorCompaction();
|
||||
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);
|
||||
Assert.assertEquals(1, this.store.getStorefiles().size());
|
||||
|
||||
|
|
|
@ -284,7 +284,7 @@ public class TestHRegion {
|
|||
@Test
|
||||
public void testCloseCarryingSnapshot() throws IOException {
|
||||
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.
|
||||
byte [] value = Bytes.toBytes(method);
|
||||
// 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,
|
||||
COLUMN_FAMILY_BYTES);
|
||||
|
||||
Store store = region.getStore(COLUMN_FAMILY_BYTES);
|
||||
HStore store = region.getStore(COLUMN_FAMILY_BYTES);
|
||||
// Get some random bytes.
|
||||
byte [] value = Bytes.toBytes(method);
|
||||
faultyLog.setStoreFlushCtx(store.createFlushContext(12345));
|
||||
|
@ -350,7 +350,7 @@ public class TestHRegion {
|
|||
} finally {
|
||||
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);
|
||||
HBaseTestingUtility.closeRegionAndWAL(region);
|
||||
}
|
||||
|
@ -382,7 +382,7 @@ public class TestHRegion {
|
|||
FSHLog hLog = new FSHLog(fs, rootDir, "testMemstoreSizeWithFlushCanceling", CONF);
|
||||
HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog,
|
||||
COLUMN_FAMILY_BYTES);
|
||||
Store store = region.getStore(COLUMN_FAMILY_BYTES);
|
||||
HStore store = region.getStore(COLUMN_FAMILY_BYTES);
|
||||
assertEquals(0, region.getMemstoreSize());
|
||||
|
||||
// Put some value and make sure flush could be completed normally
|
||||
|
@ -394,7 +394,7 @@ public class TestHRegion {
|
|||
assertTrue(onePutSize > 0);
|
||||
region.flush(true);
|
||||
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
|
||||
RegionCoprocessorHost normalCPHost = region.getCoprocessorHost();
|
||||
|
@ -406,13 +406,13 @@ public class TestHRegion {
|
|||
region.flush(true);
|
||||
assertEquals("memstoreSize should NOT be zero", onePutSize, region.getMemstoreSize());
|
||||
assertEquals("flushable size should NOT be zero", onePutSize,
|
||||
store.getSizeToFlush().getDataSize());
|
||||
store.getFlushableSize().getDataSize());
|
||||
|
||||
// set normalCPHost and flush again, the snapshot will be flushed
|
||||
region.setCoprocessorHost(normalCPHost);
|
||||
region.flush(true);
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -424,7 +424,7 @@ public class TestHRegion {
|
|||
FSHLog hLog = new FSHLog(fs, rootDir, testName, CONF);
|
||||
HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog,
|
||||
COLUMN_FAMILY_BYTES);
|
||||
Store store = region.getStore(COLUMN_FAMILY_BYTES);
|
||||
HStore store = region.getStore(COLUMN_FAMILY_BYTES);
|
||||
assertEquals(0, region.getMemstoreSize());
|
||||
|
||||
// Put one value
|
||||
|
@ -450,7 +450,7 @@ public class TestHRegion {
|
|||
long expectedSize = onePutSize * 2;
|
||||
assertEquals("memstoreSize should be incremented", expectedSize, region.getMemstoreSize());
|
||||
assertEquals("flushable size should be incremented", expectedSize,
|
||||
store.getSizeToFlush().getDataSize());
|
||||
store.getFlushableSize().getDataSize());
|
||||
|
||||
region.setCoprocessorHost(null);
|
||||
HBaseTestingUtility.closeRegionAndWAL(region);
|
||||
|
@ -565,7 +565,7 @@ public class TestHRegion {
|
|||
p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[])null));
|
||||
region.put(p1);
|
||||
// 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);
|
||||
storeFlushCtx.prepare();
|
||||
// Now add two entries to the foreground memstore.
|
||||
|
@ -699,7 +699,7 @@ public class TestHRegion {
|
|||
}
|
||||
MonitoredTask status = TaskMonitor.get().createStatus(method);
|
||||
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);
|
||||
}
|
||||
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
|
||||
|
@ -751,7 +751,7 @@ public class TestHRegion {
|
|||
long recoverSeqId = 1030;
|
||||
MonitoredTask status = TaskMonitor.get().createStatus(method);
|
||||
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);
|
||||
}
|
||||
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
|
||||
|
@ -796,7 +796,7 @@ public class TestHRegion {
|
|||
dos.close();
|
||||
|
||||
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);
|
||||
}
|
||||
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, null);
|
||||
|
@ -854,7 +854,7 @@ public class TestHRegion {
|
|||
long recoverSeqId = 1030;
|
||||
Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
MonitoredTask status = TaskMonitor.get().createStatus(method);
|
||||
for (Store store : region.getStores()) {
|
||||
for (HStore store : region.getStores()) {
|
||||
maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
|
||||
}
|
||||
long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
|
||||
|
@ -3713,7 +3713,7 @@ public class TestHRegion {
|
|||
|
||||
if (i != 0 && i % compactInterval == 0) {
|
||||
region.compact(true);
|
||||
for (Store store : region.getStores()) {
|
||||
for (HStore store : region.getStores()) {
|
||||
store.closeAndArchiveCompactedFiles();
|
||||
}
|
||||
}
|
||||
|
@ -3893,7 +3893,7 @@ public class TestHRegion {
|
|||
// Compact regularly to avoid creating too many files and exceeding
|
||||
// the ulimit.
|
||||
region.compact(false);
|
||||
for (Store store : region.getStores()) {
|
||||
for (HStore store : region.getStores()) {
|
||||
store.closeAndArchiveCompactedFiles();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,8 +35,6 @@ import static org.mockito.Mockito.times;
|
|||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
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.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
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.Get;
|
||||
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.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult;
|
||||
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.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.protobuf.ProtobufUtil;
|
||||
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.EventType;
|
||||
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
|
||||
|
@ -127,7 +128,7 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// per test fields
|
||||
private Path rootDir;
|
||||
private HTableDescriptor htd;
|
||||
private TableDescriptor htd;
|
||||
private long time;
|
||||
private RegionServerServices rss;
|
||||
private HRegionInfo primaryHri, secondaryHri;
|
||||
|
@ -146,11 +147,11 @@ public class TestHRegionReplayEvents {
|
|||
rootDir = new Path(dir + method);
|
||||
TEST_UTIL.getConfiguration().set(HConstants.HBASE_DIR, rootDir.toString());
|
||||
method = name.getMethodName();
|
||||
|
||||
htd = new HTableDescriptor(TableName.valueOf(method));
|
||||
TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(TableName.valueOf(method));
|
||||
for (byte[] family : families) {
|
||||
htd.addFamily(new HColumnDescriptor(family));
|
||||
builder.addColumnFamily(ColumnFamilyDescriptorBuilder.of(family));
|
||||
}
|
||||
htd = builder.build();
|
||||
|
||||
time = System.currentTimeMillis();
|
||||
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null);
|
||||
|
@ -338,10 +339,10 @@ public class TestHRegionReplayEvents {
|
|||
if (flushDesc != null) {
|
||||
// first verify that everything is replayed and visible before flush event replay
|
||||
verifyData(secondaryRegion, 0, lastReplayed, cq, families);
|
||||
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long storeMemstoreSize = store.getMemStoreSize();
|
||||
HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long storeMemstoreSize = store.getMemStoreSize().getHeapSize();
|
||||
long regionMemstoreSize = secondaryRegion.getMemstoreSize();
|
||||
long storeFlushableSize = store.getFlushableSize();
|
||||
long storeFlushableSize = store.getFlushableSize().getHeapSize();
|
||||
long storeSize = store.getSize();
|
||||
long storeSizeUncompressed = store.getStoreSizeUncompressed();
|
||||
if (flushDesc.getAction() == FlushAction.START_FLUSH) {
|
||||
|
@ -351,7 +352,7 @@ public class TestHRegionReplayEvents {
|
|||
assertEquals(result.flushOpSeqId, flushDesc.getFlushSequenceNumber());
|
||||
|
||||
// assert that the store memstore is smaller now
|
||||
long newStoreMemstoreSize = store.getMemStoreSize();
|
||||
long newStoreMemstoreSize = store.getMemStoreSize().getHeapSize();
|
||||
LOG.info("Memstore size reduced by:"
|
||||
+ StringUtils.humanReadableInt(newStoreMemstoreSize - storeMemstoreSize));
|
||||
assertTrue(storeMemstoreSize > newStoreMemstoreSize);
|
||||
|
@ -362,10 +363,10 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// assert that the flush files are picked
|
||||
expectedStoreFileCount++;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
long newFlushableSize = store.getFlushableSize();
|
||||
long newFlushableSize = store.getFlushableSize().getHeapSize();
|
||||
assertTrue(storeFlushableSize > newFlushableSize);
|
||||
|
||||
// assert that the region memstore is smaller now
|
||||
|
@ -383,7 +384,7 @@ public class TestHRegionReplayEvents {
|
|||
secondaryRegion.replayWALCompactionMarker(compactionDesc, true, false, Long.MAX_VALUE);
|
||||
|
||||
// assert that the compaction is applied
|
||||
for (Store store : secondaryRegion.getStores()) {
|
||||
for (HStore store : secondaryRegion.getStores()) {
|
||||
if (store.getColumnFamilyName().equals("cf1")) {
|
||||
assertEquals(1, store.getStorefilesCount());
|
||||
} else {
|
||||
|
@ -401,7 +402,7 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
LOG.info("-- Verifying edits from primary. Ensuring that files are not deleted");
|
||||
verifyData(primaryRegion, 0, lastReplayed, cq, families);
|
||||
for (Store store : primaryRegion.getStores()) {
|
||||
for (HStore store : primaryRegion.getStores()) {
|
||||
if (store.getColumnFamilyName().equals("cf1")) {
|
||||
assertEquals(1, store.getStorefilesCount());
|
||||
} else {
|
||||
|
@ -437,10 +438,10 @@ public class TestHRegionReplayEvents {
|
|||
= WALEdit.getFlushDescriptor(entry.getEdit().getCells().get(0));
|
||||
if (flushDesc != null) {
|
||||
// first verify that everything is replayed and visible before flush event replay
|
||||
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long storeMemstoreSize = store.getMemStoreSize();
|
||||
HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long storeMemstoreSize = store.getMemStoreSize().getHeapSize();
|
||||
long regionMemstoreSize = secondaryRegion.getMemstoreSize();
|
||||
long storeFlushableSize = store.getFlushableSize();
|
||||
long storeFlushableSize = store.getFlushableSize().getHeapSize();
|
||||
|
||||
if (flushDesc.getAction() == FlushAction.START_FLUSH) {
|
||||
startFlushDesc = flushDesc;
|
||||
|
@ -452,7 +453,7 @@ public class TestHRegionReplayEvents {
|
|||
assertTrue(storeFlushableSize > 0);
|
||||
|
||||
// assert that the store memstore is smaller now
|
||||
long newStoreMemstoreSize = store.getMemStoreSize();
|
||||
long newStoreMemstoreSize = store.getMemStoreSize().getHeapSize();
|
||||
LOG.info("Memstore size reduced by:"
|
||||
+ StringUtils.humanReadableInt(newStoreMemstoreSize - storeMemstoreSize));
|
||||
assertTrue(storeMemstoreSize > newStoreMemstoreSize);
|
||||
|
@ -571,7 +572,7 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// no store files in the region
|
||||
int expectedStoreFileCount = 0;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
long regionMemstoreSize = secondaryRegion.getMemstoreSize();
|
||||
|
@ -586,11 +587,11 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// assert that the flush files are picked
|
||||
expectedStoreFileCount++;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long newFlushableSize = store.getFlushableSize();
|
||||
HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long newFlushableSize = store.getFlushableSize().getHeapSize();
|
||||
assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped
|
||||
|
||||
// assert that the region memstore is same as before
|
||||
|
@ -661,7 +662,7 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// no store files in the region
|
||||
int expectedStoreFileCount = 0;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
long regionMemstoreSize = secondaryRegion.getMemstoreSize();
|
||||
|
@ -676,11 +677,11 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// assert that the flush files are picked
|
||||
expectedStoreFileCount++;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long newFlushableSize = store.getFlushableSize();
|
||||
HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long newFlushableSize = store.getFlushableSize().getHeapSize();
|
||||
assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped
|
||||
|
||||
// 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
|
||||
int expectedStoreFileCount = 0;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
long regionMemstoreSize = secondaryRegion.getMemstoreSize();
|
||||
|
@ -772,8 +773,8 @@ public class TestHRegionReplayEvents {
|
|||
assertTrue(commitFlushDesc.getFlushSequenceNumber() > 0);
|
||||
|
||||
// ensure all files are visible in secondary
|
||||
for (Store store : secondaryRegion.getStores()) {
|
||||
assertTrue(store.getMaxSequenceId() <= secondaryRegion.getReadPoint(null));
|
||||
for (HStore store : secondaryRegion.getStores()) {
|
||||
assertTrue(store.getMaxSequenceId().orElse(0L) <= secondaryRegion.getReadPoint(null));
|
||||
}
|
||||
|
||||
LOG.info("-- Replaying flush commit in secondary" + commitFlushDesc);
|
||||
|
@ -781,11 +782,11 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// assert that the flush files are picked
|
||||
expectedStoreFileCount++;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long newFlushableSize = store.getFlushableSize();
|
||||
HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long newFlushableSize = store.getFlushableSize().getHeapSize();
|
||||
if (droppableMemstore) {
|
||||
assertTrue(newFlushableSize == 0); // assert that the memstore is dropped
|
||||
} else {
|
||||
|
@ -860,7 +861,7 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// no store files in the region
|
||||
int expectedStoreFileCount = 0;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
long regionMemstoreSize = secondaryRegion.getMemstoreSize();
|
||||
|
@ -872,11 +873,11 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// assert that the flush files are picked
|
||||
expectedStoreFileCount++;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long newFlushableSize = store.getFlushableSize();
|
||||
HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
long newFlushableSize = store.getFlushableSize().getHeapSize();
|
||||
assertTrue(newFlushableSize == 0);
|
||||
|
||||
// assert that the region memstore is empty
|
||||
|
@ -941,7 +942,7 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// no store files in the region
|
||||
int expectedStoreFileCount = 0;
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
|
||||
|
@ -951,11 +952,11 @@ public class TestHRegionReplayEvents {
|
|||
|
||||
// assert that the flush files are picked
|
||||
expectedStoreFileCount = 2; // two flushes happened
|
||||
for (Store s : secondaryRegion.getStores()) {
|
||||
for (HStore s : secondaryRegion.getStores()) {
|
||||
assertEquals(expectedStoreFileCount, s.getStorefilesCount());
|
||||
}
|
||||
Store store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
MemstoreSize newSnapshotSize = store.getSizeOfSnapshot();
|
||||
HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1"));
|
||||
MemstoreSize newSnapshotSize = store.getSnapshotSize();
|
||||
assertTrue(newSnapshotSize.getDataSize() == 0);
|
||||
|
||||
// assert that the region memstore is empty
|
||||
|
|
|
@ -1,5 +1,4 @@
|
|||
/*
|
||||
*
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
|
@ -16,18 +15,18 @@
|
|||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
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.permission.FsPermission;
|
||||
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.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
|
||||
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.Scan;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
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.FSUtils;
|
||||
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.WALFactory;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
|
@ -109,7 +113,7 @@ import org.mockito.Mockito;
|
|||
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 })
|
||||
public class TestHStore {
|
||||
|
@ -117,6 +121,7 @@ public class TestHStore {
|
|||
@Rule
|
||||
public TestName name = new TestName();
|
||||
|
||||
HRegion region;
|
||||
HStore store;
|
||||
byte [] table = Bytes.toBytes("table");
|
||||
byte [] family = Bytes.toBytes("family");
|
||||
|
@ -138,8 +143,8 @@ public class TestHStore {
|
|||
long id = System.currentTimeMillis();
|
||||
Get get = new Get(row);
|
||||
|
||||
private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString();
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private static final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString();
|
||||
|
||||
|
||||
/**
|
||||
|
@ -164,55 +169,51 @@ public class TestHStore {
|
|||
init(methodName, TEST_UTIL.getConfiguration());
|
||||
}
|
||||
|
||||
private Store init(String methodName, Configuration conf) throws IOException {
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
private HStore init(String methodName, Configuration conf) throws IOException {
|
||||
// some of the tests write 4 versions and then flush
|
||||
// (with HBASE-4241, lower versions are collected on flush)
|
||||
hcd.setMaxVersions(4);
|
||||
return init(methodName, conf, hcd);
|
||||
return init(methodName, conf,
|
||||
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 {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
|
||||
return init(methodName, conf, htd, hcd);
|
||||
return init(methodName, conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)), hcd);
|
||||
}
|
||||
|
||||
private HStore init(String methodName, Configuration conf, HTableDescriptor htd,
|
||||
HColumnDescriptor hcd) throws IOException {
|
||||
return init(methodName, conf, htd, hcd, null);
|
||||
private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,
|
||||
ColumnFamilyDescriptor hcd) throws IOException {
|
||||
return init(methodName, conf, builder, hcd, null);
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
private HStore init(String methodName, Configuration conf, HTableDescriptor htd,
|
||||
HColumnDescriptor hcd, MyStoreHook hook) throws IOException {
|
||||
return init(methodName, conf, htd, hcd, hook, false);
|
||||
private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,
|
||||
ColumnFamilyDescriptor hcd, MyStoreHook hook) throws IOException {
|
||||
return init(methodName, conf, builder, hcd, hook, false);
|
||||
}
|
||||
@SuppressWarnings("deprecation")
|
||||
private HStore init(String methodName, Configuration conf, HTableDescriptor htd,
|
||||
HColumnDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {
|
||||
//Setting up a Store
|
||||
Path basedir = new Path(DIR+methodName);
|
||||
|
||||
private void initHRegion(String methodName, Configuration conf, TableDescriptorBuilder builder,
|
||||
ColumnFamilyDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {
|
||||
TableDescriptor htd = builder.addColumnFamily(hcd).build();
|
||||
Path basedir = new Path(DIR + methodName);
|
||||
Path tableDir = FSUtils.getTableDir(basedir, htd.getTableName());
|
||||
final Path logdir = new Path(basedir, AbstractFSWALProvider.getWALDirectoryName(methodName));
|
||||
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
|
||||
fs.delete(logdir, true);
|
||||
|
||||
if (htd.hasFamily(hcd.getName())) {
|
||||
htd.modifyFamily(hcd);
|
||||
} else {
|
||||
htd.addFamily(hcd);
|
||||
}
|
||||
ChunkCreator.initialize(MemStoreLABImpl.CHUNK_SIZE_DEFAULT, false,
|
||||
MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null);
|
||||
HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
|
||||
final Configuration walConf = new Configuration(conf);
|
||||
Configuration walConf = new Configuration(conf);
|
||||
FSUtils.setRootDir(walConf, basedir);
|
||||
final WALFactory wals = new WALFactory(walConf, null, methodName);
|
||||
HRegion region = new HRegion(tableDir, wals.getWAL(info.getEncodedNameAsBytes(),
|
||||
info.getTable().getNamespace()), fs, conf, info, htd, null);
|
||||
WALFactory wals = new WALFactory(walConf, null, methodName);
|
||||
region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info),
|
||||
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) {
|
||||
store = new HStore(region, hcd, conf);
|
||||
} else {
|
||||
|
@ -293,13 +294,14 @@ public class TestHStore {
|
|||
Configuration conf = HBaseConfiguration.create();
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setCompressionType(Compression.Algorithm.GZ);
|
||||
hcd.setDataBlockEncoding(DataBlockEncoding.DIFF);
|
||||
ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(family)
|
||||
.setCompressionType(Compression.Algorithm.GZ).setDataBlockEncoding(DataBlockEncoding.DIFF)
|
||||
.build();
|
||||
init(name.getMethodName(), conf, hcd);
|
||||
|
||||
// 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();
|
||||
writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1)));
|
||||
writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2)));
|
||||
|
@ -335,10 +337,8 @@ public class TestHStore {
|
|||
// Set the compaction threshold higher to avoid normal compactions.
|
||||
conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 5);
|
||||
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMinVersions(minVersions);
|
||||
hcd.setTimeToLive(ttl);
|
||||
init(name.getMethodName() + "-" + minVersions, conf, hcd);
|
||||
init(name.getMethodName() + "-" + minVersions, conf, ColumnFamilyDescriptorBuilder
|
||||
.newBuilder(family).setMinVersions(minVersions).setTimeToLive(ttl).build());
|
||||
|
||||
long storeTtl = this.store.getScanInfo().getTtl();
|
||||
long sleepTime = storeTtl / storeFileNum;
|
||||
|
@ -599,6 +599,22 @@ public class TestHStore {
|
|||
@After
|
||||
public void tearDown() throws Exception {
|
||||
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
|
||||
|
@ -824,17 +840,19 @@ public class TestHStore {
|
|||
|
||||
// HTD overrides XML.
|
||||
--anyValue;
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
htd.setConfiguration(CONFIG_KEY, Long.toString(anyValue));
|
||||
init(name.getMethodName() + "-htd", conf, htd, hcd);
|
||||
init(name.getMethodName() + "-htd", conf, TableDescriptorBuilder
|
||||
.newBuilder(TableName.valueOf(table)).setValue(CONFIG_KEY, Long.toString(anyValue)),
|
||||
ColumnFamilyDescriptorBuilder.of(family));
|
||||
assertTrue(store.throttleCompaction(anyValue + 1));
|
||||
assertFalse(store.throttleCompaction(anyValue));
|
||||
|
||||
// HCD overrides them both.
|
||||
--anyValue;
|
||||
hcd.setConfiguration(CONFIG_KEY, Long.toString(anyValue));
|
||||
init(name.getMethodName() + "-hcd", conf, htd, hcd);
|
||||
init(name.getMethodName() + "-hcd", conf,
|
||||
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));
|
||||
assertFalse(store.throttleCompaction(anyValue));
|
||||
}
|
||||
|
@ -862,7 +880,7 @@ public class TestHStore {
|
|||
private void addStoreFile() throws IOException {
|
||||
HStoreFile f = this.store.getStorefiles().iterator().next();
|
||||
Path storedir = f.getPath().getParent();
|
||||
long seqid = this.store.getMaxSequenceId();
|
||||
long seqid = this.store.getMaxSequenceId().orElse(0L);
|
||||
Configuration c = TEST_UTIL.getConfiguration();
|
||||
FileSystem fs = FileSystem.get(c);
|
||||
HFileContext fileContext = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();
|
||||
|
@ -989,20 +1007,23 @@ public class TestHStore {
|
|||
public void testNumberOfMemStoreScannersAfterFlush() throws IOException {
|
||||
long seqId = 100;
|
||||
long timestamp = System.currentTimeMillis();
|
||||
Cell cell0 = CellUtil.createCell(row, family, qf1, timestamp,
|
||||
KeyValue.Type.Put.getCode(), qf1);
|
||||
Cell cell0 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
|
||||
.setQualifier(qf1).setTimestamp(timestamp).setType(KeyValue.Type.Put.getCode())
|
||||
.setValue(qf1).build();
|
||||
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,
|
||||
KeyValue.Type.Put.getCode(), qf1);
|
||||
Cell cell1 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
|
||||
.setQualifier(qf2).setTimestamp(timestamp).setType(KeyValue.Type.Put.getCode())
|
||||
.setValue(qf1).build();
|
||||
CellUtil.setSequenceId(cell1, seqId);
|
||||
testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1));
|
||||
|
||||
seqId = 101;
|
||||
timestamp = System.currentTimeMillis();
|
||||
Cell cell2 = CellUtil.createCell(row2, family, qf2, timestamp,
|
||||
KeyValue.Type.Put.getCode(), qf1);
|
||||
Cell cell2 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row2).setFamily(family)
|
||||
.setQualifier(qf2).setTimestamp(timestamp).setType(KeyValue.Type.Put.getCode())
|
||||
.setValue(qf1).build();
|
||||
CellUtil.setSequenceId(cell2, seqId);
|
||||
testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1, cell2));
|
||||
}
|
||||
|
@ -1046,15 +1067,16 @@ public class TestHStore {
|
|||
}
|
||||
}
|
||||
|
||||
private Cell createCell(byte[] qualifier, long ts, long sequenceId, byte[] value) throws IOException {
|
||||
Cell c = CellUtil.createCell(row, family, qualifier, ts, KeyValue.Type.Put.getCode(), value);
|
||||
CellUtil.setSequenceId(c, sequenceId);
|
||||
return c;
|
||||
private Cell createCell(byte[] qualifier, long ts, long sequenceId, byte[] value)
|
||||
throws IOException {
|
||||
return createCell(row, qualifier, ts, sequenceId, value);
|
||||
}
|
||||
|
||||
private Cell createCell(byte[] row, byte[] qualifier, long ts, long sequenceId, byte[] value)
|
||||
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);
|
||||
return c;
|
||||
}
|
||||
|
@ -1148,8 +1170,6 @@ public class TestHStore {
|
|||
private void testFlushBeforeCompletingScan(MyListHook hook, Filter filter, int expectedSize)
|
||||
throws IOException, InterruptedException {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMaxVersions(1);
|
||||
byte[] r0 = Bytes.toBytes("row0");
|
||||
byte[] r1 = Bytes.toBytes("row1");
|
||||
byte[] r2 = Bytes.toBytes("row2");
|
||||
|
@ -1159,12 +1179,14 @@ public class TestHStore {
|
|||
MemstoreSize memStoreSize = new MemstoreSize();
|
||||
long ts = EnvironmentEdgeManager.currentTime();
|
||||
long seqId = 100;
|
||||
init(name.getMethodName(), conf, new HTableDescriptor(TableName.valueOf(table)), hcd, new MyStoreHook() {
|
||||
@Override
|
||||
public long getSmallestReadPoint(HStore store) {
|
||||
return seqId + 3;
|
||||
}
|
||||
});
|
||||
init(name.getMethodName(), conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)),
|
||||
ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(1).build(),
|
||||
new MyStoreHook() {
|
||||
@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
|
||||
store.add(createCell(r0, qf1, ts, seqId, value0), memStoreSize);
|
||||
store.add(createCell(r0, qf2, ts, seqId, value0), memStoreSize);
|
||||
|
@ -1210,9 +1232,8 @@ public class TestHStore {
|
|||
public void testCreateScannerAndSnapshotConcurrently() throws IOException, InterruptedException {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore.class.getName());
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
|
||||
init(name.getMethodName(), conf, hcd);
|
||||
init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
|
||||
.setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
|
||||
byte[] value = Bytes.toBytes("value");
|
||||
MemstoreSize memStoreSize = new MemstoreSize();
|
||||
long ts = EnvironmentEdgeManager.currentTime();
|
||||
|
@ -1402,9 +1423,8 @@ public class TestHStore {
|
|||
conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushSize));
|
||||
// Set the lower threshold to invoke the "MERGE" policy
|
||||
conf.set(MemStoreCompactor.COMPACTING_MEMSTORE_THRESHOLD_KEY, String.valueOf(0));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setInMemoryCompaction(MemoryCompactionPolicy.BASIC);
|
||||
init(name.getMethodName(), conf, hcd);
|
||||
init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
|
||||
.setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
|
||||
byte[] value = Bytes.toBytes("thisisavarylargevalue");
|
||||
MemstoreSize memStoreSize = new MemstoreSize();
|
||||
long ts = EnvironmentEdgeManager.currentTime();
|
||||
|
@ -1439,18 +1459,57 @@ public class TestHStore {
|
|||
storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
|
||||
}
|
||||
|
||||
private MyStore initMyStore(String methodName, Configuration conf, MyStoreHook hook)
|
||||
throws IOException {
|
||||
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
|
||||
HColumnDescriptor hcd = new HColumnDescriptor(family);
|
||||
hcd.setMaxVersions(5);
|
||||
return (MyStore) init(methodName, conf, htd, hcd, hook);
|
||||
@Test
|
||||
public void testAge() throws IOException {
|
||||
long currentTime = System.currentTimeMillis();
|
||||
ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
|
||||
edge.setValue(currentTime);
|
||||
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;
|
||||
|
||||
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 {
|
||||
super(region, family, confParam);
|
||||
this.hook = hook;
|
||||
|
@ -1473,8 +1532,10 @@ public class TestHStore {
|
|||
}
|
||||
|
||||
private abstract class MyStoreHook {
|
||||
|
||||
void getScanners(MyStore store) throws IOException {
|
||||
}
|
||||
|
||||
long getSmallestReadPoint(HStore store) {
|
||||
return store.getHRegion().getSmallestReadPoint();
|
||||
}
|
||||
|
@ -1482,13 +1543,10 @@ public class TestHStore {
|
|||
|
||||
@Test
|
||||
public void testSwitchingPreadtoStreamParallelyWithCompactionDischarger() throws Exception {
|
||||
int flushSize = 500;
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
conf.set("hbase.hstore.engine.class", DummyStoreEngine.class.getName());
|
||||
conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 0);
|
||||
// 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() {});
|
||||
MemstoreSize memStoreSize = new MemstoreSize();
|
||||
long ts = System.currentTimeMillis();
|
||||
|
@ -1514,7 +1572,6 @@ public class TestHStore {
|
|||
flushStore(store, seqID);
|
||||
|
||||
assertEquals(3, store.getStorefilesCount());
|
||||
ScanInfo scanInfo = store.getScanInfo();
|
||||
Scan scan = new Scan();
|
||||
scan.addFamily(family);
|
||||
Collection<HStoreFile> storefiles2 = store.getStorefiles();
|
||||
|
@ -1541,7 +1598,6 @@ public class TestHStore {
|
|||
ArrayList<HStoreFile> actualStorefiles1 = Lists.newArrayList(storefiles2);
|
||||
actualStorefiles1.removeAll(actualStorefiles);
|
||||
// Do compaction
|
||||
List<Exception> exceptions = new ArrayList<Exception>();
|
||||
MyThread thread = new MyThread(storeScanner);
|
||||
thread.start();
|
||||
store.replaceStoreFiles(actualStorefiles, actualStorefiles1);
|
||||
|
@ -1678,7 +1734,7 @@ public class TestHStore {
|
|||
public Object[] toArray() {return delegatee.toArray();}
|
||||
|
||||
@Override
|
||||
public <T> T[] toArray(T[] a) {return delegatee.toArray(a);}
|
||||
public <R> R[] toArray(R[] a) {return delegatee.toArray(a);}
|
||||
|
||||
@Override
|
||||
public boolean add(T e) {
|
||||
|
|
|
@ -45,6 +45,8 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
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.io.HFileLink;
|
||||
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
||||
|
@ -205,11 +207,10 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
@Test
|
||||
public void testEmptyStoreFileRestrictKeyRanges() throws Exception {
|
||||
StoreFileReader reader = mock(StoreFileReader.class);
|
||||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
HStore store = mock(HStore.class);
|
||||
byte[] cf = Bytes.toBytes("ty");
|
||||
when(hcd.getName()).thenReturn(cf);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
ColumnFamilyDescriptor cfd = ColumnFamilyDescriptorBuilder.of(cf);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(cfd);
|
||||
StoreFileScanner scanner =
|
||||
new StoreFileScanner(reader, mock(HFileScanner.class), false, false, 0, 0, true);
|
||||
Scan scan = new Scan();
|
||||
|
@ -526,10 +527,8 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
|
||||
Scan scan = new Scan(row.getBytes(),row.getBytes());
|
||||
scan.addColumn("family".getBytes(), "family:col".getBytes());
|
||||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
when(hcd.getName()).thenReturn(Bytes.toBytes("family"));
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
|
||||
boolean exists = scanner.shouldUseScanner(scan, store, Long.MIN_VALUE);
|
||||
if (i % 2 == 0) {
|
||||
if (!exists) falseNeg++;
|
||||
|
@ -713,10 +712,8 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
StoreFileScanner scanner = getStoreFileScanner(reader, false, false);
|
||||
assertEquals(expKeys[x], reader.generalBloomFilter.getKeyCount());
|
||||
|
||||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
when(hcd.getName()).thenReturn(Bytes.toBytes("family"));
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of("family"));
|
||||
// check false positives rate
|
||||
int falsePos = 0;
|
||||
int falseNeg = 0;
|
||||
|
@ -857,10 +854,8 @@ public class TestHStoreFile extends HBaseTestCase {
|
|||
|
||||
HStoreFile hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
|
||||
BloomType.NONE, true);
|
||||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
when(hcd.getName()).thenReturn(family);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(hcd);
|
||||
HStore store = mock(HStore.class);
|
||||
when(store.getColumnFamilyDescriptor()).thenReturn(ColumnFamilyDescriptorBuilder.of(family));
|
||||
hsf.initReader();
|
||||
StoreFileReader reader = hsf.getReader();
|
||||
StoreFileScanner scanner = getStoreFileScanner(reader, false, false);
|
||||
|
|
|
@ -99,7 +99,7 @@ public class TestKeepDeletes {
|
|||
// keep 3 versions, rows do not expire
|
||||
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3,
|
||||
HConstants.FOREVER, KeepDeletedCells.TRUE);
|
||||
Region region = hbu.createLocalHRegion(htd, null, null);
|
||||
HRegion region = hbu.createLocalHRegion(htd, null, null);
|
||||
|
||||
long ts = EnvironmentEdgeManager.currentTime();
|
||||
Put p = new Put(T1, ts);
|
||||
|
@ -241,7 +241,7 @@ public class TestKeepDeletes {
|
|||
// KEEP_DELETED_CELLS is NOT enabled
|
||||
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 3,
|
||||
HConstants.FOREVER, KeepDeletedCells.FALSE);
|
||||
Region region = hbu.createLocalHRegion(htd, null, null);
|
||||
HRegion region = hbu.createLocalHRegion(htd, null, null);
|
||||
|
||||
long ts = EnvironmentEdgeManager.currentTime();
|
||||
Put p = new Put(T1, ts);
|
||||
|
@ -408,7 +408,7 @@ public class TestKeepDeletes {
|
|||
public void testDeleteMarkerExpirationEmptyStore() throws Exception {
|
||||
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1,
|
||||
HConstants.FOREVER, KeepDeletedCells.TRUE);
|
||||
Region region = hbu.createLocalHRegion(htd, null, null);
|
||||
HRegion region = hbu.createLocalHRegion(htd, null, null);
|
||||
|
||||
long ts = EnvironmentEdgeManager.currentTime();
|
||||
|
||||
|
@ -451,7 +451,7 @@ public class TestKeepDeletes {
|
|||
public void testDeleteMarkerExpiration() throws Exception {
|
||||
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1,
|
||||
HConstants.FOREVER, KeepDeletedCells.TRUE);
|
||||
Region region = hbu.createLocalHRegion(htd, null, null);
|
||||
HRegion region = hbu.createLocalHRegion(htd, null, null);
|
||||
|
||||
long ts = EnvironmentEdgeManager.currentTime();
|
||||
|
||||
|
@ -514,7 +514,7 @@ public class TestKeepDeletes {
|
|||
public void testWithOldRow() throws Exception {
|
||||
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1,
|
||||
HConstants.FOREVER, KeepDeletedCells.TRUE);
|
||||
Region region = hbu.createLocalHRegion(htd, null, null);
|
||||
HRegion region = hbu.createLocalHRegion(htd, null, null);
|
||||
|
||||
long ts = EnvironmentEdgeManager.currentTime();
|
||||
|
||||
|
@ -674,7 +674,7 @@ public class TestKeepDeletes {
|
|||
public void testDeleteMarkerVersioning() throws Exception {
|
||||
HTableDescriptor htd = hbu.createTableDescriptor(name.getMethodName(), 0, 1,
|
||||
HConstants.FOREVER, KeepDeletedCells.TRUE);
|
||||
Region region = hbu.createLocalHRegion(htd, null, null);
|
||||
HRegion region = hbu.createLocalHRegion(htd, null, null);
|
||||
|
||||
long ts = EnvironmentEdgeManager.currentTime();
|
||||
Put p = new Put(T1, ts);
|
||||
|
@ -818,7 +818,7 @@ public class TestKeepDeletes {
|
|||
public void testWithMinVersions() throws Exception {
|
||||
HTableDescriptor htd =
|
||||
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
|
||||
|
||||
|
@ -897,7 +897,7 @@ public class TestKeepDeletes {
|
|||
public void testWithTTL() throws Exception {
|
||||
HTableDescriptor htd =
|
||||
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
|
||||
|
||||
|
@ -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();
|
||||
s.setRaw(true);
|
||||
// use max versions from the store(s)
|
||||
|
|
|
@ -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_BYTES;
|
||||
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.assertNotNull;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
@ -181,8 +182,8 @@ public class TestMajorCompaction {
|
|||
|
||||
public void majorCompactionWithDataBlockEncoding(boolean inCacheOnly)
|
||||
throws Exception {
|
||||
Map<Store, HFileDataBlockEncoder> replaceBlockCache = new HashMap<>();
|
||||
for (Store store : r.getStores()) {
|
||||
Map<HStore, HFileDataBlockEncoder> replaceBlockCache = new HashMap<>();
|
||||
for (HStore store : r.getStores()) {
|
||||
HFileDataBlockEncoder blockEncoder = store.getDataBlockEncoder();
|
||||
replaceBlockCache.put(store, blockEncoder);
|
||||
final DataBlockEncoding inCache = DataBlockEncoding.PREFIX;
|
||||
|
@ -194,7 +195,7 @@ public class TestMajorCompaction {
|
|||
majorCompaction();
|
||||
|
||||
// restore settings
|
||||
for (Entry<Store, HFileDataBlockEncoder> entry : replaceBlockCache.entrySet()) {
|
||||
for (Entry<HStore, HFileDataBlockEncoder> entry : replaceBlockCache.entrySet()) {
|
||||
((HStore)entry.getKey()).setDataBlockEncoderInTest(entry.getValue());
|
||||
}
|
||||
}
|
||||
|
@ -211,11 +212,11 @@ public class TestMajorCompaction {
|
|||
// Default is that there only 3 (MAXVERSIONS) versions allowed per column.
|
||||
//
|
||||
// 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());
|
||||
|
||||
// see if CompactionProgress is in place but null
|
||||
for (Store store : r.getStores()) {
|
||||
for (HStore store : r.getStores()) {
|
||||
assertNull(store.getCompactionProgress());
|
||||
}
|
||||
|
||||
|
@ -224,7 +225,7 @@ public class TestMajorCompaction {
|
|||
|
||||
// see if CompactionProgress has done its thing on at least one store
|
||||
int storeCount = 0;
|
||||
for (Store store : r.getStores()) {
|
||||
for (HStore store : r.getStores()) {
|
||||
CompactionProgress progress = store.getCompactionProgress();
|
||||
if( progress != null ) {
|
||||
++storeCount;
|
||||
|
@ -240,8 +241,7 @@ public class TestMajorCompaction {
|
|||
secondRowBytes[START_KEY_BYTES.length - 1]++;
|
||||
|
||||
// Always 3 versions if that is what max versions is.
|
||||
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).
|
||||
setMaxVersions(100));
|
||||
result = r.get(new Get(secondRowBytes).addFamily(COLUMN_FAMILY_TEXT).readVersions(100));
|
||||
LOG.debug("Row " + Bytes.toStringBinary(secondRowBytes) + " after " +
|
||||
"initial compaction: " + result);
|
||||
assertEquals("Invalid number of versions of row "
|
||||
|
@ -260,26 +260,26 @@ public class TestMajorCompaction {
|
|||
r.delete(delete);
|
||||
|
||||
// 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());
|
||||
|
||||
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());
|
||||
|
||||
// Add a bit of data and flush. Start adding at 'bbb'.
|
||||
createSmallerStoreFile(this.r);
|
||||
r.flush(true);
|
||||
// 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());
|
||||
|
||||
// Force major compaction.
|
||||
r.compact(true);
|
||||
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());
|
||||
|
||||
// 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
|
||||
// Lower TTL and expire to ensure that all our entries have been wiped
|
||||
final int ttl = 1000;
|
||||
for (Store hstore : r.getStores()) {
|
||||
HStore store = ((HStore) hstore);
|
||||
for (HStore store : r.getStores()) {
|
||||
ScanInfo old = store.getScanInfo();
|
||||
ScanInfo si = new ScanInfo(old.getConfiguration(), old.getFamily(), old.getMinVersions(),
|
||||
old.getMaxVersions(), ttl, old.getKeepDeletedCells(), old.getPreadMaxBytes(), 0,
|
||||
|
@ -411,7 +410,7 @@ public class TestMajorCompaction {
|
|||
*/
|
||||
@Test
|
||||
public void testNonUserMajorCompactionRequest() throws Exception {
|
||||
Store store = r.getStore(COLUMN_FAMILY);
|
||||
HStore store = r.getStore(COLUMN_FAMILY);
|
||||
createStoreFile(r);
|
||||
for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
|
||||
createStoreFile(r);
|
||||
|
@ -431,14 +430,14 @@ public class TestMajorCompaction {
|
|||
*/
|
||||
@Test
|
||||
public void testUserMajorCompactionRequest() throws IOException{
|
||||
Store store = r.getStore(COLUMN_FAMILY);
|
||||
HStore store = r.getStore(COLUMN_FAMILY);
|
||||
createStoreFile(r);
|
||||
for (int i = 0; i < MAX_FILES_TO_COMPACT + 1; i++) {
|
||||
createStoreFile(r);
|
||||
}
|
||||
store.triggerMajorCompaction();
|
||||
CompactionRequest request =
|
||||
store.requestCompaction(Store.PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null).get()
|
||||
store.requestCompaction(PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null).get()
|
||||
.getRequest();
|
||||
assertNotNull("Expected to receive a compaction request", request);
|
||||
assertEquals(
|
||||
|
|
|
@ -31,14 +31,14 @@ import org.apache.hadoop.hbase.HBaseTestCase;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
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.Get;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
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.wal.WAL;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
|
@ -46,7 +46,6 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
|
||||
/**
|
||||
* Test minor compactions
|
||||
*/
|
||||
|
@ -57,7 +56,7 @@ public class TestMinorCompaction {
|
|||
private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU();
|
||||
protected Configuration conf = UTIL.getConfiguration();
|
||||
|
||||
private Region r = null;
|
||||
private HRegion r = null;
|
||||
private HTableDescriptor htd = null;
|
||||
private int compactionThreshold;
|
||||
private byte[] firstRowBytes, secondRowBytes, thirdRowBytes;
|
||||
|
@ -205,7 +204,7 @@ public class TestMinorCompaction {
|
|||
assertEquals(compactionThreshold, result.size());
|
||||
|
||||
// do a compaction
|
||||
Store store2 = r.getStore(fam2);
|
||||
HStore store2 = r.getStore(fam2);
|
||||
int numFiles1 = store2.getStorefiles().size();
|
||||
assertTrue("Was expecting to see 4 store files", numFiles1 > compactionThreshold); // > 3
|
||||
((HStore)store2).compactRecentForTestingAssumingDefaultPolicy(compactionThreshold); // = 3
|
||||
|
|
|
@ -298,7 +298,7 @@ public class TestMobStoreCompaction {
|
|||
}
|
||||
|
||||
private int countStoreFiles() throws IOException {
|
||||
Store store = region.getStore(COLUMN_FAMILY);
|
||||
HStore store = region.getStore(COLUMN_FAMILY);
|
||||
return store.getStorefilesCount();
|
||||
}
|
||||
|
||||
|
|
|
@ -152,9 +152,9 @@ public class TestPerColumnFamilyFlush {
|
|||
long smallestSeqCF3 = region.getOldestSeqIdOfStore(FAMILY3);
|
||||
|
||||
// Find the sizes of the memstores of each CF.
|
||||
MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
|
||||
|
||||
// Get the overall smallest LSN in the region's memstores.
|
||||
long smallestSeqInRegionCurrentMemstore = getWAL(region)
|
||||
|
@ -184,9 +184,9 @@ public class TestPerColumnFamilyFlush {
|
|||
MemstoreSize oldCF3MemstoreSize = cf3MemstoreSize;
|
||||
|
||||
// Recalculate everything
|
||||
cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
|
||||
cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
|
||||
cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
|
||||
totalMemstoreSize = region.getMemstoreSize();
|
||||
smallestSeqInRegionCurrentMemstore = getWAL(region)
|
||||
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
|
@ -216,15 +216,15 @@ public class TestPerColumnFamilyFlush {
|
|||
}
|
||||
|
||||
// How much does the CF3 memstore occupy? Will be used later.
|
||||
oldCF3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
oldCF3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
|
||||
|
||||
// Flush again
|
||||
region.flush(false);
|
||||
|
||||
// Recalculate everything
|
||||
cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
|
||||
cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
|
||||
cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
|
||||
totalMemstoreSize = region.getMemstoreSize();
|
||||
smallestSeqInRegionCurrentMemstore = getWAL(region)
|
||||
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
|
@ -288,9 +288,9 @@ public class TestPerColumnFamilyFlush {
|
|||
long totalMemstoreSize = region.getMemstoreSize();
|
||||
|
||||
// Find the sizes of the memstores of each CF.
|
||||
MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
|
||||
|
||||
// Some other sanity checks.
|
||||
assertTrue(cf1MemstoreSize.getDataSize() > 0);
|
||||
|
@ -305,9 +305,9 @@ public class TestPerColumnFamilyFlush {
|
|||
// Flush!
|
||||
region.flush(false);
|
||||
|
||||
cf1MemstoreSize = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
cf2MemstoreSize = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
cf3MemstoreSize = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize();
|
||||
cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize();
|
||||
cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize();
|
||||
totalMemstoreSize = region.getMemstoreSize();
|
||||
long smallestSeqInRegionCurrentMemstore =
|
||||
region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
|
@ -380,9 +380,9 @@ public class TestPerColumnFamilyFlush {
|
|||
totalMemstoreSize = desiredRegion.getMemstoreSize();
|
||||
|
||||
// Find the sizes of the memstores of each CF.
|
||||
cf1MemstoreSize = desiredRegion.getStore(FAMILY1).getSizeOfMemStore().getDataSize();
|
||||
cf2MemstoreSize = desiredRegion.getStore(FAMILY2).getSizeOfMemStore().getDataSize();
|
||||
cf3MemstoreSize = desiredRegion.getStore(FAMILY3).getSizeOfMemStore().getDataSize();
|
||||
cf1MemstoreSize = desiredRegion.getStore(FAMILY1).getMemStoreSize().getDataSize();
|
||||
cf2MemstoreSize = desiredRegion.getStore(FAMILY2).getMemStoreSize().getDataSize();
|
||||
cf3MemstoreSize = desiredRegion.getStore(FAMILY3).getMemStoreSize().getDataSize();
|
||||
|
||||
// CF1 Should have been flushed
|
||||
assertEquals(0, cf1MemstoreSize);
|
||||
|
@ -500,9 +500,9 @@ public class TestPerColumnFamilyFlush {
|
|||
}
|
||||
}
|
||||
assertEquals(maxLogs, getNumRolledLogFiles(desiredRegion));
|
||||
assertTrue(desiredRegion.getStore(FAMILY1).getMemStoreSize() > cfFlushSizeLowerBound);
|
||||
assertTrue(desiredRegion.getStore(FAMILY2).getMemStoreSize() < cfFlushSizeLowerBound);
|
||||
assertTrue(desiredRegion.getStore(FAMILY3).getMemStoreSize() < cfFlushSizeLowerBound);
|
||||
assertTrue(desiredRegion.getStore(FAMILY1).getMemStoreSize().getHeapSize() > cfFlushSizeLowerBound);
|
||||
assertTrue(desiredRegion.getStore(FAMILY2).getMemStoreSize().getHeapSize() < cfFlushSizeLowerBound);
|
||||
assertTrue(desiredRegion.getStore(FAMILY3).getMemStoreSize().getHeapSize() < cfFlushSizeLowerBound);
|
||||
table.put(createPut(1, 12345678));
|
||||
// Make numRolledLogFiles greater than maxLogs
|
||||
desiredRegionAndServer.getSecond().walRoller.requestRollAll();
|
||||
|
@ -525,9 +525,9 @@ public class TestPerColumnFamilyFlush {
|
|||
});
|
||||
LOG.info("Finished waiting on flush after too many WALs...");
|
||||
// Individual families should have been flushed.
|
||||
assertEquals(0, desiredRegion.getStore(FAMILY1).getMemStoreSize());
|
||||
assertEquals(0, desiredRegion.getStore(FAMILY2).getMemStoreSize());
|
||||
assertEquals(0, desiredRegion.getStore(FAMILY3).getMemStoreSize());
|
||||
assertEquals(0, desiredRegion.getStore(FAMILY1).getMemStoreSize().getHeapSize());
|
||||
assertEquals(0, desiredRegion.getStore(FAMILY2).getMemStoreSize().getHeapSize());
|
||||
assertEquals(0, desiredRegion.getStore(FAMILY3).getMemStoreSize().getHeapSize());
|
||||
// let WAL cleanOldLogs
|
||||
assertNull(getWAL(desiredRegion).rollWriter(true));
|
||||
assertTrue(getNumRolledLogFiles(desiredRegion) < maxLogs);
|
||||
|
|
|
@ -50,7 +50,7 @@ public class TestRegionSplitPolicy {
|
|||
private Configuration conf;
|
||||
private HTableDescriptor htd;
|
||||
private HRegion mockRegion;
|
||||
private List<Store> stores;
|
||||
private List<HStore> stores;
|
||||
private static final TableName TABLENAME = TableName.valueOf("t");
|
||||
|
||||
@Rule
|
||||
|
|
|
@ -169,9 +169,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3);
|
||||
|
||||
// Find the sizes of the memstores of each CF.
|
||||
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize();
|
||||
|
||||
// Get the overall smallest LSN in the region's memstores.
|
||||
long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region)
|
||||
|
@ -220,9 +220,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
region.flush(false);
|
||||
|
||||
// Recalculate everything
|
||||
MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize();
|
||||
|
||||
long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region)
|
||||
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
|
@ -264,7 +264,7 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
+ smallestSeqCF2PhaseII +", the smallest sequence in CF3:" + smallestSeqCF3PhaseII + "\n";
|
||||
|
||||
// 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);
|
||||
|
||||
s = s + "----After more puts into CF1 its size is:" + cf1MemstoreSizePhaseIII
|
||||
|
@ -278,9 +278,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
region.flush(false);
|
||||
|
||||
// Recalculate everything
|
||||
MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize();
|
||||
|
||||
long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region)
|
||||
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
|
@ -319,9 +319,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
region.flush(true);
|
||||
|
||||
// Recalculate everything
|
||||
MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize();
|
||||
long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region)
|
||||
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
|
||||
|
@ -410,9 +410,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
long smallestSeqCF2PhaseI = region.getOldestSeqIdOfStore(FAMILY2);
|
||||
long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3);
|
||||
// Find the sizes of the memstores of each CF.
|
||||
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize();
|
||||
// Get the overall smallest LSN in the region's memstores.
|
||||
long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region)
|
||||
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
|
@ -460,9 +460,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
/*------------------------------------------------------------------------------*/
|
||||
/* PHASE II - collect sizes */
|
||||
// Recalculate everything
|
||||
MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize();
|
||||
long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region)
|
||||
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
// Find the smallest LSNs for edits wrt to each CF.
|
||||
|
@ -509,7 +509,7 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
/*------------------------------------------------------------------------------*/
|
||||
/* PHASE III - collect sizes */
|
||||
// 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();
|
||||
|
||||
/*------------------------------------------------------------------------------*/
|
||||
|
@ -531,9 +531,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
/*------------------------------------------------------------------------------*/
|
||||
/* PHASE IV - collect sizes */
|
||||
// Recalculate everything
|
||||
MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize();
|
||||
long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region)
|
||||
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3);
|
||||
|
@ -563,9 +563,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
/*------------------------------------------------------------------------------*/
|
||||
/* PHASE V - collect sizes */
|
||||
// Recalculate everything
|
||||
MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize();
|
||||
long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region)
|
||||
.getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
long totalMemstoreSizePhaseV = region.getMemstoreSize();
|
||||
|
@ -601,9 +601,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
region.put(createPut(5, i));
|
||||
}
|
||||
|
||||
MemstoreSize cf1ActiveSizePhaseVI = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf3ActiveSizePhaseVI = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf5ActiveSizePhaseVI = region.getStore(FAMILIES[4]).getSizeOfMemStore();
|
||||
MemstoreSize cf1ActiveSizePhaseVI = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf3ActiveSizePhaseVI = region.getStore(FAMILY3).getMemStoreSize();
|
||||
MemstoreSize cf5ActiveSizePhaseVI = region.getStore(FAMILIES[4]).getMemStoreSize();
|
||||
|
||||
/*------------------------------------------------------------------------------*/
|
||||
/* PHASE VI - Flush */
|
||||
|
@ -614,9 +614,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
// Since we won't find any CF above the threshold, and hence no specific
|
||||
// store to flush, we should flush all the memstores
|
||||
// Also compacted memstores are flushed to disk, but not entirely emptied
|
||||
MemstoreSize cf1ActiveSizePhaseVII = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf3ActiveSizePhaseVII = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf5ActiveSizePhaseVII = region.getStore(FAMILIES[4]).getSizeOfMemStore();
|
||||
MemstoreSize cf1ActiveSizePhaseVII = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf3ActiveSizePhaseVII = region.getStore(FAMILY3).getMemStoreSize();
|
||||
MemstoreSize cf5ActiveSizePhaseVII = region.getStore(FAMILIES[4]).getMemStoreSize();
|
||||
|
||||
assertTrue(cf1ActiveSizePhaseVII.getDataSize() < cf1ActiveSizePhaseVI.getDataSize());
|
||||
assertTrue(cf3ActiveSizePhaseVII.getDataSize() < cf3ActiveSizePhaseVI.getDataSize());
|
||||
|
@ -663,9 +663,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
long totalMemstoreSize = region.getMemstoreSize();
|
||||
|
||||
// Find the sizes of the memstores of each CF.
|
||||
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize();
|
||||
|
||||
// Some other sanity checks.
|
||||
assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0);
|
||||
|
@ -689,7 +689,7 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
cms3.flushInMemory();
|
||||
region.flush(false);
|
||||
|
||||
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize();
|
||||
|
||||
long smallestSeqInRegionCurrentMemstorePhaseII =
|
||||
region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes());
|
||||
|
@ -798,9 +798,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
long totalMemstoreSize = region.getMemstoreSize();
|
||||
|
||||
// Find the sizes of the memstores of each CF.
|
||||
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize();
|
||||
|
||||
// Some other sanity checks.
|
||||
assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0);
|
||||
|
@ -829,9 +829,9 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
// Flush-to-disk! CF2 only should be flushed
|
||||
region.flush(false);
|
||||
|
||||
MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize();
|
||||
MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize();
|
||||
|
||||
// CF1 should be flushed in memory and just flattened, so CF1 heap overhead should be smaller
|
||||
assertTrue(cf1MemstoreSizePhaseI.getHeapSize() > cf1MemstoreSizePhaseII.getHeapSize());
|
||||
|
@ -855,7 +855,7 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
region.put(createPut(2, i));
|
||||
}
|
||||
|
||||
MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize();
|
||||
|
||||
// Flush in memory!
|
||||
((CompactingMemStore) ((HStore)region.getStore(FAMILY1)).memstore).flushInMemory();
|
||||
|
@ -871,8 +871,8 @@ public class TestWalAndCompactingMemStoreFlush {
|
|||
}
|
||||
region.flush(false);
|
||||
|
||||
MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getSizeOfMemStore();
|
||||
MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getSizeOfMemStore();
|
||||
MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize();
|
||||
MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize();
|
||||
|
||||
assertEquals(2*cf1MemstoreSizePhaseI.getDataSize(), cf1MemstoreSizePhaseIV.getDataSize());
|
||||
assertEquals(
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.OptionalLong;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -104,7 +105,7 @@ public class TestDateTieredCompactor {
|
|||
when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
|
||||
anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
|
||||
when(store.getComparator()).thenReturn(CellComparator.COMPARATOR);
|
||||
long maxSequenceId = StoreUtils.getMaxSequenceIdInList(storefiles);
|
||||
OptionalLong maxSequenceId = StoreUtils.getMaxSequenceIdInList(storefiles);
|
||||
when(store.getMaxSequenceId()).thenReturn(maxSequenceId);
|
||||
|
||||
return new DateTieredCompactor(conf, store) {
|
||||
|
|
|
@ -26,20 +26,21 @@ import java.util.Random;
|
|||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.MiniHBaseCluster;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
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.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.DisabledRegionSplitPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -65,32 +66,31 @@ public class TestFIFOCompactionPolicy {
|
|||
|
||||
private final byte[] qualifier = Bytes.toBytes("q");
|
||||
|
||||
private Store getStoreWithName(TableName tableName) {
|
||||
private HStore getStoreWithName(TableName tableName) {
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
|
||||
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
|
||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||
HRegionServer hrs = rsts.get(i).getRegionServer();
|
||||
for (Region region : hrs.getRegions(tableName)) {
|
||||
return region.getStores().iterator().next();
|
||||
return ((HRegion) region).getStores().iterator().next();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private Store prepareData() throws IOException {
|
||||
private HStore prepareData() throws IOException {
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
if (admin.tableExists(tableName)) {
|
||||
admin.disableTable(tableName);
|
||||
admin.deleteTable(tableName);
|
||||
}
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
|
||||
FIFOCompactionPolicy.class.getName());
|
||||
desc.setConfiguration(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
DisabledRegionSplitPolicy.class.getName());
|
||||
HColumnDescriptor colDesc = new HColumnDescriptor(family);
|
||||
colDesc.setTimeToLive(1); // 1 sec
|
||||
desc.addFamily(colDesc);
|
||||
TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
|
||||
FIFOCompactionPolicy.class.getName())
|
||||
.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
DisabledRegionSplitPolicy.class.getName())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setTimeToLive(1).build())
|
||||
.build();
|
||||
|
||||
admin.createTable(desc);
|
||||
Table table = TEST_UTIL.getConnection().getTable(tableName);
|
||||
|
@ -129,7 +129,7 @@ public class TestFIFOCompactionPolicy {
|
|||
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
try {
|
||||
Store store = prepareData();
|
||||
HStore store = prepareData();
|
||||
assertEquals(10, store.getStorefilesCount());
|
||||
TEST_UTIL.getAdmin().majorCompact(tableName);
|
||||
while (store.getStorefilesCount() > 1) {
|
||||
|
@ -141,9 +141,8 @@ public class TestFIFOCompactionPolicy {
|
|||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSanityCheckTTL() throws Exception
|
||||
{
|
||||
@Test
|
||||
public void testSanityCheckTTL() throws Exception {
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10000);
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
|
@ -154,25 +153,23 @@ public class TestFIFOCompactionPolicy {
|
|||
admin.disableTable(tableName);
|
||||
admin.deleteTable(tableName);
|
||||
}
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
|
||||
FIFOCompactionPolicy.class.getName());
|
||||
desc.setConfiguration(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
DisabledRegionSplitPolicy.class.getName());
|
||||
HColumnDescriptor colDesc = new HColumnDescriptor(family);
|
||||
desc.addFamily(colDesc);
|
||||
try{
|
||||
TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
|
||||
FIFOCompactionPolicy.class.getName())
|
||||
.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
DisabledRegionSplitPolicy.class.getName())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).build();
|
||||
try {
|
||||
admin.createTable(desc);
|
||||
Assert.fail();
|
||||
}catch(Exception e){
|
||||
}finally{
|
||||
} catch (Exception e) {
|
||||
} finally {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSanityCheckMinVersion() throws Exception
|
||||
{
|
||||
@Test
|
||||
public void testSanityCheckMinVersion() throws Exception {
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10000);
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
|
@ -183,27 +180,25 @@ public class TestFIFOCompactionPolicy {
|
|||
admin.disableTable(tableName);
|
||||
admin.deleteTable(tableName);
|
||||
}
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
|
||||
FIFOCompactionPolicy.class.getName());
|
||||
desc.setConfiguration(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
DisabledRegionSplitPolicy.class.getName());
|
||||
HColumnDescriptor colDesc = new HColumnDescriptor(family);
|
||||
colDesc.setTimeToLive(1); // 1 sec
|
||||
colDesc.setMinVersions(1);
|
||||
desc.addFamily(colDesc);
|
||||
try{
|
||||
TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
|
||||
FIFOCompactionPolicy.class.getName())
|
||||
.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
DisabledRegionSplitPolicy.class.getName())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setTimeToLive(1)
|
||||
.setMinVersions(1).build())
|
||||
.build();
|
||||
try {
|
||||
admin.createTable(desc);
|
||||
Assert.fail();
|
||||
}catch(Exception e){
|
||||
}finally{
|
||||
} catch (Exception e) {
|
||||
} finally {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSanityCheckBlockingStoreFiles() throws Exception
|
||||
{
|
||||
@Test
|
||||
public void testSanityCheckBlockingStoreFiles() throws Exception {
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
conf.setInt(HStore.BLOCKING_STOREFILES_KEY, 10);
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
|
@ -214,19 +209,18 @@ public class TestFIFOCompactionPolicy {
|
|||
admin.disableTable(tableName);
|
||||
admin.deleteTable(tableName);
|
||||
}
|
||||
HTableDescriptor desc = new HTableDescriptor(tableName);
|
||||
desc.setConfiguration(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
|
||||
FIFOCompactionPolicy.class.getName());
|
||||
desc.setConfiguration(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
DisabledRegionSplitPolicy.class.getName());
|
||||
HColumnDescriptor colDesc = new HColumnDescriptor(family);
|
||||
colDesc.setTimeToLive(1); // 1 sec
|
||||
desc.addFamily(colDesc);
|
||||
try{
|
||||
TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
|
||||
.setValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
|
||||
FIFOCompactionPolicy.class.getName())
|
||||
.setValue(HConstants.HBASE_REGION_SPLIT_POLICY_KEY,
|
||||
DisabledRegionSplitPolicy.class.getName())
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(family).setTimeToLive(1).build())
|
||||
.build();
|
||||
try {
|
||||
admin.createTable(desc);
|
||||
Assert.fail();
|
||||
}catch(Exception e){
|
||||
}finally{
|
||||
} catch (Exception e) {
|
||||
} finally {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,33 +22,30 @@ import static org.junit.Assert.assertTrue;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.TableName;
|
||||
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.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
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.HRegion;
|
||||
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.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreEngine;
|
||||
import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
|
||||
import org.apache.hadoop.hbase.regionserver.StripeStoreEngine;
|
||||
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.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -71,30 +68,29 @@ public class TestCompactionWithThroughputController {
|
|||
|
||||
private final byte[] qualifier = Bytes.toBytes("q");
|
||||
|
||||
private Store getStoreWithName(TableName tableName) {
|
||||
private HStore getStoreWithName(TableName tableName) {
|
||||
MiniHBaseCluster cluster = TEST_UTIL.getMiniHBaseCluster();
|
||||
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
|
||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||
HRegionServer hrs = rsts.get(i).getRegionServer();
|
||||
for (Region region : hrs.getRegions(tableName)) {
|
||||
return region.getStores().iterator().next();
|
||||
return ((HRegion) region).getStores().iterator().next();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private Store prepareData() throws IOException {
|
||||
private HStore prepareData() throws IOException {
|
||||
Admin admin = TEST_UTIL.getAdmin();
|
||||
if (admin.tableExists(tableName)) {
|
||||
admin.disableTable(tableName);
|
||||
admin.deleteTable(tableName);
|
||||
}
|
||||
Table table = TEST_UTIL.createTable(tableName, family);
|
||||
Random rand = new Random();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
for (int j = 0; j < 10; j++) {
|
||||
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));
|
||||
}
|
||||
admin.flush(tableName);
|
||||
|
@ -121,7 +117,7 @@ public class TestCompactionWithThroughputController {
|
|||
PressureAwareCompactionThroughputController.class.getName());
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
try {
|
||||
Store store = prepareData();
|
||||
HStore store = prepareData();
|
||||
assertEquals(10, store.getStorefilesCount());
|
||||
long startTime = System.currentTimeMillis();
|
||||
TEST_UTIL.getAdmin().majorCompact(tableName);
|
||||
|
@ -150,7 +146,7 @@ public class TestCompactionWithThroughputController {
|
|||
NoLimitThroughputController.class.getName());
|
||||
TEST_UTIL.startMiniCluster(1);
|
||||
try {
|
||||
Store store = prepareData();
|
||||
HStore store = prepareData();
|
||||
assertEquals(10, store.getStorefilesCount());
|
||||
long startTime = System.currentTimeMillis();
|
||||
TEST_UTIL.getAdmin().majorCompact(tableName);
|
||||
|
@ -199,10 +195,10 @@ public class TestCompactionWithThroughputController {
|
|||
TEST_UTIL.startMiniCluster(1);
|
||||
Connection conn = ConnectionFactory.createConnection(conf);
|
||||
try {
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor(family));
|
||||
htd.setCompactionEnabled(false);
|
||||
TEST_UTIL.getAdmin().createTable(htd);
|
||||
TEST_UTIL.getAdmin()
|
||||
.createTable(TableDescriptorBuilder.newBuilder(tableName)
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCompactionEnabled(false)
|
||||
.build());
|
||||
TEST_UTIL.waitTableAvailable(tableName);
|
||||
HRegionServer regionServer = TEST_UTIL.getRSForFirstRegionInTable(tableName);
|
||||
PressureAwareCompactionThroughputController throughputController =
|
||||
|
@ -256,12 +252,12 @@ public class TestCompactionWithThroughputController {
|
|||
TEST_UTIL.startMiniCluster(1);
|
||||
Connection conn = ConnectionFactory.createConnection(conf);
|
||||
try {
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor(family));
|
||||
htd.setCompactionEnabled(false);
|
||||
TEST_UTIL.getAdmin().createTable(htd);
|
||||
TEST_UTIL.getAdmin()
|
||||
.createTable(TableDescriptorBuilder.newBuilder(tableName)
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCompactionEnabled(false)
|
||||
.build());
|
||||
TEST_UTIL.waitTableAvailable(tableName);
|
||||
HStore store = (HStore) getStoreWithName(tableName);
|
||||
HStore store = getStoreWithName(tableName);
|
||||
assertEquals(0, store.getStorefilesCount());
|
||||
assertEquals(0.0, store.getCompactionPressure(), EPSILON);
|
||||
Table table = conn.getTable(tableName);
|
||||
|
|
|
@ -22,25 +22,26 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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.TableName;
|
||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
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.regionserver.DefaultStoreEngine;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreEngine;
|
||||
import org.apache.hadoop.hbase.regionserver.StripeStoreEngine;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
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.Pair;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
|
@ -73,13 +74,13 @@ public class TestFlushWithThroughputController {
|
|||
hbtu.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
private Store getStoreWithName(TableName tableName) {
|
||||
private HStore getStoreWithName(TableName tableName) {
|
||||
MiniHBaseCluster cluster = hbtu.getMiniHBaseCluster();
|
||||
List<JVMClusterUtil.RegionServerThread> rsts = cluster.getRegionServerThreads();
|
||||
for (int i = 0; i < cluster.getRegionServerThreads().size(); i++) {
|
||||
HRegionServer hrs = rsts.get(i).getRegionServer();
|
||||
for (Region region : hrs.getRegions(tableName)) {
|
||||
return region.getStores().iterator().next();
|
||||
return ((HRegion) region).getStores().iterator().next();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
@ -114,7 +115,7 @@ public class TestFlushWithThroughputController {
|
|||
hbtu.getAdmin().flush(tableName);
|
||||
duration += System.nanoTime() - startTime;
|
||||
}
|
||||
Store store = getStoreWithName(tableName);
|
||||
HStore store = getStoreWithName(tableName);
|
||||
assertEquals(NUM_FLUSHES, store.getStorefilesCount());
|
||||
double throughput = (double)store.getStorefilesSize()
|
||||
/ TimeUnit.NANOSECONDS.toSeconds(duration);
|
||||
|
@ -157,10 +158,9 @@ public class TestFlushWithThroughputController {
|
|||
3000);
|
||||
hbtu.startMiniCluster(1);
|
||||
Connection conn = ConnectionFactory.createConnection(conf);
|
||||
HTableDescriptor htd = new HTableDescriptor(tableName);
|
||||
htd.addFamily(new HColumnDescriptor(family));
|
||||
htd.setCompactionEnabled(false);
|
||||
hbtu.getAdmin().createTable(htd);
|
||||
hbtu.getAdmin().createTable(TableDescriptorBuilder.newBuilder(tableName)
|
||||
.addColumnFamily(ColumnFamilyDescriptorBuilder.of(family)).setCompactionEnabled(false)
|
||||
.build());
|
||||
hbtu.waitTableAvailable(tableName);
|
||||
HRegionServer regionServer = hbtu.getRSForFirstRegionInTable(tableName);
|
||||
PressureAwareFlushThroughputController throughputController =
|
||||
|
|
|
@ -242,10 +242,11 @@ public abstract class AbstractTestWALReplay {
|
|||
assertEquals(0, count);
|
||||
|
||||
// flush region and make major compaction
|
||||
Region region = destServer.getOnlineRegion(destRegion.getRegionInfo().getRegionName());
|
||||
HRegion region =
|
||||
(HRegion) destServer.getOnlineRegion(destRegion.getRegionInfo().getRegionName());
|
||||
region.flush(true);
|
||||
// wait to complete major compaction
|
||||
for (Store store : region.getStores()) {
|
||||
for (HStore store : region.getStores()) {
|
||||
store.triggerMajorCompaction();
|
||||
}
|
||||
region.compact(true);
|
||||
|
|
|
@ -250,18 +250,19 @@ public class TestCoprocessorScanPolicy {
|
|||
public InternalScanner preFlushScannerOpen(
|
||||
final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
|
||||
List<KeyValueScanner> scanners, InternalScanner s, long readPoint) throws IOException {
|
||||
HStore hs = (HStore) store;
|
||||
Long newTtl = ttls.get(store.getTableName());
|
||||
if (newTtl != null) {
|
||||
System.out.println("PreFlush:" + newTtl);
|
||||
}
|
||||
Integer newVersions = versions.get(store.getTableName());
|
||||
ScanInfo oldSI = store.getScanInfo();
|
||||
ScanInfo oldSI = hs.getScanInfo();
|
||||
ColumnFamilyDescriptor family = store.getColumnFamilyDescriptor();
|
||||
ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(),
|
||||
family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions,
|
||||
newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(),
|
||||
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()),
|
||||
scanners, ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(),
|
||||
HConstants.OLDEST_TIMESTAMP);
|
||||
|
@ -272,16 +273,17 @@ public class TestCoprocessorScanPolicy {
|
|||
final ObserverContext<RegionCoprocessorEnvironment> c, Store store,
|
||||
List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
|
||||
InternalScanner s,CompactionLifeCycleTracker tracker, long readPoint) throws IOException {
|
||||
HStore hs = (HStore) store;
|
||||
Long newTtl = ttls.get(store.getTableName());
|
||||
Integer newVersions = versions.get(store.getTableName());
|
||||
ScanInfo oldSI = store.getScanInfo();
|
||||
ScanInfo oldSI = hs.getScanInfo();
|
||||
ColumnFamilyDescriptor family = store.getColumnFamilyDescriptor();
|
||||
ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(),
|
||||
family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions,
|
||||
newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(),
|
||||
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()),
|
||||
scanners, scanType, store.getSmallestReadPoint(), earliestPutTs);
|
||||
}
|
||||
|
@ -292,16 +294,17 @@ public class TestCoprocessorScanPolicy {
|
|||
final NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt) throws IOException {
|
||||
TableName tn = store.getTableName();
|
||||
if (!tn.isSystemTable()) {
|
||||
HStore hs = (HStore) store;
|
||||
Long newTtl = ttls.get(store.getTableName());
|
||||
Integer newVersions = versions.get(store.getTableName());
|
||||
ScanInfo oldSI = store.getScanInfo();
|
||||
ScanInfo oldSI = hs.getScanInfo();
|
||||
ColumnFamilyDescriptor family = store.getColumnFamilyDescriptor();
|
||||
ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(),
|
||||
family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions,
|
||||
newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(),
|
||||
family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(),
|
||||
family.isNewVersionBehavior());
|
||||
return new StoreScanner((HStore) store, scanInfo, scan, targetCols, readPt);
|
||||
return new StoreScanner(hs, scanInfo, scan, targetCols, readPt);
|
||||
} else {
|
||||
return s;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue