HBASE-18038 Rename StoreFile to HStoreFile and add a StoreFile interface for CP

This commit is contained in:
zhangduo 2017-06-06 16:35:19 +08:00
parent 8bfa8aaaca
commit ee0f148c73
67 changed files with 1268 additions and 1053 deletions

View File

@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics; import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
@ -605,11 +605,8 @@ public class MergeTableRegionsProcedure
final CacheConfig cacheConf = new CacheConfig(conf, hcd); final CacheConfig cacheConf = new CacheConfig(conf, hcd);
for (StoreFileInfo storeFileInfo: storeFiles) { for (StoreFileInfo storeFileInfo: storeFiles) {
// Create reference file(s) of the region in mergedDir // Create reference file(s) of the region in mergedDir
regionFs.mergeStoreFile( regionFs.mergeStoreFile(mergedRegion, family, new HStoreFile(mfs.getFileSystem(),
mergedRegion, storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true),
family,
new StoreFile(
mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()),
mergedDir); mergedDir);
} }
} }

View File

@ -18,6 +18,8 @@
package org.apache.hadoop.hbase.master.assignment; package org.apache.hadoop.hbase.master.assignment;
import com.google.common.annotations.VisibleForTesting;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
@ -62,6 +64,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
@ -71,8 +74,6 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import com.google.common.annotations.VisibleForTesting;
/** /**
* The procedure to split a region in a table. * The procedure to split a region in a table.
* Takes lock on the parent region. * Takes lock on the parent region.
@ -525,11 +526,9 @@ public class SplitTableRegionProcedure
if (storeFiles != null && storeFiles.size() > 0) { if (storeFiles != null && storeFiles.size() > 0) {
final CacheConfig cacheConf = new CacheConfig(conf, hcd); final CacheConfig cacheConf = new CacheConfig(conf, hcd);
for (StoreFileInfo storeFileInfo: storeFiles) { for (StoreFileInfo storeFileInfo: storeFiles) {
StoreFileSplitter sfs = new StoreFileSplitter( StoreFileSplitter sfs =
regionFs, new StoreFileSplitter(regionFs, family.getBytes(), new HStoreFile(mfs.getFileSystem(),
family.getBytes(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true));
new StoreFile(
mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()));
futures.add(threadPool.submit(sfs)); futures.add(threadPool.submit(sfs));
} }
} }

View File

@ -28,7 +28,6 @@ import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.Random; import java.util.Random;
import com.google.common.collect.Lists;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -51,6 +50,8 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.SwapRegi
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.collect.Lists;
/** /**
* <p>This is a best effort load balancer. Given a Cost function F(C) =&gt; x It will * <p>This is a best effort load balancer. Given a Cost function F(C) =&gt; x It will
* randomly try and mutate the cluster to Cprime. If F(Cprime) &lt; F(C) then the * randomly try and mutate the cluster to Cprime. If F(Cprime) &lt; F(C) then the

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
/** /**
@ -46,7 +47,7 @@ public class CachedMobFile extends MobFile implements Comparable<CachedMobFile>
CacheConfig cacheConf) throws IOException { CacheConfig cacheConf) throws IOException {
// XXX: primaryReplica is only used for constructing the key of block cache so it is not a // XXX: primaryReplica is only used for constructing the key of block cache so it is not a
// critical problem if we pass the wrong value, so here we always pass true. Need to fix later. // critical problem if we pass the wrong value, so here we always pass true. Need to fix later.
StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
return new CachedMobFile(sf); return new CachedMobFile(sf);
} }

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
@ -146,7 +147,7 @@ public class MobFile {
throws IOException { throws IOException {
// XXX: primaryReplica is only used for constructing the key of block cache so it is not a // XXX: primaryReplica is only used for constructing the key of block cache so it is not a
// critical problem if we pass the wrong value, so here we always pass true. Need to fix later. // critical problem if we pass the wrong value, so here we always pass true. Need to fix later.
StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
return new MobFile(sf); return new MobFile(sf);
} }
} }

View File

@ -69,6 +69,7 @@ import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.C
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor; import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -334,7 +335,7 @@ public final class MobUtils {
LOG.debug(fileName + " is an expired file"); LOG.debug(fileName + " is an expired file");
} }
filesToClean filesToClean
.add(new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true)); .add(new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true));
} }
} catch (Exception e) { } catch (Exception e) {
LOG.error("Cannot parse the fileName " + fileName, e); LOG.error("Cannot parse the fileName " + fileName, e);
@ -722,7 +723,7 @@ public final class MobUtils {
CacheConfig cacheConfig, boolean primaryReplica) throws IOException { CacheConfig cacheConfig, boolean primaryReplica) throws IOException {
StoreFile storeFile = null; StoreFile storeFile = null;
try { try {
storeFile = new StoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica); storeFile = new HStoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica);
storeFile.initReader(); storeFile.initReader();
} catch (IOException e) { } catch (IOException e) {
LOG.error("Failed to open mob file[" + path + "], keep it in temp directory.", e); LOG.error("Failed to open mob file[" + path + "], keep it in temp directory.", e);

View File

@ -36,7 +36,6 @@ import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -74,6 +73,7 @@ import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.C
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId; import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.regionserver.ScanInfo;
import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.ScannerContext;
@ -87,6 +87,8 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import com.google.common.annotations.VisibleForTesting;
/** /**
* An implementation of {@link MobCompactor} that compacts the mob files in partitions. * An implementation of {@link MobCompactor} that compacts the mob files in partitions.
*/ */
@ -335,7 +337,7 @@ public class PartitionedMobCompactor extends MobCompactor {
for (CompactionDelPartition delPartition : request.getDelPartitions()) { for (CompactionDelPartition delPartition : request.getDelPartitions()) {
for (Path newDelPath : delPartition.listDelFiles()) { for (Path newDelPath : delPartition.listDelFiles()) {
StoreFile sf = StoreFile sf =
new StoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true); new HStoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true);
// pre-create reader of a del file to avoid race condition when opening the reader in each // pre-create reader of a del file to avoid race condition when opening the reader in each
// partition. // partition.
sf.initReader(); sf.initReader();
@ -551,7 +553,7 @@ public class PartitionedMobCompactor extends MobCompactor {
// add the selected mob files and del files into filesToCompact // add the selected mob files and del files into filesToCompact
List<StoreFile> filesToCompact = new ArrayList<>(); List<StoreFile> filesToCompact = new ArrayList<>();
for (int i = offset; i < batch + offset; i++) { for (int i = offset; i < batch + offset; i++) {
StoreFile sf = new StoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig, StoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
BloomType.NONE, true); BloomType.NONE, true);
filesToCompact.add(sf); filesToCompact.add(sf);
} }
@ -733,7 +735,7 @@ public class PartitionedMobCompactor extends MobCompactor {
continue; continue;
} }
for (int i = offset; i < batch + offset; i++) { for (int i = offset; i < batch + offset; i++) {
batchedDelFiles.add(new StoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig, batchedDelFiles.add(new HStoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig,
BloomType.NONE, true)); BloomType.NONE, true));
} }
// compact the del files in a batch. // compact the del files in a batch.

View File

