diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java index 2792ea2a380..83d5506e1db 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/MergeTableRegionsProcedure.java @@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; import org.apache.hadoop.hbase.procedure2.ProcedureMetrics; 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.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; @@ -605,11 +605,8 @@ public class MergeTableRegionsProcedure final CacheConfig cacheConf = new CacheConfig(conf, hcd); for (StoreFileInfo storeFileInfo: storeFiles) { // Create reference file(s) of the region in mergedDir - regionFs.mergeStoreFile( - mergedRegion, - family, - new StoreFile( - mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()), + regionFs.mergeStoreFile(mergedRegion, family, new HStoreFile(mfs.getFileSystem(), + storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true), mergedDir); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index a663608220e..2e2aa5df01d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.master.assignment; +import com.google.common.annotations.VisibleForTesting; + import java.io.IOException; import java.io.InputStream; 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.regionserver.HRegionFileSystem; 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.StoreFileInfo; 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.Threads; -import com.google.common.annotations.VisibleForTesting; - /** * The procedure to split a region in a table. * Takes lock on the parent region. @@ -525,11 +526,9 @@ public class SplitTableRegionProcedure if (storeFiles != null && storeFiles.size() > 0) { final CacheConfig cacheConf = new CacheConfig(conf, hcd); for (StoreFileInfo storeFileInfo: storeFiles) { - StoreFileSplitter sfs = new StoreFileSplitter( - regionFs, - family.getBytes(), - new StoreFile( - mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType())); + StoreFileSplitter sfs = + new StoreFileSplitter(regionFs, family.getBytes(), new HStoreFile(mfs.getFileSystem(), + storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true)); futures.add(threadPool.submit(sfs)); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java index 4b96bc6cf59..fd4b091a4fc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java @@ -28,7 +28,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Random; -import com.google.common.collect.Lists; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; 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.EnvironmentEdgeManager; +import com.google.common.collect.Lists; + /** *

This is a best effort load balancer. Given a Cost function F(C) => x It will * randomly try and mutate the cluster to Cprime. If F(Cprime) < F(C) then the diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java index 90d1f2d6a5a..b7ebee35b4b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/CachedMobFile.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile; /** @@ -46,7 +47,7 @@ public class CachedMobFile extends MobFile implements Comparable CacheConfig cacheConf) throws IOException { // 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. - 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); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java index 73355e8058e..7e3e36f09f8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.CacheConfig; 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.StoreFileScanner; @@ -146,7 +147,7 @@ public class MobFile { throws IOException { // 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. - 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); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java index 06c50012001..a869b7acdc9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java @@ -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.regionserver.BloomType; 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.StoreFileWriter; import org.apache.hadoop.hbase.util.Bytes; @@ -334,7 +335,7 @@ public final class MobUtils { LOG.debug(fileName + " is an expired file"); } 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) { LOG.error("Cannot parse the fileName " + fileName, e); @@ -722,7 +723,7 @@ public final class MobUtils { CacheConfig cacheConfig, boolean primaryReplica) throws IOException { StoreFile storeFile = null; try { - storeFile = new StoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica); + storeFile = new HStoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica); storeFile.initReader(); } catch (IOException e) { LOG.error("Failed to open mob file[" + path + "], keep it in temp directory.", e); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java index 05c70764bbc..5fe00020147 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java @@ -36,7 +36,6 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; -import com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; 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.regionserver.BloomType; 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.ScanType; 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.Pair; +import com.google.common.annotations.VisibleForTesting; + /** * 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 (Path newDelPath : delPartition.listDelFiles()) { 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 // partition. sf.initReader(); @@ -551,7 +553,7 @@ public class PartitionedMobCompactor extends MobCompactor { // add the selected mob files and del files into filesToCompact List filesToCompact = new ArrayList<>(); 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); filesToCompact.add(sf); } @@ -733,7 +735,7 @@ public class PartitionedMobCompactor extends MobCompactor { continue; } 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)); } // compact the del files in a batch. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java index bea3e7f0bba..e1d2ea170c7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java @@ -27,19 +27,29 @@ import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FSDataOutputStream; 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.NullWritable; 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.Job; 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.TextInputFormat; import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.hadoop.hbase.HDFSBlocksDistribution; -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; +import org.apache.hadoop.util.LineReader; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; /* * The CompactionTool allows to execute a compaction specifying a: diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java index 2d86e396562..7fe4c228756 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredStoreEngine.java @@ -58,7 +58,7 @@ public class DateTieredStoreEngine extends StoreEngine storefiles = this.storefiles; + if (storefiles.isEmpty()) { return null; } - return StoreUtils.getLargestFile(this.storefiles).getFileSplitPoint(this.kvComparator); + Optional largestFile = StoreUtils.getLargestFile(storefiles); + return largestFile.isPresent() + ? StoreUtils.getFileSplitPoint(largestFile.get(), kvComparator).orElse(null) : null; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java index 73c8a1ff148..c240df3a1d7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java @@ -293,7 +293,7 @@ public class HMobStore extends HStore { private void validateMobFile(Path path) throws IOException { StoreFile storeFile = null; try { - storeFile = new StoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig, + storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig, BloomType.NONE, isPrimaryReplicaStore()); storeFile.initReader(); } catch (IOException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index a620a253f10..7f9c766351a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -1456,7 +1456,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * time-sensitive thread. * * @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. * * @throws IOException e @@ -1497,7 +1497,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * * @param abort true if server is aborting (only during testing) * @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. * * @throws IOException e @@ -4204,7 +4204,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi Set fakeStoreFiles = new HashSet<>(files.size()); for (Path file: files) { 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); } else { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 051471ecd6d..17e255a3e18 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -18,13 +18,6 @@ */ 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.InterruptedIOException; import java.net.InetSocketAddress; @@ -53,7 +46,15 @@ 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.*; +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.classification.InterfaceAudience; 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.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 * or more StoreFiles, which stretch backwards over time. @@ -455,12 +463,12 @@ public class HStore implements Store { */ @Override public long getMaxSequenceId() { - return StoreFile.getMaxSequenceIdInList(this.getStorefiles()); + return StoreUtils.getMaxSequenceIdInList(this.getStorefiles()); } @Override 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 { 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()); storeFile.initReader(); return storeFile; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java new file mode 100644 index 00000000000..2df15f9ae77 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -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()} + *

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 comparator; + + @Override + public CacheConfig getCacheConf() { + return cacheConf; + } + + @Override + public Cell getFirstKey() { + return firstKey; + } + + @Override + public Cell getLastKey() { + return lastKey; + } + + @Override + public Comparator 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 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 _SeqId__ + 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(); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 91ff97a72d5..a5efbeddb08 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -1,5 +1,4 @@ /** - * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -18,789 +17,200 @@ */ 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.util.Collection; -import java.util.Collections; import java.util.Comparator; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.OptionalLong; -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.CellComparator; -import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HDFSBlocksDistribution; 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.HFile; -import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.hbase.util.Bytes; /** - * 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()} - *

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. + * An interface to describe a store data file. */ -@InterfaceAudience.LimitedPrivate("Coprocessor") -public class StoreFile { - private static final Log LOG = LogFactory.getLog(StoreFile.class.getName()); +@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) +@InterfaceStability.Evolving +public interface StoreFile { - public static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead"; - - private static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false; + static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead"; // Keys for fileinfo values in HFile /** 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 */ - public static final byte[] MAJOR_COMPACTION_KEY = - Bytes.toBytes("MAJOR_COMPACTION_KEY"); + static final byte[] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY"); /** 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"); /** Bloom filter Type in FileInfo */ - public static final byte[] BLOOM_FILTER_TYPE_KEY = - Bytes.toBytes("BLOOM_FILTER_TYPE"); + static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE"); /** Delete Family Count in FileInfo */ - public static final byte[] DELETE_FAMILY_COUNT = - Bytes.toBytes("DELETE_FAMILY_COUNT"); + static final byte[] DELETE_FAMILY_COUNT = Bytes.toBytes("DELETE_FAMILY_COUNT"); /** 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*/ - public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE"); + /** Key for Timerange information in metadata */ + static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE"); - /** Key for timestamp of earliest-put in metadata*/ - public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS"); + /** Key for timestamp of earliest-put in metadata */ + static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS"); - /** Key for the number of mob cells in metadata*/ - public 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 comparator; - - CacheConfig getCacheConf() { - return cacheConf; - } - - public Cell getFirstKey() { - return firstKey; - } - - public Cell getLastKey() { - return lastKey; - } - - public Comparator 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; + /** Key for the number of mob cells in metadata */ + static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT"); /** Meta key set when store file is a result of a bulk load */ - public static final byte[] BULKLOAD_TASK_KEY = - Bytes.toBytes("BULKLOAD_SOURCE_TASK"); - public static final byte[] BULKLOAD_TIME_KEY = - Bytes.toBytes("BULKLOAD_TIMESTAMP"); + static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK"); + static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP"); /** - * Map of the metadata entries in the corresponding HFile. Populated when Reader is opened - * after which it is not modified again. + * 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. */ - private Map metadataMap; + static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID"); - // StoreFile.Reader - private volatile StoreFileReader reader; + CacheConfig getCacheConf(); + + Cell getFirstKey(); + + Cell getLastKey(); + + Comparator getComparator(); + + long getMaxMemstoreTS(); /** - * Bloom filter type specified in column family configuration. Does not - * necessarily correspond to the Bloom filter type present in the HFile. + * @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a + * reference. */ - private final BloomType cfBloomType; - - /** - * 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; - } + StoreFileInfo getFileInfo(); /** * @return Path or null if this StoreFile was made with a Stream. */ - public Path getPath() { - return this.fileInfo.getPath(); - } + Path getPath(); /** * @return Returns the qualified path of this StoreFile */ - public Path getQualifiedPath() { - return this.fileInfo.getPath().makeQualified(fs.getUri(), fs.getWorkingDirectory()); - } + Path getQualifiedPath(); /** - * @return True if this is a StoreFile Reference; call - * after {@link #open()} else may get wrong answer. + * @return True if this is a StoreFile Reference. */ - public boolean isReference() { - return this.fileInfo.isReference(); - } + boolean isReference(); /** * @return True if this is HFile. */ - public boolean isHFile() { - return StoreFileInfo.isHFile(this.fileInfo.getPath()); - } + boolean isHFile(); /** * @return True if this file was made by a major compaction. */ - public boolean isMajorCompaction() { - if (this.majorCompaction == null) { - throw new NullPointerException("This has not been set yet"); - } - return this.majorCompaction.get(); - } + boolean isMajorCompactionResult(); /** * @return True if this file should not be part of a minor compaction. */ - public boolean excludeFromMinorCompaction() { - return this.excludeFromMinorCompaction; - } + boolean excludeFromMinorCompaction(); /** * @return This files maximum edit sequence id. */ - public long getMaxSequenceId() { - return this.sequenceid; - } + long getMaxSequenceId(); - public long getModificationTimeStamp() throws IOException { - return (fileInfo == null) ? 0 : fileInfo.getModificationTime(); - } + long getModificationTimeStamp() throws IOException; /** * Only used by the Striped Compaction Policy * @param key * @return value associated with the metadata key */ - public byte[] getMetadataValue(byte[] key) { - return metadataMap.get(key); - } + byte[] getMetadataValue(byte[] key); /** - * Return the largest memstoreTS found across all storefiles in - * the given list. Store files that were created by a mapreduce - * bulk load are ignored, as they do not correspond to any specific - * put operation, and thus do not have a memstoreTS associated with them. - * @return 0 if no non-bulk-load files are provided or, this is Store that - * does not yet have any store files. - */ - public static long getMaxMemstoreTSInList(Collection 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 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_'} 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. + * Check if this storefile was created by bulk load. When a hfile is bulk loaded into HBase, we + * append {@code '_SeqId_'} 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. */ - 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)); - } + boolean isBulkLoadResult(); - @VisibleForTesting - public boolean isCompactedAway() { - return compactedAway; - } - - @VisibleForTesting - public int getRefCount() { - return refCount.get(); - } + boolean isCompactedAway(); /** * @return true if the file is still used in reads */ - public boolean isReferencedInReads() { - int rc = refCount.get(); - assert rc >= 0; // we should not go negative. - return rc > 0; - } + boolean isReferencedInReads(); /** * Return the timestamp at which this bulk load file was generated. */ - public long getBulkLoadTimestamp() { - byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY); - return (bulkLoadTimestamp == null) ? 0 : Bytes.toLong(bulkLoadTimestamp); - } + OptionalLong getBulkLoadTimestamp(); /** - * @return the cached value of HDFS blocks distribution. The cached value is - * calculated when store file is opened. + * @return the cached value of HDFS blocks distribution. The cached value is calculated when store + * file is opened. */ - 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 _SeqId__ - 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(); - } + HDFSBlocksDistribution getHDFSBlockDistribution(); /** * Initialize the reader used for pread. */ - 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; - } - - 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); - } + void initReader() throws IOException; /** - * @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() */ - public StoreFileReader getReader() { - return this.reader; - } + StoreFileReader getReader(); /** * @param evictOnClose whether to evict blocks belonging to this file * @throws IOException */ - public synchronized void closeReader(boolean evictOnClose) - throws IOException { - if (this.reader != null) { - this.reader.close(evictOnClose); - this.reader = null; - } - } + void closeReader(boolean evictOnClose) throws IOException; /** * Marks the status of the file as compactedAway. */ - public void markCompactedAway() { - this.compactedAway = true; - } + void markCompactedAway(); /** * Delete this file * @throws IOException */ - 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(); - } + void deleteReader() throws IOException; /** * @return a length description of this StoreFile, suitable for debug output */ - 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=").append(getBulkLoadTimestamp()); - } else { - sb.append(", seqid=").append(getMaxSequenceId()); - } - sb.append(", majorCompaction=").append(isMajorCompaction()); + String toStringDetailed(); - return sb.toString(); - } + OptionalLong getMinimumTimestamp(); - /** - * 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 dir - */ - 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 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 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 { - @Override - public Long apply(StoreFile sf) { - return sf.getMaxSequenceId(); - } - } - - private static class GetFileSize implements Function { - @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 { - @Override - public Long apply(StoreFile sf) { - if (!sf.isBulkLoadResult()) return Long.MAX_VALUE; - return sf.getBulkLoadTimestamp(); - } - } - - private static class GetPathName implements Function { - @Override - public String apply(StoreFile sf) { - return sf.getPath().getName(); - } - } - - private static class GetMaxTimestamp implements Function { - @Override - public Long apply(StoreFile sf) { - return sf.getMaximumTimestamp() == null? (Long)Long.MAX_VALUE : sf.getMaximumTimestamp(); - } - } - } + OptionalLong getMaximumTimestamp(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java new file mode 100644 index 00000000000..961e3388fd3 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileComparators.java @@ -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 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 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 { + @Override + public Long apply(StoreFile sf) { + return sf.getMaxSequenceId(); + } + } + + private static class GetFileSize implements Function { + @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 { + @Override + public Long apply(StoreFile sf) { + return sf.getBulkLoadTimestamp().orElse(Long.MAX_VALUE); + } + } + + private static class GetPathName implements Function { + @Override + public String apply(StoreFile sf) { + return sf.getPath().getName(); + } + } + + private static class GetMaxTimestamp implements Function { + @Override + public Long apply(StoreFile sf) { + return sf.getMaximumTimestamp().orElse(Long.MAX_VALUE); + } + } +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java index 0e99c742b5e..c65618377c2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileInfo.java @@ -283,7 +283,6 @@ public class StoreFileInfo { */ public HDFSBlocksDistribution computeHDFSBlocksDistribution(final FileSystem fs) throws IOException { - // guard against the case where we get the FileStatus from link, but by the time we // call compute the file is moved again if (this.link != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java index 42c2af29a9b..13a5f019193 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.PriorityQueue; import java.util.concurrent.atomic.LongAdder; import org.apache.hadoop.fs.Path; @@ -120,13 +121,20 @@ public class StoreFileScanner implements KeyValueScanner { public static List getScannersForStoreFiles(Collection files, boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop, ScanQueryMatcher matcher, long readPt) throws IOException { + if (files.isEmpty()) { + return Collections.emptyList(); + } List scanners = new ArrayList<>(files.size()); - List sortedFiles = new ArrayList<>(files); - Collections.sort(sortedFiles, StoreFile.Comparators.SEQ_ID); boolean canOptimizeForNonNullColumn = matcher != null ? !matcher.hasNullColumnInQuery() : false; - for (int i = 0, n = sortedFiles.size(); i < n; i++) { - StoreFile sf = sortedFiles.get(i); - sf.initReader(); + PriorityQueue sortedFiles = + new PriorityQueue<>(files.size(), StoreFileComparators.SEQ_ID); + 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; if (usePread) { scanner = sf.getPreadScanner(cacheBlocks, readPt, i, canOptimizeForNonNullColumn); @@ -147,7 +155,7 @@ public class StoreFileScanner implements KeyValueScanner { boolean canUseDropBehind, long readPt) throws IOException { List scanners = new ArrayList<>(files.size()); List sortedFiles = new ArrayList<>(files); - Collections.sort(sortedFiles, StoreFile.Comparators.SEQ_ID); + Collections.sort(sortedFiles, StoreFileComparators.SEQ_ID); boolean succ = false; try { for (int i = 0, n = sortedFiles.size(); i < n; i++) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java index 88cba750b0d..2e3b6f55c74 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileWriter.java @@ -18,8 +18,11 @@ */ package org.apache.hadoop.hbase.regionserver; +import com.google.common.base.Preconditions; + import java.io.IOException; import java.net.InetSocketAddress; +import java.util.UUID; import org.apache.commons.logging.Log; 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.KeyValue; 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.HFile; 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.Bytes; 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.RowColBloomContext; 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 * local because it is an implementation detail of the HBase regionserver. @@ -359,6 +358,18 @@ public class StoreFileWriter implements CellSink, ShipperListener { return writer; } + /** + * @param fs + * @param dir Directory to create file in. + * @return random filename inside passed dir + */ + 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", justification="Will not overflow") public static class Builder { @@ -496,7 +507,7 @@ public class StoreFileWriter implements CellSink, ShipperListener { FSUtils.setStoragePolicy(this.fs, dir, policyName); if (filePath == null) { - filePath = StoreFile.getUniqueFile(fs, dir); + filePath = getUniqueFile(fs, dir); if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) { bloomType = BloomType.NONE; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java index c7bf78d486c..6990e91c0fa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java @@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.regionserver; -import com.google.common.annotations.VisibleForTesting; - import java.io.IOException; import java.io.InterruptedIOException; 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.EnvironmentEdgeManager; +import com.google.common.annotations.VisibleForTesting; + /** * Scanner scans both the memstore and the Store. Coalesce KeyValue stream into List<KeyValue> * for a single row. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java index 196c889aeb6..5623adb848f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java @@ -20,7 +20,14 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; 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; /** @@ -28,14 +35,14 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; */ @InterfaceAudience.Private public class StoreUtils { + + private static final Log LOG = LogFactory.getLog(StoreUtils.class); + /** * Creates a deterministic hash code for store file collection. */ - public static Integer getDeterministicRandomSeed(final Collection files) { - if (files != null && !files.isEmpty()) { - return files.iterator().next().getPath().getName().hashCode(); - } - return null; + public static OptionalInt getDeterministicRandomSeed(Collection files) { + return files.stream().mapToInt(f -> f.getPath().getName().hashCode()).findFirst(); } /** @@ -70,18 +77,73 @@ public class StoreUtils { * @param candidates The files to choose from. * @return The largest file; null if no file has a reader. */ - static StoreFile getLargestFile(final Collection candidates) { - long maxSize = -1L; - StoreFile largestSf = null; - for (StoreFile sf : candidates) { - StoreFileReader r = sf.getReader(); - if (r == null) continue; - long size = r.length(); - if (size > maxSize) { - maxSize = size; - largestSf = sf; + static Optional getLargestFile(Collection candidates) { + return candidates.stream().filter(f -> f.getReader() != null) + .max((f1, f2) -> Long.compare(f1.getReader().length(), f2.getReader().length())); + } + + /** + * Return the largest memstoreTS found across all storefiles in the given list. Store files that + * were created by a mapreduce bulk load are ignored, as they do not correspond to any specific + * put operation, and thus do not have a memstoreTS associated with them. + * @return 0 if no non-bulk-load files are provided or, this is Store that does not yet have any + * store files. + */ + public static long getMaxMemstoreTSInList(Collection 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 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 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(); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java index ef86e2fc74e..3c7469eccb1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFileManager.java @@ -28,6 +28,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.TreeMap; 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.HConstants; 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.regionserver.compactions.StripeCompactionPolicy; import org.apache.hadoop.hbase.util.Bytes; @@ -216,7 +216,12 @@ public class StripeStoreFileManager return original; } - @Override + private byte[] getSplitPoint(Collection sfs) throws IOException { + Optional 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 * stripes, unless it causes significant imbalance between split sides' sizes. In that @@ -224,6 +229,7 @@ public class StripeStoreFileManager * minimize imbalance. * @return The split point, or null if no split is possible. */ + @Override public byte[] getSplitPoint() throws IOException { if (this.getStorefileCount() == 0) return null; 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 " + newRatio + " configured ratio " + config.getMaxSplitImbalance()); // Ok, we may get better ratio, get it. - return StoreUtils.getLargestFile(state.stripeFiles.get( - isRightLarger ? rightIndex : leftIndex)).getFileSplitPoint(this.cellComparator); + return getSplitPoint(state.stripeFiles.get(isRightLarger ? rightIndex : leftIndex)); } private byte[] getSplitPointFromAllFiles() throws IOException { ConcatenatedLists sfs = new ConcatenatedLists<>(); sfs.addSublist(state.level0Files); sfs.addAllSublists(state.stripeFiles); - if (sfs.isEmpty()) return null; - return StoreUtils.getLargestFile(sfs).getFileSplitPoint(this.cellComparator); + return getSplitPoint(sfs); } 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. for (int insertBefore = 0; ; ++insertBefore) { 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); break; } @@ -1071,6 +1075,6 @@ public class StripeStoreFileManager @Override public Comparator getStoreFileComparator() { - return StoreFile.Comparators.SEQ_ID; + return StoreFileComparators.SEQ_ID; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java index 3d4f9a13480..716a820e58b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionRequest.java @@ -18,24 +18,19 @@ */ 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.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.InterfaceStability; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileReader; 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. @@ -43,7 +38,7 @@ import org.apache.hadoop.util.StringUtils; @InterfaceAudience.LimitedPrivate({ "coprocessor" }) @InterfaceStability.Evolving public class CompactionRequest implements Comparable { - private static final Log LOG = LogFactory.getLog(CompactionRequest.class); + // was this compaction promoted to an off-peak private boolean isOffPeak = false; private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR } @@ -207,27 +202,15 @@ public class CompactionRequest implements Comparable { @Override public String toString() { - String fsList = Joiner.on(", ").join( - Collections2.transform(Collections2.filter( - this.getFiles(), - new Predicate() { - @Override - public boolean apply(StoreFile sf) { - return sf.getReader() != null; - } - }), new Function() { - @Override - public String apply(StoreFile sf) { - return StringUtils.humanReadableInt( - (sf.getReader() == null) ? 0 : sf.getReader().length()); - } - })); + String fsList = filesToCompact.stream().filter(f -> f.getReader() != null) + .map(f -> TraditionalBinaryPrefix.long2String(f.getReader().length(), "", 1)) + .collect(Collectors.joining(", ")); - return "regionName=" + regionName + ", storeName=" + storeName + - ", fileCount=" + this.getFiles().size() + - ", fileSize=" + StringUtils.humanReadableInt(totalSize) + - ((fsList.isEmpty()) ? "" : " (" + fsList + ")") + - ", priority=" + priority + ", time=" + timeInNanos; + return "regionName=" + regionName + ", storeName=" + storeName + ", fileCount=" + + this.getFiles().size() + ", fileSize=" + + TraditionalBinaryPrefix.long2String(totalSize, "", 1) + + ((fsList.isEmpty()) ? "" : " (" + fsList + ")") + ", priority=" + priority + ", time=" + + timeInNanos; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 0ba500a2932..463ed86b954 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.hbase.regionserver.compactions; -import com.google.common.io.Closeables; - import java.io.IOException; import java.io.InterruptedIOException; 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.util.StringUtils.TraditionalBinaryPrefix; +import com.google.common.io.Closeables; + /** * 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). diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java index 6413ee6d16b..de461e9b4f8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java @@ -18,17 +18,12 @@ */ 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.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.OptionalLong; import org.apache.commons.logging.Log; 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.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 * Cassandra's for the following benefits: @@ -135,24 +136,24 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { boolean[] filesInWindow = new boolean[boundaries.size()]; for (StoreFile file: filesToCompact) { - Long minTimestamp = file.getMinimumTimestamp(); - long oldest = (minTimestamp == null) ? Long.MIN_VALUE : now - minTimestamp.longValue(); + OptionalLong minTimestamp = file.getMinimumTimestamp(); + long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE; if (cfTTL != Long.MAX_VALUE && oldest >= cfTTL) { LOG.debug("Major compaction triggered on store " + this + "; for TTL maintenance"); return true; } - if (!file.isMajorCompaction() || file.isBulkLoadResult()) { + if (!file.isMajorCompactionResult() || file.isBulkLoadResult()) { LOG.debug("Major compaction triggered on store " + this + ", because there are new files and time since last major compaction " + (now - lowTimestamp) + "ms"); return true; } - int lowerWindowIndex = Collections.binarySearch(boundaries, - minTimestamp == null ? (Long)Long.MAX_VALUE : minTimestamp); - int upperWindowIndex = Collections.binarySearch(boundaries, - file.getMaximumTimestamp() == null ? (Long)Long.MAX_VALUE : file.getMaximumTimestamp()); + int lowerWindowIndex = + Collections.binarySearch(boundaries, minTimestamp.orElse(Long.MAX_VALUE)); + int upperWindowIndex = + Collections.binarySearch(boundaries, file.getMaximumTimestamp().orElse(Long.MAX_VALUE)); // Handle boundary conditions and negative values of binarySearch lowerWindowIndex = (lowerWindowIndex < 0) ? Math.abs(lowerWindowIndex + 2) : lowerWindowIndex; upperWindowIndex = (upperWindowIndex < 0) ? Math.abs(upperWindowIndex + 2) : upperWindowIndex; @@ -220,8 +221,8 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { for (StoreFile storeFile : candidateSelection) { // if there is out-of-order data, // we put them in the same window as the last file in increasing order - maxTimestampSeen = Math.max(maxTimestampSeen, - storeFile.getMaximumTimestamp() == null? Long.MIN_VALUE : storeFile.getMaximumTimestamp()); + maxTimestampSeen = + Math.max(maxTimestampSeen, storeFile.getMaximumTimestamp().orElse(Long.MIN_VALUE)); storefileMaxTimestampPairs.add(new Pair<>(storeFile, maxTimestampSeen)); } Collections.reverse(storefileMaxTimestampPairs); @@ -288,23 +289,18 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { } /** - * Return a list of boundaries for multiple compaction output - * in ascending order. + * Return a list of boundaries for multiple compaction output in ascending order. */ private List getCompactBoundariesForMajor(Collection filesToCompact, long now) { - long minTimestamp = Long.MAX_VALUE; - for (StoreFile file : filesToCompact) { - minTimestamp = - Math.min(minTimestamp, - file.getMinimumTimestamp() == null ? Long.MAX_VALUE : file.getMinimumTimestamp()); - } + long minTimestamp = + filesToCompact.stream().mapToLong(f -> f.getMinimumTimestamp().orElse(Long.MAX_VALUE)).min() + .orElse(Long.MAX_VALUE); List boundaries = new ArrayList<>(); // Add startMillis of all windows between now and min timestamp - for (CompactionWindow window = getIncomingWindow(now); - window.compareToTimestamp(minTimestamp) > 0; - window = window.nextEarlierWindow()) { + for (CompactionWindow window = getIncomingWindow(now); window + .compareToTimestamp(minTimestamp) > 0; window = window.nextEarlierWindow()) { boundaries.add(window.startMillis()); } boundaries.add(Long.MIN_VALUE); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java index b1203c5ccdb..0eb94332f4d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java @@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter; import org.apache.hadoop.hbase.regionserver.InternalScanner; 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.security.User; @@ -47,7 +47,7 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor compact(final CompactionRequest request, final List lowerBoundaries, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java index 2ee051b115d..28f669bdd47 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/RatioBasedCompactionPolicy.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.OptionalLong; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -79,9 +80,9 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy { if (filesToCompact.size() == 1) { // Single file StoreFile sf = filesToCompact.iterator().next(); - Long minTimestamp = sf.getMinimumTimestamp(); - long oldest = (minTimestamp == null) ? Long.MIN_VALUE : now - minTimestamp.longValue(); - if (sf.isMajorCompaction() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) { + OptionalLong minTimestamp = sf.getMinimumTimestamp(); + long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE; + if (sf.isMajorCompactionResult() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) { float blockLocalityIndex = sf.getHDFSBlockDistribution().getBlockLocalityIndex( RSRpcServices.getHostname(comConf.conf, false)); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java index 42b57a4891c..232e5528a4f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/SortedCompactionPolicy.java @@ -10,15 +10,11 @@ */ 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.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.OptionalInt; import java.util.Random; 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.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. */ @@ -118,30 +119,31 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { * @param filesToCompact * @return When to run next major compaction */ - public long getNextMajorCompactTime(final Collection filesToCompact) { + public long getNextMajorCompactTime(Collection filesToCompact) { // default = 24hrs - long ret = comConf.getMajorCompactionPeriod(); - if (ret > 0) { - // default = 20% = +/- 4.8 hrs - double jitterPct = comConf.getMajorCompactionJitter(); - if (jitterPct > 0) { - long jitter = Math.round(ret * jitterPct); - // deterministic jitter avoids a major compaction storm on restart - Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact); - if (seed != null) { - // Synchronized to ensure one user of random instance at a time. - double rnd = -1; - synchronized (this) { - this.random.setSeed(seed); - rnd = this.random.nextDouble(); - } - ret += jitter - Math.round(2L * jitter * rnd); - } else { - ret = 0; // If seed is null, then no storefiles == no major compaction - } - } + long period = comConf.getMajorCompactionPeriod(); + if (period <= 0) { + return period; + } + // default = 20% = +/- 4.8 hrs + double jitterPct = comConf.getMajorCompactionJitter(); + if (jitterPct <= 0) { + return period; + } + // deterministic jitter avoids a major compaction storm on restart + OptionalInt seed = StoreUtils.getDeterministicRandomSeed(filesToCompact); + if (seed.isPresent()) { + // Synchronized to ensure one user of random instance at a time. + double rnd; + synchronized (this) { + this.random.setSeed(seed.getAsInt()); + rnd = this.random.nextDouble(); + } + long jitter = Math.round(period * jitterPct); + return period + jitter - Math.round(2L * jitter * rnd); + } else { + return 0L; } - return ret; } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java index c5b24e928f0..7ba5312bcbf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java @@ -18,9 +18,6 @@ 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.IOException; 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.errorhandling.ForeignExceptionSnare; 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.HRegionFileSystem; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; 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.FSTableDescriptors; import org.apache.hadoop.hbase.util.FSUtils; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java index 22d6fe1a423..48b98dfd856 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java @@ -24,9 +24,9 @@ import java.io.IOException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; 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.CompoundBloomFilter; import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterBase; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java index 24b5051a4d4..c508b028231 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/SimpleRegionObserver.java @@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CoprocessorEnvironment; 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.Delete; 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.Reference; 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.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.Leases; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index 274a76ef6be..3533f8a243f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -22,15 +22,14 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotSame; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.lang.reflect.Field; -import java.util.Arrays; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; 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.HFile; 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.regionserver.BloomType; 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.FSUtils; 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.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java index 6647ffecd55..9b69411a31f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/TestMobFile.java @@ -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.HFileContextBuilder; 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.StoreFileWriter; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -61,7 +61,7 @@ public class TestMobFile extends TestCase { MobTestUtil.writeStoreFile(writer, caseName); 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[] qualify = Bytes.toBytes(caseName); @@ -113,7 +113,7 @@ public class TestMobFile extends TestCase { MobTestUtil.writeStoreFile(writer, getName()); 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()); assertTrue(mobFile.getScanner() instanceof StoreFileScanner); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java index 47a1c247b33..398d14dd192 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java @@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.mob.MobFileName; import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.regionserver.BloomType; 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.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; @@ -814,7 +815,7 @@ public class TestMobCompactor { Assert.assertTrue(hasFiles); Path path = files[0].getPath(); 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); sf.initReader(); HFile.Reader reader = sf.getReader().getHFileReader(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java index f65e2245f97..f93ce98f25f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestPartitionedMobCompactor.java @@ -18,9 +18,20 @@ */ package org.apache.hadoop.hbase.mob.compactions; +import static org.junit.Assert.assertTrue; + import java.io.IOException; 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.RejectedExecutionException; 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.FileSystem; 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.TableName; 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.io.hfile.CacheConfig; 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.PartitionedMobCompactionRequest.CompactionDelPartition; 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.FSUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.junit.AfterClass; import org.junit.Assert; -import static org.junit.Assert.assertTrue; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; @@ -516,7 +542,7 @@ public class TestPartitionedMobCompactor { for (CompactionDelPartition delPartition : request.getDelPartitions()) { for (Path newDelPath : delPartition.listDelFiles()) { 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 // each partition. sf.initReader(); @@ -851,7 +877,7 @@ public class TestPartitionedMobCompactor { List sfs = new ArrayList<>(); int size = 0; 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); } List scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java index bc7f32a4f7d..7bc500231f2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java @@ -25,6 +25,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.collect.Sets; + import java.io.IOException; import java.util.Collections; import java.util.List; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java index 7229c409eef..efbac639951 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/AbstractTestDateTieredCompactionPolicy.java @@ -17,19 +17,20 @@ */ 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.util.ArrayList; import java.util.Arrays; 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.DateTieredCompactionRequest; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; -import static org.junit.Assert.*; public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java index e36d16f1095..2635e2d3169 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/DataBlockEncodingTool.java @@ -592,7 +592,7 @@ public class DataBlockEncodingTool { Path path = new Path(hfilePath); CacheConfig cacheConf = new CacheConfig(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(); StoreFileReader reader = hsf.getReader(); reader.loadFileInfo(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java index f47fc4e7221..bde09348734 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/EncodedSeekPerformanceTest.java @@ -59,7 +59,7 @@ public class EncodedSeekPerformanceTest { List allKeyValues = new ArrayList<>(); // read all of the key values - StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(), + StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(), path, configuration, cacheConf, BloomType.NONE, true); storeFile.initReader(); StoreFileReader reader = storeFile.getReader(); @@ -89,7 +89,7 @@ public class EncodedSeekPerformanceTest { private void runTest(Path path, DataBlockEncoding blockEncoding, List seeks) throws IOException { // read all of the key values - StoreFile storeFile = new StoreFile(testingUtility.getTestFileSystem(), + StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(), path, configuration, cacheConf, BloomType.NONE, true); storeFile.initReader(); long totalSize = 0; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java index 91b85d37dae..6fa951ee5cd 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java @@ -19,11 +19,14 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.util.Arrays; import java.util.Map; +import java.util.OptionalLong; import java.util.TreeMap; 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.HDFSBlocksDistribution; 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; /** 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; boolean isRef = false; long ageInDisk; @@ -64,18 +67,13 @@ public class MockStoreFile extends StoreFile { this.length = newLen; } - @Override - byte[] getFileSplitPoint(CellComparator comparator) throws IOException { - return this.splitPoint; - } - @Override public long getMaxSequenceId() { return sequenceid; } @Override - public boolean isMajorCompaction() { + public boolean isMajorCompactionResult() { return isMajor; } @@ -110,14 +108,14 @@ public class MockStoreFile extends StoreFile { this.entryCount = entryCount; } - public Long getMinimumTimestamp() { - return (timeRangeTracker == null) ? - null : timeRangeTracker.getMin(); + public OptionalLong getMinimumTimestamp() { + return timeRangeTracker == null ? OptionalLong.empty() + : OptionalLong.of(timeRangeTracker.getMin()); } - public Long getMaximumTimestamp() { - return (timeRangeTracker == null) ? - null : timeRangeTracker.getMax(); + public OptionalLong getMaximumTimestamp() { + return timeRangeTracker == null ? OptionalLong.empty() + : OptionalLong.of(timeRangeTracker.getMax()); } @Override @@ -184,6 +182,39 @@ public class MockStoreFile extends StoreFile { public void close(boolean evictOnClose) throws IOException { // 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(); + } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java index efe06058fbc..8fad1576dc8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCacheOnWriteInSchema.java @@ -218,7 +218,7 @@ public class TestCacheOnWriteInSchema { private void readStoreFile(Path path) throws IOException { CacheConfig cacheConf = store.getCacheConfig(); 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(); HFile.Reader reader = sf.getReader().getHFileReader(); try { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java index e32036877e8..d68f07e6e10 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveConcurrentClose.java @@ -23,6 +23,14 @@ import static org.junit.Assert.assertFalse; import static org.mockito.Mockito.mock; 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.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -45,14 +53,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; 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 * and HRegion.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java index e7fcf18a107..7fb7f214880 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionArchiveIOException.java @@ -30,6 +30,12 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; 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.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -53,11 +59,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; - /** * Tests that archiving compacted files behaves correctly when encountering exceptions. */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java index 57a5f599daa..384608ca2a4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompoundBloomFilter.java @@ -43,9 +43,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.hadoop.hbase.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.io.hfile.BlockCache; 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.HFileContext; 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.Bytes; import org.apache.hadoop.hbase.util.BloomFilterUtil; +import org.apache.hadoop.hbase.util.Bytes; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -200,7 +200,7 @@ public class TestCompoundBloomFilter { private void readStoreFile(int t, BloomType bt, List kvs, 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(); StoreFileReader r = sf.getReader(); final boolean pread = true; // does not really matter diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java index 4f6d4c225b1..9acf24479b0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionKeyRotation.java @@ -17,7 +17,8 @@ */ 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.security.Key; @@ -36,8 +37,6 @@ 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.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.Predicate; 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.security.EncryptionUtil; 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.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java index cd1f1e76fe8..3837e940111 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEncryptionRandomKeying.java @@ -17,7 +17,8 @@ */ 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.util.ArrayList; @@ -29,8 +30,6 @@ 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.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Put; 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.hfile.CacheConfig; 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.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java index e231b601195..daddb5c94ea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestFSErrorsExposed.java @@ -91,10 +91,10 @@ public class TestFSErrorsExposed { .withOutputDir(hfilePath) .withFileContext(meta) .build(); - TestStoreFile.writeStoreFile( + TestHStoreFile.writeStoreFile( 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); sf.initReader(); StoreFileReader reader = sf.getReader(); @@ -141,10 +141,10 @@ public class TestFSErrorsExposed { .withOutputDir(hfilePath) .withFileContext(meta) .build(); - TestStoreFile.writeStoreFile( + TestHStoreFile.writeStoreFile( 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); List scanners = StoreFileScanner.getScannersForStoreFiles( diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java index 0f24a24f0c2..2eb88f4d151 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java @@ -18,11 +18,26 @@ */ 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.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; 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.TableName; import org.apache.hadoop.hbase.Tag; -import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Scan; @@ -61,19 +75,6 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; 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) public class TestHMobStore { public static final Log LOG = LogFactory.getLog(TestHMobStore.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index ed0b9ddec0d..4f46c88ccd8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -116,7 +116,6 @@ import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.ColumnCountGetFilter; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; 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.FilterList; import org.apache.hadoop.hbase.filter.NullComparator; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java index 6eed7df9faa..5467c3f6a60 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java @@ -35,6 +35,8 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.google.common.collect.Lists; + import java.io.FileNotFoundException; import java.io.IOException; 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.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; +import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl; +import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult; +import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; +import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType; @@ -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.EventType; 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.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; @@ -98,8 +100,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; 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 * region replicas @@ -1350,7 +1350,7 @@ public class TestHRegionReplayEvents { } @Test - public void testRefreshStoreFiles() throws IOException { + public void testRefresStoreFiles() throws IOException { assertEquals(0, primaryRegion.getStoreFileList(families).size()); assertEquals(0, secondaryRegion.getStoreFileList(families).size()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java index 0ac51533898..61fe2cc1cdf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionServerBulkLoad.java @@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertThat; +import com.google.common.collect.Lists; + import java.io.IOException; import java.io.InterruptedIOException; 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.HFileContextBuilder; 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.generated.AdminProtos; 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.RegionServerTests; 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.Parameters; -import com.google.common.collect.Lists; - /** * Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of * the region server's bullkLoad functionality. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java similarity index 94% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java index d1444c97a84..7070a801ee1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.OptionalLong; import java.util.TreeSet; import java.util.concurrent.atomic.AtomicInteger; @@ -78,8 +79,8 @@ import static org.mockito.Mockito.when; * Test HStoreFile */ @Category({RegionServerTests.class, SmallTests.class}) -public class TestStoreFile extends HBaseTestCase { - private static final Log LOG = LogFactory.getLog(TestStoreFile.class); +public class TestHStoreFile extends HBaseTestCase { + private static final Log LOG = LogFactory.getLog(TestHStoreFile.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration()); private static String ROOT_DIR = TEST_UTIL.getDataTestDir("TestStoreFile").toString(); @@ -117,7 +118,7 @@ public class TestStoreFile extends HBaseTestCase { writeStoreFile(writer); 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); } @@ -169,7 +170,7 @@ public class TestStoreFile extends HBaseTestCase { writeStoreFile(writer); 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(); StoreFileReader reader = hsf.getReader(); // 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 HRegionInfo splitHri = new HRegionInfo(hri.getTable(), null, midRow); 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(); // Now confirm that I can read from the reference and that it only gets // keys from top half of the file. @@ -242,8 +243,8 @@ public class TestStoreFile extends HBaseTestCase { // Try to open store file from link StoreFileInfo storeFileInfo = new StoreFileInfo(testConf, this.fs, linkFilePath); - StoreFile hsf = - new StoreFile(this.fs, storeFileInfo, testConf, cacheConf, BloomType.NONE, true); + HStoreFile hsf = + new HStoreFile(this.fs, storeFileInfo, testConf, cacheConf, BloomType.NONE, true); assertTrue(storeFileInfo.isLink()); hsf.initReader(); @@ -296,7 +297,7 @@ public class TestStoreFile extends HBaseTestCase { // /clone/splitB// HRegionInfo splitHriA = new HRegionInfo(hri.getTable(), null, SPLITKEY); 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(); Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top 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. // 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(); // 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 // 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(); // 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); } - private void checkHalfHFile(final HRegionFileSystem regionFs, final StoreFile f) + private void checkHalfHFile(final HRegionFileSystem regionFs, final HStoreFile f) throws IOException { f.initReader(); Cell midkey = f.getReader().midkey(); @@ -353,10 +354,10 @@ public class TestStoreFile extends HBaseTestCase { midRow, null); Path bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, midRow, false); // 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(); 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(); StoreFileReader bottom = bottomF.getReader(); ByteBuffer previous = null; @@ -416,7 +417,7 @@ public class TestStoreFile extends HBaseTestCase { 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(); top = topF.getReader(); // Now read from the top. @@ -456,7 +457,7 @@ public class TestStoreFile extends HBaseTestCase { bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, badmidkey, false); 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(); bottom = bottomF.getReader(); first = true; @@ -754,7 +755,7 @@ public class TestStoreFile extends HBaseTestCase { @Test 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, 99, 1000, -1, "/foo/126"), 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 * same way that they're passed. */ - private void assertOrdering(Comparator comparator, StoreFile ... sfs) { - ArrayList sorted = Lists.newArrayList(sfs); + private void assertOrdering(Comparator comparator, HStoreFile ... sfs) { + ArrayList sorted = Lists.newArrayList(sfs); Collections.shuffle(sorted); Collections.sort(sorted, comparator); LOG.debug("sfs: " + Joiner.on(",").join(sfs)); @@ -778,19 +779,19 @@ public class TestStoreFile extends HBaseTestCase { /** * Create a mock StoreFile with the given attributes. */ - private StoreFile mockStoreFile(boolean bulkLoad, + private HStoreFile mockStoreFile(boolean bulkLoad, long size, long bulkTimestamp, long seqId, String path) { - StoreFile mock = Mockito.mock(StoreFile.class); + HStoreFile mock = Mockito.mock(HStoreFile.class); StoreFileReader reader = Mockito.mock(StoreFileReader.class); Mockito.doReturn(size).when(reader).length(); Mockito.doReturn(reader).when(mock).getReader(); 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(new Path(path)).when(mock).getPath(); String name = "mock storefile, bulkLoad=" + bulkLoad + @@ -855,7 +856,7 @@ public class TestStoreFile extends HBaseTestCase { writer.appendMetadata(0, false); 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); Store store = mock(Store.class); HColumnDescriptor hcd = mock(HColumnDescriptor.class); @@ -913,7 +914,7 @@ public class TestStoreFile extends HBaseTestCase { CacheConfig cacheConf = new CacheConfig(conf); Path pathCowOff = new Path(baseDir, "123456789"); 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); LOG.debug(hsf.getPath().toString()); @@ -936,7 +937,7 @@ public class TestStoreFile extends HBaseTestCase { cacheConf = new CacheConfig(conf); Path pathCowOn = new Path(baseDir, "123456788"); 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); // Read this file, we should see 3 hits @@ -953,13 +954,13 @@ public class TestStoreFile extends HBaseTestCase { reader.close(cacheConf.shouldEvictOnClose()); // 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(); StoreFileReader readerOne = hsf.getReader(); readerOne.loadFileInfo(); StoreFileScanner scannerOne = getStoreFileScanner(readerOne, true, true); 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(); StoreFileReader readerTwo = hsf.getReader(); readerTwo.loadFileInfo(); @@ -992,7 +993,7 @@ public class TestStoreFile extends HBaseTestCase { // Let's close the first file with evict on close turned on conf.setBoolean("hbase.rs.evictblocksonclose", true); 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(); reader = hsf.getReader(); 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 conf.setBoolean("hbase.rs.evictblocksonclose", false); 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(); reader = hsf.getReader(); reader.close(cacheConf.shouldEvictOnClose()); @@ -1018,7 +1019,7 @@ public class TestStoreFile extends HBaseTestCase { } 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 { FileSystem fs = regionFs.getFileSystem(); Path path = regionFs.splitStoreFile(hri, family, sf, splitKey, isTopRef, null); @@ -1093,8 +1094,8 @@ public class TestStoreFile extends HBaseTestCase { .build(); writer.close(); - StoreFile storeFile = - new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true); + HStoreFile storeFile = + new HStoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true); storeFile.initReader(); StoreFileReader reader = storeFile.getReader(); @@ -1103,4 +1104,3 @@ public class TestStoreFile extends HBaseTestCase { assertEquals(dataBlockEncoderAlgo.getNameInBytes(), value); } } - diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java index 1997b31afa6..155c6b6c98a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMobStoreCompaction.java @@ -37,7 +37,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; 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.Durability; import org.apache.hadoop.hbase.client.Put; @@ -293,7 +300,7 @@ public class TestMobStoreCompaction { if (fs.exists(mobDirPath)) { FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath); 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(); Map fileInfo = sf.getReader().loadFileInfo(); byte[] count = fileInfo.get(StoreFile.MOB_CELLS_COUNT); @@ -408,14 +415,14 @@ public class TestMobStoreCompaction { int size = 0; if (fs.exists(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); if (StoreFileInfo.isDelFile(sf.getPath())) { numDelfiles++; } } - List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, false, false, + List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, false, false, HConstants.LATEST_TIMESTAMP); Scan scan = new Scan(); scan.setMaxVersions(hcd.getMaxVersions()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java index 5c12bc5e3f2..72a968cb673 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionReplicas.java @@ -225,7 +225,7 @@ public class TestRegionReplicas { } @Test(timeout = 300000) - public void testRefreshStoreFiles() throws Exception { + public void testRefresStoreFiles() throws Exception { // enable store file refreshing final int refreshPeriod = 2000; // 2 sec HTU.getConfiguration().setInt("hbase.hstore.compactionThreshold", 100); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java index 2dfdf5bac1a..8d72aa217c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReversibleScanners.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Lists; + import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -66,8 +68,6 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; -import com.google.common.collect.Lists; - /** * Test cases against ReversibleKeyValueScanner */ @@ -113,7 +113,7 @@ public class TestReversibleScanners { .withFileContext(hFileContext).build(); 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); List scanners = StoreFileScanner @@ -167,10 +167,10 @@ public class TestReversibleScanners { writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1, 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); - 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); /** * Test without MVCC @@ -257,10 +257,10 @@ public class TestReversibleScanners { writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1, 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); - 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); ScanType scanType = ScanType.USER_SCAN; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java index 2630ce0a585..da086eac75b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSplitTransactionOnCluster.java @@ -72,12 +72,11 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.exceptions.UnexpectedStateException; import org.apache.hadoop.hbase.master.HMaster; 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.State; import org.apache.hadoop.hbase.master.assignment.AssignmentManager; 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.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java index ff213b8fe98..0f3d1983fe9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java @@ -27,7 +27,6 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import com.google.common.collect.Lists; import java.io.IOException; import java.lang.ref.SoftReference; import java.security.PrivilegedExceptionAction; @@ -45,7 +44,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -100,6 +99,8 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.mockito.Mockito; +import com.google.common.collect.Lists; + /** * Test class for the Store */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java index b2739e1e07d..8c460a6c56e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java @@ -30,6 +30,7 @@ import static org.mockito.Mockito.when; import java.util.ArrayList; import java.util.Arrays; +import java.util.OptionalLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -108,6 +109,7 @@ public class TestStripeStoreEngine { .thenReturn(StripeStoreFileManager.INVALID_KEY); when(sf.getReader()).thenReturn(mock(StoreFileReader.class)); when(sf.getPath()).thenReturn(new Path("moo")); + when(sf.getBulkLoadTimestamp()).thenReturn(OptionalLong.empty()); return sf; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java index ca13263f9cd..76959c6c13c 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreFileManager.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver; 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.assertFalse; 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. MockStoreFile sf5 = createFile(5, 0); - sf5.splitPoint = new byte[1]; + sf5.splitPoint = new byte[] { 1 }; manager.insertNewFiles(al(sf5)); 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. 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. MockStoreFile sf6 = createFile(6, 0, OPEN_KEY, OPEN_KEY); - sf6.splitPoint = new byte[1]; + sf6.splitPoint = new byte[] { 2 }; manager.addCompactionResults(al(), al(sf6)); - assertEquals(sf6.splitPoint, manager.getSplitPoint()); + assertArrayEquals(sf6.splitPoint, manager.getSplitPoint()); } @Test @@ -572,7 +573,7 @@ public class TestStripeStoreFileManager { private static MockStoreFile createFile( long size, long seqNum, byte[] startKey, byte[] endKey) throws Exception { FileSystem fs = TEST_UTIL.getTestFileSystem(); - Path testFilePath = StoreFile.getUniqueFile(fs, CFDIR); + Path testFilePath = StoreFileWriter.getUniqueFile(fs, CFDIR); fs.create(testFilePath).close(); MockStoreFile sf = new MockStoreFile(TEST_UTIL, testFilePath, size, 0, false, seqNum); if (startKey != null) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java index cb97d276818..96fee2459d2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/MockStoreFileGenerator.java @@ -18,20 +18,21 @@ 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.List; import java.util.Random; -import com.google.common.base.Objects; import org.apache.commons.lang.RandomStringUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileReader; 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. */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java index 0a84fe96e84..2dbc26f097b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicies.java @@ -18,29 +18,28 @@ package org.apache.hadoop.hbase.regionserver.compactions; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -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 static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import static org.mockito.Mockito.mock; - -import static org.mockito.Mockito.when; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +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}) @RunWith(Parameterized.class) diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java index 08fc7bff8d5..a71e76607e1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactedHFilesDischarger.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionServerServices; @@ -203,13 +204,13 @@ public class TestCompactedHFilesDischarger { int usedReaderCount = 0; int unusedReaderCount = 0; for (StoreFile file : storefiles) { - if (file.getRefCount() == 3) { + if (((HStoreFile) file).getRefCount() == 3) { usedReaderCount++; } } compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); for(StoreFile file : compactedfiles) { - assertEquals("Refcount should be 3", 0, file.getRefCount()); + assertEquals("Refcount should be 3", 0, ((HStoreFile) file).getRefCount()); unusedReaderCount++; } // Though there are files we are not using them for reads @@ -274,14 +275,14 @@ public class TestCompactedHFilesDischarger { int usedReaderCount = 0; int unusedReaderCount = 0; for (StoreFile file : storefiles) { - if (file.getRefCount() == 0) { + if (((HStoreFile) file).getRefCount() == 0) { unusedReaderCount++; } } compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); for(StoreFile file : compactedfiles) { - assertEquals("Refcount should be 3", 3, file.getRefCount()); + assertEquals("Refcount should be 3", 3, ((HStoreFile) file).getRefCount()); usedReaderCount++; } // The newly compacted file will not be used by any scanner @@ -307,13 +308,13 @@ public class TestCompactedHFilesDischarger { usedReaderCount = 0; unusedReaderCount = 0; for (StoreFile file : storefiles) { - if (file.getRefCount() == 3) { + if (((HStoreFile) file).getRefCount() == 3) { usedReaderCount++; } } compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); for(StoreFile file : compactedfiles) { - assertEquals("Refcount should be 0", 0, file.getRefCount()); + assertEquals("Refcount should be 0", 0, ((HStoreFile) file).getRefCount()); unusedReaderCount++; } // Though there are files we are not using them for reads diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java index e5906396ad5..2ab931e5fd2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestDateTieredCompactor.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFile; 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.StoreFileWritersCapture; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; @@ -103,7 +104,7 @@ public class TestDateTieredCompactor { when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(), anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers); when(store.getComparator()).thenReturn(CellComparator.COMPARATOR); - long maxSequenceId = StoreFile.getMaxSequenceIdInList(storefiles); + long maxSequenceId = StoreUtils.getMaxSequenceIdInList(storefiles); when(store.getMaxSequenceId()).thenReturn(maxSequenceId); return new DateTieredCompactor(conf, store) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java index b839fc38642..74279acfd74 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java @@ -36,12 +36,16 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.List; +import java.util.OptionalLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -84,9 +88,6 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; import org.mockito.ArgumentMatcher; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - @RunWith(Parameterized.class) @Category({RegionServerTests.class, SmallTests.class}) public class TestStripeCompactionPolicy { @@ -753,6 +754,7 @@ public class TestStripeCompactionPolicy { when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyLong(), anyBoolean())).thenReturn(mock(StoreFileScanner.class)); when(sf.getReader()).thenReturn(r); + when(sf.getBulkLoadTimestamp()).thenReturn(OptionalLong.empty()); return sf; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java index 83486791a53..7ac5f3477bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/visibility/TestVisibilityLabels.java @@ -27,6 +27,8 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.protobuf.ByteString; + import java.io.IOException; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; @@ -72,8 +74,6 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TestName; -import com.google.protobuf.ByteString; - /** * Base test class for visibility labels basic features */ diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java index dd3e631ad11..236994abead 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileTestUtil.java @@ -18,6 +18,11 @@ */ 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.fs.FileSystem; 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.regionserver.StoreFile; -import java.io.IOException; - -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.fail; - /** * Utility class for HFile-related testing. */ @@ -130,8 +130,7 @@ public class HFileTestUtil { writer.append(kv); } } finally { - writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, - Bytes.toBytes(System.currentTimeMillis())); + writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())); writer.close(); } }