@ -27,19 +27,29 @@ import java.util.Set;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured; import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.mapreduce.JobUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.util.LineReader;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobContext;
@ -47,21 +57,9 @@ import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.lib.input.FileSplit; import org.apache.hadoop.mapreduce.lib.input.FileSplit;
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.util.LineReader;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.util.Tool;
import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.mapreduce.JobUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
/* /*
* The CompactionTool allows to execute a compaction specifying a: * The CompactionTool allows to execute a compaction specifying a:

View File

@ -58,7 +58,7 @@ public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
throws IOException { throws IOException {
this.compactionPolicy = new DateTieredCompactionPolicy(conf, store); this.compactionPolicy = new DateTieredCompactionPolicy(conf, store);
this.storeFileManager = this.storeFileManager =
new DefaultStoreFileManager(kvComparator, StoreFile.Comparators.SEQ_ID_MAX_TIMESTAMP, conf, new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, conf,
compactionPolicy.getConf()); compactionPolicy.getConf());
this.storeFlusher = new DefaultStoreFlusher(conf, store); this.storeFlusher = new DefaultStoreFlusher(conf, store);
this.compactor = new DateTieredCompactor(conf, store); this.compactor = new DateTieredCompactor(conf, store);

View File

@ -69,7 +69,7 @@ public class DefaultStoreEngine extends StoreEngine<
createCompactionPolicy(conf, store); createCompactionPolicy(conf, store);
createStoreFlusher(conf, store); createStoreFlusher(conf, store);
storeFileManager = storeFileManager =
new DefaultStoreFileManager(kvComparator, StoreFile.Comparators.SEQ_ID, conf, new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID, conf,
compactionPolicy.getConf()); compactionPolicy.getConf());
} }

View File

@ -25,6 +25,7 @@ import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Optional;
import com.google.common.collect.ImmutableCollection; import com.google.common.collect.ImmutableCollection;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
@ -172,10 +173,13 @@ class DefaultStoreFileManager implements StoreFileManager {
@Override @Override
public final byte[] getSplitPoint() throws IOException { public final byte[] getSplitPoint() throws IOException {
if (this.storefiles.isEmpty()) { List<StoreFile> storefiles = this.storefiles;
if (storefiles.isEmpty()) {
return null; return null;
} }
return StoreUtils.getLargestFile(this.storefiles).getFileSplitPoint(this.kvComparator); Optional<StoreFile> largestFile = StoreUtils.getLargestFile(storefiles);
return largestFile.isPresent()
? StoreUtils.getFileSplitPoint(largestFile.get(), kvComparator).orElse(null) : null;
} }
@Override @Override

View File

@ -293,7 +293,7 @@ public class HMobStore extends HStore {
private void validateMobFile(Path path) throws IOException { private void validateMobFile(Path path) throws IOException {
StoreFile storeFile = null; StoreFile storeFile = null;
try { try {
storeFile = new StoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig, storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig,
BloomType.NONE, isPrimaryReplicaStore()); BloomType.NONE, isPrimaryReplicaStore());
storeFile.initReader(); storeFile.initReader();
} catch (IOException e) { } catch (IOException e) {

View File

@ -1456,7 +1456,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* time-sensitive thread. * time-sensitive thread.
* *
* @return Vector of all the storage files that the HRegion's component * @return Vector of all the storage files that the HRegion's component
* HStores make use of. It's a list of all HStoreFile objects. Returns empty * HStores make use of. It's a list of all StoreFile objects. Returns empty
* vector if already closed and null if judged that it should not close. * vector if already closed and null if judged that it should not close.
* *
* @throws IOException e * @throws IOException e
@ -1497,7 +1497,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* *
* @param abort true if server is aborting (only during testing) * @param abort true if server is aborting (only during testing)
* @return Vector of all the storage files that the HRegion's component * @return Vector of all the storage files that the HRegion's component
* HStores make use of. It's a list of HStoreFile objects. Can be null if * HStores make use of. It's a list of StoreFile objects. Can be null if
* we are not to close at this time or we are already closed. * we are not to close at this time or we are already closed.
* *
* @throws IOException e * @throws IOException e
@ -4204,7 +4204,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
Set<StoreFile> fakeStoreFiles = new HashSet<>(files.size()); Set<StoreFile> fakeStoreFiles = new HashSet<>(files.size());
for (Path file: files) { for (Path file: files) {
fakeStoreFiles.add( fakeStoreFiles.add(
new StoreFile(getRegionFileSystem().getFileSystem(), file, this.conf, null, null, true)); new HStoreFile(getRegionFileSystem().getFileSystem(), file, this.conf, null, null, true));
} }
getRegionFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles); getRegionFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
} else { } else {

View File

@ -18,13 +18,6 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableCollection;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
@ -53,7 +46,15 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CompoundConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.FailedArchiveException; import org.apache.hadoop.hbase.backup.FailedArchiveException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
@ -90,6 +91,13 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableCollection;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
/** /**
* A Store holds a column family in a Region. Its a memstore and a set of zero * A Store holds a column family in a Region. Its a memstore and a set of zero
* or more StoreFiles, which stretch backwards over time. * or more StoreFiles, which stretch backwards over time.
@ -455,12 +463,12 @@ public class HStore implements Store {
*/ */
@Override @Override
public long getMaxSequenceId() { public long getMaxSequenceId() {
return StoreFile.getMaxSequenceIdInList(this.getStorefiles()); return StoreUtils.getMaxSequenceIdInList(this.getStorefiles());
} }
@Override @Override
public long getMaxMemstoreTS() { public long getMaxMemstoreTS() {
return StoreFile.getMaxMemstoreTSInList(this.getStorefiles()); return StoreUtils.getMaxMemstoreTSInList(this.getStorefiles());
} }
/** /**
@ -655,7 +663,7 @@ public class HStore implements Store {
private StoreFile createStoreFileAndReader(final StoreFileInfo info) throws IOException { private StoreFile createStoreFileAndReader(final StoreFileInfo info) throws IOException {
info.setRegionCoprocessorHost(this.region.getCoprocessorHost()); info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
StoreFile storeFile = new StoreFile(this.getFileSystem(), info, this.conf, this.cacheConf, StoreFile storeFile = new HStoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
this.family.getBloomFilterType(), isPrimaryReplicaStore()); this.family.getBloomFilterType(), isPrimaryReplicaStore());
storeFile.initReader(); storeFile.initReader();
return storeFile; return storeFile;

View File

@ -0,0 +1,560 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Collections;
import java.util.Comparator;
import java.util.Map;
import java.util.OptionalLong;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.annotations.VisibleForTesting;
/**
* A Store data file. Stores usually have one or more of these files. They
* are produced by flushing the memstore to disk. To
* create, instantiate a writer using {@link StoreFileWriter.Builder}
* and append data. Be sure to add any metadata before calling close on the
* Writer (Use the appendMetadata convenience methods). On close, a StoreFile
* is sitting in the Filesystem. To refer to it, create a StoreFile instance
* passing filesystem and path. To read, call {@link #initReader()}
* <p>StoreFiles may also reference store files in another Store.
*
* The reason for this weird pattern where you use a different instance for the
* writer and a reader is that we write once but read a lot more.
*/
@InterfaceAudience.Private
public class HStoreFile implements StoreFile {
private static final Log LOG = LogFactory.getLog(HStoreFile.class.getName());
private static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false;
private final StoreFileInfo fileInfo;
private final FileSystem fs;
// Block cache configuration and reference.
private final CacheConfig cacheConf;
// Counter that is incremented every time a scanner is created on the
// store file. It is decremented when the scan on the store file is
// done.
private final AtomicInteger refCount = new AtomicInteger(0);
private final boolean noReadahead;
private final boolean primaryReplica;
// Indicates if the file got compacted
private volatile boolean compactedAway = false;
// Keys for metadata stored in backing HFile.
// Set when we obtain a Reader.
private long sequenceid = -1;
// max of the MemstoreTS in the KV's in this store
// Set when we obtain a Reader.
private long maxMemstoreTS = -1;
// firstKey, lastkey and cellComparator will be set when openReader.
private Cell firstKey;
private Cell lastKey;
private Comparator<Cell> comparator;
@Override
public CacheConfig getCacheConf() {
return cacheConf;
}
@Override
public Cell getFirstKey() {
return firstKey;
}
@Override
public Cell getLastKey() {
return lastKey;
}
@Override
public Comparator<Cell> getComparator() {
return comparator;
}
@Override
public long getMaxMemstoreTS() {
return maxMemstoreTS;
}
// If true, this file was product of a major compaction. Its then set
// whenever you get a Reader.
private AtomicBoolean majorCompaction = null;
// If true, this file should not be included in minor compactions.
// It's set whenever you get a Reader.
private boolean excludeFromMinorCompaction = false;
/**
* Map of the metadata entries in the corresponding HFile. Populated when Reader is opened
* after which it is not modified again.
*/
private Map<byte[], byte[]> metadataMap;
// StoreFile.Reader
private volatile StoreFileReader reader;
/**
* Bloom filter type specified in column family configuration. Does not
* necessarily correspond to the Bloom filter type present in the HFile.
*/
private final BloomType cfBloomType;
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs The current file system to use.
* @param p The path of the file.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified by column family
* configuration. This may or may not be the same as the Bloom filter type actually
* present in the HFile, because column family configuration might change. If this is
* {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @deprecated Now we will specific whether the StoreFile is for primary replica when
* constructing, so please use {@link #HStoreFile(FileSystem, Path, Configuration,
* CacheConfig, BloomType, boolean)} directly.
*/
@Deprecated
public HStoreFile(final FileSystem fs, final Path p, final Configuration conf,
final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType);
}
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs The current file system to use.
* @param p The path of the file.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified by column family
* configuration. This may or may not be the same as the Bloom filter type actually
* present in the HFile, because column family configuration might change. If this is
* {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @param primaryReplica true if this is a store file for primary replica, otherwise false.
* @throws IOException
*/
public HStoreFile(FileSystem fs, Path p, Configuration conf, CacheConfig cacheConf,
BloomType cfBloomType, boolean primaryReplica) throws IOException {
this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType, primaryReplica);
}
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs The current file system to use.
* @param fileInfo The store file information.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified by column family
* configuration. This may or may not be the same as the Bloom filter type actually
* present in the HFile, because column family configuration might change. If this is
* {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @deprecated Now we will specific whether the StoreFile is for primary replica when
* constructing, so please use {@link #HStoreFile(FileSystem, StoreFileInfo,
* Configuration, CacheConfig, BloomType, boolean)} directly.
*/
@Deprecated
public HStoreFile(final FileSystem fs, final StoreFileInfo fileInfo, final Configuration conf,
final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
this(fs, fileInfo, conf, cacheConf, cfBloomType, true);
}
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs fs The current file system to use.
* @param fileInfo The store file information.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified by column
* family configuration. This may or may not be the same as the Bloom filter type
* actually present in the HFile, because column family configuration might change. If
* this is {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @param primaryReplica true if this is a store file for primary replica, otherwise false.
*/
public HStoreFile(FileSystem fs, StoreFileInfo fileInfo, Configuration conf, CacheConfig cacheConf,
BloomType cfBloomType, boolean primaryReplica) {
this.fs = fs;
this.fileInfo = fileInfo;
this.cacheConf = cacheConf;
this.noReadahead =
conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD);
if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
this.cfBloomType = cfBloomType;
} else {
LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " + "cfBloomType=" +
cfBloomType + " (disabled in config)");
this.cfBloomType = BloomType.NONE;
}
this.primaryReplica = primaryReplica;
}
@Override
public StoreFileInfo getFileInfo() {
return this.fileInfo;
}
@Override
public Path getPath() {
return this.fileInfo.getPath();
}
@Override
public Path getQualifiedPath() {
return this.fileInfo.getPath().makeQualified(fs.getUri(), fs.getWorkingDirectory());
}
@Override
public boolean isReference() {
return this.fileInfo.isReference();
}
@Override
public boolean isHFile() {
return StoreFileInfo.isHFile(this.fileInfo.getPath());
}
@Override
public boolean isMajorCompactionResult() {
if (this.majorCompaction == null) {
throw new NullPointerException("This has not been set yet");
}
return this.majorCompaction.get();
}
@Override
public boolean excludeFromMinorCompaction() {
return this.excludeFromMinorCompaction;
}
@Override
public long getMaxSequenceId() {
return this.sequenceid;
}
@Override
public long getModificationTimeStamp() throws IOException {
return fileInfo.getModificationTime();
}
@Override
public byte[] getMetadataValue(byte[] key) {
return metadataMap.get(key);
}
@Override
public boolean isBulkLoadResult() {
boolean bulkLoadedHFile = false;
String fileName = this.getPath().getName();
int startPos = fileName.indexOf("SeqId_");
if (startPos != -1) {
bulkLoadedHFile = true;
}
return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY));
}
@Override
public boolean isCompactedAway() {
return compactedAway;
}
@VisibleForTesting
public int getRefCount() {
return refCount.get();
}
@Override
public boolean isReferencedInReads() {
int rc = refCount.get();
assert rc >= 0; // we should not go negative.
return rc > 0;
}
@Override
public OptionalLong getBulkLoadTimestamp() {
byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY);
return bulkLoadTimestamp == null ? OptionalLong.empty()
: OptionalLong.of(Bytes.toLong(bulkLoadTimestamp));
}
@Override
public HDFSBlocksDistribution getHDFSBlockDistribution() {
return this.fileInfo.getHDFSBlockDistribution();
}
/**
* Opens reader on this store file. Called by Constructor.
* @throws IOException
* @see #closeReader(boolean)
*/
private void open() throws IOException {
if (this.reader != null) {
throw new IllegalAccessError("Already open");
}
// Open the StoreFile.Reader
this.reader = fileInfo.open(this.fs, this.cacheConf, false, noReadahead ? 0L : -1L,
primaryReplica, refCount, true);
// Load up indices and fileinfo. This also loads Bloom filter type.
metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
// Read in our metadata.
byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
if (b != null) {
// By convention, if halfhfile, top half has a sequence number > bottom
// half. Thats why we add one in below. Its done for case the two halves
// are ever merged back together --rare. Without it, on open of store,
// since store files are distinguished by sequence id, the one half would
// subsume the other.
this.sequenceid = Bytes.toLong(b);
if (fileInfo.isTopReference()) {
this.sequenceid += 1;
}
}
if (isBulkLoadResult()){
// generate the sequenceId from the fileName
// fileName is of the form <randomName>_SeqId_<id-when-loaded>_
String fileName = this.getPath().getName();
// Use lastIndexOf() to get the last, most recent bulk load seqId.
int startPos = fileName.lastIndexOf("SeqId_");
if (startPos != -1) {
this.sequenceid = Long.parseLong(fileName.substring(startPos + 6,
fileName.indexOf('_', startPos + 6)));
// Handle reference files as done above.
if (fileInfo.isTopReference()) {
this.sequenceid += 1;
}
}
// SKIP_RESET_SEQ_ID only works in bulk loaded file.
// In mob compaction, the hfile where the cells contain the path of a new mob file is bulk
// loaded to hbase, these cells have the same seqIds with the old ones. We do not want
// to reset new seqIds for them since this might make a mess of the visibility of cells that
// have the same row key but different seqIds.
boolean skipResetSeqId = isSkipResetSeqId(metadataMap.get(SKIP_RESET_SEQ_ID));
if (skipResetSeqId) {
// increase the seqId when it is a bulk loaded file from mob compaction.
this.sequenceid += 1;
}
this.reader.setSkipResetSeqId(skipResetSeqId);
this.reader.setBulkLoaded(true);
}
this.reader.setSequenceID(this.sequenceid);
b = metadataMap.get(HFile.Writer.MAX_MEMSTORE_TS_KEY);
if (b != null) {
this.maxMemstoreTS = Bytes.toLong(b);
}
b = metadataMap.get(MAJOR_COMPACTION_KEY);
if (b != null) {
boolean mc = Bytes.toBoolean(b);
if (this.majorCompaction == null) {
this.majorCompaction = new AtomicBoolean(mc);
} else {
this.majorCompaction.set(mc);
}
} else {
// Presume it is not major compacted if it doesn't explicity say so
// HFileOutputFormat explicitly sets the major compacted key.
this.majorCompaction = new AtomicBoolean(false);
}
b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
BloomType hfileBloomType = reader.getBloomFilterType();
if (cfBloomType != BloomType.NONE) {
reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
if (hfileBloomType != cfBloomType) {
LOG.info("HFile Bloom filter type for "
+ reader.getHFileReader().getName() + ": " + hfileBloomType
+ ", but " + cfBloomType + " specified in column family "
+ "configuration");
}
} else if (hfileBloomType != BloomType.NONE) {
LOG.info("Bloom filter turned off by CF config for "
+ reader.getHFileReader().getName());
}
// load delete family bloom filter
reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
try {
this.reader.timeRange = TimeRangeTracker.getTimeRange(metadataMap.get(TIMERANGE_KEY));
} catch (IllegalArgumentException e) {
LOG.error("Error reading timestamp range data from meta -- " +
"proceeding without", e);
this.reader.timeRange = null;
}
// initialize so we can reuse them after reader closed.
firstKey = reader.getFirstKey();
lastKey = reader.getLastKey();
comparator = reader.getComparator();
}
@Override
public void initReader() throws IOException {
if (reader == null) {
try {
open();
} catch (Exception e) {
try {
boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true;
this.closeReader(evictOnClose);
} catch (IOException ee) {
LOG.warn("failed to close reader", ee);
}
throw e;
}
}
}
private StoreFileReader createStreamReader(boolean canUseDropBehind) throws IOException {
initReader();
StoreFileReader reader = fileInfo.open(this.fs, this.cacheConf, canUseDropBehind, -1L,
primaryReplica, refCount, false);
reader.copyFields(this.reader);
return reader;
}
@Override
public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
boolean canOptimizeForNonNullColumn) {
return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder,
canOptimizeForNonNullColumn);
}
@Override
public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
throws IOException {
return createStreamReader(canUseDropBehind).getStoreFileScanner(cacheBlocks, false,
isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn);
}
@Override
public StoreFileReader getReader() {
return this.reader;
}
@Override
public synchronized void closeReader(boolean evictOnClose)
throws IOException {
if (this.reader != null) {
this.reader.close(evictOnClose);
this.reader = null;
}
}
@Override
public void markCompactedAway() {
this.compactedAway = true;
}
@Override
public void deleteReader() throws IOException {
boolean evictOnClose =
cacheConf != null? cacheConf.shouldEvictOnClose(): true;
closeReader(evictOnClose);
this.fs.delete(getPath(), true);
}
@Override
public String toString() {
return this.fileInfo.toString();
}
@Override
public String toStringDetailed() {
StringBuilder sb = new StringBuilder();
sb.append(this.getPath().toString());
sb.append(", isReference=").append(isReference());
sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
if (isBulkLoadResult()) {
sb.append(", bulkLoadTS=");
OptionalLong bulkLoadTS = getBulkLoadTimestamp();
if (bulkLoadTS.isPresent()) {
sb.append(bulkLoadTS.getAsLong());
} else {
sb.append("NotPresent");
}
} else {
sb.append(", seqid=").append(getMaxSequenceId());
}
sb.append(", majorCompaction=").append(isMajorCompactionResult());
return sb.toString();
}
/**
* Gets whether to skip resetting the sequence id for cells.
* @param skipResetSeqId The byte array of boolean.
* @return Whether to skip resetting the sequence id.
*/
private boolean isSkipResetSeqId(byte[] skipResetSeqId) {
if (skipResetSeqId != null && skipResetSeqId.length == 1) {
return Bytes.toBoolean(skipResetSeqId);
}
return false;
}
@Override
public OptionalLong getMinimumTimestamp() {
TimeRange tr = getReader().timeRange;
return tr != null ? OptionalLong.of(tr.getMin()) : OptionalLong.empty();
}
@Override
public OptionalLong getMaximumTimestamp() {
TimeRange tr = getReader().timeRange;
return tr != null ? OptionalLong.of(tr.getMax()) : OptionalLong.empty();
}
}

View File

@ -1,5 +1,4 @@
/** /**
*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -18,789 +17,200 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Ordering;
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.Map; import java.util.OptionalLong;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.BlockType; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
/** /**
* A Store data file. Stores usually have one or more of these files. They * An interface to describe a store data file.
* are produced by flushing the memstore to disk. To
* create, instantiate a writer using {@link StoreFileWriter.Builder}
* and append data. Be sure to add any metadata before calling close on the
* Writer (Use the appendMetadata convenience methods). On close, a StoreFile
* is sitting in the Filesystem. To refer to it, create a StoreFile instance
* passing filesystem and path. To read, call {@link #initReader()}
* <p>StoreFiles may also reference store files in another Store.
*
* The reason for this weird pattern where you use a different instance for the
* writer and a reader is that we write once but read a lot more.
*/ */
@InterfaceAudience.LimitedPrivate("Coprocessor") @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
public class StoreFile { @InterfaceStability.Evolving
private static final Log LOG = LogFactory.getLog(StoreFile.class.getName()); public interface StoreFile {
public static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead"; static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
private static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false;
// Keys for fileinfo values in HFile // Keys for fileinfo values in HFile
/** Max Sequence ID in FileInfo */ /** Max Sequence ID in FileInfo */
public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY"); static final byte[] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
/** Major compaction flag in FileInfo */ /** Major compaction flag in FileInfo */
public static final byte[] MAJOR_COMPACTION_KEY = static final byte[] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
Bytes.toBytes("MAJOR_COMPACTION_KEY");
/** Minor compaction flag in FileInfo */ /** Minor compaction flag in FileInfo */
public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY = static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION"); Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
/** Bloom filter Type in FileInfo */ /** Bloom filter Type in FileInfo */
public static final byte[] BLOOM_FILTER_TYPE_KEY = static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
Bytes.toBytes("BLOOM_FILTER_TYPE");
/** Delete Family Count in FileInfo */ /** Delete Family Count in FileInfo */
public static final byte[] DELETE_FAMILY_COUNT = static final byte[] DELETE_FAMILY_COUNT = Bytes.toBytes("DELETE_FAMILY_COUNT");
Bytes.toBytes("DELETE_FAMILY_COUNT");
/** Last Bloom filter key in FileInfo */ /** Last Bloom filter key in FileInfo */
public static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY"); static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
/** Key for Timerange information in metadata*/ /** Key for Timerange information in metadata */
public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE"); static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
/** Key for timestamp of earliest-put in metadata*/ /** Key for timestamp of earliest-put in metadata */
public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS"); static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
/** Key for the number of mob cells in metadata*/ /** Key for the number of mob cells in metadata */
public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT"); static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
private final StoreFileInfo fileInfo;
private final FileSystem fs;
// Block cache configuration and reference.
private final CacheConfig cacheConf;
// Counter that is incremented every time a scanner is created on the
// store file. It is decremented when the scan on the store file is
// done.
private final AtomicInteger refCount = new AtomicInteger(0);
private final boolean noReadahead;
private final boolean primaryReplica;
// Indicates if the file got compacted
private volatile boolean compactedAway = false;
// Keys for metadata stored in backing HFile.
// Set when we obtain a Reader.
private long sequenceid = -1;
// max of the MemstoreTS in the KV's in this store
// Set when we obtain a Reader.
private long maxMemstoreTS = -1;
// firstKey, lastkey and cellComparator will be set when openReader.
private Cell firstKey;
private Cell lastKey;
private Comparator<Cell> comparator;
CacheConfig getCacheConf() {
return cacheConf;
}
public Cell getFirstKey() {
return firstKey;
}
public Cell getLastKey() {
return lastKey;
}
public Comparator<Cell> getComparator() {
return comparator;
}
public long getMaxMemstoreTS() {
return maxMemstoreTS;
}
public void setMaxMemstoreTS(long maxMemstoreTS) {
this.maxMemstoreTS = maxMemstoreTS;
}
// If true, this file was product of a major compaction. Its then set
// whenever you get a Reader.
private AtomicBoolean majorCompaction = null;
// If true, this file should not be included in minor compactions.
// It's set whenever you get a Reader.
private boolean excludeFromMinorCompaction = false;
/** Meta key set when store file is a result of a bulk load */ /** Meta key set when store file is a result of a bulk load */
public static final byte[] BULKLOAD_TASK_KEY = static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK");
Bytes.toBytes("BULKLOAD_SOURCE_TASK"); static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP");
public static final byte[] BULKLOAD_TIME_KEY =
Bytes.toBytes("BULKLOAD_TIMESTAMP");
/** /**
* Map of the metadata entries in the corresponding HFile. Populated when Reader is opened * Key for skipping resetting sequence id in metadata. For bulk loaded hfiles, the scanner resets
* after which it is not modified again. * the cell seqId with the latest one, if this metadata is set as true, the reset is skipped.
*/ */
private Map<byte[], byte[]> metadataMap; static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
// StoreFile.Reader CacheConfig getCacheConf();
private volatile StoreFileReader reader;
Cell getFirstKey();
Cell getLastKey();
Comparator<Cell> getComparator();
long getMaxMemstoreTS();
/** /**
* Bloom filter type specified in column family configuration. Does not * @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a
* necessarily correspond to the Bloom filter type present in the HFile. * reference.
*/ */
private final BloomType cfBloomType; StoreFileInfo getFileInfo();
/**
* Key for skipping resetting sequence id in metadata.
* For bulk loaded hfiles, the scanner resets the cell seqId with the latest one,
* if this metadata is set as true, the reset is skipped.
*/
public static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs The current file system to use.
* @param p The path of the file.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified by column family
* configuration. This may or may not be the same as the Bloom filter type actually
* present in the HFile, because column family configuration might change. If this is
* {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @deprecated Now we will specific whether the StoreFile is for primary replica when
* constructing, so please use
* {@link #StoreFile(FileSystem, Path, Configuration, CacheConfig, BloomType, boolean)}
* directly.
*/
@Deprecated
public StoreFile(final FileSystem fs, final Path p, final Configuration conf,
final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType);
}
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs The current file system to use.
* @param p The path of the file.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified by column family
* configuration. This may or may not be the same as the Bloom filter type actually
* present in the HFile, because column family configuration might change. If this is
* {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @param primaryReplica true if this is a store file for primary replica, otherwise false.
* @throws IOException
*/
public StoreFile(FileSystem fs, Path p, Configuration conf, CacheConfig cacheConf,
BloomType cfBloomType, boolean primaryReplica) throws IOException {
this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType, primaryReplica);
}
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs The current file system to use.
* @param fileInfo The store file information.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified by column family
* configuration. This may or may not be the same as the Bloom filter type actually
* present in the HFile, because column family configuration might change. If this is
* {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @deprecated Now we will specific whether the StoreFile is for primary replica when
* constructing, so please use
* {@link #StoreFile(FileSystem, StoreFileInfo, Configuration, CacheConfig, BloomType, boolean)}
* directly.
*/
@Deprecated
public StoreFile(final FileSystem fs, final StoreFileInfo fileInfo, final Configuration conf,
final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
this(fs, fileInfo, conf, cacheConf, cfBloomType, true);
}
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs fs The current file system to use.
* @param fileInfo The store file information.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType cfBloomType The bloom type to use for this store file as specified by column
* family configuration. This may or may not be the same as the Bloom filter type
* actually present in the HFile, because column family configuration might change. If
* this is {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @param primaryReplica true if this is a store file for primary replica, otherwise false.
*/
public StoreFile(FileSystem fs, StoreFileInfo fileInfo, Configuration conf, CacheConfig cacheConf,
BloomType cfBloomType, boolean primaryReplica) {
this.fs = fs;
this.fileInfo = fileInfo;
this.cacheConf = cacheConf;
this.noReadahead =
conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD);
if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
this.cfBloomType = cfBloomType;
} else {
LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " + "cfBloomType=" +
cfBloomType + " (disabled in config)");
this.cfBloomType = BloomType.NONE;
}
this.primaryReplica = primaryReplica;
}
/**
* @return the StoreFile object associated to this StoreFile.
* null if the StoreFile is not a reference.
*/
public StoreFileInfo getFileInfo() {
return this.fileInfo;
}
/** /**
* @return Path or null if this StoreFile was made with a Stream. * @return Path or null if this StoreFile was made with a Stream.
*/ */
public Path getPath() { Path getPath();
return this.fileInfo.getPath();
}
/** /**
* @return Returns the qualified path of this StoreFile * @return Returns the qualified path of this StoreFile
*/ */
public Path getQualifiedPath() { Path getQualifiedPath();
return this.fileInfo.getPath().makeQualified(fs.getUri(), fs.getWorkingDirectory());
}
/** /**
* @return True if this is a StoreFile Reference; call * @return True if this is a StoreFile Reference.
* after {@link #open()} else may get wrong answer.
*/ */
public boolean isReference() { boolean isReference();
return this.fileInfo.isReference();
}
/** /**
* @return True if this is HFile. * @return True if this is HFile.
*/ */
public boolean isHFile() { boolean isHFile();
return StoreFileInfo.isHFile(this.fileInfo.getPath());
}
/** /**
* @return True if this file was made by a major compaction. * @return True if this file was made by a major compaction.
*/ */
public boolean isMajorCompaction() { boolean isMajorCompactionResult();
if (this.majorCompaction == null) {
throw new NullPointerException("This has not been set yet");
}
return this.majorCompaction.get();
}
/** /**
* @return True if this file should not be part of a minor compaction. * @return True if this file should not be part of a minor compaction.
*/ */
public boolean excludeFromMinorCompaction() { boolean excludeFromMinorCompaction();
return this.excludeFromMinorCompaction;
}
/** /**
* @return This files maximum edit sequence id. * @return This files maximum edit sequence id.
*/ */
public long getMaxSequenceId() { long getMaxSequenceId();
return this.sequenceid;
}
public long getModificationTimeStamp() throws IOException { long getModificationTimeStamp() throws IOException;
return (fileInfo == null) ? 0 : fileInfo.getModificationTime();
}
/** /**
* Only used by the Striped Compaction Policy * Only used by the Striped Compaction Policy
* @param key * @param key
* @return value associated with the metadata key * @return value associated with the metadata key
*/ */
public byte[] getMetadataValue(byte[] key) { byte[] getMetadataValue(byte[] key);
return metadataMap.get(key);
}
/** /**
* Return the largest memstoreTS found across all storefiles in * Check if this storefile was created by bulk load. When a hfile is bulk loaded into HBase, we
* the given list. Store files that were created by a mapreduce * append {@code '_SeqId_<id-when-loaded>'} to the hfile name, unless
* bulk load are ignored, as they do not correspond to any specific * "hbase.mapreduce.bulkload.assign.sequenceNumbers" is explicitly turned off. If
* put operation, and thus do not have a memstoreTS associated with them. * "hbase.mapreduce.bulkload.assign.sequenceNumbers" is turned off, fall back to
* @return 0 if no non-bulk-load files are provided or, this is Store that * BULKLOAD_TIME_KEY.
* does not yet have any store files.
*/
public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
long max = 0;
for (StoreFile sf : sfs) {
if (!sf.isBulkLoadResult()) {
max = Math.max(max, sf.getMaxMemstoreTS());
}
}
return 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<StoreFile> sfs) {
long max = 0;
for (StoreFile sf : sfs) {
max = Math.max(max, sf.getMaxSequenceId());
}
return max;
}
/**
* Check if this storefile was created by bulk load.
* When a hfile is bulk loaded into HBase, we append
* {@code '_SeqId_<id-when-loaded>'} to the hfile name, unless
* "hbase.mapreduce.bulkload.assign.sequenceNumbers" is
* explicitly turned off.
* If "hbase.mapreduce.bulkload.assign.sequenceNumbers"
* is turned off, fall back to BULKLOAD_TIME_KEY.
* @return true if this storefile was created by bulk load. * @return true if this storefile was created by bulk load.
*/ */
public boolean isBulkLoadResult() { boolean isBulkLoadResult();
boolean bulkLoadedHFile = false;
String fileName = this.getPath().getName();
int startPos = fileName.indexOf("SeqId_");
if (startPos != -1) {
bulkLoadedHFile = true;
}
return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY));
}
@VisibleForTesting boolean isCompactedAway();
public boolean isCompactedAway() {
return compactedAway;
}
@VisibleForTesting
public int getRefCount() {
return refCount.get();
}
/** /**
* @return true if the file is still used in reads * @return true if the file is still used in reads
*/ */
public boolean isReferencedInReads() { boolean isReferencedInReads();
int rc = refCount.get();
assert rc >= 0; // we should not go negative.
return rc > 0;
}
/** /**
* Return the timestamp at which this bulk load file was generated. * Return the timestamp at which this bulk load file was generated.
*/ */
public long getBulkLoadTimestamp() { OptionalLong getBulkLoadTimestamp();
byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY);
return (bulkLoadTimestamp == null) ? 0 : Bytes.toLong(bulkLoadTimestamp);
}
/** /**
* @return the cached value of HDFS blocks distribution. The cached value is * @return the cached value of HDFS blocks distribution. The cached value is calculated when store
* calculated when store file is opened. * file is opened.
*/ */
public HDFSBlocksDistribution getHDFSBlockDistribution() { HDFSBlocksDistribution getHDFSBlockDistribution();
return this.fileInfo.getHDFSBlockDistribution();
}
/**
* Opens reader on this store file. Called by Constructor.
* @throws IOException
* @see #closeReader(boolean)
*/
private void open() throws IOException {
if (this.reader != null) {
throw new IllegalAccessError("Already open");
}
// Open the StoreFile.Reader
this.reader = fileInfo.open(this.fs, this.cacheConf, false, noReadahead ? 0L : -1L,
primaryReplica, refCount, true);
// Load up indices and fileinfo. This also loads Bloom filter type.
metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
// Read in our metadata.
byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
if (b != null) {
// By convention, if halfhfile, top half has a sequence number > bottom
// half. Thats why we add one in below. Its done for case the two halves
// are ever merged back together --rare. Without it, on open of store,
// since store files are distinguished by sequence id, the one half would
// subsume the other.
this.sequenceid = Bytes.toLong(b);
if (fileInfo.isTopReference()) {
this.sequenceid += 1;
}
}
if (isBulkLoadResult()){
// generate the sequenceId from the fileName
// fileName is of the form <randomName>_SeqId_<id-when-loaded>_
String fileName = this.getPath().getName();
// Use lastIndexOf() to get the last, most recent bulk load seqId.
int startPos = fileName.lastIndexOf("SeqId_");
if (startPos != -1) {
this.sequenceid = Long.parseLong(fileName.substring(startPos + 6,
fileName.indexOf('_', startPos + 6)));
// Handle reference files as done above.
if (fileInfo.isTopReference()) {
this.sequenceid += 1;
}
}
// SKIP_RESET_SEQ_ID only works in bulk loaded file.
// In mob compaction, the hfile where the cells contain the path of a new mob file is bulk
// loaded to hbase, these cells have the same seqIds with the old ones. We do not want
// to reset new seqIds for them since this might make a mess of the visibility of cells that
// have the same row key but different seqIds.
boolean skipResetSeqId = isSkipResetSeqId(metadataMap.get(SKIP_RESET_SEQ_ID));
if (skipResetSeqId) {
// increase the seqId when it is a bulk loaded file from mob compaction.
this.sequenceid += 1;
}
this.reader.setSkipResetSeqId(skipResetSeqId);
this.reader.setBulkLoaded(true);
}
this.reader.setSequenceID(this.sequenceid);
b = metadataMap.get(HFile.Writer.MAX_MEMSTORE_TS_KEY);
if (b != null) {
this.maxMemstoreTS = Bytes.toLong(b);
}
b = metadataMap.get(MAJOR_COMPACTION_KEY);
if (b != null) {
boolean mc = Bytes.toBoolean(b);
if (this.majorCompaction == null) {
this.majorCompaction = new AtomicBoolean(mc);
} else {
this.majorCompaction.set(mc);
}
} else {
// Presume it is not major compacted if it doesn't explicity say so
// HFileOutputFormat explicitly sets the major compacted key.
this.majorCompaction = new AtomicBoolean(false);
}
b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
BloomType hfileBloomType = reader.getBloomFilterType();
if (cfBloomType != BloomType.NONE) {
reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
if (hfileBloomType != cfBloomType) {
LOG.info("HFile Bloom filter type for "
+ reader.getHFileReader().getName() + ": " + hfileBloomType
+ ", but " + cfBloomType + " specified in column family "
+ "configuration");
}
} else if (hfileBloomType != BloomType.NONE) {
LOG.info("Bloom filter turned off by CF config for "
+ reader.getHFileReader().getName());
}
// load delete family bloom filter
reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
try {
this.reader.timeRange = TimeRangeTracker.getTimeRange(metadataMap.get(TIMERANGE_KEY));
} catch (IllegalArgumentException e) {
LOG.error("Error reading timestamp range data from meta -- " +
"proceeding without", e);
this.reader.timeRange = null;
}
// initialize so we can reuse them after reader closed.
firstKey = reader.getFirstKey();
lastKey = reader.getLastKey();
comparator = reader.getComparator();
}
/** /**
* Initialize the reader used for pread. * Initialize the reader used for pread.
*/ */
public void initReader() throws IOException { void initReader() throws IOException;
if (reader == null) {
try {
open();
} catch (Exception e) {
try {
boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true;
this.closeReader(evictOnClose);
} catch (IOException ee) {
LOG.warn("failed to close reader", ee);
}
throw e;
}
}
}
private StoreFileReader createStreamReader(boolean canUseDropBehind) throws IOException {
initReader();
StoreFileReader reader = fileInfo.open(this.fs, this.cacheConf, canUseDropBehind, -1L,
primaryReplica, refCount, false);
reader.copyFields(this.reader);
return reader;
}
public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
boolean canOptimizeForNonNullColumn) {
return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder,
canOptimizeForNonNullColumn);
}
public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
throws IOException {
return createStreamReader(canUseDropBehind).getStoreFileScanner(cacheBlocks, false,
isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn);
}
/** /**
* @return Current reader. Must call initReader first else returns null. * Must be called after initReader.
*/
StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
boolean canOptimizeForNonNullColumn);
StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
throws IOException;
/**
* @return Current reader. Must call initReader first else returns null.
* @see #initReader() * @see #initReader()
*/ */
public StoreFileReader getReader() { StoreFileReader getReader();
return this.reader;
}
/** /**
* @param evictOnClose whether to evict blocks belonging to this file * @param evictOnClose whether to evict blocks belonging to this file
* @throws IOException * @throws IOException
*/ */
public synchronized void closeReader(boolean evictOnClose) void closeReader(boolean evictOnClose) throws IOException;
throws IOException {
if (this.reader != null) {
this.reader.close(evictOnClose);
this.reader = null;
}
}
/** /**
* Marks the status of the file as compactedAway. * Marks the status of the file as compactedAway.
*/ */
public void markCompactedAway() { void markCompactedAway();
this.compactedAway = true;
}
/** /**
* Delete this file * Delete this file
* @throws IOException * @throws IOException
*/ */
public void deleteReader() throws IOException { void deleteReader() throws IOException;
boolean evictOnClose =
cacheConf != null? cacheConf.shouldEvictOnClose(): true;
closeReader(evictOnClose);
this.fs.delete(getPath(), true);
}
@Override
public String toString() {
return this.fileInfo.toString();
}
/** /**
* @return a length description of this StoreFile, suitable for debug output * @return a length description of this StoreFile, suitable for debug output
*/ */
public String toStringDetailed() { String toStringDetailed();
StringBuilder sb = new StringBuilder();
sb.append(this.getPath().toString());
sb.append(", isReference=").append(isReference());
sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
if (isBulkLoadResult()) {
sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
} else {
sb.append(", seqid=").append(getMaxSequenceId());
}
sb.append(", majorCompaction=").append(isMajorCompaction());
return sb.toString(); OptionalLong getMinimumTimestamp();
}
/** OptionalLong getMaximumTimestamp();
* Gets whether to skip resetting the sequence id for cells.
* @param skipResetSeqId The byte array of boolean.
* @return Whether to skip resetting the sequence id.
*/
private boolean isSkipResetSeqId(byte[] skipResetSeqId) {
if (skipResetSeqId != null && skipResetSeqId.length == 1) {
return Bytes.toBoolean(skipResetSeqId);
}
return false;
}
/**
* @param fs
* @param dir Directory to create file in.
* @return random filename inside passed <code>dir</code>
*/
public static Path getUniqueFile(final FileSystem fs, final Path dir)
throws IOException {
if (!fs.getFileStatus(dir).isDirectory()) {
throw new IOException("Expecting " + dir.toString() +
" to be a directory");
}
return new Path(dir, UUID.randomUUID().toString().replaceAll("-", ""));
}
public Long getMinimumTimestamp() {
return getReader().timeRange == null? null: getReader().timeRange.getMin();
}
public Long getMaximumTimestamp() {
return getReader().timeRange == null? null: getReader().timeRange.getMax();
}
/**
* Gets the approximate mid-point of this file that is optimal for use in splitting it.
* @param comparator Comparator used to compare KVs.
* @return The split point row, or null if splitting is not possible, or reader is null.
*/
byte[] getFileSplitPoint(CellComparator comparator) throws IOException {
if (this.reader == null) {
LOG.warn("Storefile " + this + " Reader is null; cannot get split point");
return null;
}
// Get first, last, and mid keys. Midkey is the key that starts block
// in middle of hfile. Has column and timestamp. Need to return just
// the row we want to split on as midkey.
Cell midkey = this.reader.midkey();
if (midkey != null) {
Cell firstKey = this.reader.getFirstKey();
Cell lastKey = this.reader.getLastKey();
// if the midkey is the same as the first or last keys, we cannot (ever) split this region.
if (comparator.compareRows(midkey, firstKey) == 0
|| comparator.compareRows(midkey, lastKey) == 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("cannot split because midkey is the same as first or last row");
}
return null;
}
return CellUtil.cloneRow(midkey);
}
return null;
}
/**
* Useful comparators for comparing StoreFiles.
*/
public abstract static class Comparators {
/**
* Comparator that compares based on the Sequence Ids of the
* the StoreFiles. Bulk loads that did not request a seq ID
* are given a seq id of -1; thus, they are placed before all non-
* bulk loads, and bulk loads with sequence Id. Among these files,
* the size is used to determine the ordering, then bulkLoadTime.
* If there are ties, the path name is used as a tie-breaker.
*/
public static final Comparator<StoreFile> SEQ_ID =
Ordering.compound(ImmutableList.of(
Ordering.natural().onResultOf(new GetSeqId()),
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
Ordering.natural().onResultOf(new GetBulkTime()),
Ordering.natural().onResultOf(new GetPathName())
));
/**
* Comparator for time-aware compaction. SeqId is still the first
* ordering criterion to maintain MVCC.
*/
public static final Comparator<StoreFile> SEQ_ID_MAX_TIMESTAMP =
Ordering.compound(ImmutableList.of(
Ordering.natural().onResultOf(new GetSeqId()),
Ordering.natural().onResultOf(new GetMaxTimestamp()),
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
Ordering.natural().onResultOf(new GetBulkTime()),
Ordering.natural().onResultOf(new GetPathName())
));
private static class GetSeqId implements Function<StoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
return sf.getMaxSequenceId();
}
}
private static class GetFileSize implements Function<StoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
if (sf.getReader() != null) {
return sf.getReader().length();
} else {
// the reader may be null for the compacted files and if the archiving
// had failed.
return -1L;
}
}
}
private static class GetBulkTime implements Function<StoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
return sf.getBulkLoadTimestamp();
}
}
private static class GetPathName implements Function<StoreFile, String> {
@Override
public String apply(StoreFile sf) {
return sf.getPath().getName();
}
}
private static class GetMaxTimestamp implements Function<StoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
return sf.getMaximumTimestamp() == null? (Long)Long.MAX_VALUE : sf.getMaximumTimestamp();
}
}
}
} }

View File

@ -0,0 +1,96 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Ordering;
import java.util.Comparator;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Useful comparators for comparing StoreFiles.
*/
@InterfaceAudience.Private
final class StoreFileComparators {
/**
* Comparator that compares based on the Sequence Ids of the the StoreFiles. Bulk loads that did
* not request a seq ID are given a seq id of -1; thus, they are placed before all non- bulk
* loads, and bulk loads with sequence Id. Among these files, the size is used to determine the
* ordering, then bulkLoadTime. If there are ties, the path name is used as a tie-breaker.
*/
public static final Comparator<StoreFile> SEQ_ID =
Ordering.compound(ImmutableList.of(Ordering.natural().onResultOf(new GetSeqId()),
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
Ordering.natural().onResultOf(new GetBulkTime()),
Ordering.natural().onResultOf(new GetPathName())));
/**
* Comparator for time-aware compaction. SeqId is still the first ordering criterion to maintain
* MVCC.
*/
public static final Comparator<StoreFile> SEQ_ID_MAX_TIMESTAMP =
Ordering.compound(ImmutableList.of(Ordering.natural().onResultOf(new GetSeqId()),
Ordering.natural().onResultOf(new GetMaxTimestamp()),
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
Ordering.natural().onResultOf(new GetBulkTime()),
Ordering.natural().onResultOf(new GetPathName())));
private static class GetSeqId implements Function<StoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
return sf.getMaxSequenceId();
}
}
private static class GetFileSize implements Function<StoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
if (sf.getReader() != null) {
return sf.getReader().length();
} else {
// the reader may be null for the compacted files and if the archiving
// had failed.
return -1L;
}
}
}
private static class GetBulkTime implements Function<StoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
return sf.getBulkLoadTimestamp().orElse(Long.MAX_VALUE);
}
}
private static class GetPathName implements Function<StoreFile, String> {
@Override
public String apply(StoreFile sf) {
return sf.getPath().getName();
}
}
private static class GetMaxTimestamp implements Function<StoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
return sf.getMaximumTimestamp().orElse(Long.MAX_VALUE);
}
}
}

View File

@ -283,7 +283,6 @@ public class StoreFileInfo {
*/ */
public HDFSBlocksDistribution computeHDFSBlocksDistribution(final FileSystem fs) public HDFSBlocksDistribution computeHDFSBlocksDistribution(final FileSystem fs)
throws IOException { throws IOException {
// guard against the case where we get the FileStatus from link, but by the time we // guard against the case where we get the FileStatus from link, but by the time we
// call compute the file is moved again // call compute the file is moved again
if (this.link != null) { if (this.link != null) {

View File

@ -25,6 +25,7 @@ import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.PriorityQueue;
import java.util.concurrent.atomic.LongAdder; import java.util.concurrent.atomic.LongAdder;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -120,13 +121,20 @@ public class StoreFileScanner implements KeyValueScanner {
public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files, public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop, boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop,
ScanQueryMatcher matcher, long readPt) throws IOException { ScanQueryMatcher matcher, long readPt) throws IOException {
if (files.isEmpty()) {
return Collections.emptyList();
}
List<StoreFileScanner> scanners = new ArrayList<>(files.size()); List<StoreFileScanner> scanners = new ArrayList<>(files.size());
List<StoreFile> sortedFiles = new ArrayList<>(files);
Collections.sort(sortedFiles, StoreFile.Comparators.SEQ_ID);
boolean canOptimizeForNonNullColumn = matcher != null ? !matcher.hasNullColumnInQuery() : false; boolean canOptimizeForNonNullColumn = matcher != null ? !matcher.hasNullColumnInQuery() : false;
for (int i = 0, n = sortedFiles.size(); i < n; i++) { PriorityQueue<StoreFile> sortedFiles =
StoreFile sf = sortedFiles.get(i); new PriorityQueue<>(files.size(), StoreFileComparators.SEQ_ID);
sf.initReader(); for (StoreFile file : files) {
// The sort function needs metadata so we need to open reader first before sorting the list.
file.initReader();
sortedFiles.add(file);
}
for (int i = 0, n = files.size(); i < n; i++) {
StoreFile sf = sortedFiles.remove();
StoreFileScanner scanner; StoreFileScanner scanner;
if (usePread) { if (usePread) {
scanner = sf.getPreadScanner(cacheBlocks, readPt, i, canOptimizeForNonNullColumn); scanner = sf.getPreadScanner(cacheBlocks, readPt, i, canOptimizeForNonNullColumn);
@ -147,7 +155,7 @@ public class StoreFileScanner implements KeyValueScanner {
boolean canUseDropBehind, long readPt) throws IOException { boolean canUseDropBehind, long readPt) throws IOException {
List<StoreFileScanner> scanners = new ArrayList<>(files.size()); List<StoreFileScanner> scanners = new ArrayList<>(files.size());
List<StoreFile> sortedFiles = new ArrayList<>(files); List<StoreFile> sortedFiles = new ArrayList<>(files);
Collections.sort(sortedFiles, StoreFile.Comparators.SEQ_ID); Collections.sort(sortedFiles, StoreFileComparators.SEQ_ID);
boolean succ = false; boolean succ = false;
try { try {
for (int i = 0, n = sortedFiles.size(); i < n; i++) { for (int i = 0, n = sortedFiles.size(); i < n; i++) {

View File

@ -18,8 +18,11 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import com.google.common.base.Preconditions;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.util.UUID;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -33,7 +36,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -42,13 +44,10 @@ import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.BloomFilterWriter; import org.apache.hadoop.hbase.util.BloomFilterWriter;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.RowBloomContext; import org.apache.hadoop.hbase.util.RowBloomContext;
import org.apache.hadoop.hbase.util.RowColBloomContext; import org.apache.hadoop.hbase.util.RowColBloomContext;
import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.WritableUtils;
import com.google.common.base.Preconditions;
/** /**
* A StoreFile writer. Use this to read/write HBase Store Files. It is package * A StoreFile writer. Use this to read/write HBase Store Files. It is package
* local because it is an implementation detail of the HBase regionserver. * local because it is an implementation detail of the HBase regionserver.
@ -359,6 +358,18 @@ public class StoreFileWriter implements CellSink, ShipperListener {
return writer; return writer;
} }
/**
* @param fs
* @param dir Directory to create file in.
* @return random filename inside passed <code>dir</code>
*/
static Path getUniqueFile(final FileSystem fs, final Path dir) throws IOException {
if (!fs.getFileStatus(dir).isDirectory()) {
throw new IOException("Expecting " + dir.toString() + " to be a directory");
}
return new Path(dir, UUID.randomUUID().toString().replaceAll("-", ""));
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ICAST_INTEGER_MULTIPLY_CAST_TO_LONG", @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ICAST_INTEGER_MULTIPLY_CAST_TO_LONG",
justification="Will not overflow") justification="Will not overflow")
public static class Builder { public static class Builder {
@ -496,7 +507,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
FSUtils.setStoragePolicy(this.fs, dir, policyName); FSUtils.setStoragePolicy(this.fs, dir, policyName);
if (filePath == null) { if (filePath == null) {
filePath = StoreFile.getUniqueFile(fs, dir); filePath = getUniqueFile(fs, dir);
if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) { if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
bloomType = BloomType.NONE; bloomType = BloomType.NONE;
} }

View File

@ -19,8 +19,6 @@
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import com.google.common.annotations.VisibleForTesting;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -55,6 +53,8 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;
import org.apache.hadoop.hbase.util.CollectionUtils; import org.apache.hadoop.hbase.util.CollectionUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.annotations.VisibleForTesting;
/** /**
* Scanner scans both the memstore and the Store. Coalesce KeyValue stream into List&lt;KeyValue&gt; * Scanner scans both the memstore and the Store. Coalesce KeyValue stream into List&lt;KeyValue&gt;
* for a single row. * for a single row.

View File

@ -20,7 +20,14 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException; import java.io.IOException;
import java.util.Collection; import java.util.Collection;
import java.util.Optional;
import java.util.OptionalInt;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
/** /**
@ -28,14 +35,14 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class StoreUtils { public class StoreUtils {
private static final Log LOG = LogFactory.getLog(StoreUtils.class);
/** /**
* Creates a deterministic hash code for store file collection. * Creates a deterministic hash code for store file collection.
*/ */
public static Integer getDeterministicRandomSeed(final Collection<StoreFile> files) { public static OptionalInt getDeterministicRandomSeed(Collection<StoreFile> files) {
if (files != null && !files.isEmpty()) { return files.stream().mapToInt(f -> f.getPath().getName().hashCode()).findFirst();
return files.iterator().next().getPath().getName().hashCode();
}
return null;
} }
/** /**
@ -70,18 +77,73 @@ public class StoreUtils {
* @param candidates The files to choose from. * @param candidates The files to choose from.
* @return The largest file; null if no file has a reader. * @return The largest file; null if no file has a reader.
*/ */
static StoreFile getLargestFile(final Collection<StoreFile> candidates) { static Optional<StoreFile> getLargestFile(Collection<StoreFile> candidates) {
long maxSize = -1L; return candidates.stream().filter(f -> f.getReader() != null)
StoreFile largestSf = null; .max((f1, f2) -> Long.compare(f1.getReader().length(), f2.getReader().length()));
for (StoreFile sf : candidates) { }
StoreFileReader r = sf.getReader();
if (r == null) continue; /**
long size = r.length(); * Return the largest memstoreTS found across all storefiles in the given list. Store files that
if (size > maxSize) { * were created by a mapreduce bulk load are ignored, as they do not correspond to any specific
maxSize = size; * put operation, and thus do not have a memstoreTS associated with them.
largestSf = sf; * @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<StoreFile> sfs) {
long max = 0;
for (StoreFile sf : sfs) {
if (!sf.isBulkLoadResult()) {
max = Math.max(max, sf.getMaxMemstoreTS());
} }
} }
return largestSf; return 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<StoreFile> sfs) {
long max = 0;
for (StoreFile sf : sfs) {
max = Math.max(max, sf.getMaxSequenceId());
}
return max;
}
/**
* Gets the approximate mid-point of the given file that is optimal for use in splitting it.
* @param file the store file
* @param comparator Comparator used to compare KVs.
* @return The split point row, or null if splitting is not possible, or reader is null.
*/
static Optional<byte[]> getFileSplitPoint(StoreFile file, CellComparator comparator)
throws IOException {
StoreFileReader reader = file.getReader();
if (reader == null) {
LOG.warn("Storefile " + file + " Reader is null; cannot get split point");
return Optional.empty();
}
// Get first, last, and mid keys. Midkey is the key that starts block
// in middle of hfile. Has column and timestamp. Need to return just
// the row we want to split on as midkey.
Cell midkey = reader.midkey();
if (midkey != null) {
Cell firstKey = reader.getFirstKey();
Cell lastKey = reader.getLastKey();
// if the midkey is the same as the first or last keys, we cannot (ever) split this region.
if (comparator.compareRows(midkey, firstKey) == 0 ||
comparator.compareRows(midkey, lastKey) == 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("cannot split because midkey is the same as first or last row");
}
return Optional.empty();
}
return Optional.of(CellUtil.cloneRow(midkey));
}
return Optional.empty();
} }
} }

View File

@ -28,6 +28,7 @@ import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.TreeMap; import java.util.TreeMap;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -38,7 +39,6 @@ import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KeyOnlyKeyValue;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -216,7 +216,12 @@ public class StripeStoreFileManager
return original; return original;
} }
@Override private byte[] getSplitPoint(Collection<StoreFile> sfs) throws IOException {
Optional<StoreFile> largestFile = StoreUtils.getLargestFile(sfs);
return largestFile.isPresent()
? StoreUtils.getFileSplitPoint(largestFile.get(), cellComparator).orElse(null) : null;
}
/** /**
* Override of getSplitPoint that determines the split point as the boundary between two * Override of getSplitPoint that determines the split point as the boundary between two
* stripes, unless it causes significant imbalance between split sides' sizes. In that * stripes, unless it causes significant imbalance between split sides' sizes. In that
@ -224,6 +229,7 @@ public class StripeStoreFileManager
* minimize imbalance. * minimize imbalance.
* @return The split point, or null if no split is possible. * @return The split point, or null if no split is possible.
*/ */
@Override
public byte[] getSplitPoint() throws IOException { public byte[] getSplitPoint() throws IOException {
if (this.getStorefileCount() == 0) return null; if (this.getStorefileCount() == 0) return null;
if (state.stripeFiles.size() <= 1) { if (state.stripeFiles.size() <= 1) {
@ -271,16 +277,14 @@ public class StripeStoreFileManager
LOG.debug("Splitting the stripe - ratio w/o split " + ratio + ", ratio with split " LOG.debug("Splitting the stripe - ratio w/o split " + ratio + ", ratio with split "
+ newRatio + " configured ratio " + config.getMaxSplitImbalance()); + newRatio + " configured ratio " + config.getMaxSplitImbalance());
// Ok, we may get better ratio, get it. // Ok, we may get better ratio, get it.
return StoreUtils.getLargestFile(state.stripeFiles.get( return getSplitPoint(state.stripeFiles.get(isRightLarger ? rightIndex : leftIndex));
isRightLarger ? rightIndex : leftIndex)).getFileSplitPoint(this.cellComparator);
} }
private byte[] getSplitPointFromAllFiles() throws IOException { private byte[] getSplitPointFromAllFiles() throws IOException {
ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>(); ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
sfs.addSublist(state.level0Files); sfs.addSublist(state.level0Files);
sfs.addAllSublists(state.stripeFiles); sfs.addAllSublists(state.stripeFiles);
if (sfs.isEmpty()) return null; return getSplitPoint(sfs);
return StoreUtils.getLargestFile(sfs).getFileSplitPoint(this.cellComparator);
} }
private double getMidStripeSplitRatio(long smallerSize, long largerSize, long lastLargerSize) { private double getMidStripeSplitRatio(long smallerSize, long largerSize, long lastLargerSize) {
@ -639,7 +643,7 @@ public class StripeStoreFileManager
// we will store the file in reverse order by seqNum from the outset. // we will store the file in reverse order by seqNum from the outset.
for (int insertBefore = 0; ; ++insertBefore) { for (int insertBefore = 0; ; ++insertBefore) {
if (insertBefore == stripe.size() if (insertBefore == stripe.size()
|| (StoreFile.Comparators.SEQ_ID.compare(sf, stripe.get(insertBefore)) >= 0)) { || (StoreFileComparators.SEQ_ID.compare(sf, stripe.get(insertBefore)) >= 0)) {
stripe.add(insertBefore, sf); stripe.add(insertBefore, sf);
break; break;
} }
@ -1071,6 +1075,6 @@ public class StripeStoreFileManager
@Override @Override
public Comparator<StoreFile> getStoreFileComparator() { public Comparator<StoreFile> getStoreFileComparator() {
return StoreFile.Comparators.SEQ_ID; return StoreFileComparators.SEQ_ID;
} }
} }

View File

@ -18,24 +18,19 @@
*/ */
package org.apache.hadoop.hbase.regionserver.compactions; package org.apache.hadoop.hbase.regionserver.compactions;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.Collections2;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileReader; import org.apache.hadoop.hbase.regionserver.StoreFileReader;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import com.google.common.base.Preconditions;
/** /**
* This class holds all logical details necessary to run a compaction. * This class holds all logical details necessary to run a compaction.
@ -43,7 +38,7 @@ import org.apache.hadoop.util.StringUtils;
@InterfaceAudience.LimitedPrivate({ "coprocessor" }) @InterfaceAudience.LimitedPrivate({ "coprocessor" })
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class CompactionRequest implements Comparable<CompactionRequest> { public class CompactionRequest implements Comparable<CompactionRequest> {
private static final Log LOG = LogFactory.getLog(CompactionRequest.class);
// was this compaction promoted to an off-peak // was this compaction promoted to an off-peak
private boolean isOffPeak = false; private boolean isOffPeak = false;
private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR } private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR }
@ -207,27 +202,15 @@ public class CompactionRequest implements Comparable<CompactionRequest> {
@Override @Override
public String toString() { public String toString() {
String fsList = Joiner.on(", ").join( String fsList = filesToCompact.stream().filter(f -> f.getReader() != null)
Collections2.transform(Collections2.filter( .map(f -> TraditionalBinaryPrefix.long2String(f.getReader().length(), "", 1))
this.getFiles(), .collect(Collectors.joining(", "));
new Predicate<StoreFile>() {
@Override
public boolean apply(StoreFile sf) {
return sf.getReader() != null;
}
}), new Function<StoreFile, String>() {
@Override
public String apply(StoreFile sf) {
return StringUtils.humanReadableInt(
(sf.getReader() == null) ? 0 : sf.getReader().length());
}
}));
return "regionName=" + regionName + ", storeName=" + storeName + return "regionName=" + regionName + ", storeName=" + storeName + ", fileCount=" +
", fileCount=" + this.getFiles().size() + this.getFiles().size() + ", fileSize=" +
", fileSize=" + StringUtils.humanReadableInt(totalSize) + TraditionalBinaryPrefix.long2String(totalSize, "", 1) +
((fsList.isEmpty()) ? "" : " (" + fsList + ")") + ((fsList.isEmpty()) ? "" : " (" + fsList + ")") + ", priority=" + priority + ", time=" +
", priority=" + priority + ", time=" + timeInNanos; timeInNanos;
} }
/** /**

View File

@ -17,8 +17,6 @@
*/ */
package org.apache.hadoop.hbase.regionserver.compactions; package org.apache.hadoop.hbase.regionserver.compactions;
import com.google.common.io.Closeables;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -60,6 +58,8 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import com.google.common.io.Closeables;
/** /**
* A compactor is a compaction algorithm associated a given policy. Base class also contains * A compactor is a compaction algorithm associated a given policy. Base class also contains
* reusable parts for implementing compactors (what is common and what isn't is evolving). * reusable parts for implementing compactors (what is common and what isn't is evolving).

View File

@ -18,17 +18,12 @@
*/ */
package org.apache.hadoop.hbase.regionserver.compactions; package org.apache.hadoop.hbase.regionserver.compactions;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.PeekingIterator;
import com.google.common.math.LongMath;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.OptionalLong;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -44,6 +39,12 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.hadoop.hbase.util.ReflectionUtils;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.PeekingIterator;
import com.google.common.math.LongMath;
/** /**
* HBASE-15181 This is a simple implementation of date-based tiered compaction similar to * HBASE-15181 This is a simple implementation of date-based tiered compaction similar to
* Cassandra's for the following benefits: * Cassandra's for the following benefits:
@ -135,24 +136,24 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
boolean[] filesInWindow = new boolean[boundaries.size()]; boolean[] filesInWindow = new boolean[boundaries.size()];
for (StoreFile file: filesToCompact) { for (StoreFile file: filesToCompact) {
Long minTimestamp = file.getMinimumTimestamp(); OptionalLong minTimestamp = file.getMinimumTimestamp();
long oldest = (minTimestamp == null) ? Long.MIN_VALUE : now - minTimestamp.longValue(); long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE;
if (cfTTL != Long.MAX_VALUE && oldest >= cfTTL) { if (cfTTL != Long.MAX_VALUE && oldest >= cfTTL) {
LOG.debug("Major compaction triggered on store " + this LOG.debug("Major compaction triggered on store " + this
+ "; for TTL maintenance"); + "; for TTL maintenance");
return true; return true;
} }
if (!file.isMajorCompaction() || file.isBulkLoadResult()) { if (!file.isMajorCompactionResult() || file.isBulkLoadResult()) {
LOG.debug("Major compaction triggered on store " + this LOG.debug("Major compaction triggered on store " + this
+ ", because there are new files and time since last major compaction " + ", because there are new files and time since last major compaction "
+ (now - lowTimestamp) + "ms"); + (now - lowTimestamp) + "ms");
return true; return true;
} }
int lowerWindowIndex = Collections.binarySearch(boundaries, int lowerWindowIndex =
minTimestamp == null ? (Long)Long.MAX_VALUE : minTimestamp); Collections.binarySearch(boundaries, minTimestamp.orElse(Long.MAX_VALUE));
int upperWindowIndex = Collections.binarySearch(boundaries, int upperWindowIndex =
file.getMaximumTimestamp() == null ? (Long)Long.MAX_VALUE : file.getMaximumTimestamp()); Collections.binarySearch(boundaries, file.getMaximumTimestamp().orElse(Long.MAX_VALUE));
// Handle boundary conditions and negative values of binarySearch // Handle boundary conditions and negative values of binarySearch
lowerWindowIndex = (lowerWindowIndex < 0) ? Math.abs(lowerWindowIndex + 2) : lowerWindowIndex; lowerWindowIndex = (lowerWindowIndex < 0) ? Math.abs(lowerWindowIndex + 2) : lowerWindowIndex;
upperWindowIndex = (upperWindowIndex < 0) ? Math.abs(upperWindowIndex + 2) : upperWindowIndex; upperWindowIndex = (upperWindowIndex < 0) ? Math.abs(upperWindowIndex + 2) : upperWindowIndex;
@ -220,8 +221,8 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
for (StoreFile storeFile : candidateSelection) { for (StoreFile storeFile : candidateSelection) {
// if there is out-of-order data, // if there is out-of-order data,
// we put them in the same window as the last file in increasing order // we put them in the same window as the last file in increasing order
maxTimestampSeen = Math.max(maxTimestampSeen, maxTimestampSeen =
storeFile.getMaximumTimestamp() == null? Long.MIN_VALUE : storeFile.getMaximumTimestamp()); Math.max(maxTimestampSeen, storeFile.getMaximumTimestamp().orElse(Long.MIN_VALUE));
storefileMaxTimestampPairs.add(new Pair<>(storeFile, maxTimestampSeen)); storefileMaxTimestampPairs.add(new Pair<>(storeFile, maxTimestampSeen));
} }
Collections.reverse(storefileMaxTimestampPairs); Collections.reverse(storefileMaxTimestampPairs);
@ -288,23 +289,18 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
} }
/** /**
* Return a list of boundaries for multiple compaction output * Return a list of boundaries for multiple compaction output in ascending order.
* in ascending order.
*/ */
private List<Long> getCompactBoundariesForMajor(Collection<StoreFile> filesToCompact, long now) { private List<Long> getCompactBoundariesForMajor(Collection<StoreFile> filesToCompact, long now) {
long minTimestamp = Long.MAX_VALUE; long minTimestamp =
for (StoreFile file : filesToCompact) { filesToCompact.stream().mapToLong(f -> f.getMinimumTimestamp().orElse(Long.MAX_VALUE)).min()
minTimestamp = .orElse(Long.MAX_VALUE);
Math.min(minTimestamp,
file.getMinimumTimestamp() == null ? Long.MAX_VALUE : file.getMinimumTimestamp());
}
List<Long> boundaries = new ArrayList<>(); List<Long> boundaries = new ArrayList<>();
// Add startMillis of all windows between now and min timestamp // Add startMillis of all windows between now and min timestamp
for (CompactionWindow window = getIncomingWindow(now); for (CompactionWindow window = getIncomingWindow(now); window
window.compareToTimestamp(minTimestamp) > 0; .compareToTimestamp(minTimestamp) > 0; window = window.nextEarlierWindow()) {
window = window.nextEarlierWindow()) {
boundaries.add(window.startMillis()); boundaries.add(window.startMillis());
} }
boundaries.add(Long.MIN_VALUE); boundaries.add(Long.MIN_VALUE);

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter; import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter;
import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreUtils;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
@ -47,7 +47,7 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTiered
private boolean needEmptyFile(CompactionRequest request) { private boolean needEmptyFile(CompactionRequest request) {
// if we are going to compact the last N files, then we need to emit an empty file to retain the // if we are going to compact the last N files, then we need to emit an empty file to retain the
// maxSeqId if we haven't written out anything. // maxSeqId if we haven't written out anything.
return StoreFile.getMaxSequenceIdInList(request.getFiles()) == store.getMaxSequenceId(); return StoreUtils.getMaxSequenceIdInList(request.getFiles()) == store.getMaxSequenceId();
} }
public List<Path> compact(final CompactionRequest request, final List<Long> lowerBoundaries, public List<Path> compact(final CompactionRequest request, final List<Long> lowerBoundaries,

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.OptionalLong;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
@ -79,9 +80,9 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
if (filesToCompact.size() == 1) { if (filesToCompact.size() == 1) {
// Single file // Single file
StoreFile sf = filesToCompact.iterator().next(); StoreFile sf = filesToCompact.iterator().next();
Long minTimestamp = sf.getMinimumTimestamp(); OptionalLong minTimestamp = sf.getMinimumTimestamp();
long oldest = (minTimestamp == null) ? Long.MIN_VALUE : now - minTimestamp.longValue(); long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE;
if (sf.isMajorCompaction() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) { if (sf.isMajorCompactionResult() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) {
float blockLocalityIndex = float blockLocalityIndex =
sf.getHDFSBlockDistribution().getBlockLocalityIndex( sf.getHDFSBlockDistribution().getBlockLocalityIndex(
RSRpcServices.getHostname(comConf.conf, false)); RSRpcServices.getHostname(comConf.conf, false));

View File

@ -10,15 +10,11 @@
*/ */
package org.apache.hadoop.hbase.regionserver.compactions; package org.apache.hadoop.hbase.regionserver.compactions;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.Collections2;
import com.google.common.collect.Lists;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.OptionalInt;
import java.util.Random; import java.util.Random;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
@ -29,6 +25,11 @@ import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.regionserver.StoreUtils;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.Collections2;
import com.google.common.collect.Lists;
/** /**
* An abstract compaction policy that select files on seq id order. * An abstract compaction policy that select files on seq id order.
*/ */
@ -118,30 +119,31 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
* @param filesToCompact * @param filesToCompact
* @return When to run next major compaction * @return When to run next major compaction
*/ */
public long getNextMajorCompactTime(final Collection<StoreFile> filesToCompact) { public long getNextMajorCompactTime(Collection<StoreFile> filesToCompact) {
// default = 24hrs // default = 24hrs
long ret = comConf.getMajorCompactionPeriod(); long period = comConf.getMajorCompactionPeriod();
if (ret > 0) { if (period <= 0) {
// default = 20% = +/- 4.8 hrs return period;
double jitterPct = comConf.getMajorCompactionJitter(); }
if (jitterPct > 0) { // default = 20% = +/- 4.8 hrs
long jitter = Math.round(ret * jitterPct); double jitterPct = comConf.getMajorCompactionJitter();
// deterministic jitter avoids a major compaction storm on restart if (jitterPct <= 0) {
Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact); return period;
if (seed != null) { }
// Synchronized to ensure one user of random instance at a time. // deterministic jitter avoids a major compaction storm on restart
double rnd = -1; OptionalInt seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
synchronized (this) { if (seed.isPresent()) {
this.random.setSeed(seed); // Synchronized to ensure one user of random instance at a time.
rnd = this.random.nextDouble(); double rnd;
} synchronized (this) {
ret += jitter - Math.round(2L * jitter * rnd); this.random.setSeed(seed.getAsInt());
} else { rnd = this.random.nextDouble();
ret = 0; // If seed is null, then no storefiles == no major compaction }
} long jitter = Math.round(period * jitterPct);
} return period + jitter - Math.round(2L * jitter * rnd);
} else {
return 0L;
} }
return ret;
} }
/** /**

View File

@ -18,9 +18,6 @@
package org.apache.hadoop.hbase.snapshot; package org.apache.hadoop.hbase.snapshot;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -45,15 +42,17 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;

View File

@ -24,9 +24,9 @@ import java.io.IOException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter; import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterBase; import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterBase;

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.Reference; import org.apache.hadoop.hbase.io.Reference;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.Leases; import org.apache.hadoop.hbase.regionserver.Leases;

View File

@ -22,15 +22,14 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import java.io.IOException; import java.io.IOException;
import java.io.UnsupportedEncodingException; import java.io.UnsupportedEncodingException;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.util.Arrays;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -81,7 +80,6 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFile.Reader; import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
import org.apache.hadoop.hbase.mapreduce.TestImportTSVWithTTLs.TTLCheckingObserver;
import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
@ -93,7 +91,6 @@ import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.hbase.util.Writables;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.SmallTests;
@ -61,7 +61,7 @@ public class TestMobFile extends TestCase {
MobTestUtil.writeStoreFile(writer, caseName); MobTestUtil.writeStoreFile(writer, caseName);
MobFile mobFile = MobFile mobFile =
new MobFile(new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true)); new MobFile(new HStoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true));
byte[] family = Bytes.toBytes(caseName); byte[] family = Bytes.toBytes(caseName);
byte[] qualify = Bytes.toBytes(caseName); byte[] qualify = Bytes.toBytes(caseName);
@ -113,7 +113,7 @@ public class TestMobFile extends TestCase {
MobTestUtil.writeStoreFile(writer, getName()); MobTestUtil.writeStoreFile(writer, getName());
MobFile mobFile = MobFile mobFile =
new MobFile(new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true)); new MobFile(new HStoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true));
assertNotNull(mobFile.getScanner()); assertNotNull(mobFile.getScanner());
assertTrue(mobFile.getScanner() instanceof StoreFileScanner); assertTrue(mobFile.getScanner() instanceof StoreFileScanner);
} }

View File

@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.mob.MobFileName;
import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
@ -814,7 +815,7 @@ public class TestMobCompactor {
Assert.assertTrue(hasFiles); Assert.assertTrue(hasFiles);
Path path = files[0].getPath(); Path path = files[0].getPath();
CacheConfig cacheConf = new CacheConfig(conf); CacheConfig cacheConf = new CacheConfig(conf);
StoreFile sf = new StoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf, StoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
sf.initReader(); sf.initReader();
HFile.Reader reader = sf.getReader().getHFileReader(); HFile.Reader reader = sf.getReader().getHFileReader();

View File

@ -18,9 +18,20 @@
*/ */
package org.apache.hadoop.hbase.mob.compactions; package org.apache.hadoop.hbase.mob.compactions;
import static org.junit.Assert.assertTrue;
import java.io.IOException; import java.io.IOException;
import java.text.ParseException; import java.text.ParseException;
import java.util.*; import java.util.ArrayList;
import java.util.Calendar;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.UUID;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.RejectedExecutionHandler;
@ -34,11 +45,16 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.Cell;
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.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.Type; import org.apache.hadoop.hbase.KeyValue.Type;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
import org.apache.hadoop.hbase.regionserver.*;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -49,13 +65,23 @@ import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType; import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType;
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition; import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition;
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition; import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.ScanInfo;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.regionserver.StoreScanner;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import static org.junit.Assert.assertTrue;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
@ -516,7 +542,7 @@ public class TestPartitionedMobCompactor {
for (CompactionDelPartition delPartition : request.getDelPartitions()) { for (CompactionDelPartition delPartition : request.getDelPartitions()) {
for (Path newDelPath : delPartition.listDelFiles()) { for (Path newDelPath : delPartition.listDelFiles()) {
StoreFile sf = StoreFile sf =
new StoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true); new HStoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true);
// pre-create reader of a del file to avoid race condition when opening the reader in // pre-create reader of a del file to avoid race condition when opening the reader in
// each partition. // each partition.
sf.initReader(); sf.initReader();
@ -851,7 +877,7 @@ public class TestPartitionedMobCompactor {
List<StoreFile> sfs = new ArrayList<>(); List<StoreFile> sfs = new ArrayList<>();
int size = 0; int size = 0;
for (Path path : paths) { for (Path path : paths) {
StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
sfs.add(sf); sfs.add(sf);
} }
List<KeyValueScanner> scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs, List<KeyValueScanner> scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs,

View File

@ -25,6 +25,8 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import com.google.common.collect.Sets;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;

View File

@ -17,19 +17,20 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy;
import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest; import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
import static org.junit.Assert.*;
public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy { public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy {

View File

@ -592,7 +592,7 @@ public class DataBlockEncodingTool {
Path path = new Path(hfilePath); Path path = new Path(hfilePath);
CacheConfig cacheConf = new CacheConfig(conf); CacheConfig cacheConf = new CacheConfig(conf);
FileSystem fs = FileSystem.get(conf); FileSystem fs = FileSystem.get(conf);
StoreFile hsf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); StoreFile hsf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
hsf.initReader(); hsf.initReader();
StoreFileReader reader = hsf.getReader(); StoreFileReader reader = hsf.getReader();
reader.loadFileInfo(); reader.loadFileInfo();

View File

@ -59,7 +59,7 @@ public class EncodedSeekPerformanceTest {
List<Cell> allKeyValues = new ArrayList<>(); List<Cell> allKeyValues = new ArrayList<>();
// read all of the key values // read all of the key values
StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(), StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
path, configuration, cacheConf, BloomType.NONE, true); path, configuration, cacheConf, BloomType.NONE, true);
storeFile.initReader(); storeFile.initReader();
StoreFileReader reader = storeFile.getReader(); StoreFileReader reader = storeFile.getReader();
@ -89,7 +89,7 @@ public class EncodedSeekPerformanceTest {
private void runTest(Path path, DataBlockEncoding blockEncoding, private void runTest(Path path, DataBlockEncoding blockEncoding,
List<Cell> seeks) throws IOException { List<Cell> seeks) throws IOException {
// read all of the key values // read all of the key values
StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(), StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
path, configuration, cacheConf, BloomType.NONE, true); path, configuration, cacheConf, BloomType.NONE, true);
storeFile.initReader(); storeFile.initReader();
long totalSize = 0; long totalSize = 0;

View File

@ -19,11 +19,14 @@
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import java.util.Map; import java.util.Map;
import java.util.OptionalLong;
import java.util.TreeMap; import java.util.TreeMap;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@ -31,7 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/** A mock used so our tests don't deal with actual StoreFiles */ /** A mock used so our tests don't deal with actual StoreFiles */
public class MockStoreFile extends StoreFile { public class MockStoreFile extends HStoreFile {
long length = 0; long length = 0;
boolean isRef = false; boolean isRef = false;
long ageInDisk; long ageInDisk;
@ -64,18 +67,13 @@ public class MockStoreFile extends StoreFile {
this.length = newLen; this.length = newLen;
} }
@Override
byte[] getFileSplitPoint(CellComparator comparator) throws IOException {
return this.splitPoint;
}
@Override @Override
public long getMaxSequenceId() { public long getMaxSequenceId() {
return sequenceid; return sequenceid;
} }
@Override @Override
public boolean isMajorCompaction() { public boolean isMajorCompactionResult() {
return isMajor; return isMajor;
} }
@ -110,14 +108,14 @@ public class MockStoreFile extends StoreFile {
this.entryCount = entryCount; this.entryCount = entryCount;
} }
public Long getMinimumTimestamp() { public OptionalLong getMinimumTimestamp() {
return (timeRangeTracker == null) ? return timeRangeTracker == null ? OptionalLong.empty()
null : timeRangeTracker.getMin(); : OptionalLong.of(timeRangeTracker.getMin());
} }
public Long getMaximumTimestamp() { public OptionalLong getMaximumTimestamp() {
return (timeRangeTracker == null) ? return timeRangeTracker == null ? OptionalLong.empty()
null : timeRangeTracker.getMax(); : OptionalLong.of(timeRangeTracker.getMax());
} }
@Override @Override
@ -184,6 +182,39 @@ public class MockStoreFile extends StoreFile {
public void close(boolean evictOnClose) throws IOException { public void close(boolean evictOnClose) throws IOException {
// no-op // no-op
} }
@Override
public Cell getLastKey() {
if (splitPoint != null) {
return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length + 1));
} else {
return null;
}
}
@Override
public Cell midkey() throws IOException {
if (splitPoint != null) {
return CellUtil.createCell(splitPoint);
} else {
return null;
}
}
@Override
public Cell getFirstKey() {
if (splitPoint != null) {
return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length - 1));
} else {
return null;
}
}
}; };
} }
@Override
public OptionalLong getBulkLoadTimestamp() {
// we always return false for isBulkLoadResult so we do not have a bulk load timestamp
return OptionalLong.empty();
}
} }

View File

@ -218,7 +218,7 @@ public class TestCacheOnWriteInSchema {
private void readStoreFile(Path path) throws IOException { private void readStoreFile(Path path) throws IOException {
CacheConfig cacheConf = store.getCacheConfig(); CacheConfig cacheConf = store.getCacheConfig();
BlockCache cache = cacheConf.getBlockCache(); BlockCache cache = cacheConf.getBlockCache();
StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true); StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
sf.initReader(); sf.initReader();
HFile.Reader reader = sf.getReader().getHFileReader(); HFile.Reader reader = sf.getReader().getHFileReader();
try { try {

View File

@ -23,6 +23,14 @@ import static org.junit.Assert.assertFalse;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -45,14 +53,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName; import org.junit.rules.TestName;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
/** /**
* Tests a race condition between archiving of compacted files in CompactedHFilesDischarger chore * Tests a race condition between archiving of compacted files in CompactedHFilesDischarger chore
* and HRegion.close(); * and HRegion.close();

View File

@ -30,6 +30,12 @@ import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@ -53,11 +59,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName; import org.junit.rules.TestName;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
/** /**
* Tests that archiving compacted files behaves correctly when encountering exceptions. * Tests that archiving compacted files behaves correctly when encountering exceptions.
*/ */

View File

@ -43,9 +43,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.io.hfile.RandomKeyValueUtil;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
@ -54,9 +51,12 @@ import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.io.hfile.RandomKeyValueUtil;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.BloomFilterUtil; import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
@ -200,7 +200,7 @@ public class TestCompoundBloomFilter {
private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs, private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
Path sfPath) throws IOException { Path sfPath) throws IOException {
StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt, true); StoreFile sf = new HStoreFile(fs, sfPath, conf, cacheConf, bt, true);
sf.initReader(); sf.initReader();
StoreFileReader r = sf.getReader(); StoreFileReader r = sf.getReader();
final boolean pread = true; // does not really matter final boolean pread = true; // does not really matter

View File

@ -17,7 +17,8 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.*; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException; import java.io.IOException;
import java.security.Key; import java.security.Key;
@ -36,8 +37,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter.Predicate; import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -49,8 +48,9 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.security.EncryptionUtil; import org.apache.hadoop.hbase.security.EncryptionUtil;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Rule; import org.junit.Rule;

View File

@ -17,7 +17,8 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.*; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.security.Key; import java.security.Key;
import java.util.ArrayList; import java.util.ArrayList;
@ -29,8 +30,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
@ -38,8 +37,9 @@ import org.apache.hadoop.hbase.io.crypto.Encryption;
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;

View File

@ -91,10 +91,10 @@ public class TestFSErrorsExposed {
.withOutputDir(hfilePath) .withOutputDir(hfilePath)
.withFileContext(meta) .withFileContext(meta)
.build(); .build();
TestStoreFile.writeStoreFile( TestHStoreFile.writeStoreFile(
writer, Bytes.toBytes("cf"), Bytes.toBytes("qual")); writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf, StoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
sf.initReader(); sf.initReader();
StoreFileReader reader = sf.getReader(); StoreFileReader reader = sf.getReader();
@ -141,10 +141,10 @@ public class TestFSErrorsExposed {
.withOutputDir(hfilePath) .withOutputDir(hfilePath)
.withFileContext(meta) .withFileContext(meta)
.build(); .build();
TestStoreFile.writeStoreFile( TestHStoreFile.writeStoreFile(
writer, Bytes.toBytes("cf"), Bytes.toBytes("qual")); writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf, StoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles( List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(

View File

@ -18,11 +18,26 @@
*/ */
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.security.Key;
import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
import java.util.NavigableSet;
import java.util.concurrent.ConcurrentSkipListSet;
import javax.crypto.spec.SecretKeySpec;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ArrayBackedTag;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
@ -35,7 +50,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.ArrayBackedTag;
import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagType;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
@ -61,19 +75,6 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName; import org.junit.rules.TestName;
import org.mockito.Mockito; import org.mockito.Mockito;
import javax.crypto.spec.SecretKeySpec;
import java.io.IOException;
import java.security.Key;
import java.security.SecureRandom;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
import java.util.NavigableSet;
import java.util.concurrent.ConcurrentSkipListSet;
@Category(MediumTests.class) @Category(MediumTests.class)
public class TestHMobStore { public class TestHMobStore {
public static final Log LOG = LogFactory.getLog(TestHMobStore.class); public static final Log LOG = LogFactory.getLog(TestHMobStore.class);

View File

@ -116,7 +116,6 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterAllFilter;
import org.apache.hadoop.hbase.filter.FilterBase; import org.apache.hadoop.hbase.filter.FilterBase;
import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.NullComparator; import org.apache.hadoop.hbase.filter.NullComparator;

View File

@ -35,6 +35,8 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import com.google.common.collect.Lists;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -64,6 +66,10 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.executor.ExecutorService; import org.apache.hadoop.hbase.executor.ExecutorService;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl;
import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
@ -75,10 +81,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescript
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl;
import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -98,8 +100,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName; import org.junit.rules.TestName;
import com.google.common.collect.Lists;
/** /**
* Tests of HRegion methods for replaying flush, compaction, region open, etc events for secondary * Tests of HRegion methods for replaying flush, compaction, region open, etc events for secondary
* region replicas * region replicas
@ -1350,7 +1350,7 @@ public class TestHRegionReplayEvents {
} }
@Test @Test
public void testRefreshStoreFiles() throws IOException { public void testRefresStoreFiles() throws IOException {
assertEquals(0, primaryRegion.getStoreFileList(families).size()); assertEquals(0, primaryRegion.getStoreFileList(families).size());
assertEquals(0, secondaryRegion.getStoreFileList(families).size()); assertEquals(0, secondaryRegion.getStoreFileList(families).size());

View File

@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
import static org.hamcrest.core.Is.is; import static org.hamcrest.core.Is.is;
import static org.junit.Assert.assertThat; import static org.junit.Assert.assertThat;
import com.google.common.collect.Lists;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException; import java.io.InterruptedIOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -63,11 +65,11 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext; import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -81,8 +83,6 @@ import org.junit.runner.RunWith;
import org.junit.runners.Parameterized; import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters; import org.junit.runners.Parameterized.Parameters;
import com.google.common.collect.Lists;
/** /**
* Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of * Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of
* the region server's bullkLoad functionality. * the region server's bullkLoad functionality.

View File

@ -26,6 +26,7 @@ import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.OptionalLong;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -78,8 +79,8 @@ import static org.mockito.Mockito.when;
* Test HStoreFile * Test HStoreFile
*/ */
@Category({RegionServerTests.class, SmallTests.class}) @Category({RegionServerTests.class, SmallTests.class})
public class TestStoreFile extends HBaseTestCase { public class TestHStoreFile extends HBaseTestCase {
private static final Log LOG = LogFactory.getLog(TestStoreFile.class); private static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration()); private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
private static String ROOT_DIR = TEST_UTIL.getDataTestDir("TestStoreFile").toString(); private static String ROOT_DIR = TEST_UTIL.getDataTestDir("TestStoreFile").toString();
@ -117,7 +118,7 @@ public class TestStoreFile extends HBaseTestCase {
writeStoreFile(writer); writeStoreFile(writer);
Path sfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath()); Path sfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
StoreFile sf = new StoreFile(this.fs, sfPath, conf, cacheConf, BloomType.NONE, true); HStoreFile sf = new HStoreFile(this.fs, sfPath, conf, cacheConf, BloomType.NONE, true);
checkHalfHFile(regionFs, sf); checkHalfHFile(regionFs, sf);
} }
@ -169,7 +170,7 @@ public class TestStoreFile extends HBaseTestCase {
writeStoreFile(writer); writeStoreFile(writer);
Path hsfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath()); Path hsfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
StoreFile hsf = new StoreFile(this.fs, hsfPath, conf, cacheConf, BloomType.NONE, true); HStoreFile hsf = new HStoreFile(this.fs, hsfPath, conf, cacheConf, BloomType.NONE, true);
hsf.initReader(); hsf.initReader();
StoreFileReader reader = hsf.getReader(); StoreFileReader reader = hsf.getReader();
// Split on a row, not in middle of row. Midkey returned by reader // Split on a row, not in middle of row. Midkey returned by reader
@ -184,7 +185,7 @@ public class TestStoreFile extends HBaseTestCase {
// Make a reference // Make a reference
HRegionInfo splitHri = new HRegionInfo(hri.getTable(), null, midRow); HRegionInfo splitHri = new HRegionInfo(hri.getTable(), null, midRow);
Path refPath = splitStoreFile(regionFs, splitHri, TEST_FAMILY, hsf, midRow, true); Path refPath = splitStoreFile(regionFs, splitHri, TEST_FAMILY, hsf, midRow, true);
StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf, BloomType.NONE, true); HStoreFile refHsf = new HStoreFile(this.fs, refPath, conf, cacheConf, BloomType.NONE, true);
refHsf.initReader(); refHsf.initReader();
// Now confirm that I can read from the reference and that it only gets // Now confirm that I can read from the reference and that it only gets
// keys from top half of the file. // keys from top half of the file.
@ -242,8 +243,8 @@ public class TestStoreFile extends HBaseTestCase {
// Try to open store file from link // Try to open store file from link
StoreFileInfo storeFileInfo = new StoreFileInfo(testConf, this.fs, linkFilePath); StoreFileInfo storeFileInfo = new StoreFileInfo(testConf, this.fs, linkFilePath);
StoreFile hsf = HStoreFile hsf =
new StoreFile(this.fs, storeFileInfo, testConf, cacheConf, BloomType.NONE, true); new HStoreFile(this.fs, storeFileInfo, testConf, cacheConf, BloomType.NONE, true);
assertTrue(storeFileInfo.isLink()); assertTrue(storeFileInfo.isLink());
hsf.initReader(); hsf.initReader();
@ -296,7 +297,7 @@ public class TestStoreFile extends HBaseTestCase {
// <root>/clone/splitB/<cf>/<reftohfilelink> // <root>/clone/splitB/<cf>/<reftohfilelink>
HRegionInfo splitHriA = new HRegionInfo(hri.getTable(), null, SPLITKEY); HRegionInfo splitHriA = new HRegionInfo(hri.getTable(), null, SPLITKEY);
HRegionInfo splitHriB = new HRegionInfo(hri.getTable(), SPLITKEY, null); HRegionInfo splitHriB = new HRegionInfo(hri.getTable(), SPLITKEY, null);
StoreFile f = new StoreFile(fs, linkFilePath, testConf, cacheConf, BloomType.NONE, true); HStoreFile f = new HStoreFile(fs, linkFilePath, testConf, cacheConf, BloomType.NONE, true);
f.initReader(); f.initReader();
Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top
Path pathB = splitStoreFile(cloneRegionFs, splitHriB, TEST_FAMILY, f, SPLITKEY, false);// bottom Path pathB = splitStoreFile(cloneRegionFs, splitHriB, TEST_FAMILY, f, SPLITKEY, false);// bottom
@ -308,7 +309,7 @@ public class TestStoreFile extends HBaseTestCase {
// reference to a hfile link. This code in StoreFile that handles this case. // reference to a hfile link. This code in StoreFile that handles this case.
// Try to open store file from link // Try to open store file from link
StoreFile hsfA = new StoreFile(this.fs, pathA, testConf, cacheConf, BloomType.NONE, true); HStoreFile hsfA = new HStoreFile(this.fs, pathA, testConf, cacheConf, BloomType.NONE, true);
hsfA.initReader(); hsfA.initReader();
// Now confirm that I can read from the ref to link // Now confirm that I can read from the ref to link
@ -321,7 +322,7 @@ public class TestStoreFile extends HBaseTestCase {
assertTrue(count > 0); // read some rows here assertTrue(count > 0); // read some rows here
// Try to open store file from link // Try to open store file from link
StoreFile hsfB = new StoreFile(this.fs, pathB, testConf, cacheConf, BloomType.NONE, true); HStoreFile hsfB = new HStoreFile(this.fs, pathB, testConf, cacheConf, BloomType.NONE, true);
hsfB.initReader(); hsfB.initReader();
// Now confirm that I can read from the ref to link // Now confirm that I can read from the ref to link
@ -338,7 +339,7 @@ public class TestStoreFile extends HBaseTestCase {
assertEquals((LAST_CHAR - FIRST_CHAR + 1) * (LAST_CHAR - FIRST_CHAR + 1), count); assertEquals((LAST_CHAR - FIRST_CHAR + 1) * (LAST_CHAR - FIRST_CHAR + 1), count);
} }
private void checkHalfHFile(final HRegionFileSystem regionFs, final StoreFile f) private void checkHalfHFile(final HRegionFileSystem regionFs, final HStoreFile f)
throws IOException { throws IOException {
f.initReader(); f.initReader();
Cell midkey = f.getReader().midkey(); Cell midkey = f.getReader().midkey();
@ -353,10 +354,10 @@ public class TestStoreFile extends HBaseTestCase {
midRow, null); midRow, null);
Path bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, midRow, false); Path bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, midRow, false);
// Make readers on top and bottom. // Make readers on top and bottom.
StoreFile topF = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true); HStoreFile topF = new HStoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true);
topF.initReader(); topF.initReader();
StoreFileReader top = topF.getReader(); StoreFileReader top = topF.getReader();
StoreFile bottomF = new StoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true); HStoreFile bottomF = new HStoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true);
bottomF.initReader(); bottomF.initReader();
StoreFileReader bottom = bottomF.getReader(); StoreFileReader bottom = bottomF.getReader();
ByteBuffer previous = null; ByteBuffer previous = null;
@ -416,7 +417,7 @@ public class TestStoreFile extends HBaseTestCase {
assertNull(bottomPath); assertNull(bottomPath);
topF = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true); topF = new HStoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true);
topF.initReader(); topF.initReader();
top = topF.getReader(); top = topF.getReader();
// Now read from the top. // Now read from the top.
@ -456,7 +457,7 @@ public class TestStoreFile extends HBaseTestCase {
bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, badmidkey, false); bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, badmidkey, false);
assertNull(topPath); assertNull(topPath);
bottomF = new StoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true); bottomF = new HStoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true);
bottomF.initReader(); bottomF.initReader();
bottom = bottomF.getReader(); bottom = bottomF.getReader();
first = true; first = true;
@ -754,7 +755,7 @@ public class TestStoreFile extends HBaseTestCase {
@Test @Test
public void testSeqIdComparator() { public void testSeqIdComparator() {
assertOrdering(StoreFile.Comparators.SEQ_ID, mockStoreFile(true, 100, 1000, -1, "/foo/123"), assertOrdering(StoreFileComparators.SEQ_ID, mockStoreFile(true, 100, 1000, -1, "/foo/123"),
mockStoreFile(true, 100, 1000, -1, "/foo/124"), mockStoreFile(true, 100, 1000, -1, "/foo/124"),
mockStoreFile(true, 99, 1000, -1, "/foo/126"), mockStoreFile(true, 99, 1000, -1, "/foo/126"),
mockStoreFile(true, 98, 2000, -1, "/foo/126"), mockStoreFile(false, 3453, -1, 1, "/foo/1"), mockStoreFile(true, 98, 2000, -1, "/foo/126"), mockStoreFile(false, 3453, -1, 1, "/foo/1"),
@ -766,8 +767,8 @@ public class TestStoreFile extends HBaseTestCase {
* Assert that the given comparator orders the given storefiles in the * Assert that the given comparator orders the given storefiles in the
* same way that they're passed. * same way that they're passed.
*/ */
private void assertOrdering(Comparator<StoreFile> comparator, StoreFile ... sfs) { private void assertOrdering(Comparator<? super HStoreFile> comparator, HStoreFile ... sfs) {
ArrayList<StoreFile> sorted = Lists.newArrayList(sfs); ArrayList<HStoreFile> sorted = Lists.newArrayList(sfs);
Collections.shuffle(sorted); Collections.shuffle(sorted);
Collections.sort(sorted, comparator); Collections.sort(sorted, comparator);
LOG.debug("sfs: " + Joiner.on(",").join(sfs)); LOG.debug("sfs: " + Joiner.on(",").join(sfs));
@ -778,19 +779,19 @@ public class TestStoreFile extends HBaseTestCase {
/** /**
* Create a mock StoreFile with the given attributes. * Create a mock StoreFile with the given attributes.
*/ */
private StoreFile mockStoreFile(boolean bulkLoad, private HStoreFile mockStoreFile(boolean bulkLoad,
long size, long size,
long bulkTimestamp, long bulkTimestamp,
long seqId, long seqId,
String path) { String path) {
StoreFile mock = Mockito.mock(StoreFile.class); HStoreFile mock = Mockito.mock(HStoreFile.class);
StoreFileReader reader = Mockito.mock(StoreFileReader.class); StoreFileReader reader = Mockito.mock(StoreFileReader.class);
Mockito.doReturn(size).when(reader).length(); Mockito.doReturn(size).when(reader).length();
Mockito.doReturn(reader).when(mock).getReader(); Mockito.doReturn(reader).when(mock).getReader();
Mockito.doReturn(bulkLoad).when(mock).isBulkLoadResult(); Mockito.doReturn(bulkLoad).when(mock).isBulkLoadResult();
Mockito.doReturn(bulkTimestamp).when(mock).getBulkLoadTimestamp(); Mockito.doReturn(OptionalLong.of(bulkTimestamp)).when(mock).getBulkLoadTimestamp();
Mockito.doReturn(seqId).when(mock).getMaxSequenceId(); Mockito.doReturn(seqId).when(mock).getMaxSequenceId();
Mockito.doReturn(new Path(path)).when(mock).getPath(); Mockito.doReturn(new Path(path)).when(mock).getPath();
String name = "mock storefile, bulkLoad=" + bulkLoad + String name = "mock storefile, bulkLoad=" + bulkLoad +
@ -855,7 +856,7 @@ public class TestStoreFile extends HBaseTestCase {
writer.appendMetadata(0, false); writer.appendMetadata(0, false);
writer.close(); writer.close();
StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf, HStoreFile hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
Store store = mock(Store.class); Store store = mock(Store.class);
HColumnDescriptor hcd = mock(HColumnDescriptor.class); HColumnDescriptor hcd = mock(HColumnDescriptor.class);
@ -913,7 +914,7 @@ public class TestStoreFile extends HBaseTestCase {
CacheConfig cacheConf = new CacheConfig(conf); CacheConfig cacheConf = new CacheConfig(conf);
Path pathCowOff = new Path(baseDir, "123456789"); Path pathCowOff = new Path(baseDir, "123456789");
StoreFileWriter writer = writeStoreFile(conf, cacheConf, pathCowOff, 3); StoreFileWriter writer = writeStoreFile(conf, cacheConf, pathCowOff, 3);
StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf, HStoreFile hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
LOG.debug(hsf.getPath().toString()); LOG.debug(hsf.getPath().toString());
@ -936,7 +937,7 @@ public class TestStoreFile extends HBaseTestCase {
cacheConf = new CacheConfig(conf); cacheConf = new CacheConfig(conf);
Path pathCowOn = new Path(baseDir, "123456788"); Path pathCowOn = new Path(baseDir, "123456788");
writer = writeStoreFile(conf, cacheConf, pathCowOn, 3); writer = writeStoreFile(conf, cacheConf, pathCowOn, 3);
hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf, hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
// Read this file, we should see 3 hits // Read this file, we should see 3 hits
@ -953,13 +954,13 @@ public class TestStoreFile extends HBaseTestCase {
reader.close(cacheConf.shouldEvictOnClose()); reader.close(cacheConf.shouldEvictOnClose());
// Let's read back the two files to ensure the blocks exactly match // Let's read back the two files to ensure the blocks exactly match
hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true); hsf = new HStoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
hsf.initReader(); hsf.initReader();
StoreFileReader readerOne = hsf.getReader(); StoreFileReader readerOne = hsf.getReader();
readerOne.loadFileInfo(); readerOne.loadFileInfo();
StoreFileScanner scannerOne = getStoreFileScanner(readerOne, true, true); StoreFileScanner scannerOne = getStoreFileScanner(readerOne, true, true);
scannerOne.seek(KeyValue.LOWESTKEY); scannerOne.seek(KeyValue.LOWESTKEY);
hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true); hsf = new HStoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
hsf.initReader(); hsf.initReader();
StoreFileReader readerTwo = hsf.getReader(); StoreFileReader readerTwo = hsf.getReader();
readerTwo.loadFileInfo(); readerTwo.loadFileInfo();
@ -992,7 +993,7 @@ public class TestStoreFile extends HBaseTestCase {
// Let's close the first file with evict on close turned on // Let's close the first file with evict on close turned on
conf.setBoolean("hbase.rs.evictblocksonclose", true); conf.setBoolean("hbase.rs.evictblocksonclose", true);
cacheConf = new CacheConfig(conf); cacheConf = new CacheConfig(conf);
hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true); hsf = new HStoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
hsf.initReader(); hsf.initReader();
reader = hsf.getReader(); reader = hsf.getReader();
reader.close(cacheConf.shouldEvictOnClose()); reader.close(cacheConf.shouldEvictOnClose());
@ -1006,7 +1007,7 @@ public class TestStoreFile extends HBaseTestCase {
// Let's close the second file with evict on close turned off // Let's close the second file with evict on close turned off
conf.setBoolean("hbase.rs.evictblocksonclose", false); conf.setBoolean("hbase.rs.evictblocksonclose", false);
cacheConf = new CacheConfig(conf); cacheConf = new CacheConfig(conf);
hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true); hsf = new HStoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
hsf.initReader(); hsf.initReader();
reader = hsf.getReader(); reader = hsf.getReader();
reader.close(cacheConf.shouldEvictOnClose()); reader.close(cacheConf.shouldEvictOnClose());
@ -1018,7 +1019,7 @@ public class TestStoreFile extends HBaseTestCase {
} }
private Path splitStoreFile(final HRegionFileSystem regionFs, final HRegionInfo hri, private Path splitStoreFile(final HRegionFileSystem regionFs, final HRegionInfo hri,
final String family, final StoreFile sf, final byte[] splitKey, boolean isTopRef) final String family, final HStoreFile sf, final byte[] splitKey, boolean isTopRef)
throws IOException { throws IOException {
FileSystem fs = regionFs.getFileSystem(); FileSystem fs = regionFs.getFileSystem();
Path path = regionFs.splitStoreFile(hri, family, sf, splitKey, isTopRef, null); Path path = regionFs.splitStoreFile(hri, family, sf, splitKey, isTopRef, null);
@ -1093,8 +1094,8 @@ public class TestStoreFile extends HBaseTestCase {
.build(); .build();
writer.close(); writer.close();
StoreFile storeFile = HStoreFile storeFile =
new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true); new HStoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true);
storeFile.initReader(); storeFile.initReader();
StoreFileReader reader = storeFile.getReader(); StoreFileReader reader = storeFile.getReader();
@ -1103,4 +1104,3 @@ public class TestStoreFile extends HBaseTestCase {
assertEquals(dataBlockEncoderAlgo.getNameInBytes(), value); assertEquals(dataBlockEncoderAlgo.getNameInBytes(), value);
} }
} }

View File

@ -37,7 +37,14 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.*; import org.apache.hadoop.hbase.Cell;
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.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -293,7 +300,7 @@ public class TestMobStoreCompaction {
if (fs.exists(mobDirPath)) { if (fs.exists(mobDirPath)) {
FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath); FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath);
for (FileStatus file : files) { for (FileStatus file : files) {
StoreFile sf = new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true); StoreFile sf = new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true);
sf.initReader(); sf.initReader();
Map<byte[], byte[]> fileInfo = sf.getReader().loadFileInfo(); Map<byte[], byte[]> fileInfo = sf.getReader().loadFileInfo();
byte[] count = fileInfo.get(StoreFile.MOB_CELLS_COUNT); byte[] count = fileInfo.get(StoreFile.MOB_CELLS_COUNT);
@ -408,14 +415,14 @@ public class TestMobStoreCompaction {
int size = 0; int size = 0;
if (fs.exists(mobDirPath)) { if (fs.exists(mobDirPath)) {
for (FileStatus f : fs.listStatus(mobDirPath)) { for (FileStatus f : fs.listStatus(mobDirPath)) {
StoreFile sf = new StoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true); StoreFile sf = new HStoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true);
sfs.add(sf); sfs.add(sf);
if (StoreFileInfo.isDelFile(sf.getPath())) { if (StoreFileInfo.isDelFile(sf.getPath())) {
numDelfiles++; numDelfiles++;
} }
} }
List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, false, false, List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, false, false,
HConstants.LATEST_TIMESTAMP); HConstants.LATEST_TIMESTAMP);
Scan scan = new Scan(); Scan scan = new Scan();
scan.setMaxVersions(hcd.getMaxVersions()); scan.setMaxVersions(hcd.getMaxVersions());

View File

@ -225,7 +225,7 @@ public class TestRegionReplicas {
} }
@Test(timeout = 300000) @Test(timeout = 300000)
public void testRefreshStoreFiles() throws Exception { public void testRefresStoreFiles() throws Exception {
// enable store file refreshing // enable store file refreshing
final int refreshPeriod = 2000; // 2 sec final int refreshPeriod = 2000; // 2 sec
HTU.getConfiguration().setInt("hbase.hstore.compactionThreshold", 100); HTU.getConfiguration().setInt("hbase.hstore.compactionThreshold", 100);

View File

@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import com.google.common.collect.Lists;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
@ -66,8 +68,6 @@ import org.junit.Test;
import org.junit.experimental.categories.Category; import org.junit.experimental.categories.Category;
import org.junit.rules.TestName; import org.junit.rules.TestName;
import com.google.common.collect.Lists;
/** /**
* Test cases against ReversibleKeyValueScanner * Test cases against ReversibleKeyValueScanner
*/ */
@ -113,7 +113,7 @@ public class TestReversibleScanners {
.withFileContext(hFileContext).build(); .withFileContext(hFileContext).build();
writeStoreFile(writer); writeStoreFile(writer);
StoreFile sf = new StoreFile(fs, writer.getPath(), TEST_UTIL.getConfiguration(), cacheConf, StoreFile sf = new HStoreFile(fs, writer.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
List<StoreFileScanner> scanners = StoreFileScanner List<StoreFileScanner> scanners = StoreFileScanner
@ -167,10 +167,10 @@ public class TestReversibleScanners {
writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1, writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1,
writer2 }); writer2 });
StoreFile sf1 = new StoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf, StoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
StoreFile sf2 = new StoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf, StoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
/** /**
* Test without MVCC * Test without MVCC
@ -257,10 +257,10 @@ public class TestReversibleScanners {
writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1, writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1,
writer2 }); writer2 });
StoreFile sf1 = new StoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf, StoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
StoreFile sf2 = new StoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf, StoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
BloomType.NONE, true); BloomType.NONE, true);
ScanType scanType = ScanType.USER_SCAN; ScanType scanType = ScanType.USER_SCAN;

View File

@ -72,12 +72,11 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException; import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterRpcServices; import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.master.NoSuchProcedureException;
import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.RegionState.State; import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager; import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.master.NoSuchProcedureException;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;

View File

@ -27,7 +27,6 @@ import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times; import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verify;
import com.google.common.collect.Lists;
import java.io.IOException; import java.io.IOException;
import java.lang.ref.SoftReference; import java.lang.ref.SoftReference;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
@ -45,7 +44,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -100,6 +99,8 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName; import org.junit.rules.TestName;
import org.mockito.Mockito; import org.mockito.Mockito;
import com.google.common.collect.Lists;
/** /**
* Test class for the Store * Test class for the Store
*/ */

View File

@ -30,6 +30,7 @@ import static org.mockito.Mockito.when;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.OptionalLong;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -108,6 +109,7 @@ public class TestStripeStoreEngine {
.thenReturn(StripeStoreFileManager.INVALID_KEY); .thenReturn(StripeStoreFileManager.INVALID_KEY);
when(sf.getReader()).thenReturn(mock(StoreFileReader.class)); when(sf.getReader()).thenReturn(mock(StoreFileReader.class));
when(sf.getPath()).thenReturn(new Path("moo")); when(sf.getPath()).thenReturn(new Path("moo"));
when(sf.getBulkLoadTimestamp()).thenReturn(OptionalLong.empty());
return sf; return sf;
} }

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY; import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNull; import static org.junit.Assert.assertNull;
@ -179,20 +180,20 @@ public class TestStripeStoreFileManager {
// If there are no stripes, should pick midpoint from the biggest file in L0. // If there are no stripes, should pick midpoint from the biggest file in L0.
MockStoreFile sf5 = createFile(5, 0); MockStoreFile sf5 = createFile(5, 0);
sf5.splitPoint = new byte[1]; sf5.splitPoint = new byte[] { 1 };
manager.insertNewFiles(al(sf5)); manager.insertNewFiles(al(sf5));
manager.insertNewFiles(al(createFile(1, 0))); manager.insertNewFiles(al(createFile(1, 0)));
assertEquals(sf5.splitPoint, manager.getSplitPoint()); assertArrayEquals(sf5.splitPoint, manager.getSplitPoint());
// Same if there's one stripe but the biggest file is still in L0. // Same if there's one stripe but the biggest file is still in L0.
manager.addCompactionResults(al(), al(createFile(2, 0, OPEN_KEY, OPEN_KEY))); manager.addCompactionResults(al(), al(createFile(2, 0, OPEN_KEY, OPEN_KEY)));
assertEquals(sf5.splitPoint, manager.getSplitPoint()); assertArrayEquals(sf5.splitPoint, manager.getSplitPoint());
// If the biggest file is in the stripe, should get from it. // If the biggest file is in the stripe, should get from it.
MockStoreFile sf6 = createFile(6, 0, OPEN_KEY, OPEN_KEY); MockStoreFile sf6 = createFile(6, 0, OPEN_KEY, OPEN_KEY);
sf6.splitPoint = new byte[1]; sf6.splitPoint = new byte[] { 2 };
manager.addCompactionResults(al(), al(sf6)); manager.addCompactionResults(al(), al(sf6));
assertEquals(sf6.splitPoint, manager.getSplitPoint()); assertArrayEquals(sf6.splitPoint, manager.getSplitPoint());
} }
@Test @Test
@ -572,7 +573,7 @@ public class TestStripeStoreFileManager {
private static MockStoreFile createFile( private static MockStoreFile createFile(
long size, long seqNum, byte[] startKey, byte[] endKey) throws Exception { long size, long seqNum, byte[] startKey, byte[] endKey) throws Exception {
FileSystem fs = TEST_UTIL.getTestFileSystem(); FileSystem fs = TEST_UTIL.getTestFileSystem();
Path testFilePath = StoreFile.getUniqueFile(fs, CFDIR); Path testFilePath = StoreFileWriter.getUniqueFile(fs, CFDIR);
fs.create(testFilePath).close(); fs.create(testFilePath).close();
MockStoreFile sf = new MockStoreFile(TEST_UTIL, testFilePath, size, 0, false, seqNum); MockStoreFile sf = new MockStoreFile(TEST_UTIL, testFilePath, size, 0, false, seqNum);
if (startKey != null) { if (startKey != null) {

View File

@ -18,20 +18,21 @@
package org.apache.hadoop.hbase.regionserver.compactions; package org.apache.hadoop.hbase.regionserver.compactions;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import com.google.common.base.Objects;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.Random; import java.util.Random;
import com.google.common.base.Objects;
import org.apache.commons.lang.RandomStringUtils; import org.apache.commons.lang.RandomStringUtils;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileReader; import org.apache.hadoop.hbase.regionserver.StoreFileReader;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
/** /**
* Base class of objects that can create mock store files with a given size. * Base class of objects that can create mock store files with a given size.
*/ */

View File

@ -18,29 +18,28 @@
package org.apache.hadoop.hbase.regionserver.compactions; package org.apache.hadoop.hbase.regionserver.compactions;
import org.apache.commons.logging.Log; import static org.mockito.Mockito.mock;
import org.apache.commons.logging.LogFactory; import static org.mockito.Mockito.when;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import static org.mockito.Mockito.mock; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import static org.mockito.Mockito.when; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@Category({RegionServerTests.class, MediumTests.class}) @Category({RegionServerTests.class, MediumTests.class})
@RunWith(Parameterized.class) @RunWith(Parameterized.class)

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger; import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.RegionServerServices;
@ -203,13 +204,13 @@ public class TestCompactedHFilesDischarger {
int usedReaderCount = 0; int usedReaderCount = 0;
int unusedReaderCount = 0; int unusedReaderCount = 0;
for (StoreFile file : storefiles) { for (StoreFile file : storefiles) {
if (file.getRefCount() == 3) { if (((HStoreFile) file).getRefCount() == 3) {
usedReaderCount++; usedReaderCount++;
} }
} }
compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
for(StoreFile file : compactedfiles) { for(StoreFile file : compactedfiles) {
assertEquals("Refcount should be 3", 0, file.getRefCount()); assertEquals("Refcount should be 3", 0, ((HStoreFile) file).getRefCount());
unusedReaderCount++; unusedReaderCount++;
} }
// Though there are files we are not using them for reads // Though there are files we are not using them for reads
@ -274,14 +275,14 @@ public class TestCompactedHFilesDischarger {
int usedReaderCount = 0; int usedReaderCount = 0;
int unusedReaderCount = 0; int unusedReaderCount = 0;
for (StoreFile file : storefiles) { for (StoreFile file : storefiles) {
if (file.getRefCount() == 0) { if (((HStoreFile) file).getRefCount() == 0) {
unusedReaderCount++; unusedReaderCount++;
} }
} }
compactedfiles = compactedfiles =
((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
for(StoreFile file : compactedfiles) { for(StoreFile file : compactedfiles) {
assertEquals("Refcount should be 3", 3, file.getRefCount()); assertEquals("Refcount should be 3", 3, ((HStoreFile) file).getRefCount());
usedReaderCount++; usedReaderCount++;
} }
// The newly compacted file will not be used by any scanner // The newly compacted file will not be used by any scanner
@ -307,13 +308,13 @@ public class TestCompactedHFilesDischarger {
usedReaderCount = 0; usedReaderCount = 0;
unusedReaderCount = 0; unusedReaderCount = 0;
for (StoreFile file : storefiles) { for (StoreFile file : storefiles) {
if (file.getRefCount() == 3) { if (((HStoreFile) file).getRefCount() == 3) {
usedReaderCount++; usedReaderCount++;
} }
} }
compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
for(StoreFile file : compactedfiles) { for(StoreFile file : compactedfiles) {
assertEquals("Refcount should be 0", 0, file.getRefCount()); assertEquals("Refcount should be 0", 0, ((HStoreFile) file).getRefCount());
unusedReaderCount++; unusedReaderCount++;
} }
// Though there are files we are not using them for reads // Though there are files we are not using them for reads

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreUtils;
import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner; import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner;
import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture; import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
@ -103,7 +104,7 @@ public class TestDateTieredCompactor {
when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(), when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers); anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
when(store.getComparator()).thenReturn(CellComparator.COMPARATOR); when(store.getComparator()).thenReturn(CellComparator.COMPARATOR);
long maxSequenceId = StoreFile.getMaxSequenceIdInList(storefiles); long maxSequenceId = StoreUtils.getMaxSequenceIdInList(storefiles);
when(store.getMaxSequenceId()).thenReturn(maxSequenceId); when(store.getMaxSequenceId()).thenReturn(maxSequenceId);
return new DateTieredCompactor(conf, store) { return new DateTieredCompactor(conf, store) {

View File

@ -36,12 +36,16 @@ import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.OptionalLong;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -84,9 +88,6 @@ import org.junit.runners.Parameterized.Parameter;
import org.junit.runners.Parameterized.Parameters; import org.junit.runners.Parameterized.Parameters;
import org.mockito.ArgumentMatcher; import org.mockito.ArgumentMatcher;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
@Category({RegionServerTests.class, SmallTests.class}) @Category({RegionServerTests.class, SmallTests.class})
public class TestStripeCompactionPolicy { public class TestStripeCompactionPolicy {
@ -753,6 +754,7 @@ public class TestStripeCompactionPolicy {
when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyLong(), when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyLong(),
anyBoolean())).thenReturn(mock(StoreFileScanner.class)); anyBoolean())).thenReturn(mock(StoreFileScanner.class));
when(sf.getReader()).thenReturn(r); when(sf.getReader()).thenReturn(r);
when(sf.getBulkLoadTimestamp()).thenReturn(OptionalLong.empty());
return sf; return sf;
} }

View File

@ -27,6 +27,8 @@ import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import com.google.protobuf.ByteString;
import java.io.IOException; import java.io.IOException;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
import java.util.ArrayList; import java.util.ArrayList;
@ -72,8 +74,6 @@ import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.TestName; import org.junit.rules.TestName;
import com.google.protobuf.ByteString;
/** /**
* Base test class for visibility labels basic features * Base test class for visibility labels basic features
*/ */

View File

@ -18,6 +18,11 @@
*/ */
package org.apache.hadoop.hbase.util; package org.apache.hadoop.hbase.util;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.fail;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -40,11 +45,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
import java.io.IOException;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.fail;
/** /**
* Utility class for HFile-related testing. * Utility class for HFile-related testing.
*/ */
@ -130,8 +130,7 @@ public class HFileTestUtil {
writer.append(kv); writer.append(kv);
} }
} finally { } finally {
writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis()));
Bytes.toBytes(System.currentTimeMillis()));
writer.close(); writer.close();
} }
} }