From a5f84430a305db6a1eddd45f82d19babf43a8d01 Mon Sep 17 00:00:00 2001 From: zhangduo Date: Sun, 24 Sep 2017 19:22:16 +0800 Subject: [PATCH] HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface --- .../hadoop/hbase/backup/util/RestoreTool.java | 4 +- .../example/ZooKeeperScanPolicyObserver.java | 68 ++- .../hbase/mapreduce/HFileOutputFormat2.java | 30 +- .../hbase/regionserver/CompactionTool.java | 6 +- .../mapreduce/TestHFileOutputFormat2.java | 4 +- .../hadoop/hbase/backup/HFileArchiver.java | 30 +- .../hbase/coprocessor/RegionObserver.java | 4 +- .../hadoop/hbase/io/HalfStoreFileReader.java | 23 +- .../apache/hadoop/hbase/io/hfile/HFile.java | 11 +- .../hbase/io/hfile/HFilePrettyPrinter.java | 2 +- .../hbase/io/hfile/HFileReaderImpl.java | 36 +- .../assignment/SplitTableRegionProcedure.java | 31 +- .../balancer/StochasticLoadBalancer.java | 5 +- .../hadoop/hbase/mob/CachedMobFile.java | 7 +- .../hbase/mob/DefaultMobStoreCompactor.java | 2 +- .../hbase/mob/DefaultMobStoreFlusher.java | 9 +- .../org/apache/hadoop/hbase/mob/MobFile.java | 15 +- .../hadoop/hbase/mob/MobStoreEngine.java | 8 +- .../org/apache/hadoop/hbase/mob/MobUtils.java | 11 +- .../PartitionedMobCompactionRequest.java | 13 +- .../compactions/PartitionedMobCompactor.java | 58 +- .../regionserver/ChangedReadersObserver.java | 4 +- .../regionserver/CompactingMemStore.java | 4 +- .../regionserver/DateTieredStoreEngine.java | 8 +- .../regionserver/DefaultStoreEngine.java | 20 +- .../regionserver/DefaultStoreFileManager.java | 155 ++---- .../regionserver/DefaultStoreFlusher.java | 2 +- .../hadoop/hbase/regionserver/HMobStore.java | 8 +- .../hadoop/hbase/regionserver/HRegion.java | 92 +-- .../hbase/regionserver/HRegionFileSystem.java | 44 +- .../hbase/regionserver/HRegionServer.java | 78 +-- .../hadoop/hbase/regionserver/HStore.java | 524 +++++++++--------- .../hadoop/hbase/regionserver/HStoreFile.java | 170 +++--- .../hbase/regionserver/MemStoreCompactor.java | 31 +- .../MemStoreCompactorSegmentsIterator.java | 8 +- .../hbase/regionserver/MobStoreScanner.java | 4 +- .../regionserver/RegionCoprocessorHost.java | 54 +- .../hbase/regionserver/RegionSplitPolicy.java | 15 +- .../regionserver/ReversedMobStoreScanner.java | 4 +- .../regionserver/ReversedStoreScanner.java | 4 +- .../hadoop/hbase/regionserver/Store.java | 121 +--- .../hbase/regionserver/StoreEngine.java | 16 +- .../hadoop/hbase/regionserver/StoreFile.java | 128 +---- .../regionserver/StoreFileComparators.java | 28 +- .../hbase/regionserver/StoreFileManager.java | 42 +- .../hbase/regionserver/StoreFileReader.java | 46 +- .../hbase/regionserver/StoreFileScanner.java | 24 +- .../hbase/regionserver/StoreFileWriter.java | 42 +- .../hbase/regionserver/StoreFlusher.java | 6 +- .../hbase/regionserver/StoreScanner.java | 29 +- .../hadoop/hbase/regionserver/StoreUtils.java | 88 ++- .../hbase/regionserver/StripeStoreEngine.java | 8 +- .../regionserver/StripeStoreFileManager.java | 225 ++++---- .../regionserver/StripeStoreFlusher.java | 6 +- .../AbstractMultiOutputCompactor.java | 6 +- .../compactions/CompactionContext.java | 11 +- .../compactions/CompactionPolicy.java | 8 +- .../compactions/CompactionRequest.java | 12 +- .../regionserver/compactions/Compactor.java | 26 +- .../DateTieredCompactionPolicy.java | 38 +- .../DateTieredCompactionRequest.java | 4 +- .../compactions/DateTieredCompactor.java | 6 +- .../compactions/DefaultCompactor.java | 10 +- .../ExploringCompactionPolicy.java | 40 +- .../compactions/FIFOCompactionPolicy.java | 29 +- .../RatioBasedCompactionPolicy.java | 18 +- .../compactions/SortedCompactionPolicy.java | 60 +- .../compactions/StripeCompactionPolicy.java | 92 ++- .../compactions/StripeCompactor.java | 6 +- .../hbase/snapshot/SnapshotManifest.java | 21 +- .../hbase/tool/LoadIncrementalHFiles.java | 25 +- .../hadoop/hbase/util/BloomFilterFactory.java | 14 +- .../apache/hadoop/hbase/util/HBaseFsck.java | 17 +- .../hadoop/hbase/util/RowBloomContext.java | 7 +- .../hadoop/hbase/util/RowColBloomContext.java | 7 +- .../hbase-webapps/regionserver/region.jsp | 2 +- .../apache/hadoop/hbase/TestIOFencing.java | 12 +- .../hadoop/hbase/client/TestAdmin1.java | 17 +- ...tAvoidCellReferencesIntoShippedBlocks.java | 13 +- .../hbase/client/TestFromClientSide.java | 7 +- .../coprocessor/SimpleRegionObserver.java | 15 +- .../TestRegionObserverScannerOpenHook.java | 3 +- .../hbase/io/TestHalfStoreFileReader.java | 4 +- .../hadoop/hbase/io/hfile/TestHFile.java | 4 +- .../hbase/io/hfile/TestHFileBlockIndex.java | 8 +- .../hadoop/hbase/io/hfile/TestHFileSeek.java | 4 +- .../mob/compactions/TestMobCompactor.java | 6 +- .../TestPartitionedMobCompactor.java | 74 ++- .../quotas/SpaceQuotaHelperForTests.java | 12 +- ...bstractTestDateTieredCompactionPolicy.java | 24 +- .../regionserver/DataBlockEncodingTool.java | 2 +- .../EncodedSeekPerformanceTest.java | 8 +- ...MockStoreFile.java => MockHStoreFile.java} | 35 +- .../regionserver/NoOpScanPolicyObserver.java | 15 +- .../TestCacheOnWriteInSchema.java | 2 +- .../hbase/regionserver/TestCompaction.java | 41 +- .../TestCompactionArchiveConcurrentClose.java | 15 +- .../TestCompactionArchiveIOException.java | 18 +- .../regionserver/TestCompactionPolicy.java | 26 +- .../regionserver/TestCompoundBloomFilter.java | 2 +- .../TestDefaultCompactSelection.java | 10 +- .../regionserver/TestDefaultStoreEngine.java | 10 +- .../TestEncryptionKeyRotation.java | 24 +- .../TestEncryptionRandomKeying.java | 4 +- .../regionserver/TestFSErrorsExposed.java | 6 +- .../hbase/regionserver/TestHMobStore.java | 6 +- .../hbase/regionserver/TestHRegion.java | 135 ++--- .../regionserver/TestHRegionReplayEvents.java | 8 +- .../TestHRegionServerBulkLoad.java | 16 +- .../{TestStore.java => TestHStore.java} | 149 ++--- .../hbase/regionserver/TestHStoreFile.java | 15 +- .../regionserver/TestMajorCompaction.java | 8 +- .../regionserver/TestMobStoreCompaction.java | 16 +- .../regionserver/TestRegionReplicas.java | 15 +- .../regionserver/TestRegionSplitPolicy.java | 13 +- .../regionserver/TestReversibleScanners.java | 45 +- .../regionserver/TestScannerWithBulkload.java | 9 +- .../TestSplitTransactionOnCluster.java | 17 +- .../regionserver/TestStripeStoreEngine.java | 12 +- .../TestStripeStoreFileManager.java | 155 +++--- .../regionserver/TestSwitchToStreamRead.java | 11 +- .../ConstantSizeFileListGenerator.java | 11 +- .../compactions/EverythingPolicy.java | 12 +- .../ExplicitFileListGenerator.java | 11 +- .../GaussianFileListGenerator.java | 11 +- .../compactions/MockStoreFileGenerator.java | 24 +- .../PerfTestCompactionPolicies.java | 38 +- .../SemiConstantSizeFileListGenerator.java | 11 +- .../SinusoidalFileListGenerator.java | 13 +- .../compactions/SpikyFileListGenerator.java | 11 +- .../compactions/StoreFileListGenerator.java | 9 +- .../TestCompactedHFilesDischarger.java | 63 +-- .../compactions/TestCompactor.java | 6 +- .../compactions/TestDateTieredCompactor.java | 16 +- .../TestStripeCompactionPolicy.java | 89 +-- .../compactions/TestStripeCompactor.java | 8 +- .../wal/AbstractTestWALReplay.java | 17 +- .../visibility/TestVisibilityLabels.java | 10 +- .../hadoop/hbase/util/HFileTestUtil.java | 4 +- .../hbase/util/TestCoprocessorScanPolicy.java | 13 +- .../hbase/util/TestHBaseFsckEncryption.java | 13 +- .../hadoop/hbase/spark/HBaseContext.scala | 10 +- 142 files changed, 2049 insertions(+), 2276 deletions(-) rename hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/{MockStoreFile.java => MockHStoreFile.java} (82%) rename hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/{TestStore.java => TestHStore.java} (93%) diff --git a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java index ca0d02669f3..b00351b12e0 100644 --- a/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java +++ b/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/util/RestoreTool.java @@ -440,8 +440,8 @@ public class RestoreTool { final byte[] first, last; try { reader.loadFileInfo(); - first = reader.getFirstRowKey(); - last = reader.getLastRowKey(); + first = reader.getFirstRowKey().get(); + last = reader.getLastRowKey().get(); LOG.debug("Trying to figure out region boundaries hfile=" + hfile + " first=" + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last)); diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java index 6b316649186..a7304036bbe 100644 --- a/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java +++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java @@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.zookeeper.KeeperException; @@ -130,32 +129,32 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { @Override public void process(WatchedEvent event) { - switch(event.getType()) { - case NodeDataChanged: - case NodeCreated: - try { - // get data and re-watch - data = zk.getData(node, this, null); - LOG.debug("Read asynchronously: "+(data == null ? "null" : Bytes.toLong(data))); - } catch (InterruptedException ix) { - } catch (KeeperException kx) { - needSetup = true; - } - break; + switch (event.getType()) { + case NodeDataChanged: + case NodeCreated: + try { + // get data and re-watch + data = zk.getData(node, this, null); + LOG.debug("Read asynchronously: " + (data == null ? "null" : Bytes.toLong(data))); + } catch (InterruptedException ix) { + } catch (KeeperException kx) { + needSetup = true; + } + break; - case NodeDeleted: - try { - // just re-watch - zk.exists(node, this); - data = null; - } catch (InterruptedException ix) { - } catch (KeeperException kx) { - needSetup = true; - } - break; + case NodeDeleted: + try { + // just re-watch + zk.exists(node, this); + data = null; + } catch (InterruptedException ix) { + } catch (KeeperException kx) { + needSetup = true; + } + break; - default: - // ignore + default: + // ignore } } } @@ -166,15 +165,13 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { if (!re.getSharedData().containsKey(zkkey)) { // there is a short race here // in the worst case we create a watcher that will be notified once - re.getSharedData().putIfAbsent( - zkkey, - new ZKWatcher(re.getRegionServerServices().getZooKeeper() - .getRecoverableZooKeeper().getZooKeeper())); + re.getSharedData().putIfAbsent(zkkey, new ZKWatcher( + re.getRegionServerServices().getZooKeeper().getRecoverableZooKeeper().getZooKeeper())); } } protected ScanInfo getScanInfo(Store store, RegionCoprocessorEnvironment e) { - byte[] data = ((ZKWatcher)e.getSharedData().get(zkkey)).getData(); + byte[] data = ((ZKWatcher) e.getSharedData().get(zkkey)).getData(); if (data == null) { return null; } @@ -182,8 +179,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { if (oldSI.getTtl() == Long.MAX_VALUE) { return null; } - long ttl = Math.max(EnvironmentEdgeManager.currentTime() - - Bytes.toLong(data), oldSI.getTtl()); + long ttl = Math.max(EnvironmentEdgeManager.currentTime() - Bytes.toLong(data), oldSI.getTtl()); return new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), ttl, oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); } @@ -197,7 +193,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { // take default action return null; } - return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, + return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP); } @@ -210,7 +206,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { // take default action return null; } - return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, scanType, + return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, scanType, store.getSmallestReadPoint(), earliestPutTs); } @@ -223,7 +219,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver { // take default action return null; } - return new StoreScanner(store, scanInfo, scan, targetCols, - ((HStore)store).getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED)); + return new StoreScanner((HStore) store, scanInfo, scan, targetCols, + ((HStore) store).getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED)); } } diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java index 41a9839a8c5..e8b7d11eb10 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/HFileOutputFormat2.java @@ -17,6 +17,11 @@ */ package org.apache.hadoop.hbase.mapreduce; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TASK_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY; + import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.InetSocketAddress; @@ -43,8 +48,12 @@ 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.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueUtil; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Connection; @@ -52,10 +61,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.fs.HFileSystem; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.compress.Compression.Algorithm; @@ -65,13 +72,9 @@ 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.HFileWriterImpl; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; @@ -85,6 +88,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; @@ -413,13 +417,13 @@ public class HFileOutputFormat2 private void close(final StoreFileWriter w) throws IOException { if (w != null) { - w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, + w.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())); - w.appendFileInfo(StoreFile.BULKLOAD_TASK_KEY, + w.appendFileInfo(BULKLOAD_TASK_KEY, Bytes.toBytes(context.getTaskAttemptID().toString())); - w.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY, + w.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(true)); - w.appendFileInfo(StoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY, + w.appendFileInfo(EXCLUDE_FROM_MINOR_COMPACTION_KEY, Bytes.toBytes(compactionExclude)); w.appendTrackedTimestampsToMetadata(); w.close(); diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java index bb0145996f8..79123402d69 100644 --- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java +++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionTool.java @@ -38,7 +38,6 @@ 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.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.mapreduce.JobUtil; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; @@ -62,6 +61,7 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; import org.apache.hadoop.util.LineReader; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; /* * The CompactionTool allows to execute a compaction specifying a: @@ -165,11 +165,11 @@ public class CompactionTool extends Configured implements Tool { if (!compaction.isPresent()) { break; } - List storeFiles = + List storeFiles = store.compact(compaction.get(), NoLimitThroughputController.INSTANCE); if (storeFiles != null && !storeFiles.isEmpty()) { if (keepCompactedFiles && deleteCompacted) { - for (StoreFile storeFile: storeFiles) { + for (HStoreFile storeFile: storeFiles) { fs.delete(storeFile.getPath(), false); } } diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java index cbff2deb003..0b5a9292165 100644 --- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java +++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hbase.mapreduce; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -85,7 +86,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -1162,7 +1162,7 @@ public class TestHFileOutputFormat2 { Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf), true, conf); Map fileInfo = reader.loadFileInfo(); - byte[] bloomFilter = fileInfo.get(StoreFile.BLOOM_FILTER_TYPE_KEY); + byte[] bloomFilter = fileInfo.get(BLOOM_FILTER_TYPE_KEY); if (bloomFilter == null) bloomFilter = Bytes.toBytes("NONE"); assertEquals("Incorrect bloom filter used for column family " + familyStr + "(reader: " + reader + ")", diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java index d17546e8787..4321dc8c8dc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/HFileArchiver.java @@ -33,14 +33,14 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.io.MultipleIOException; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.base.Function; import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Collections2; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; /** - * Utility class to handle the removal of HFiles (or the respective {@link StoreFile StoreFiles}) + * Utility class to handle the removal of HFiles (or the respective {@link HStoreFile StoreFiles}) * for a HRegion from the {@link FileSystem}. The hfiles will be archived or deleted, depending on * the state of the system. */ @@ -226,7 +226,7 @@ public class HFileArchiver { * @throws IOException if the files could not be correctly disposed. */ public static void archiveStoreFiles(Configuration conf, FileSystem fs, HRegionInfo regionInfo, - Path tableDir, byte[] family, Collection compactedFiles) + Path tableDir, byte[] family, Collection compactedFiles) throws IOException, FailedArchiveException { // sometimes in testing, we don't have rss, so we need to check for that @@ -479,13 +479,13 @@ public class HFileArchiver { * @throws IOException if a file cannot be deleted. All files will be attempted to deleted before * throwing the exception, rather than failing at the first file. */ - private static void deleteStoreFilesWithoutArchiving(Collection compactedFiles) + private static void deleteStoreFilesWithoutArchiving(Collection compactedFiles) throws IOException { LOG.debug("Deleting store files without archiving."); List errors = new ArrayList<>(0); - for (StoreFile hsf : compactedFiles) { + for (HStoreFile hsf : compactedFiles) { try { - hsf.deleteReader(); + hsf.deleteStoreFile(); } catch (IOException e) { LOG.error("Failed to delete store file:" + hsf.getPath()); errors.add(e); @@ -524,16 +524,16 @@ public class HFileArchiver { } /** - * Convert the {@link StoreFile} into something we can manage in the archive + * Convert the {@link HStoreFile} into something we can manage in the archive * methods */ - private static class StoreToFile extends FileConverter { + private static class StoreToFile extends FileConverter { public StoreToFile(FileSystem fs) { super(fs); } @Override - public File apply(StoreFile input) { + public File apply(HStoreFile input) { return new FileableStoreFile(fs, input); } } @@ -656,20 +656,20 @@ public class HFileArchiver { } /** - * {@link File} adapter for a {@link StoreFile} living on a {@link FileSystem} + * {@link File} adapter for a {@link HStoreFile} living on a {@link FileSystem} * . */ private static class FileableStoreFile extends File { - StoreFile file; + HStoreFile file; - public FileableStoreFile(FileSystem fs, StoreFile store) { + public FileableStoreFile(FileSystem fs, HStoreFile store) { super(fs); this.file = store; } @Override public void delete() throws IOException { - file.deleteReader(); + file.deleteStoreFile(); } @Override @@ -690,7 +690,7 @@ public class HFileArchiver { @Override public void close() throws IOException { - file.closeReader(true); + file.closeStoreFile(true); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java index 9318b9aaaa3..60e5f40ba37 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java @@ -189,7 +189,7 @@ public interface RegionObserver extends Coprocessor { * @param tracker tracker used to track the life cycle of a compaction */ default void preCompactSelection(ObserverContext c, Store store, - List candidates, CompactionLifeCycleTracker tracker) throws IOException {} + List candidates, CompactionLifeCycleTracker tracker) throws IOException {} /** * Called after the {@link StoreFile}s to compact have been selected from the available @@ -200,7 +200,7 @@ public interface RegionObserver extends Coprocessor { * @param tracker tracker used to track the life cycle of a compaction */ default void postCompactSelection(ObserverContext c, Store store, - ImmutableList selected, CompactionLifeCycleTracker tracker) {} + ImmutableList selected, CompactionLifeCycleTracker tracker) {} /** * Called prior to writing the {@link StoreFile}s selected for compaction into a new diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java index 0c915d12466..18ddb6a77e6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/HalfStoreFileReader.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.io; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; @@ -48,7 +49,7 @@ import org.apache.hadoop.hbase.util.Bytes; *

This type works in tandem with the {@link Reference} type. This class * is used reading while Reference is used writing. * - *

This file is not splitable. Calls to {@link #midkey()} return null. + *

This file is not splitable. Calls to {@link #midKey()} return null. */ @InterfaceAudience.Private public class HalfStoreFileReader extends StoreFileReader { @@ -60,7 +61,7 @@ public class HalfStoreFileReader extends StoreFileReader { protected final Cell splitCell; - private Cell firstKey = null; + private Optional firstKey = null; private boolean firstKeySeeked = false; @@ -258,8 +259,8 @@ public class HalfStoreFileReader extends StoreFileReader { @Override public boolean seekBefore(Cell key) throws IOException { if (top) { - Cell fk = getFirstKey(); - if (getComparator().compareKeyIgnoresMvcc(key, fk) <= 0) { + Optional fk = getFirstKey(); + if (getComparator().compareKeyIgnoresMvcc(key, fk.get()) <= 0) { return false; } } else { @@ -303,7 +304,7 @@ public class HalfStoreFileReader extends StoreFileReader { } @Override - public Cell getLastKey() { + public Optional getLastKey() { if (top) { return super.getLastKey(); } @@ -311,7 +312,7 @@ public class HalfStoreFileReader extends StoreFileReader { HFileScanner scanner = getScanner(true, true); try { if (scanner.seekBefore(this.splitCell)) { - return scanner.getKey(); + return Optional.ofNullable(scanner.getKey()); } } catch (IOException e) { LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e); @@ -320,22 +321,22 @@ public class HalfStoreFileReader extends StoreFileReader { scanner.close(); } } - return null; + return Optional.empty(); } @Override - public Cell midkey() throws IOException { + public Optional midKey() throws IOException { // Returns null to indicate file is not splitable. - return null; + return Optional.empty(); } @Override - public Cell getFirstKey() { + public Optional getFirstKey() { if (!firstKeySeeked) { HFileScanner scanner = getScanner(true, true, false); try { if (scanner.seekTo()) { - this.firstKey = scanner.getKey(); + this.firstKey = Optional.ofNullable(scanner.getKey()); } firstKeySeeked = true; } catch (IOException e) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 3d6cdaf26b2..9cf00b33740 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -31,6 +31,7 @@ import java.util.Collection; import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -434,21 +435,21 @@ public class HFile { Map loadFileInfo() throws IOException; - Cell getLastKey(); + Optional getLastKey(); - Cell midkey() throws IOException; + Optional midKey() throws IOException; long length(); long getEntries(); - Cell getFirstKey(); + Optional getFirstKey(); long indexSize(); - byte[] getFirstRowKey(); + Optional getFirstRowKey(); - byte[] getLastRowKey(); + Optional getLastRowKey(); FixedFileTrailer getTrailer(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java index dcfffb899c4..45243509807 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java @@ -544,7 +544,7 @@ public class HFilePrettyPrinter extends Configured implements Tool { } try { - out.println("Mid-key: " + (CellUtil.getCellKeyAsString(reader.midkey()))); + out.println("Mid-key: " + reader.midKey().map(CellUtil::getCellKeyAsString)); } catch (Exception e) { out.println ("Unable to retrieve the midkey"); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java index bf722de56ed..381279aa664 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.security.Key; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.logging.Log; @@ -338,14 +339,12 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { } } - private String toStringFirstKey() { - if(getFirstKey() == null) - return null; - return CellUtil.getCellKeyAsString(getFirstKey()); + private Optional toStringFirstKey() { + return getFirstKey().map(CellUtil::getCellKeyAsString); } - private String toStringLastKey() { - return CellUtil.toString(getLastKey(), false); + private Optional toStringLastKey() { + return getLastKey().map(CellUtil::getCellKeyAsString); } @Override @@ -382,12 +381,12 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { * first KeyValue. */ @Override - public Cell getFirstKey() { + public Optional getFirstKey() { if (dataBlockIndexReader == null) { throw new BlockIndexNotLoadedException(); } - return dataBlockIndexReader.isEmpty() ? null - : dataBlockIndexReader.getRootBlockKey(0); + return dataBlockIndexReader.isEmpty() ? Optional.empty() + : Optional.of(dataBlockIndexReader.getRootBlockKey(0)); } /** @@ -397,10 +396,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { * @return the first row key, or null if the file is empty. */ @Override - public byte[] getFirstRowKey() { - Cell firstKey = getFirstKey(); + public Optional getFirstRowKey() { // We have to copy the row part to form the row key alone - return firstKey == null? null: CellUtil.cloneRow(firstKey); + return getFirstKey().map(CellUtil::cloneRow); } /** @@ -410,9 +408,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { * @return the last row key, or null if the file is empty. */ @Override - public byte[] getLastRowKey() { - Cell lastKey = getLastKey(); - return lastKey == null? null: CellUtil.cloneRow(lastKey); + public Optional getLastRowKey() { + // We have to copy the row part to form the row key alone + return getLastKey().map(CellUtil::cloneRow); } /** @return number of KV entries in this HFile */ @@ -1550,8 +1548,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { * key */ @Override - public Cell getLastKey() { - return dataBlockIndexReader.isEmpty() ? null : lastKeyCell; + public Optional getLastKey() { + return dataBlockIndexReader.isEmpty() ? Optional.empty() : Optional.of(lastKeyCell); } /** @@ -1560,8 +1558,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { * @throws IOException */ @Override - public Cell midkey() throws IOException { - return dataBlockIndexReader.midkey(); + public Optional midKey() throws IOException { + return Optional.ofNullable(dataBlockIndexReader.midkey()); } @Override 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 98cd16d1abe..f67aa5bd475 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,12 +18,14 @@ package org.apache.hadoop.hbase.master.assignment; -import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; - import java.io.IOException; import java.io.InterruptedIOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -41,7 +43,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.MasterSwitchType; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.RegionReplicaUtil; @@ -58,19 +60,21 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil; import org.apache.hadoop.hbase.procedure2.ProcedureMetrics; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.quotas.QuotaExceededException; -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; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState; /** * The procedure to split a region in a table. @@ -636,8 +640,8 @@ public class SplitTableRegionProcedure } } - private Pair splitStoreFile(final HRegionFileSystem regionFs, - final byte[] family, final StoreFile sf) throws IOException { + private Pair splitStoreFile(HRegionFileSystem regionFs, byte[] family, HStoreFile sf) + throws IOException { if (LOG.isDebugEnabled()) { LOG.debug("pid=" + getProcId() + " splitting started for store file: " + sf.getPath() + " for region: " + getParentRegion().getShortNameToLog()); @@ -663,7 +667,7 @@ public class SplitTableRegionProcedure private class StoreFileSplitter implements Callable> { private final HRegionFileSystem regionFs; private final byte[] family; - private final StoreFile sf; + private final HStoreFile sf; /** * Constructor that takes what it needs to split @@ -671,8 +675,7 @@ public class SplitTableRegionProcedure * @param family Family that contains the store file * @param sf which file */ - public StoreFileSplitter(final HRegionFileSystem regionFs, final byte[] family, - final StoreFile sf) { + public StoreFileSplitter(HRegionFileSystem regionFs, byte[] family, HStoreFile sf) { this.regionFs = regionFs; this.sf = sf; this.family = family; 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 c1fef921636..957c182b270 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.balancer; import com.google.common.annotations.VisibleForTesting; + import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; @@ -42,7 +43,6 @@ import org.apache.hadoop.hbase.RegionLoad; import org.apache.hadoop.hbase.ServerLoad; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action; @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.MoveRegi import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.SwapRegionsAction; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.base.Optional; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; @@ -1241,7 +1242,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { /** * Compute a cost of a potential cluster configuration based upon where - * {@link org.apache.hadoop.hbase.regionserver.StoreFile}s are located. + * {@link org.apache.hadoop.hbase.regionserver.HStoreFile}s are located. */ static abstract class LocalityBasedCostFunction extends CostFunction { 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 9dc32bef30e..397570cca66 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 @@ -24,11 +24,10 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.yetus.audience.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.yetus.audience.InterfaceAudience; /** * Cached mob file. @@ -39,7 +38,7 @@ public class CachedMobFile extends MobFile implements Comparable private long accessCount; private AtomicLong referenceCount = new AtomicLong(0); - public CachedMobFile(StoreFile sf) { + public CachedMobFile(HStoreFile sf) { super(sf); } @@ -47,7 +46,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 HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); + HStoreFile 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/DefaultMobStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java index 1badeb25ca2..e8ada97c72e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreCompactor.java @@ -91,7 +91,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor { } }; - public DefaultMobStoreCompactor(Configuration conf, Store store) { + public DefaultMobStoreCompactor(Configuration conf, HStore store) { super(conf, store); // The mob cells reside in the mob-enabled column family which is held by HMobStore. // During the compaction, the compactor reads the cells from the mob files and diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java index bb0301e5ee5..bef73f21760 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/DefaultMobStoreFlusher.java @@ -32,19 +32,19 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher; import org.apache.hadoop.hbase.regionserver.HMobStore; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.StringUtils; +import org.apache.yetus.audience.InterfaceAudience; /** * An implementation of the StoreFlusher. It extends the DefaultStoreFlusher. @@ -70,8 +70,11 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher { private Path targetPath; private HMobStore mobStore; - public DefaultMobStoreFlusher(Configuration conf, Store store) throws IOException { + public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException { super(conf, store); + if (!(store instanceof HMobStore)) { + throw new IllegalArgumentException("The store " + store + " is not a HMobStore"); + } mobCellValueSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold(); this.targetPath = MobUtils.getMobFamilyPath(conf, store.getTableName(), store.getColumnFamilyName()); 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 929bfd29be5..91702c1986d 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 @@ -26,12 +26,11 @@ 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.yetus.audience.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; +import org.apache.yetus.audience.InterfaceAudience; /** * The mob file. @@ -39,13 +38,13 @@ import org.apache.hadoop.hbase.regionserver.StoreFileScanner; @InterfaceAudience.Private public class MobFile { - private StoreFile sf; + private HStoreFile sf; // internal use only for sub classes protected MobFile() { } - protected MobFile(StoreFile sf) { + protected MobFile(HStoreFile sf) { this.sf = sf; } @@ -56,7 +55,7 @@ public class MobFile { * @throws IOException */ public StoreFileScanner getScanner() throws IOException { - List sfs = new ArrayList<>(); + List sfs = new ArrayList<>(); sfs.add(sf); List sfScanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, false, false, sf.getMaxMemstoreTS()); @@ -86,7 +85,7 @@ public class MobFile { public Cell readCell(Cell search, boolean cacheMobBlocks, long readPt) throws IOException { Cell result = null; StoreFileScanner scanner = null; - List sfs = new ArrayList<>(); + List sfs = new ArrayList<>(); sfs.add(sf); try { List sfScanners = StoreFileScanner.getScannersForStoreFiles(sfs, @@ -129,7 +128,7 @@ public class MobFile { */ public void close() throws IOException { if (sf != null) { - sf.closeReader(false); + sf.closeStoreFile(false); sf = null; } } @@ -147,7 +146,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 HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); + HStoreFile 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/MobStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java index bdd336f23cd..ee1fe7db14f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobStoreEngine.java @@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.mob; import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; -import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.yetus.audience.InterfaceAudience; /** * MobStoreEngine creates the mob specific compactor, and store flusher. @@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.regionserver.Store; public class MobStoreEngine extends DefaultStoreEngine { @Override - protected void createStoreFlusher(Configuration conf, Store store) throws IOException { + protected void createStoreFlusher(Configuration conf, HStore store) throws IOException { // When using MOB, we use DefaultMobStoreFlusher always // Just use the compactor and compaction policy as that in DefaultStoreEngine. We can have MOB // specific compactor and policy when that is implemented. @@ -42,7 +42,7 @@ public class MobStoreEngine extends DefaultStoreEngine { * Creates the DefaultMobCompactor. */ @Override - protected void createCompactor(Configuration conf, Store store) throws IOException { + protected void createCompactor(Configuration conf, HStore store) throws IOException { compactor = new DefaultMobStoreCompactor(conf, store); } } 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 2592b72ad49..54f13736698 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 @@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.backup.HFileArchiver; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy; import org.apache.hadoop.hbase.client.Scan; @@ -70,7 +69,6 @@ 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; import org.apache.hadoop.hbase.util.ChecksumType; @@ -78,6 +76,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.hadoop.hbase.util.Threads; +import org.apache.yetus.audience.InterfaceAudience; /** * The mob utilities @@ -315,7 +314,7 @@ public final class MobUtils { // no file found return; } - List filesToClean = new ArrayList<>(); + List filesToClean = new ArrayList<>(); int deletedFileCount = 0; for (FileStatus file : stats) { String fileName = file.getPath().getName(); @@ -467,7 +466,7 @@ public final class MobUtils { * @throws IOException */ public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName, - Path tableDir, byte[] family, Collection storeFiles) throws IOException { + Path tableDir, byte[] family, Collection storeFiles) throws IOException { HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family, storeFiles); } @@ -721,7 +720,7 @@ public final class MobUtils { */ private static void validateMobFile(Configuration conf, FileSystem fs, Path path, CacheConfig cacheConfig, boolean primaryReplica) throws IOException { - StoreFile storeFile = null; + HStoreFile storeFile = null; try { storeFile = new HStoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica); storeFile.initReader(); @@ -730,7 +729,7 @@ public final class MobUtils { throw e; } finally { if (storeFile != null) { - storeFile.closeReader(false); + storeFile.closeStoreFile(false); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java index 79167791d6c..aaf545b1ffb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactionRequest.java @@ -25,10 +25,10 @@ import java.util.List; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; /** * An implementation of {@link MobCompactionRequest} that is used in @@ -106,8 +106,7 @@ public class PartitionedMobCompactionRequest extends MobCompactionRequest { * Set start key of this partition, only if the input startKey is less than * the current start key. */ - public void setStartKey(final byte[] startKey) - { + public void setStartKey(final byte[] startKey) { if ((this.startKey == null) || (Bytes.compareTo(startKey, this.startKey) < 0)) { this.startKey = startKey; } @@ -227,7 +226,7 @@ public class PartitionedMobCompactionRequest extends MobCompactionRequest { */ protected static class CompactionDelPartition { private List delFiles = new ArrayList(); - private List storeFiles = new ArrayList<>(); + private List storeFiles = new ArrayList<>(); private CompactionDelPartitionId id; public CompactionDelPartition(CompactionDelPartitionId id) { @@ -241,11 +240,11 @@ public class PartitionedMobCompactionRequest extends MobCompactionRequest { void addDelFile(FileStatus file) { delFiles.add(file.getPath()); } - public void addStoreFile(final StoreFile file) { + public void addStoreFile(HStoreFile file) { storeFiles.add(file); } - public List getStoreFiles() { + public List getStoreFiles() { return storeFiles; } 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 e40f3a75711..1fc290272b3 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 @@ -18,6 +18,10 @@ */ package org.apache.hadoop.hbase.mob.compactions; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.SKIP_RESET_SEQ_ID; + import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; @@ -50,7 +54,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagUtil; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; @@ -75,7 +78,6 @@ 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; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; @@ -85,6 +87,7 @@ import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; @@ -225,8 +228,8 @@ public class PartitionedMobCompactor extends MobCompactor { // Get delId from the file try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) { - delId.setStartKey(reader.getFirstRowKey()); - delId.setEndKey(reader.getLastRowKey()); + delId.setStartKey(reader.getFirstRowKey().get()); + delId.setEndKey(reader.getLastRowKey().get()); } CompactionDelPartition delPartition = delFilesToCompact.get(delId); if (delPartition == null) { @@ -266,8 +269,8 @@ public class PartitionedMobCompactor extends MobCompactor { // get startKey and endKey from the file and update partition // TODO: is it possible to skip read of most hfiles? try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) { - compactionPartition.setStartKey(reader.getFirstRowKey()); - compactionPartition.setEndKey(reader.getLastRowKey()); + compactionPartition.setStartKey(reader.getFirstRowKey().get()); + compactionPartition.setEndKey(reader.getLastRowKey().get()); } } @@ -335,7 +338,7 @@ public class PartitionedMobCompactor extends MobCompactor { try { for (CompactionDelPartition delPartition : request.getDelPartitions()) { for (Path newDelPath : delPartition.listDelFiles()) { - StoreFile sf = + HStoreFile sf = 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. @@ -361,7 +364,8 @@ public class PartitionedMobCompactor extends MobCompactor { for (CompactionDelPartition delPartition : request.getDelPartitions()) { LOG.info(delPartition.listDelFiles()); try { - MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delPartition.getStoreFiles()); + MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), + delPartition.getStoreFiles()); } catch (IOException e) { LOG.error("Failed to archive the del files " + delPartition.getStoreFiles(), e); } @@ -397,11 +401,11 @@ public class PartitionedMobCompactor extends MobCompactor { } @VisibleForTesting - List getListOfDelFilesForPartition(final CompactionPartition partition, + List getListOfDelFilesForPartition(final CompactionPartition partition, final List delPartitions) { // Binary search for startKey and endKey - List result = new ArrayList<>(); + List result = new ArrayList<>(); DelPartitionComparator comparator = new DelPartitionComparator(false); CompactionDelPartitionId id = new CompactionDelPartitionId(null, partition.getStartKey()); @@ -473,7 +477,7 @@ public class PartitionedMobCompactor extends MobCompactor { // Search the delPartitions and collect all the delFiles for the partition // One optimization can do is that if there is no del file, we do not need to // come up with startKey/endKey. - List delFiles = getListOfDelFilesForPartition(partition, + List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); results.put(partition.getPartitionId(), pool.submit(new Callable>() { @@ -521,7 +525,7 @@ public class PartitionedMobCompactor extends MobCompactor { */ private List compactMobFilePartition(PartitionedMobCompactionRequest request, CompactionPartition partition, - List delFiles, + List delFiles, Connection connection, Table table) throws IOException { if (MobUtils.isMobFileExpired(column, EnvironmentEdgeManager.currentTime(), @@ -550,9 +554,9 @@ public class PartitionedMobCompactor extends MobCompactor { // clean the bulkload directory to avoid loading old files. fs.delete(bulkloadPathOfPartition, true); // add the selected mob files and del files into filesToCompact - List filesToCompact = new ArrayList<>(); + List filesToCompact = new ArrayList<>(); for (int i = offset; i < batch + offset; i++) { - StoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig, + HStoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig, BloomType.NONE, true); filesToCompact.add(sf); } @@ -572,10 +576,10 @@ public class PartitionedMobCompactor extends MobCompactor { * Closes the readers of store files. * @param storeFiles The store files to be closed. */ - private void closeStoreFileReaders(List storeFiles) { - for (StoreFile storeFile : storeFiles) { + private void closeStoreFileReaders(List storeFiles) { + for (HStoreFile storeFile : storeFiles) { try { - storeFile.closeReader(true); + storeFile.closeStoreFile(true); } catch (IOException e) { LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e); } @@ -600,14 +604,14 @@ public class PartitionedMobCompactor extends MobCompactor { private void compactMobFilesInBatch(PartitionedMobCompactionRequest request, CompactionPartition partition, Connection connection, Table table, - List filesToCompact, int batch, + List filesToCompact, int batch, Path bulkloadPathOfPartition, Path bulkloadColumnPath, List newFiles) throws IOException { // open scanner to the selected mob files and del files. StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES); // the mob files to be compacted, not include the del files. - List mobFilesToCompact = filesToCompact.subList(0, batch); + List mobFilesToCompact = filesToCompact.subList(0, batch); // Pair(maxSeqId, cellsCount) Pair fileInfo = getFileInfo(mobFilesToCompact); // open writers for the mob files and new ref store files. @@ -726,7 +730,7 @@ public class PartitionedMobCompactor extends MobCompactor { if (delFilePaths.size() - offset < compactionBatchSize) { batch = delFilePaths.size() - offset; } - List batchedDelFiles = new ArrayList<>(); + List batchedDelFiles = new ArrayList<>(); if (batch == 1) { // only one file left, do not compact it, directly add it to the new files. paths.add(delFilePaths.get(offset)); @@ -753,7 +757,7 @@ public class PartitionedMobCompactor extends MobCompactor { * @throws IOException if IO failure is encountered */ private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request, - List delFiles) throws IOException { + List delFiles) throws IOException { // create a scanner for the del files. StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES); StoreFileWriter writer = null; @@ -803,7 +807,7 @@ public class PartitionedMobCompactor extends MobCompactor { * @return The store scanner. * @throws IOException if IO failure is encountered */ - private StoreScanner createScanner(List filesToCompact, ScanType scanType) + private StoreScanner createScanner(List filesToCompact, ScanType scanType) throws IOException { List scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact, false, true, false, false, HConstants.LATEST_TIMESTAMP); @@ -864,8 +868,8 @@ public class PartitionedMobCompactor extends MobCompactor { throws IOException { if (writer != null) { writer.appendMetadata(maxSeqId, false); - writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime)); - writer.appendFileInfo(StoreFile.SKIP_RESET_SEQ_ID, Bytes.toBytes(true)); + writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime)); + writer.appendFileInfo(SKIP_RESET_SEQ_ID, Bytes.toBytes(true)); try { writer.close(); } catch (IOException e) { @@ -880,14 +884,14 @@ public class PartitionedMobCompactor extends MobCompactor { * @return The pair of the max seqId and number of cells of the store files. * @throws IOException if IO failure is encountered */ - private Pair getFileInfo(List storeFiles) throws IOException { + private Pair getFileInfo(List storeFiles) throws IOException { long maxSeqId = 0; long maxKeyCount = 0; - for (StoreFile sf : storeFiles) { + for (HStoreFile sf : storeFiles) { // the readers will be closed later after the merge. maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId()); sf.initReader(); - byte[] count = sf.getReader().loadFileInfo().get(StoreFile.MOB_CELLS_COUNT); + byte[] count = sf.getReader().loadFileInfo().get(MOB_CELLS_COUNT); if (count != null) { maxKeyCount += Bytes.toLong(count); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java index 07b72e283a5..065fd37848f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ChangedReadersObserver.java @@ -16,11 +16,11 @@ * 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.List; + import org.apache.yetus.audience.InterfaceAudience; /** @@ -40,5 +40,5 @@ public interface ChangedReadersObserver { * @param memStoreScanners scanner of current memstore * @throws IOException e */ - void updateReaders(List sfs, List memStoreScanners) throws IOException; + void updateReaders(List sfs, List memStoreScanners) throws IOException; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java index c31a1cf2b13..da502c2fb85 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java @@ -69,7 +69,7 @@ public class CompactingMemStore extends AbstractMemStore { private static final double IN_MEMORY_FLUSH_THRESHOLD_FACTOR_DEFAULT = 0.25; private static final Log LOG = LogFactory.getLog(CompactingMemStore.class); - private Store store; + private HStore store; private RegionServicesForStores regionServices; private CompactionPipeline pipeline; private MemStoreCompactor compactor; @@ -341,7 +341,7 @@ public class CompactingMemStore extends AbstractMemStore { return store.getSmallestReadPoint(); } - public Store getStore() { + public HStore getStore() { return store; } 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 25e1609129f..f7c18f94719 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 @@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.security.User; public class DateTieredStoreEngine extends StoreEngine { @Override - public boolean needsCompaction(List filesCompacting) { + public boolean needsCompaction(List filesCompacting) { return compactionPolicy.needsCompaction(storeFileManager.getStorefiles(), filesCompacting); } @@ -54,7 +54,7 @@ public class DateTieredStoreEngine extends StoreEngine preSelect(List filesCompacting) { + public List preSelect(List filesCompacting) { return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStorefiles(), filesCompacting); } @Override - public boolean select(List filesCompacting, boolean isUserCompaction, + public boolean select(List filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak, boolean forceMajor) throws IOException { request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting, isUserCompaction, mayUseOffPeak, forceMajor); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java index 70421e1d349..58f8bbbb6ac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreEngine.java @@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseInterfaceAudience; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy; @@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPoli import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.ReflectionUtils; +import org.apache.yetus.audience.InterfaceAudience; /** * Default StoreEngine creates the default compactor, policy, and store file manager, or @@ -57,14 +57,14 @@ public class DefaultStoreEngine extends StoreEngine< DEFAULT_COMPACTION_POLICY_CLASS = ExploringCompactionPolicy.class; @Override - public boolean needsCompaction(List filesCompacting) { + public boolean needsCompaction(List filesCompacting) { return compactionPolicy.needsCompaction( this.storeFileManager.getStorefiles(), filesCompacting); } @Override protected void createComponents( - Configuration conf, Store store, CellComparator kvComparator) throws IOException { + Configuration conf, HStore store, CellComparator kvComparator) throws IOException { createCompactor(conf, store); createCompactionPolicy(conf, store); createStoreFlusher(conf, store); @@ -73,17 +73,17 @@ public class DefaultStoreEngine extends StoreEngine< compactionPolicy.getConf()); } - protected void createCompactor(Configuration conf, Store store) throws IOException { + protected void createCompactor(Configuration conf, HStore store) throws IOException { String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName()); try { compactor = ReflectionUtils.instantiateWithCustomCtor(className, - new Class[] { Configuration.class, Store.class }, new Object[] { conf, store }); + new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store }); } catch (Exception e) { throw new IOException("Unable to load configured compactor '" + className + "'", e); } } - protected void createCompactionPolicy(Configuration conf, Store store) throws IOException { + protected void createCompactionPolicy(Configuration conf, HStore store) throws IOException { String className = conf.get( DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName()); try { @@ -95,12 +95,12 @@ public class DefaultStoreEngine extends StoreEngine< } } - protected void createStoreFlusher(Configuration conf, Store store) throws IOException { + protected void createStoreFlusher(Configuration conf, HStore store) throws IOException { String className = conf.get( DEFAULT_STORE_FLUSHER_CLASS_KEY, DEFAULT_STORE_FLUSHER_CLASS.getName()); try { storeFlusher = ReflectionUtils.instantiateWithCustomCtor(className, - new Class[] { Configuration.class, Store.class }, new Object[] { conf, store }); + new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store }); } catch (Exception e) { throw new IOException("Unable to load configured store flusher '" + className + "'", e); } @@ -113,7 +113,7 @@ public class DefaultStoreEngine extends StoreEngine< private class DefaultCompactionContext extends CompactionContext { @Override - public boolean select(List filesCompacting, boolean isUserCompaction, + public boolean select(List filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak, boolean forceMajor) throws IOException { request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting, isUserCompaction, mayUseOffPeak, forceMajor); @@ -127,7 +127,7 @@ public class DefaultStoreEngine extends StoreEngine< } @Override - public List preSelect(List filesCompacting) { + public List preSelect(List filesCompacting) { return compactionPolicy.preSelectCompactionForCoprocessor( storeFileManager.getStorefiles(), filesCompacting); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java index 95945c69cea..e1f31bbe444 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFileManager.java @@ -19,13 +19,12 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -33,12 +32,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.KeyValue; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables; /** * Default implementation of StoreFileManager. Not thread-safe. @@ -47,27 +46,27 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; class DefaultStoreFileManager implements StoreFileManager { private static final Log LOG = LogFactory.getLog(DefaultStoreFileManager.class); - private final CellComparator kvComparator; + private final CellComparator cellComparator; private final CompactionConfiguration comConf; private final int blockingFileCount; - private final Comparator storeFileComparator; + private final Comparator storeFileComparator; /** * List of store files inside this store. This is an immutable list that * is atomically replaced when its contents change. */ - private volatile ImmutableList storefiles = null; + private volatile ImmutableList storefiles = ImmutableList.of(); /** * List of compacted files inside this store that needs to be excluded in reads * because further new reads will be using only the newly created files out of compaction. * These compacted files will be deleted/cleared once all the existing readers on these * compacted files are done. */ - private volatile List compactedfiles = null; + private volatile ImmutableList compactedfiles = ImmutableList.of(); - public DefaultStoreFileManager(CellComparator kvComparator, - Comparator storeFileComparator, Configuration conf, + public DefaultStoreFileManager(CellComparator cellComparator, + Comparator storeFileComparator, Configuration conf, CompactionConfiguration comConf) { - this.kvComparator = kvComparator; + this.cellComparator = cellComparator; this.storeFileComparator = storeFileComparator; this.comConf = comConf; this.blockingFileCount = @@ -75,39 +74,37 @@ class DefaultStoreFileManager implements StoreFileManager { } @Override - public void loadFiles(List storeFiles) { - sortAndSetStoreFiles(storeFiles); + public void loadFiles(List storeFiles) { + this.storefiles = ImmutableList.sortedCopyOf(storeFileComparator, storeFiles); } @Override - public final Collection getStorefiles() { - // TODO: I can return a null list of StoreFiles? That'll mess up clients. St.Ack 20151111 + public final Collection getStorefiles() { return storefiles; } @Override - public Collection getCompactedfiles() { + public Collection getCompactedfiles() { return compactedfiles; } @Override - public void insertNewFiles(Collection sfs) throws IOException { - ArrayList newFiles = new ArrayList<>(storefiles); - newFiles.addAll(sfs); - sortAndSetStoreFiles(newFiles); + public void insertNewFiles(Collection sfs) throws IOException { + this.storefiles = + ImmutableList.sortedCopyOf(storeFileComparator, Iterables.concat(this.storefiles, sfs)); } @Override - public ImmutableCollection clearFiles() { - ImmutableList result = storefiles; + public ImmutableCollection clearFiles() { + ImmutableList result = storefiles; storefiles = ImmutableList.of(); return result; } @Override - public Collection clearCompactedFiles() { - List result = compactedfiles; - compactedfiles = new ArrayList<>(); + public Collection clearCompactedFiles() { + List result = compactedfiles; + compactedfiles = ImmutableList.of(); return result; } @@ -118,60 +115,39 @@ class DefaultStoreFileManager implements StoreFileManager { @Override public final int getCompactedFilesCount() { - if (compactedfiles == null) { - return 0; - } return compactedfiles.size(); } @Override - public void addCompactionResults( - Collection newCompactedfiles, Collection results) { - ArrayList newStoreFiles = Lists.newArrayList(storefiles); - newStoreFiles.removeAll(newCompactedfiles); - if (!results.isEmpty()) { - newStoreFiles.addAll(results); - } - sortAndSetStoreFiles(newStoreFiles); - ArrayList updatedCompactedfiles = null; - if (this.compactedfiles != null) { - updatedCompactedfiles = new ArrayList<>(this.compactedfiles); - updatedCompactedfiles.addAll(newCompactedfiles); - } else { - updatedCompactedfiles = new ArrayList<>(newCompactedfiles); - } - markCompactedAway(newCompactedfiles); - this.compactedfiles = sortCompactedfiles(updatedCompactedfiles); - } - - // Mark the files as compactedAway once the storefiles and compactedfiles list is finalized - // Let a background thread close the actual reader on these compacted files and also - // ensure to evict the blocks from block cache so that they are no longer in - // cache - private void markCompactedAway(Collection compactedFiles) { - for (StoreFile file : compactedFiles) { - file.markCompactedAway(); - } + public void addCompactionResults(Collection newCompactedfiles, + Collection results) { + this.storefiles = ImmutableList.sortedCopyOf(storeFileComparator, Iterables + .concat(Iterables.filter(storefiles, sf -> !newCompactedfiles.contains(sf)), results)); + // Mark the files as compactedAway once the storefiles and compactedfiles list is finalized + // Let a background thread close the actual reader on these compacted files and also + // ensure to evict the blocks from block cache so that they are no longer in + // cache + newCompactedfiles.forEach(HStoreFile::markCompactedAway); + this.compactedfiles = ImmutableList.sortedCopyOf(storeFileComparator, + Iterables.concat(this.compactedfiles, newCompactedfiles)); } @Override - public void removeCompactedFiles(Collection removedCompactedfiles) throws IOException { - ArrayList updatedCompactedfiles = null; - if (this.compactedfiles != null) { - updatedCompactedfiles = new ArrayList<>(this.compactedfiles); - updatedCompactedfiles.removeAll(removedCompactedfiles); - this.compactedfiles = sortCompactedfiles(updatedCompactedfiles); - } + public void removeCompactedFiles(Collection removedCompactedfiles) + throws IOException { + this.compactedfiles = + this.compactedfiles.stream().filter(sf -> !removedCompactedfiles.contains(sf)) + .sorted(storeFileComparator).collect(ImmutableList.toImmutableList()); } @Override - public final Iterator getCandidateFilesForRowKeyBefore(final KeyValue targetKey) { - return new ArrayList<>(Lists.reverse(this.storefiles)).iterator(); + public final Iterator getCandidateFilesForRowKeyBefore(KeyValue targetKey) { + return this.storefiles.reverse().iterator(); } @Override - public Iterator updateCandidateFilesForRowKeyBefore( - Iterator candidateFiles, final KeyValue targetKey, final Cell candidate) { + public Iterator updateCandidateFilesForRowKeyBefore( + Iterator candidateFiles, KeyValue targetKey, Cell candidate) { // Default store has nothing useful to do here. // TODO: move this comment when implementing Level: // Level store can trim the list by range, removing all the files which cannot have @@ -180,18 +156,12 @@ class DefaultStoreFileManager implements StoreFileManager { } @Override - public final byte[] getSplitPoint() throws IOException { - List storefiles = this.storefiles; - if (storefiles.isEmpty()) { - return null; - } - Optional largestFile = StoreUtils.getLargestFile(storefiles); - return largestFile.isPresent() - ? StoreUtils.getFileSplitPoint(largestFile.get(), kvComparator).orElse(null) : null; + public final Optional getSplitPoint() throws IOException { + return StoreUtils.getSplitPoint(storefiles, cellComparator); } @Override - public final Collection getFilesForScan(byte[] startRow, boolean includeStartRow, + public final Collection getFilesForScan(byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow) { // We cannot provide any useful input and already have the files sorted by seqNum. return getStorefiles(); @@ -204,35 +174,20 @@ class DefaultStoreFileManager implements StoreFileManager { } @Override - public Collection getUnneededFiles(long maxTs, List filesCompacting) { - Collection expiredStoreFiles = null; - ImmutableList files = storefiles; + public Collection getUnneededFiles(long maxTs, List filesCompacting) { + ImmutableList files = storefiles; // 1) We can never get rid of the last file which has the maximum seqid. // 2) Files that are not the latest can't become one due to (1), so the rest are fair game. - for (int i = 0; i < files.size() - 1; ++i) { - StoreFile sf = files.get(i); + return files.stream().limit(Math.max(0, files.size() - 1)).filter(sf -> { long fileTs = sf.getReader().getMaxTimestamp(); if (fileTs < maxTs && !filesCompacting.contains(sf)) { - LOG.info("Found an expired store file: " + sf.getPath() - + " whose maxTimeStamp is " + fileTs + ", which is below " + maxTs); - if (expiredStoreFiles == null) { - expiredStoreFiles = new ArrayList<>(); - } - expiredStoreFiles.add(sf); + LOG.info("Found an expired store file: " + sf.getPath() + " whose maxTimeStamp is " + + fileTs + ", which is below " + maxTs); + return true; + } else { + return false; } - } - return expiredStoreFiles; - } - - private void sortAndSetStoreFiles(List storeFiles) { - Collections.sort(storeFiles, storeFileComparator); - storefiles = ImmutableList.copyOf(storeFiles); - } - - private List sortCompactedfiles(List storefiles) { - // Sorting may not be really needed here for the compacted files? - Collections.sort(storefiles, storeFileComparator); - return new ArrayList<>(storefiles); + }).collect(Collectors.toList()); } @Override @@ -246,7 +201,7 @@ class DefaultStoreFileManager implements StoreFileManager { } @Override - public Comparator getStoreFileComparator() { + public Comparator getStoreFileComparator() { return storeFileComparator; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java index 14c3f921450..a5dd9f71094 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultStoreFlusher.java @@ -39,7 +39,7 @@ public class DefaultStoreFlusher extends StoreFlusher { private static final Log LOG = LogFactory.getLog(DefaultStoreFlusher.class); private final Object flushLock = new Object(); - public DefaultStoreFlusher(Configuration conf, Store store) { + public DefaultStoreFlusher(Configuration conf, HStore store) { super(conf, store); } 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 43e4a172c6a..95bbf743a99 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 @@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagUtil; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.filter.Filter; @@ -59,6 +58,7 @@ import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.hbase.util.IdLock; +import org.apache.yetus.audience.InterfaceAudience; /** * The store implementation to save MOBs (medium objects), it extends the HStore. @@ -166,7 +166,7 @@ public class HMobStore extends HStore { * Creates the mob store engine. */ @Override - protected StoreEngine createStoreEngine(Store store, Configuration conf, + protected StoreEngine createStoreEngine(HStore store, Configuration conf, CellComparator cellComparator) throws IOException { MobStoreEngine engine = new MobStoreEngine(); engine.createComponents(conf, store, cellComparator); @@ -291,7 +291,7 @@ public class HMobStore extends HStore { * @param path the path to the mob file */ private void validateMobFile(Path path) throws IOException { - StoreFile storeFile = null; + HStoreFile storeFile = null; try { storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig, BloomType.NONE, isPrimaryReplicaStore()); @@ -301,7 +301,7 @@ public class HMobStore extends HStore { throw e; } finally { if (storeFile != null) { - storeFile.closeReader(false); + storeFile.closeStoreFile(false); } } } 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 9554d7fbc3b..4fa2c704ebc 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY; import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent; import java.io.EOFException; @@ -100,7 +101,6 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.UnknownScannerException; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Delete; @@ -147,9 +147,34 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; +import org.apache.hadoop.hbase.snapshot.SnapshotManifest; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CancelableProgressable; +import org.apache.hadoop.hbase.util.ClassSize; +import org.apache.hadoop.hbase.util.CollectionUtils; +import org.apache.hadoop.hbase.util.CompressionTest; +import org.apache.hadoop.hbase.util.EncryptionTest; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.HashedBytes; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.hadoop.hbase.wal.WALSplitter; +import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay; +import org.apache.hadoop.io.MultipleIOException; +import org.apache.hadoop.util.StringUtils; +import org.apache.htrace.Trace; +import org.apache.htrace.TraceScope; +import org.apache.yetus.audience.InterfaceAudience; + import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; @@ -172,29 +197,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.snapshot.SnapshotDescriptionUtils; -import org.apache.hadoop.hbase.snapshot.SnapshotManifest; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.CancelableProgressable; -import org.apache.hadoop.hbase.util.ClassSize; -import org.apache.hadoop.hbase.util.CollectionUtils; -import org.apache.hadoop.hbase.util.CompressionTest; -import org.apache.hadoop.hbase.util.EncryptionTest; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.util.FSUtils; -import org.apache.hadoop.hbase.util.HashedBytes; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.wal.WALKey; -import org.apache.hadoop.hbase.wal.WALSplitter; -import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay; -import org.apache.hadoop.io.MultipleIOException; -import org.apache.hadoop.util.StringUtils; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; @SuppressWarnings("deprecation") @InterfaceAudience.Private @@ -1066,12 +1068,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi private NavigableMap> getStoreFiles() { NavigableMap> allStoreFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR); for (HStore store : stores.values()) { - Collection storeFiles = store.getStorefiles(); + Collection storeFiles = store.getStorefiles(); if (storeFiles == null) { continue; } List storeFileNames = new ArrayList<>(); - for (StoreFile storeFile : storeFiles) { + for (HStoreFile storeFile : storeFiles) { storeFileNames.add(storeFile.getPath()); } allStoreFiles.put(store.getColumnFamilyDescriptor().getName(), storeFileNames); @@ -1124,7 +1126,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi public HDFSBlocksDistribution getHDFSBlocksDistribution() { HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); stores.values().stream().filter(s -> s.getStorefiles() != null) - .flatMap(s -> s.getStorefiles().stream()).map(StoreFile::getHDFSBlockDistribution) + .flatMap(s -> s.getStorefiles().stream()).map(HStoreFile::getHDFSBlockDistribution) .forEachOrdered(hdfsBlocksDistribution::add); return hdfsBlocksDistribution; } @@ -1384,7 +1386,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi LOG.info("DEBUG LIST ALL FILES"); for (HStore store : this.stores.values()) { LOG.info("store " + store.getColumnFamilyName()); - for (StoreFile sf : store.getStorefiles()) { + for (HStoreFile sf : store.getStorefiles()) { LOG.info(sf.toStringDetailed()); } } @@ -1458,7 +1460,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * because a Snapshot was not properly persisted. The region is put in closing mode, and the * caller MUST abort after this. */ - public Map> close() throws IOException { + public Map> close() throws IOException { return close(false); } @@ -1499,7 +1501,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * because a Snapshot was not properly persisted. The region is put in closing mode, and the * caller MUST abort after this. */ - public Map> close(final boolean abort) throws IOException { + public Map> close(boolean abort) throws IOException { // Only allow one thread to close at a time. Serialize them so dual // threads attempting to close will run up against each other. MonitoredTask status = TaskMonitor.get().createStatus( @@ -1537,7 +1539,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH", justification="I think FindBugs is confused") - private Map> doClose(final boolean abort, MonitoredTask status) + private Map> doClose(boolean abort, MonitoredTask status) throws IOException { if (isClosed()) { LOG.warn("Region " + this + " already closed"); @@ -1632,13 +1634,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - Map> result = new TreeMap<>(Bytes.BYTES_COMPARATOR); + Map> result = new TreeMap<>(Bytes.BYTES_COMPARATOR); if (!stores.isEmpty()) { // initialize the thread pool for closing stores in parallel. ThreadPoolExecutor storeCloserThreadPool = getStoreOpenAndCloseThreadPool("StoreCloserThread-" + getRegionInfo().getRegionNameAsString()); - CompletionService>> completionService = + CompletionService>> completionService = new ExecutorCompletionService<>(storeCloserThreadPool); // close each store in parallel @@ -1654,18 +1656,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } completionService - .submit(new Callable>>() { + .submit(new Callable>>() { @Override - public Pair> call() throws IOException { + public Pair> call() throws IOException { return new Pair<>(store.getColumnFamilyDescriptor().getName(), store.close()); } }); } try { for (int i = 0; i < stores.size(); i++) { - Future>> future = completionService.take(); - Pair> storeFiles = future.get(); - List familyFiles = result.get(storeFiles.getFirst()); + Future>> future = completionService.take(); + Pair> storeFiles = future.get(); + List familyFiles = result.get(storeFiles.getFirst()); if (familyFiles == null) { familyFiles = new ArrayList<>(); result.put(storeFiles.getFirst(), familyFiles); @@ -1874,11 +1876,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException { long result = Long.MAX_VALUE; for (HStore store : stores.values()) { - Collection storeFiles = store.getStorefiles(); + Collection storeFiles = store.getStorefiles(); if (storeFiles == null) { continue; } - for (StoreFile file : storeFiles) { + for (HStoreFile file : storeFiles) { StoreFileReader sfReader = file.getReader(); if (sfReader == null) { continue; @@ -1888,7 +1890,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi continue; } if (majorCompactionOnly) { - byte[] val = reader.loadFileInfo().get(StoreFile.MAJOR_COMPACTION_KEY); + byte[] val = reader.loadFileInfo().get(MAJOR_COMPACTION_KEY); if (val == null || !Bytes.toBoolean(val)) { continue; } @@ -4182,7 +4184,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // If this flag is set, make use of the hfile archiving by making recovered.edits a fake // column family. Have to fake out file type too by casting our recovered.edits as storefiles String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regiondir).getName(); - Set fakeStoreFiles = new HashSet<>(files.size()); + Set fakeStoreFiles = new HashSet<>(files.size()); for (Path file: files) { fakeStoreFiles.add( new HStoreFile(getRegionFileSystem().getFileSystem(), file, this.conf, null, null, true)); @@ -5296,11 +5298,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi throw new IllegalArgumentException( "No column family : " + new String(column) + " available"); } - Collection storeFiles = store.getStorefiles(); + Collection storeFiles = store.getStorefiles(); if (storeFiles == null) { continue; } - for (StoreFile storeFile : storeFiles) { + for (HStoreFile storeFile : storeFiles) { storeFileNames.add(storeFile.getPath().toString()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java index 3cb5bdbe78f..3f4246688c1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java @@ -25,7 +25,7 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.Map; +import java.util.Optional; import java.util.UUID; import org.apache.commons.logging.Log; @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.backup.HFileArchiver; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.fs.HFileSystem; @@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.util.FSHDFSUtils; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; @@ -481,20 +481,6 @@ public class HRegionFileSystem { return dstPath; } - /** - * Moves multiple store files to the relative region's family store directory. - * @param storeFiles list of store files divided by family - * @throws IOException - */ - void commitStoreFiles(final Map> storeFiles) throws IOException { - for (Map.Entry> es: storeFiles.entrySet()) { - String familyName = Bytes.toString(es.getKey()); - for (StoreFile sf: es.getValue()) { - commitStoreFile(familyName, sf.getPath()); - } - } - } - /** * Archives the specified store file from the specified family. * @param familyName Family that contains the store files @@ -513,7 +499,7 @@ public class HRegionFileSystem { * @param storeFiles set of store files to remove * @throws IOException if the archiving fails */ - public void removeStoreFiles(final String familyName, final Collection storeFiles) + public void removeStoreFiles(String familyName, Collection storeFiles) throws IOException { HFileArchiver.archiveStoreFiles(this.conf, this.fs, this.regionInfoForFs, this.tableDir, Bytes.toBytes(familyName), storeFiles); @@ -671,9 +657,8 @@ public class HRegionFileSystem { * @return Path to created reference. * @throws IOException */ - public Path splitStoreFile(final HRegionInfo hri, final String familyName, final StoreFile f, - final byte[] splitRow, final boolean top, RegionSplitPolicy splitPolicy) - throws IOException { + public Path splitStoreFile(HRegionInfo hri, String familyName, HStoreFile f, byte[] splitRow, + boolean top, RegionSplitPolicy splitPolicy) throws IOException { if (splitPolicy == null || !splitPolicy.skipStoreFileRangeCheck(familyName)) { // Check whether the split row lies in the range of the store file // If it is outside the range, return directly. @@ -682,28 +667,28 @@ public class HRegionFileSystem { if (top) { //check if larger than last key. Cell splitKey = CellUtil.createFirstOnRow(splitRow); - Cell lastKey = f.getLastKey(); + Optional lastKey = f.getLastKey(); // If lastKey is null means storefile is empty. - if (lastKey == null) { + if (!lastKey.isPresent()) { return null; } - if (f.getComparator().compare(splitKey, lastKey) > 0) { + if (f.getComparator().compare(splitKey, lastKey.get()) > 0) { return null; } } else { //check if smaller than first key Cell splitKey = CellUtil.createLastOnRow(splitRow); - Cell firstKey = f.getFirstKey(); + Optional firstKey = f.getFirstKey(); // If firstKey is null means storefile is empty. - if (firstKey == null) { + if (!firstKey.isPresent()) { return null; } - if (f.getComparator().compare(splitKey, firstKey) < 0) { + if (f.getComparator().compare(splitKey, firstKey.get()) < 0) { return null; } } } finally { - f.closeReader(f.getCacheConf() != null ? f.getCacheConf().shouldEvictOnClose() : true); + f.closeStoreFile(f.getCacheConf() != null ? f.getCacheConf().shouldEvictOnClose() : true); } } @@ -791,9 +776,8 @@ public class HRegionFileSystem { * @return Path to created reference. * @throws IOException */ - public Path mergeStoreFile(final HRegionInfo mergedRegion, final String familyName, - final StoreFile f, final Path mergedDir) - throws IOException { + public Path mergeStoreFile(HRegionInfo mergedRegion, String familyName, HStoreFile f, + Path mergedDir) throws IOException { Path referenceDir = new Path(new Path(mergedDir, mergedRegion.getEncodedName()), familyName); // A whole reference to the store file. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index f648c2fbff0..394826c5160 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -18,7 +18,6 @@ */ package org.apache.hadoop.hbase.regionserver; - import java.io.IOException; import java.io.InterruptedIOException; import java.lang.Thread.UncaughtExceptionHandler; @@ -84,7 +83,6 @@ import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.YouAreDeadException; import org.apache.hadoop.hbase.ZNodeClearer; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionUtils; @@ -141,6 +139,44 @@ import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.UserProvider; +import org.apache.hadoop.hbase.trace.SpanReceiverHost; +import org.apache.hadoop.hbase.util.Addressing; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CompressionTest; +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.hbase.util.HasThread; +import org.apache.hadoop.hbase.util.JSONBean; +import org.apache.hadoop.hbase.util.JvmPauseMonitor; +import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; +import org.apache.hadoop.hbase.util.Sleeper; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.hadoop.hbase.util.VersionInfo; +import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; +import org.apache.hadoop.hbase.wal.NettyAsyncFSWALConfigHelper; +import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALFactory; +import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; +import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; +import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; +import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher; +import org.apache.hadoop.hbase.zookeeper.ZKClusterId; +import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.hadoop.ipc.RemoteException; +import org.apache.hadoop.metrics2.util.MBeans; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.NoNodeException; +import org.apache.zookeeper.data.Stat; + import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps; @@ -177,42 +213,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse; -import org.apache.hadoop.hbase.trace.SpanReceiverHost; -import org.apache.hadoop.hbase.util.Addressing; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.CompressionTest; -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.hbase.util.HasThread; -import org.apache.hadoop.hbase.util.JSONBean; -import org.apache.hadoop.hbase.util.JvmPauseMonitor; -import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; -import org.apache.hadoop.hbase.util.Sleeper; -import org.apache.hadoop.hbase.util.Threads; -import org.apache.hadoop.hbase.util.VersionInfo; -import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; -import org.apache.hadoop.hbase.wal.NettyAsyncFSWALConfigHelper; -import org.apache.hadoop.hbase.wal.WAL; -import org.apache.hadoop.hbase.wal.WALFactory; -import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker; -import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; -import org.apache.hadoop.hbase.zookeeper.MetaTableLocator; -import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher; -import org.apache.hadoop.hbase.zookeeper.ZKClusterId; -import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.hadoop.ipc.RemoteException; -import org.apache.hadoop.metrics2.util.MBeans; -import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hadoop.util.StringUtils; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.KeeperException.NoNodeException; -import org.apache.zookeeper.data.Stat; import sun.misc.Signal; import sun.misc.SignalHandler; @@ -3172,7 +3172,7 @@ public class HRegionServer extends HasThread implements for (int i = 0; i < regionEncodedName.size(); ++i) { Region regionToClose = this.getFromOnlineRegions(regionEncodedName.get(i)); if (regionToClose != null) { - Map> hstoreFiles = null; + Map> hstoreFiles = null; Exception exceptionToThrow = null; try{ hstoreFiles = ((HRegion)regionToClose).close(false); 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 daad24114b2..de41087c042 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 @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; import java.util.Optional; +import java.util.OptionalDouble; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.CompletionService; @@ -43,6 +44,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Predicate; +import java.util.function.ToLongFunction; +import java.util.stream.Collectors; +import java.util.stream.LongStream; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -171,7 +175,7 @@ public class HStore implements Store { private ScanInfo scanInfo; // TODO: ideally, this should be part of storeFileManager, as we keep passing this to it. - final List filesCompacting = Lists.newArrayList(); + final List filesCompacting = Lists.newArrayList(); // All access must be synchronized. private final Set changedReaderObservers = @@ -335,7 +339,7 @@ public class HStore implements Store { * @param kvComparator KVComparator for storeFileManager. * @return StoreEngine to use. */ - protected StoreEngine createStoreEngine(Store store, Configuration conf, + protected StoreEngine createStoreEngine(HStore store, Configuration conf, CellComparator kvComparator) throws IOException { return StoreEngine.create(store, conf, comparator); } @@ -517,12 +521,12 @@ public class HStore implements Store { * from the given directory. * @throws IOException */ - private List loadStoreFiles() throws IOException { + private List loadStoreFiles() throws IOException { Collection files = fs.getStoreFiles(getColumnFamilyName()); return openStoreFiles(files); } - private List openStoreFiles(Collection files) throws IOException { + private List openStoreFiles(Collection files) throws IOException { if (files == null || files.isEmpty()) { return new ArrayList<>(); } @@ -530,28 +534,21 @@ public class HStore implements Store { ThreadPoolExecutor storeFileOpenerThreadPool = this.region.getStoreFileOpenAndCloseThreadPool("StoreFileOpenerThread-" + this.getColumnFamilyName()); - CompletionService completionService = new ExecutorCompletionService<>(storeFileOpenerThreadPool); + CompletionService completionService = new ExecutorCompletionService<>(storeFileOpenerThreadPool); int totalValidStoreFile = 0; - for (final StoreFileInfo storeFileInfo: files) { + for (StoreFileInfo storeFileInfo : files) { // open each store file in parallel - completionService.submit(new Callable() { - @Override - public StoreFile call() throws IOException { - StoreFile storeFile = createStoreFileAndReader(storeFileInfo); - return storeFile; - } - }); + completionService.submit(() -> this.createStoreFileAndReader(storeFileInfo)); totalValidStoreFile++; } - ArrayList results = new ArrayList<>(files.size()); + ArrayList results = new ArrayList<>(files.size()); IOException ioe = null; try { for (int i = 0; i < totalValidStoreFile; i++) { try { - Future future = completionService.take(); - StoreFile storeFile = future.get(); + HStoreFile storeFile = completionService.take().get(); if (storeFile != null) { long length = storeFile.getReader().length(); this.storeSize += length; @@ -574,9 +571,9 @@ public class HStore implements Store { // close StoreFile readers boolean evictOnClose = cacheConf != null? cacheConf.shouldEvictOnClose(): true; - for (StoreFile file : results) { + for (HStoreFile file : results) { try { - if (file != null) file.closeReader(evictOnClose); + if (file != null) file.closeStoreFile(evictOnClose); } catch (IOException e) { LOG.warn(e.getMessage()); } @@ -618,19 +615,18 @@ public class HStore implements Store { */ private void refreshStoreFilesInternal(Collection newFiles) throws IOException { StoreFileManager sfm = storeEngine.getStoreFileManager(); - Collection currentFiles = sfm.getStorefiles(); - Collection compactedFiles = sfm.getCompactedfiles(); + Collection currentFiles = sfm.getStorefiles(); + Collection compactedFiles = sfm.getCompactedfiles(); if (currentFiles == null) currentFiles = Collections.emptySet(); if (newFiles == null) newFiles = Collections.emptySet(); if (compactedFiles == null) compactedFiles = Collections.emptySet(); - HashMap currentFilesSet = new HashMap<>(currentFiles.size()); - for (StoreFile sf : currentFiles) { + HashMap currentFilesSet = new HashMap<>(currentFiles.size()); + for (HStoreFile sf : currentFiles) { currentFilesSet.put(sf.getFileInfo(), sf); } - HashMap compactedFilesSet = - new HashMap(compactedFiles.size()); - for (StoreFile sf : compactedFiles) { + HashMap compactedFilesSet = new HashMap<>(compactedFiles.size()); + for (HStoreFile sf : compactedFiles) { compactedFilesSet.put(sf.getFileInfo(), sf); } @@ -647,13 +643,13 @@ public class HStore implements Store { LOG.info("Refreshing store files for region " + this.getRegionInfo().getRegionNameAsString() + " files to add: " + toBeAddedFiles + " files to remove: " + toBeRemovedFiles); - Set toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size()); + Set toBeRemovedStoreFiles = new HashSet<>(toBeRemovedFiles.size()); for (StoreFileInfo sfi : toBeRemovedFiles) { toBeRemovedStoreFiles.add(currentFilesSet.get(sfi)); } // try to open the files - List openedFiles = openStoreFiles(toBeAddedFiles); + List openedFiles = openStoreFiles(toBeAddedFiles); // propogate the file changes to the underlying store file manager replaceStoreFiles(toBeRemovedStoreFiles, openedFiles); //won't throw an exception @@ -668,14 +664,14 @@ public class HStore implements Store { completeCompaction(toBeRemovedStoreFiles); } - private StoreFile createStoreFileAndReader(final Path p) throws IOException { + private HStoreFile createStoreFileAndReader(final Path p) throws IOException { StoreFileInfo info = new StoreFileInfo(conf, this.getFileSystem(), p); return createStoreFileAndReader(info); } - private StoreFile createStoreFileAndReader(final StoreFileInfo info) throws IOException { + private HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException { info.setRegionCoprocessorHost(this.region.getCoprocessorHost()); - StoreFile storeFile = new HStoreFile(this.getFileSystem(), info, this.conf, this.cacheConf, + HStoreFile storeFile = new HStoreFile(this.getFileSystem(), info, this.conf, this.cacheConf, this.family.getBloomFilterType(), isPrimaryReplicaStore()); storeFile.initReader(); return storeFile; @@ -734,12 +730,12 @@ public class HStore implements Store { * @return All store files. */ @Override - public Collection getStorefiles() { + public Collection getStorefiles() { return this.storeEngine.getStoreFileManager().getStorefiles(); } @Override - public Collection getCompactedFiles() { + public Collection getCompactedFiles() { return this.storeEngine.getStoreFileManager().getCompactedfiles(); } @@ -756,19 +752,19 @@ public class HStore implements Store { isPrimaryReplicaStore(), conf); reader.loadFileInfo(); - byte[] firstKey = reader.getFirstRowKey(); - Preconditions.checkState(firstKey != null, "First key can not be null"); - Cell lk = reader.getLastKey(); - Preconditions.checkState(lk != null, "Last key can not be null"); - byte[] lastKey = CellUtil.cloneRow(lk); + Optional firstKey = reader.getFirstRowKey(); + Preconditions.checkState(firstKey.isPresent(), "First key can not be null"); + Optional lk = reader.getLastKey(); + Preconditions.checkState(lk.isPresent(), "Last key can not be null"); + byte[] lastKey = CellUtil.cloneRow(lk.get()); - LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey) + + LOG.debug("HFile bounds: first=" + Bytes.toStringBinary(firstKey.get()) + " last=" + Bytes.toStringBinary(lastKey)); LOG.debug("Region bounds: first=" + Bytes.toStringBinary(getRegionInfo().getStartKey()) + " last=" + Bytes.toStringBinary(getRegionInfo().getEndKey())); - if (!this.getRegionInfo().containsRange(firstKey, lastKey)) { + if (!this.getRegionInfo().containsRange(firstKey.get(), lastKey)) { throw new WrongRegionException( "Bulk load file " + srcPath.toString() + " does not fit inside region " + this.getRegionInfo().getRegionNameAsString()); @@ -842,7 +838,7 @@ public class HStore implements Store { LOG.info("Loaded HFile " + srcPath + " into store '" + getColumnFamilyName() + "' as " + dstPath + " - updating store file list."); - StoreFile sf = createStoreFileAndReader(dstPath); + HStoreFile sf = createStoreFileAndReader(dstPath); bulkLoadHFile(sf); LOG.info("Successfully loaded store file " + srcPath + " into store " + this @@ -852,11 +848,11 @@ public class HStore implements Store { } public void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException { - StoreFile sf = createStoreFileAndReader(fileInfo); + HStoreFile sf = createStoreFileAndReader(fileInfo); bulkLoadHFile(sf); } - private void bulkLoadHFile(StoreFile sf) throws IOException { + private void bulkLoadHFile(HStoreFile sf) throws IOException { StoreFileReader r = sf.getReader(); this.storeSize += r.length(); this.totalUncompressedBytes += r.getTotalUncompressedBytes(); @@ -883,13 +879,13 @@ public class HStore implements Store { } @Override - public ImmutableCollection close() throws IOException { + public ImmutableCollection close() throws IOException { this.archiveLock.lock(); this.lock.writeLock().lock(); try { // Clear so metrics doesn't find them. - ImmutableCollection result = storeEngine.getStoreFileManager().clearFiles(); - Collection compactedfiles = + ImmutableCollection result = storeEngine.getStoreFileManager().clearFiles(); + Collection compactedfiles = storeEngine.getStoreFileManager().clearCompactedFiles(); // clear the compacted files if (compactedfiles != null && !compactedfiles.isEmpty()) { @@ -904,13 +900,13 @@ public class HStore implements Store { // close each store file in parallel CompletionService completionService = new ExecutorCompletionService<>(storeFileCloserThreadPool); - for (final StoreFile f : result) { + for (HStoreFile f : result) { completionService.submit(new Callable() { @Override public Void call() throws IOException { boolean evictOnClose = cacheConf != null? cacheConf.shouldEvictOnClose(): true; - f.closeReader(evictOnClose); + f.closeStoreFile(evictOnClose); return null; } }); @@ -1012,20 +1008,20 @@ public class HStore implements Store { throw lastException; } - /* + /** * @param path The pathname of the tmp file into which the store was flushed * @param logCacheFlushId * @param status - * @return StoreFile created. + * @return store file created. * @throws IOException */ - private StoreFile commitFile(final Path path, final long logCacheFlushId, MonitoredTask status) + private HStoreFile commitFile(Path path, long logCacheFlushId, MonitoredTask status) throws IOException { // Write-out finished successfully, move into the right spot Path dstPath = fs.commitStoreFile(getColumnFamilyName(), path); status.setStatus("Flushing " + this + ": reopening flushed file"); - StoreFile sf = createStoreFileAndReader(dstPath); + HStoreFile sf = createStoreFileAndReader(dstPath); StoreFileReader r = sf.getReader(); this.storeSize += r.length(); @@ -1041,35 +1037,32 @@ public class HStore implements Store { @Override public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, - boolean isCompaction, boolean includeMVCCReadpoint, - boolean includesTag) - throws IOException { + boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag) throws IOException { return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint, - includesTag, false); + includesTag, false); } - /* + /** * @param maxKeyCount * @param compression Compression algorithm to use * @param isCompaction whether we are creating a new file in a compaction - * @param includesMVCCReadPoint - whether to include MVCC or not + * @param includeMVCCReadpoint - whether to include MVCC or not * @param includesTag - includesTag or not * @return Writer for a new StoreFile in the tmp dir. */ @Override public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag, - boolean shouldDropBehind) - throws IOException { + boolean shouldDropBehind) throws IOException { return createWriterInTmp(maxKeyCount, compression, isCompaction, includeMVCCReadpoint, - includesTag, shouldDropBehind, null); + includesTag, shouldDropBehind, null); } - /* + /** * @param maxKeyCount * @param compression Compression algorithm to use * @param isCompaction whether we are creating a new file in a compaction - * @param includesMVCCReadPoint - whether to include MVCC or not + * @param includeMVCCReadpoint - whether to include MVCC or not * @param includesTag - includesTag or not * @return Writer for a new StoreFile in the tmp dir. */ @@ -1078,8 +1071,7 @@ public class HStore implements Store { @Override public StoreFileWriter createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag, - boolean shouldDropBehind, final TimeRangeTracker trt) - throws IOException { + boolean shouldDropBehind, final TimeRangeTracker trt) throws IOException { final CacheConfig writerCacheConf; if (isCompaction) { // Don't cache data on write on compactions. @@ -1133,15 +1125,18 @@ public class HStore implements Store { } - /* + private long getTotalSize(Collection sfs) { + return sfs.stream().mapToLong(sf -> sf.getReader().length()).sum(); + } + + /** * Change storeFiles adding into place the Reader produced by this new flush. * @param sfs Store files * @param snapshotId * @throws IOException * @return Whether compaction is required. */ - private boolean updateStorefiles(final List sfs, final long snapshotId) - throws IOException { + private boolean updateStorefiles(List sfs, long snapshotId) throws IOException { this.lock.writeLock().lock(); try { this.storeEngine.getStoreFileManager().insertNewFiles(sfs); @@ -1159,10 +1154,7 @@ public class HStore implements Store { // notify to be called here - only in case of flushes notifyChangedReadersObservers(sfs); if (LOG.isTraceEnabled()) { - long totalSize = 0; - for (StoreFile sf : sfs) { - totalSize += sf.getReader().length(); - } + long totalSize = getTotalSize(sfs); String traceMessage = "FLUSH time,count,size,store size,store files [" + EnvironmentEdgeManager.currentTime() + "," + sfs.size() + "," + totalSize + "," + storeSize + "," + storeEngine.getStoreFileManager().getStorefileCount() + "]"; @@ -1171,11 +1163,11 @@ public class HStore implements Store { return needsCompaction(); } - /* + /** * Notify all observers that set of Readers has changed. * @throws IOException */ - private void notifyChangedReadersObservers(List sfs) throws IOException { + private void notifyChangedReadersObservers(List sfs) throws IOException { for (ChangedReadersObserver o : this.changedReaderObservers) { List memStoreScanners; this.lock.readLock().lock(); @@ -1190,13 +1182,39 @@ public class HStore implements Store { /** * Get all scanners with no filtering based on TTL (that happens further down the line). + * @param cacheBlocks cache the blocks or not + * @param usePread true to use pread, false if not + * @param isCompaction true if the scanner is created for compaction + * @param matcher the scan query matcher + * @param startRow the start row + * @param stopRow the stop row + * @param readPt the read point of the current scan + * @return all scanners for this store + */ + public List getScanners(boolean cacheBlocks, boolean isGet, boolean usePread, + boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt) + throws IOException { + return getScanners(cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, false, + readPt); + } + + /** + * Get all scanners with no filtering based on TTL (that happens further down the line). + * @param cacheBlocks cache the blocks or not + * @param usePread true to use pread, false if not + * @param isCompaction true if the scanner is created for compaction + * @param matcher the scan query matcher + * @param startRow the start row + * @param includeStartRow true to include start row, false if not + * @param stopRow the stop row + * @param includeStopRow true to include stop row, false if not + * @param readPt the read point of the current scan * @return all scanners for this store */ - @Override public List getScanners(boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt) throws IOException { - Collection storeFilesToScan; + Collection storeFilesToScan; List memStoreScanners; this.lock.readLock().lock(); try { @@ -1221,8 +1239,45 @@ public class HStore implements Store { return scanners; } - @Override - public List getScanners(List files, boolean cacheBlocks, + /** + * Create scanners on the given files and if needed on the memstore with no filtering based on TTL + * (that happens further down the line). + * @param files the list of files on which the scanners has to be created + * @param cacheBlocks cache the blocks or not + * @param usePread true to use pread, false if not + * @param isCompaction true if the scanner is created for compaction + * @param matcher the scan query matcher + * @param startRow the start row + * @param stopRow the stop row + * @param readPt the read point of the current scan + * @param includeMemstoreScanner true if memstore has to be included + * @return scanners on the given files and on the memstore if specified + */ + public List getScanners(List files, boolean cacheBlocks, + boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, + byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner) + throws IOException { + return getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, + false, readPt, includeMemstoreScanner); + } + + /** + * Create scanners on the given files and if needed on the memstore with no filtering based on TTL + * (that happens further down the line). + * @param files the list of files on which the scanners has to be created + * @param cacheBlocks ache the blocks or not + * @param usePread true to use pread, false if not + * @param isCompaction true if the scanner is created for compaction + * @param matcher the scan query matcher + * @param startRow the start row + * @param includeStartRow true to include start row, false if not + * @param stopRow the stop row + * @param includeStopRow true to include stop row, false if not + * @param readPt the read point of the current scan + * @param includeMemstoreScanner true if memstore has to be included + * @return scanners on the given files and on the memstore if specified + */ + public List getScanners(List files, boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner) throws IOException { @@ -1305,16 +1360,16 @@ public class HStore implements Store { * @return Storefile we compacted into or null if we failed or opted out early. */ @Override - public List compact(CompactionContext compaction, + public List compact(CompactionContext compaction, ThroughputController throughputController) throws IOException { return compact(compaction, throughputController, null); } @Override - public List compact(CompactionContext compaction, + public List compact(CompactionContext compaction, ThroughputController throughputController, User user) throws IOException { assert compaction != null; - List sfs = null; + List sfs = null; CompactionRequest cr = compaction.getRequest(); try { // Do all sanity checking in here if we have a valid CompactionRequest @@ -1322,7 +1377,7 @@ public class HStore implements Store { // block below long compactionStartTime = EnvironmentEdgeManager.currentTime(); assert compaction.hasSelection(); - Collection filesToCompact = cr.getFiles(); + Collection filesToCompact = cr.getFiles(); assert !filesToCompact.isEmpty(); synchronized (filesCompacting) { // sanity check: we're compacting files that this store knows about @@ -1338,7 +1393,6 @@ public class HStore implements Store { // Commence the compaction. List newFiles = compaction.compact(throughputController, user); - long outputBytes = 0L; // TODO: get rid of this! if (!this.conf.getBoolean("hbase.hstore.compaction.complete", true)) { LOG.warn("hbase.hstore.compaction.complete is set to false"); @@ -1347,8 +1401,8 @@ public class HStore implements Store { cacheConf != null? cacheConf.shouldEvictOnClose(): true; for (Path newFile : newFiles) { // Create storefile around what we wrote with a reader on it. - StoreFile sf = createStoreFileAndReader(newFile); - sf.closeReader(evictOnClose); + HStoreFile sf = createStoreFileAndReader(newFile); + sf.closeStoreFile(evictOnClose); sfs.add(sf); } return sfs; @@ -1364,10 +1418,7 @@ public class HStore implements Store { compactedCellsCount += getCompactionProgress().totalCompactingKVs; compactedCellsSize += getCompactionProgress().totalCompactedSize; } - - for (StoreFile sf : sfs) { - outputBytes += sf.getReader().length(); - } + long outputBytes = getTotalSize(sfs); // At this point the store will use new files for all new scanners. completeCompaction(filesToCompact); // update store size. @@ -1387,12 +1438,12 @@ public class HStore implements Store { } } - private List moveCompatedFilesIntoPlace(CompactionRequest cr, List newFiles, + private List moveCompatedFilesIntoPlace(CompactionRequest cr, List newFiles, User user) throws IOException { - List sfs = new ArrayList<>(newFiles.size()); + List sfs = new ArrayList<>(newFiles.size()); for (Path newFile : newFiles) { assert newFile != null; - StoreFile sf = moveFileIntoPlace(newFile); + HStoreFile sf = moveFileIntoPlace(newFile); if (this.getCoprocessorHost() != null) { getCoprocessorHost().postCompact(this, sf, cr.getTracker(), user); } @@ -1403,7 +1454,7 @@ public class HStore implements Store { } // Package-visible for tests - StoreFile moveFileIntoPlace(final Path newFile) throws IOException { + HStoreFile moveFileIntoPlace(Path newFile) throws IOException { validateStoreFile(newFile); // Move the file into the right spot Path destPath = fs.commitStoreFile(getColumnFamilyName(), newFile); @@ -1415,17 +1466,15 @@ public class HStore implements Store { * @param filesCompacted Files compacted (input). * @param newFiles Files from compaction. */ - private void writeCompactionWalRecord(Collection filesCompacted, - Collection newFiles) throws IOException { - if (region.getWAL() == null) return; - List inputPaths = new ArrayList<>(filesCompacted.size()); - for (StoreFile f : filesCompacted) { - inputPaths.add(f.getPath()); - } - List outputPaths = new ArrayList<>(newFiles.size()); - for (StoreFile f : newFiles) { - outputPaths.add(f.getPath()); + private void writeCompactionWalRecord(Collection filesCompacted, + Collection newFiles) throws IOException { + if (region.getWAL() == null) { + return; } + List inputPaths = + filesCompacted.stream().map(HStoreFile::getPath).collect(Collectors.toList()); + List outputPaths = + newFiles.stream().map(HStoreFile::getPath).collect(Collectors.toList()); HRegionInfo info = this.region.getRegionInfo(); CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(info, family.getName(), inputPaths, outputPaths, fs.getStoreDir(getColumnFamilyDescriptor().getNameAsString())); @@ -1437,8 +1486,8 @@ public class HStore implements Store { } @VisibleForTesting - void replaceStoreFiles(final Collection compactedFiles, - final Collection result) throws IOException { + void replaceStoreFiles(Collection compactedFiles, Collection result) + throws IOException { this.lock.writeLock().lock(); try { this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result); @@ -1455,7 +1504,7 @@ public class HStore implements Store { * @param compactionStartTime Start time. */ private void logCompactionEndMessage( - CompactionRequest cr, List sfs, long now, long compactionStartTime) { + CompactionRequest cr, List sfs, long now, long compactionStartTime) { StringBuilder message = new StringBuilder( "Completed" + (cr.isMajor() ? " major" : "") + " compaction of " + cr.getFiles().size() + (cr.isAllFiles() ? " (all)" : "") + " file(s) in " @@ -1463,7 +1512,7 @@ public class HStore implements Store { if (sfs.isEmpty()) { message.append("none, "); } else { - for (StoreFile sf: sfs) { + for (HStoreFile sf: sfs) { message.append(sf.getPath().getName()); message.append("(size="); message.append(TraditionalBinaryPrefix.long2String(sf.getReader().length(), "", 1)); @@ -1479,10 +1528,7 @@ public class HStore implements Store { LOG.info(message.toString()); if (LOG.isTraceEnabled()) { int fileCount = storeEngine.getStoreFileManager().getStorefileCount(); - long resultSize = 0; - for (StoreFile sf : sfs) { - resultSize += sf.getReader().length(); - } + long resultSize = getTotalSize(sfs); String traceMessage = "COMPACTION start,end,size out,files in,files out,store size," + "store files [" + compactionStartTime + "," + now + "," + resultSize + "," + cr.getFiles().size() + "," + sfs.size() + "," + storeSize + "," + fileCount + "]"; @@ -1496,9 +1542,8 @@ public class HStore implements Store { * See HBASE-2231. * @param compaction */ - public void replayCompactionMarker(CompactionDescriptor compaction, - boolean pickCompactionFiles, boolean removeFiles) - throws IOException { + public void replayCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles, + boolean removeFiles) throws IOException { LOG.debug("Completing compaction from the WAL marker"); List compactionInputs = compaction.getCompactionInputList(); List compactionOutputs = Lists.newArrayList(compaction.getCompactionOutputList()); @@ -1525,23 +1570,23 @@ public class HStore implements Store { } //some of the input files might already be deleted - List inputStoreFiles = new ArrayList<>(compactionInputs.size()); - for (StoreFile sf : this.getStorefiles()) { + List inputStoreFiles = new ArrayList<>(compactionInputs.size()); + for (HStoreFile sf : this.getStorefiles()) { if (inputFiles.contains(sf.getPath().getName())) { inputStoreFiles.add(sf); } } // check whether we need to pick up the new files - List outputStoreFiles = new ArrayList<>(compactionOutputs.size()); + List outputStoreFiles = new ArrayList<>(compactionOutputs.size()); if (pickCompactionFiles) { - for (StoreFile sf : this.getStorefiles()) { + for (HStoreFile sf : this.getStorefiles()) { compactionOutputs.remove(sf.getPath().getName()); } for (String compactionOutput : compactionOutputs) { StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), compactionOutput); - StoreFile storeFile = createStoreFileAndReader(storeFileInfo); + HStoreFile storeFile = createStoreFileAndReader(storeFileInfo); outputStoreFiles.add(storeFile); } } @@ -1561,8 +1606,9 @@ public class HStore implements Store { * but instead makes a compaction candidate list by itself. * @param N Number of files. */ + @VisibleForTesting public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException { - List filesToCompact; + List filesToCompact; boolean isMajor; this.lock.readLock().lock(); @@ -1572,7 +1618,7 @@ public class HStore implements Store { if (!filesCompacting.isEmpty()) { // exclude all files older than the newest file we're currently // compacting. this allows us to preserve contiguity (HBASE-2856) - StoreFile last = filesCompacting.get(filesCompacting.size() - 1); + HStoreFile last = filesCompacting.get(filesCompacting.size() - 1); int idx = filesToCompact.indexOf(last); Preconditions.checkArgument(idx != -1); filesToCompact.subList(0, idx + 1).clear(); @@ -1598,11 +1644,11 @@ public class HStore implements Store { .compactForTesting(filesToCompact, isMajor); for (Path newFile: newFiles) { // Move the compaction into place. - StoreFile sf = moveFileIntoPlace(newFile); + HStoreFile sf = moveFileIntoPlace(newFile); if (this.getCoprocessorHost() != null) { this.getCoprocessorHost().postCompact(this, sf, null, null); } - replaceStoreFiles(filesToCompact, Lists.newArrayList(sf)); + replaceStoreFiles(filesToCompact, Collections.singletonList(sf)); completeCompaction(filesToCompact); } } finally { @@ -1624,7 +1670,7 @@ public class HStore implements Store { @Override public boolean isMajorCompaction() throws IOException { - for (StoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) { + for (HStoreFile sf : this.storeEngine.getStoreFileManager().getStorefiles()) { // TODO: what are these reader checks all over the place? if (sf.getReader() == null) { LOG.debug("StoreFile " + sf + " has null Reader"); @@ -1652,7 +1698,7 @@ public class HStore implements Store { synchronized (filesCompacting) { // First, see if coprocessor would want to override selection. if (this.getCoprocessorHost() != null) { - final List candidatesForCoproc = compaction.preSelect(this.filesCompacting); + final List candidatesForCoproc = compaction.preSelect(this.filesCompacting); boolean override = false; override = getCoprocessorHost().preCompactSelection(this, candidatesForCoproc, tracker, user); @@ -1688,7 +1734,7 @@ public class HStore implements Store { } // Finally, we have the resulting files list. Check if we have any files at all. request = compaction.getRequest(); - Collection selectedFiles = request.getFiles(); + Collection selectedFiles = request.getFiles(); if (selectedFiles.isEmpty()) { return Optional.empty(); } @@ -1716,7 +1762,7 @@ public class HStore implements Store { } /** Adds the files to compacting files. filesCompacting must be locked. */ - private void addToCompactingFiles(final Collection filesToAdd) { + private void addToCompactingFiles(Collection filesToAdd) { if (filesToAdd == null) return; // Check that we do not try to compact the same StoreFile twice. if (!Collections.disjoint(filesCompacting, filesToAdd)) { @@ -1734,7 +1780,7 @@ public class HStore implements Store { return; } this.lock.readLock().lock(); - Collection delSfs = null; + Collection delSfs = null; try { synchronized (filesCompacting) { long cfTtl = getStoreFileTtl(); @@ -1749,7 +1795,7 @@ public class HStore implements Store { } if (delSfs == null || delSfs.isEmpty()) return; - Collection newFiles = new ArrayList<>(); // No new files. + Collection newFiles = new ArrayList<>(); // No new files. writeCompactionWalRecord(delSfs, newFiles); replaceStoreFiles(delSfs, newFiles); completeCompaction(delSfs); @@ -1775,23 +1821,20 @@ public class HStore implements Store { } /** - * Validates a store file by opening and closing it. In HFileV2 this should - * not be an expensive operation. - * + * Validates a store file by opening and closing it. In HFileV2 this should not be an expensive + * operation. * @param path the path to the store file */ - private void validateStoreFile(Path path) - throws IOException { - StoreFile storeFile = null; + private void validateStoreFile(Path path) throws IOException { + HStoreFile storeFile = null; try { storeFile = createStoreFileAndReader(path); } catch (IOException e) { - LOG.error("Failed to open store file : " + path - + ", keeping it in tmp location", e); + LOG.error("Failed to open store file : " + path + ", keeping it in tmp location", e); throw e; } finally { if (storeFile != null) { - storeFile.closeReader(false); + storeFile.closeStoreFile(false); } } } @@ -1811,11 +1854,11 @@ public class HStore implements Store { * @param compactedFiles list of files that were compacted */ @VisibleForTesting - protected void completeCompaction(final Collection compactedFiles) + protected void completeCompaction(Collection compactedFiles) throws IOException { this.storeSize = 0L; this.totalUncompressedBytes = 0L; - for (StoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) { + for (HStoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) { StoreFileReader r = hsf.getReader(); if (r == null) { LOG.warn("StoreFile " + hsf + " has a null Reader"); @@ -1857,7 +1900,7 @@ public class HStore implements Store { } @Override - public byte[] getSplitPoint() { + public Optional getSplitPoint() { this.lock.readLock().lock(); try { // Should already be enforced by the split policy! @@ -1867,7 +1910,7 @@ public class HStore implements Store { if (LOG.isTraceEnabled()) { LOG.trace("Not splittable; has references: " + this); } - return null; + return Optional.empty(); } return this.storeEngine.getStoreFileManager().getSplitPoint(); } catch(IOException e) { @@ -1875,7 +1918,7 @@ public class HStore implements Store { } finally { this.lock.readLock().unlock(); } - return null; + return Optional.empty(); } @Override @@ -1924,24 +1967,39 @@ public class HStore implements Store { return scanner; } - @Override + /** + * Recreates the scanners on the current list of active store file scanners + * @param currentFileScanners the current set of active store file scanners + * @param cacheBlocks cache the blocks or not + * @param usePread use pread or not + * @param isCompaction is the scanner for compaction + * @param matcher the scan query matcher + * @param startRow the scan's start row + * @param includeStartRow should the scan include the start row + * @param stopRow the scan's stop row + * @param includeStopRow should the scan include the stop row + * @param readPt the read point of the current scane + * @param includeMemstoreScanner whether the current scanner should include memstorescanner + * @return list of scanners recreated on the current Scanners + * @throws IOException + */ public List recreateScanners(List currentFileScanners, boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner) throws IOException { this.lock.readLock().lock(); try { - Map name2File = + Map name2File = new HashMap<>(getStorefilesCount() + getCompactedFilesCount()); - for (StoreFile file : getStorefiles()) { + for (HStoreFile file : getStorefiles()) { name2File.put(file.getFileInfo().getActiveFileName(), file); } if (getCompactedFiles() != null) { - for (StoreFile file : getCompactedFiles()) { + for (HStoreFile file : getCompactedFiles()) { name2File.put(file.getFileInfo().getActiveFileName(), file); } } - List filesToReopen = new ArrayList<>(); + List filesToReopen = new ArrayList<>(); for (KeyValueScanner kvs : currentFileScanners) { assert kvs.isFileScanner(); if (kvs.peek() == null) { @@ -1974,87 +2032,45 @@ public class HStore implements Store { return this.storeEngine.getStoreFileManager().getCompactedFilesCount(); } + private LongStream getStoreFileCreatedTimestampStream() { + return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> { + if (sf.getReader() == null) { + LOG.warn("StoreFile " + sf + " has a null Reader"); + return false; + } else { + return true; + } + }).filter(HStoreFile::isHFile).mapToLong(sf -> sf.getFileInfo().getCreatedTimestamp()); + } + @Override public long getMaxStoreFileAge() { - long earliestTS = Long.MAX_VALUE; - for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) { - StoreFileReader r = s.getReader(); - if (r == null) { - LOG.warn("StoreFile " + s + " has a null Reader"); - continue; - } - if (!s.isHFile()) { - continue; - } - long createdTS = s.getFileInfo().getCreatedTimestamp(); - earliestTS = (createdTS < earliestTS) ? createdTS : earliestTS; - } - long now = EnvironmentEdgeManager.currentTime(); - return now - earliestTS; + return EnvironmentEdgeManager.currentTime() - + getStoreFileCreatedTimestampStream().min().orElse(Long.MAX_VALUE); } @Override public long getMinStoreFileAge() { - long latestTS = 0; - for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) { - StoreFileReader r = s.getReader(); - if (r == null) { - LOG.warn("StoreFile " + s + " has a null Reader"); - continue; - } - if (!s.isHFile()) { - continue; - } - long createdTS = s.getFileInfo().getCreatedTimestamp(); - latestTS = (createdTS > latestTS) ? createdTS : latestTS; - } - long now = EnvironmentEdgeManager.currentTime(); - return now - latestTS; + return EnvironmentEdgeManager.currentTime() - + getStoreFileCreatedTimestampStream().max().orElse(0L); } @Override public long getAvgStoreFileAge() { - long sum = 0, count = 0; - for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) { - StoreFileReader r = s.getReader(); - if (r == null) { - LOG.warn("StoreFile " + s + " has a null Reader"); - continue; - } - if (!s.isHFile()) { - continue; - } - sum += s.getFileInfo().getCreatedTimestamp(); - count++; - } - if (count == 0) { - return 0; - } - long avgTS = sum / count; - long now = EnvironmentEdgeManager.currentTime(); - return now - avgTS; + OptionalDouble avg = getStoreFileCreatedTimestampStream().average(); + return avg.isPresent() ? EnvironmentEdgeManager.currentTime() - (long) avg.getAsDouble() : 0L; } @Override public long getNumReferenceFiles() { - long numRefFiles = 0; - for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) { - if (s.isReference()) { - numRefFiles++; - } - } - return numRefFiles; + return this.storeEngine.getStoreFileManager().getStorefiles().stream() + .filter(HStoreFile::isReference).count(); } @Override public long getNumHFiles() { - long numHFiles = 0; - for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) { - if (s.isHFile()) { - numHFiles++; - } - } - return numHFiles; + return this.storeEngine.getStoreFileManager().getStorefiles().stream() + .filter(HStoreFile::isHFile).count(); } @Override @@ -2074,59 +2090,41 @@ public class HStore implements Store { return getStorefilesSize(storeFile -> storeFile.isHFile()); } - private long getStorefilesSize(Predicate predicate) { - long size = 0; - for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) { - StoreFileReader r = s.getReader(); - if (r == null) { - LOG.warn("StoreFile " + s + " has a null Reader"); - continue; + private long getStorefilesSize(Predicate predicate) { + return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> { + if (sf.getReader() == null) { + LOG.warn("StoreFile " + sf + " has a null Reader"); + return false; + } else { + return true; } - if (predicate.test(s)) { - size += r.length(); + }).filter(predicate).mapToLong(sf -> sf.getReader().length()).sum(); + } + + private long getStoreFileFieldSize(ToLongFunction f) { + return this.storeEngine.getStoreFileManager().getStorefiles().stream().filter(sf -> { + if (sf.getReader() == null) { + LOG.warn("StoreFile " + sf + " has a null Reader"); + return false; + } else { + return true; } - } - return size; + }).map(HStoreFile::getReader).mapToLong(f).sum(); } @Override public long getStorefilesIndexSize() { - long size = 0; - for (StoreFile s: this.storeEngine.getStoreFileManager().getStorefiles()) { - StoreFileReader r = s.getReader(); - if (r == null) { - LOG.warn("StoreFile " + s + " has a null Reader"); - continue; - } - size += r.indexSize(); - } - return size; + return getStoreFileFieldSize(StoreFileReader::indexSize); } @Override public long getTotalStaticIndexSize() { - long size = 0; - for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) { - StoreFileReader r = s.getReader(); - if (r == null) { - continue; - } - size += r.getUncompressedDataIndexSize(); - } - return size; + return getStoreFileFieldSize(StoreFileReader::getUncompressedDataIndexSize); } @Override public long getTotalStaticBloomSize() { - long size = 0; - for (StoreFile s : this.storeEngine.getStoreFileManager().getStorefiles()) { - StoreFileReader r = s.getReader(); - if (r == null) { - continue; - } - size += r.getTotalBloomSize(); - } - return size; + return getStoreFileFieldSize(StoreFileReader::getTotalBloomSize); } @Override @@ -2247,19 +2245,19 @@ public class HStore implements Store { if (this.tempFiles == null || this.tempFiles.isEmpty()) { return false; } - List storeFiles = new ArrayList<>(this.tempFiles.size()); + List storeFiles = new ArrayList<>(this.tempFiles.size()); for (Path storeFilePath : tempFiles) { try { - StoreFile sf = HStore.this.commitFile(storeFilePath, cacheFlushSeqNum, status); + HStoreFile sf = HStore.this.commitFile(storeFilePath, cacheFlushSeqNum, status); outputFileSize += sf.getReader().length(); storeFiles.add(sf); } catch (IOException ex) { LOG.error("Failed to commit store file " + storeFilePath, ex); // Try to delete the files we have committed before. - for (StoreFile sf : storeFiles) { + for (HStoreFile sf : storeFiles) { Path pathToDelete = sf.getPath(); try { - sf.deleteReader(); + sf.deleteStoreFile(); } catch (IOException deleteEx) { LOG.fatal("Failed to delete store file we committed, halting " + pathToDelete, ex); Runtime.getRuntime().halt(1); @@ -2269,7 +2267,7 @@ public class HStore implements Store { } } - for (StoreFile sf : storeFiles) { + for (HStoreFile sf : storeFiles) { if (HStore.this.getCoprocessorHost() != null) { HStore.this.getCoprocessorHost().postFlush(HStore.this, sf); } @@ -2305,11 +2303,11 @@ public class HStore implements Store { @Override public void replayFlush(List fileNames, boolean dropMemstoreSnapshot) throws IOException { - List storeFiles = new ArrayList<>(fileNames.size()); + List storeFiles = new ArrayList<>(fileNames.size()); for (String file : fileNames) { // open the file as a store file (hfile link, etc) StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), file); - StoreFile storeFile = createStoreFileAndReader(storeFileInfo); + HStoreFile storeFile = createStoreFileAndReader(storeFileInfo); storeFiles.add(storeFile); HStore.this.storeSize += storeFile.getReader().length(); HStore.this.totalUncompressedBytes += storeFile.getReader().getTotalUncompressedBytes(); @@ -2498,9 +2496,9 @@ public class HStore implements Store { archiveLock.lock(); try { lock.readLock().lock(); - Collection copyCompactedfiles = null; + Collection copyCompactedfiles = null; try { - Collection compactedfiles = + Collection compactedfiles = this.getStoreEngine().getStoreFileManager().getCompactedfiles(); if (compactedfiles != null && compactedfiles.size() != 0) { // Do a copy under read lock @@ -2527,10 +2525,10 @@ public class HStore implements Store { * @param compactedfiles The compacted files in this store that are not active in reads * @throws IOException */ - private void removeCompactedfiles(Collection compactedfiles) + private void removeCompactedfiles(Collection compactedfiles) throws IOException { - final List filesToRemove = new ArrayList<>(compactedfiles.size()); - for (final StoreFile file : compactedfiles) { + final List filesToRemove = new ArrayList<>(compactedfiles.size()); + for (final HStoreFile file : compactedfiles) { synchronized (file) { try { StoreFileReader r = file.getReader(); @@ -2573,7 +2571,7 @@ public class HStore implements Store { // files which were successfully archived. Otherwise we will receive a // FileNotFoundException when we attempt to re-archive them in the next go around. Collection failedFiles = fae.getFailedFiles(); - Iterator iter = filesToRemove.iterator(); + Iterator iter = filesToRemove.iterator(); while (iter.hasNext()) { if (failedFiles.contains(iter.next().getPath())) { iter.remove(); @@ -2601,7 +2599,7 @@ public class HStore implements Store { return this.memstore.isSloppy(); } - private void clearCompactedfiles(final List filesToRemove) throws IOException { + private void clearCompactedfiles(List filesToRemove) throws IOException { if (LOG.isTraceEnabled()) { LOG.trace("Clearing the compacted file " + filesToRemove + " from this store"); } 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 index c43b7885821..a79af136d4a 100644 --- 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 @@ -20,8 +20,8 @@ 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.Optional; import java.util.OptionalLong; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -32,14 +32,15 @@ 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.HDFSBlocksDistribution; -import org.apache.yetus.audience.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 org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; @@ -61,8 +62,50 @@ public class HStoreFile implements StoreFile { private static final Log LOG = LogFactory.getLog(HStoreFile.class.getName()); + 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; + // 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"); + + /** Major compaction flag in FileInfo */ + public 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 = + Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION"); + + /** Bloom filter Type in FileInfo */ + public 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"); + + /** Last Bloom filter key in FileInfo */ + public 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 timestamp of earliest-put in metadata */ + public 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"); + + /** 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"); + + /** + * 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"); + private final StoreFileInfo fileInfo; private final FileSystem fs; @@ -90,29 +133,28 @@ public class HStoreFile implements StoreFile { private long maxMemstoreTS = -1; // firstKey, lastkey and cellComparator will be set when openReader. - private Cell firstKey; + private Optional firstKey; - private Cell lastKey; + private Optional lastKey; - private Comparator comparator; + private CellComparator comparator; - @Override public CacheConfig getCacheConf() { return cacheConf; } @Override - public Cell getFirstKey() { + public Optional getFirstKey() { return firstKey; } @Override - public Cell getLastKey() { + public Optional getLastKey() { return lastKey; } @Override - public Comparator getComparator() { + public CellComparator getComparator() { return comparator; } @@ -144,27 +186,6 @@ public class HStoreFile implements StoreFile { */ 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?). @@ -184,27 +205,6 @@ public class HStoreFile implements StoreFile { 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?). @@ -235,7 +235,10 @@ public class HStoreFile implements StoreFile { this.primaryReplica = primaryReplica; } - @Override + /** + * @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a + * reference. + */ public StoreFileInfo getFileInfo() { return this.fileInfo; } @@ -283,7 +286,11 @@ public class HStoreFile implements StoreFile { return fileInfo.getModificationTime(); } - @Override + /** + * 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); } @@ -299,7 +306,6 @@ public class HStoreFile implements StoreFile { return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY)); } - @Override public boolean isCompactedAway() { return compactedAway; } @@ -309,7 +315,9 @@ public class HStoreFile implements StoreFile { return refCount.get(); } - @Override + /** + * @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. @@ -331,7 +339,7 @@ public class HStoreFile implements StoreFile { /** * Opens reader on this store file. Called by Constructor. * @throws IOException - * @see #closeReader(boolean) + * @see #closeStoreFile(boolean) */ private void open() throws IOException { if (this.reader != null) { @@ -440,7 +448,9 @@ public class HStoreFile implements StoreFile { comparator = reader.getComparator(); } - @Override + /** + * Initialize the reader used for pread. + */ public void initReader() throws IOException { if (reader == null) { try { @@ -448,7 +458,7 @@ public class HStoreFile implements StoreFile { } catch (Exception e) { try { boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true; - this.closeReader(evictOnClose); + this.closeStoreFile(evictOnClose); } catch (IOException ee) { LOG.warn("failed to close reader", ee); } @@ -465,14 +475,22 @@ public class HStoreFile implements StoreFile { return reader; } - @Override + /** + * Get a scanner which uses pread. + *

+ * Must be called after initReader. + */ public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) { return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder, canOptimizeForNonNullColumn); } - @Override + /** + * Get a scanner which uses streaming read. + *

+ * Must be called after initReader. + */ public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks, boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) throws IOException { @@ -480,31 +498,37 @@ public class HStoreFile implements StoreFile { isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn); } - @Override + /** + * @return Current reader. Must call initReader first else returns null. + * @see #initReader() + */ public StoreFileReader getReader() { return this.reader; } - @Override - public synchronized void closeReader(boolean evictOnClose) - throws IOException { + /** + * @param evictOnClose whether to evict blocks belonging to this file + * @throws IOException + */ + public synchronized void closeStoreFile(boolean evictOnClose) throws IOException { if (this.reader != null) { this.reader.close(evictOnClose); this.reader = null; } } - @Override - public void markCompactedAway() { - this.compactedAway = true; + /** + * Delete this file + * @throws IOException + */ + public void deleteStoreFile() throws IOException { + boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true; + closeStoreFile(evictOnClose); + this.fs.delete(getPath(), true); } - @Override - public void deleteReader() throws IOException { - boolean evictOnClose = - cacheConf != null? cacheConf.shouldEvictOnClose(): true; - closeReader(evictOnClose); - this.fs.delete(getPath(), true); + public void markCompactedAway() { + this.compactedAway = true; } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java index 45285178e46..8af33b6147d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactor.java @@ -246,30 +246,27 @@ public class MemStoreCompactor { MemStoreSegmentsIterator iterator = null; switch (action) { - case COMPACT: - iterator = - new MemStoreCompactorSegmentsIterator(versionedList.getStoreSegments(), - compactingMemStore.getComparator(), - compactionKVMax, compactingMemStore.getStore()); + case COMPACT: + iterator = new MemStoreCompactorSegmentsIterator(versionedList.getStoreSegments(), + compactingMemStore.getComparator(), compactionKVMax, compactingMemStore.getStore()); - result = SegmentFactory.instance().createImmutableSegmentByCompaction( + result = SegmentFactory.instance().createImmutableSegmentByCompaction( compactingMemStore.getConfiguration(), compactingMemStore.getComparator(), iterator, versionedList.getNumOfCells(), compactingMemStore.getIndexType()); - iterator.close(); - break; - case MERGE: - iterator = - new MemStoreMergerSegmentsIterator(versionedList.getStoreSegments(), - compactingMemStore.getComparator(), - compactionKVMax); + iterator.close(); + break; + case MERGE: + iterator = new MemStoreMergerSegmentsIterator(versionedList.getStoreSegments(), + compactingMemStore.getComparator(), compactionKVMax); - result = SegmentFactory.instance().createImmutableSegmentByMerge( + result = SegmentFactory.instance().createImmutableSegmentByMerge( compactingMemStore.getConfiguration(), compactingMemStore.getComparator(), iterator, versionedList.getNumOfCells(), versionedList.getStoreSegments(), compactingMemStore.getIndexType()); - iterator.close(); - break; - default: throw new RuntimeException("Unknown action " + action); // sanity check + iterator.close(); + break; + default: + throw new RuntimeException("Unknown action " + action); // sanity check } return result; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactorSegmentsIterator.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactorSegmentsIterator.java index 3d889559b00..b3ba998d116 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactorSegmentsIterator.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreCompactorSegmentsIterator.java @@ -45,10 +45,8 @@ public class MemStoreCompactorSegmentsIterator extends MemStoreSegmentsIterator private StoreScanner compactingScanner; // C-tor - public MemStoreCompactorSegmentsIterator( - List segments, - CellComparator comparator, int compactionKVMax, Store store - ) throws IOException { + public MemStoreCompactorSegmentsIterator(List segments, + CellComparator comparator, int compactionKVMax, HStore store) throws IOException { super(compactionKVMax); List scanners = new ArrayList(); @@ -108,7 +106,7 @@ public class MemStoreCompactorSegmentsIterator extends MemStoreSegmentsIterator * Creates the scanner for compacting the pipeline. * @return the scanner */ - private StoreScanner createScanner(Store store, List scanners) + private StoreScanner createScanner(HStore store, List scanners) throws IOException { // Get all available versions return new StoreScanner(store, store.getScanInfo(), OptionalInt.of(Integer.MAX_VALUE), scanners, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java index 9bdeedc40fc..b9f9af88536 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MobStoreScanner.java @@ -23,9 +23,9 @@ import java.util.List; import java.util.NavigableSet; import org.apache.hadoop.hbase.Cell; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.yetus.audience.InterfaceAudience; /** * Scanner scans both the memstore and the MOB Store. Coalesce KeyValue stream into @@ -39,7 +39,7 @@ public class MobStoreScanner extends StoreScanner { private boolean readEmptyValueOnMobCellMiss = false; private final HMobStore mobStore; - public MobStoreScanner(Store store, ScanInfo scanInfo, Scan scan, + public MobStoreScanner(HStore store, ScanInfo scanInfo, Scan scan, final NavigableSet columns, long readPt) throws IOException { super(store, scanInfo, scan, columns, readPt); cacheMobBlocks = MobUtils.isCacheMobBlocks(scan); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java index 044c4dcd565..fe0f30ee956 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java @@ -499,7 +499,7 @@ public class RegionCoprocessorHost * {@link RegionObserver#preCompactScannerOpen(ObserverContext, Store, List, ScanType, long, * InternalScanner, CompactionLifeCycleTracker, long)} */ - public InternalScanner preCompactScannerOpen(Store store, List scanners, + public InternalScanner preCompactScannerOpen(HStore store, List scanners, ScanType scanType, long earliestPutTs, CompactionLifeCycleTracker tracker, User user, long readPoint) throws IOException { return execOperationWithResult(null, @@ -514,7 +514,7 @@ public class RegionCoprocessorHost } /** - * Called prior to selecting the {@link StoreFile}s for compaction from the list of currently + * Called prior to selecting the {@link HStoreFile}s for compaction from the list of currently * available candidates. * @param store The store where compaction is being requested * @param candidates The currently available store files @@ -522,7 +522,7 @@ public class RegionCoprocessorHost * @return If {@code true}, skip the normal selection process and use the current list * @throws IOException */ - public boolean preCompactSelection(Store store, List candidates, + public boolean preCompactSelection(HStore store, List candidates, CompactionLifeCycleTracker tracker, User user) throws IOException { return execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) { @Override @@ -534,13 +534,13 @@ public class RegionCoprocessorHost } /** - * Called after the {@link StoreFile}s to be compacted have been selected from the available + * Called after the {@link HStoreFile}s to be compacted have been selected from the available * candidates. * @param store The store where compaction is being requested * @param selected The store files selected to compact * @param tracker used to track the life cycle of a compaction */ - public void postCompactSelection(Store store, ImmutableList selected, + public void postCompactSelection(HStore store, ImmutableList selected, CompactionLifeCycleTracker tracker, User user) throws IOException { execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) { @Override @@ -559,7 +559,7 @@ public class RegionCoprocessorHost * @param tracker used to track the life cycle of a compaction * @throws IOException */ - public InternalScanner preCompact(Store store, InternalScanner scanner, ScanType scanType, + public InternalScanner preCompact(HStore store, InternalScanner scanner, ScanType scanType, CompactionLifeCycleTracker tracker, User user) throws IOException { return execOperationWithResult(false, scanner, coprocessors.isEmpty() ? null : new RegionOperationWithResult(user) { @@ -578,7 +578,7 @@ public class RegionCoprocessorHost * @param tracker used to track the life cycle of a compaction * @throws IOException */ - public void postCompact(Store store, StoreFile resultFile, CompactionLifeCycleTracker tracker, + public void postCompact(HStore store, HStoreFile resultFile, CompactionLifeCycleTracker tracker, User user) throws IOException { execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) { @Override @@ -593,7 +593,7 @@ public class RegionCoprocessorHost * Invoked before a memstore flush * @throws IOException */ - public InternalScanner preFlush(final Store store, final InternalScanner scanner) + public InternalScanner preFlush(HStore store, final InternalScanner scanner) throws IOException { return execOperationWithResult(false, scanner, coprocessors.isEmpty() ? null : new RegionOperationWithResult() { @@ -623,16 +623,16 @@ public class RegionCoprocessorHost * See * {@link RegionObserver#preFlushScannerOpen(ObserverContext, Store, List, InternalScanner, long)} */ - public InternalScanner preFlushScannerOpen(final Store store, - final List scanners, final long readPoint) throws IOException { + public InternalScanner preFlushScannerOpen(HStore store, List scanners, + long readPoint) throws IOException { return execOperationWithResult(null, - coprocessors.isEmpty() ? null : new RegionOperationWithResult() { - @Override - public void call(RegionObserver oserver, ObserverContext ctx) - throws IOException { - setResult(oserver.preFlushScannerOpen(ctx, store, scanners, getResult(), readPoint)); - } - }); + coprocessors.isEmpty() ? null : new RegionOperationWithResult() { + @Override + public void call(RegionObserver oserver, ObserverContext ctx) + throws IOException { + setResult(oserver.preFlushScannerOpen(ctx, store, scanners, getResult(), readPoint)); + } + }); } /** @@ -653,7 +653,7 @@ public class RegionCoprocessorHost * Invoked after a memstore flush * @throws IOException */ - public void postFlush(final Store store, final StoreFile storeFile) throws IOException { + public void postFlush(HStore store, HStoreFile storeFile) throws IOException { execOperation(coprocessors.isEmpty() ? null : new RegionOperation() { @Override public void call(RegionObserver oserver, ObserverContext ctx) @@ -1136,16 +1136,16 @@ public class RegionCoprocessorHost * See * {@link RegionObserver#preStoreScannerOpen(ObserverContext, Store, Scan, NavigableSet, KeyValueScanner, long)} */ - public KeyValueScanner preStoreScannerOpen(final Store store, final Scan scan, - final NavigableSet targetCols, final long readPt) throws IOException { + public KeyValueScanner preStoreScannerOpen(HStore store, Scan scan, + NavigableSet targetCols, long readPt) throws IOException { return execOperationWithResult(null, - coprocessors.isEmpty() ? null : new RegionOperationWithResult() { - @Override - public void call(RegionObserver oserver, ObserverContext ctx) - throws IOException { - setResult(oserver.preStoreScannerOpen(ctx, store, scan, targetCols, getResult(), readPt)); - } - }); + coprocessors.isEmpty() ? null : new RegionOperationWithResult() { + @Override + public void call(RegionObserver oserver, ObserverContext ctx) + throws IOException { + setResult(oserver.preStoreScannerOpen(ctx, store, scan, targetCols, getResult(), readPt)); + } + }); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java index 5ccd6e3ed88..71b7b9b3db0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionSplitPolicy.java @@ -19,14 +19,15 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.List; +import java.util.Optional; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.util.ReflectionUtils; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; @@ -80,12 +81,12 @@ public abstract class RegionSplitPolicy extends Configured { byte[] splitPointFromLargestStore = null; long largestStoreSize = 0; - for (Store s : stores) { - byte[] splitPoint = s.getSplitPoint(); + for (HStore s : stores) { + Optional splitPoint = s.getSplitPoint(); // Store also returns null if it has references as way of indicating it is not splittable long storeSize = s.getSize(); - if (splitPoint != null && largestStoreSize < storeSize) { - splitPointFromLargestStore = splitPoint; + if (splitPoint.isPresent() && largestStoreSize < storeSize) { + splitPointFromLargestStore = splitPoint.get(); largestStoreSize = storeSize; } } @@ -131,7 +132,7 @@ public abstract class RegionSplitPolicy extends Configured { /** * In {@link HRegionFileSystem#splitStoreFile(org.apache.hadoop.hbase.HRegionInfo, String, - * StoreFile, byte[], boolean, RegionSplitPolicy)} we are not creating the split reference + * HStoreFile, byte[], boolean, RegionSplitPolicy)} we are not creating the split reference * if split row not lies in the StoreFile range. But in some use cases we may need to create * the split reference even when the split row not lies in the range. This method can be used * to decide, whether to skip the the StoreFile range check or not. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedMobStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedMobStoreScanner.java index bfe20ba8d1a..d64c3725c6e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedMobStoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedMobStoreScanner.java @@ -23,9 +23,9 @@ import java.util.List; import java.util.NavigableSet; import org.apache.hadoop.hbase.Cell; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.mob.MobUtils; +import org.apache.yetus.audience.InterfaceAudience; /** * ReversedMobStoreScanner extends from ReversedStoreScanner, and is used to support @@ -40,7 +40,7 @@ public class ReversedMobStoreScanner extends ReversedStoreScanner { private boolean readEmptyValueOnMobCellMiss = false; protected final HMobStore mobStore; - ReversedMobStoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet columns, + ReversedMobStoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet columns, long readPt) throws IOException { super(store, scanInfo, scan, columns, readPt); cacheMobBlocks = MobUtils.isCacheMobBlocks(scan); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java index 04e77e9c384..0089d3f3a71 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ReversedStoreScanner.java @@ -22,12 +22,12 @@ import java.io.IOException; import java.util.List; import java.util.NavigableSet; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.Scan; +import org.apache.yetus.audience.InterfaceAudience; /** * ReversedStoreScanner extends from StoreScanner, and is used to support @@ -46,7 +46,7 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner { * @param columns which columns we are scanning * @throws IOException */ - ReversedStoreScanner(Store store, ScanInfo scanInfo, Scan scan, + ReversedStoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet columns, long readPt) throws IOException { super(store, scanInfo, scan, columns, readPt); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index b7e83bf7e77..6cece0ffcd2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -28,8 +28,6 @@ import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.TableName; -import org.apache.yetus.audience.InterfaceAudience; -import org.apache.yetus.audience.InterfaceStability; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver; @@ -40,10 +38,10 @@ import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; -import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; +import org.apache.yetus.audience.InterfaceAudience; +import org.apache.yetus.audience.InterfaceStability; /** * Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or @@ -63,9 +61,9 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf // General Accessors CellComparator getComparator(); - Collection getStorefiles(); + Collection getStorefiles(); - Collection getCompactedFiles(); + Collection getCompactedFiles(); /** * Close all the readers We don't need to worry about subsequent requests because the Region @@ -73,7 +71,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf * @return the {@link StoreFile StoreFiles} that were previously being used. * @throws IOException on failure */ - Collection close() throws IOException; + Collection close() throws IOException; /** * Return a scanner for both the memstore and the HStore files. Assumes we are not in a @@ -86,105 +84,6 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf KeyValueScanner getScanner(Scan scan, final NavigableSet targetCols, long readPt) throws IOException; - /** - * Get all scanners with no filtering based on TTL (that happens further down the line). - * @param cacheBlocks cache the blocks or not - * @param usePread true to use pread, false if not - * @param isCompaction true if the scanner is created for compaction - * @param matcher the scan query matcher - * @param startRow the start row - * @param stopRow the stop row - * @param readPt the read point of the current scan - * @return all scanners for this store - */ - default List getScanners(boolean cacheBlocks, boolean isGet, boolean usePread, - boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt) - throws IOException { - return getScanners(cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, false, - readPt); - } - - /** - * Get all scanners with no filtering based on TTL (that happens further down the line). - * @param cacheBlocks cache the blocks or not - * @param usePread true to use pread, false if not - * @param isCompaction true if the scanner is created for compaction - * @param matcher the scan query matcher - * @param startRow the start row - * @param includeStartRow true to include start row, false if not - * @param stopRow the stop row - * @param includeStopRow true to include stop row, false if not - * @param readPt the read point of the current scan - * @return all scanners for this store - */ - List getScanners(boolean cacheBlocks, boolean usePread, boolean isCompaction, - ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, - boolean includeStopRow, long readPt) throws IOException; - - /** - * Recreates the scanners on the current list of active store file scanners - * @param currentFileScanners the current set of active store file scanners - * @param cacheBlocks cache the blocks or not - * @param usePread use pread or not - * @param isCompaction is the scanner for compaction - * @param matcher the scan query matcher - * @param startRow the scan's start row - * @param includeStartRow should the scan include the start row - * @param stopRow the scan's stop row - * @param includeStopRow should the scan include the stop row - * @param readPt the read point of the current scane - * @param includeMemstoreScanner whether the current scanner should include memstorescanner - * @return list of scanners recreated on the current Scanners - * @throws IOException - */ - List recreateScanners(List currentFileScanners, - boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, - byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, - boolean includeMemstoreScanner) throws IOException; - - /** - * Create scanners on the given files and if needed on the memstore with no filtering based on TTL - * (that happens further down the line). - * @param files the list of files on which the scanners has to be created - * @param cacheBlocks cache the blocks or not - * @param usePread true to use pread, false if not - * @param isCompaction true if the scanner is created for compaction - * @param matcher the scan query matcher - * @param startRow the start row - * @param stopRow the stop row - * @param readPt the read point of the current scan - * @param includeMemstoreScanner true if memstore has to be included - * @return scanners on the given files and on the memstore if specified - */ - default List getScanners(List files, boolean cacheBlocks, - boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, - byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner) - throws IOException { - return getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, - false, readPt, includeMemstoreScanner); - } - - /** - * Create scanners on the given files and if needed on the memstore with no filtering based on TTL - * (that happens further down the line). - * @param files the list of files on which the scanners has to be created - * @param cacheBlocks ache the blocks or not - * @param usePread true to use pread, false if not - * @param isCompaction true if the scanner is created for compaction - * @param matcher the scan query matcher - * @param startRow the start row - * @param includeStartRow true to include start row, false if not - * @param stopRow the stop row - * @param includeStopRow true to include stop row, false if not - * @param readPt the read point of the current scan - * @param includeMemstoreScanner true if memstore has to be included - * @return scanners on the given files and on the memstore if specified - */ - List getScanners(List files, boolean cacheBlocks, boolean usePread, - boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, - byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner) - throws IOException; - ScanInfo getScanInfo(); /** @@ -194,7 +93,6 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf FileSystem getFileSystem(); - /** * @param maxKeyCount * @param compression Compression algorithm to use @@ -269,10 +167,10 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf * @deprecated see compact(CompactionContext, ThroughputController, User) */ @Deprecated - List compact(CompactionContext compaction, + List compact(CompactionContext compaction, ThroughputController throughputController) throws IOException; - List compact(CompactionContext compaction, + List compact(CompactionContext compaction, ThroughputController throughputController, User user) throws IOException; /** @@ -297,10 +195,9 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf boolean canSplit(); /** - * Determines if Store should be split - * @return byte[] if store should be split, null otherwise. + * Determines if Store should be split. */ - byte[] getSplitPoint(); + Optional getSplitPoint(); // General accessors into the state of the store // TODO abstract some of this out into a metrics class diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java index fb5f0e4b3b1..60b3c3d0d20 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java @@ -22,13 +22,13 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.List; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.Compactor; import org.apache.hadoop.hbase.util.ReflectionUtils; +import org.apache.yetus.audience.InterfaceAudience; /** * StoreEngine is a factory that can create the objects necessary for HStore to operate. @@ -84,7 +84,7 @@ public abstract class StoreEngine filesCompacting); + public abstract boolean needsCompaction(List filesCompacting); /** * Creates an instance of a compaction context specific to this engine. @@ -97,13 +97,13 @@ public abstract class StoreEngine create( - Store store, Configuration conf, CellComparator kvComparator) throws IOException { + HStore store, Configuration conf, CellComparator cellComparator) throws IOException { String className = conf.get(STORE_ENGINE_CLASS_KEY, DEFAULT_STORE_ENGINE_CLASS.getName()); try { StoreEngine se = ReflectionUtils.instantiateWithCustomCtor( className, new Class[] { }, new Object[] { }); - se.createComponentsOnce(conf, store, kvComparator); + se.createComponentsOnce(conf, store, cellComparator); return se; } catch (Exception e) { throw new IOException("Unable to load configured store engine '" + className + "'", e); 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 d5e51eda116..0097bd76477 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 @@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; import java.util.Comparator; +import java.util.Optional; import java.util.OptionalLong; import org.apache.hadoop.fs.Path; @@ -27,73 +28,38 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceStability; -import org.apache.hadoop.hbase.io.hfile.CacheConfig; -import org.apache.hadoop.hbase.util.Bytes; /** * An interface to describe a store data file. + *

+ * NOTICE: this interface is mainly designed for coprocessor, so it will not expose + * all the internal APIs for a 'store file'. If you are implementing something inside HBase, i.e, + * not a coprocessor hook, usually you should use {@link HStoreFile} directly as it is the only + * implementation of this interface. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) @InterfaceStability.Evolving public interface StoreFile { - static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead"; - - // Keys for fileinfo values in HFile - - /** Max Sequence ID in FileInfo */ - static final byte[] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY"); - - /** Major compaction flag in FileInfo */ - static final byte[] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY"); - - /** Minor compaction flag in FileInfo */ - static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY = - Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION"); - - /** Bloom filter Type in FileInfo */ - static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE"); - - /** Delete Family Count in FileInfo */ - static final byte[] DELETE_FAMILY_COUNT = Bytes.toBytes("DELETE_FAMILY_COUNT"); - - /** Last Bloom filter key in FileInfo */ - static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY"); - - /** Key for Timerange information in metadata */ - static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE"); - - /** 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 */ - static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT"); - - /** Meta key set when store file is a result of a bulk load */ - static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK"); - static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP"); + /** + * Get the first key in this store file. + */ + Optional getFirstKey(); /** - * 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. + * Get the last key in this store file. */ - static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID"); - - CacheConfig getCacheConf(); - - Cell getFirstKey(); - - Cell getLastKey(); + Optional getLastKey(); + /** + * Get the comparator for comparing two cells. + */ Comparator getComparator(); - long getMaxMemstoreTS(); - /** - * @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a - * reference. + * Get max of the MemstoreTS in the KV's in this store file. */ - StoreFileInfo getFileInfo(); + long getMaxMemstoreTS(); /** * @return Path or null if this StoreFile was made with a Stream. @@ -130,14 +96,11 @@ public interface StoreFile { */ long getMaxSequenceId(); - long getModificationTimeStamp() throws IOException; - /** - * Only used by the Striped Compaction Policy - * @param key - * @return value associated with the metadata key + * Get the modification time of this store file. Usually will access the file system so throws + * IOException. */ - byte[] getMetadataValue(byte[] key); + long getModificationTimeStamp() throws IOException; /** * Check if this storefile was created by bulk load. When a hfile is bulk loaded into HBase, we @@ -149,13 +112,6 @@ public interface StoreFile { */ boolean isBulkLoadResult(); - boolean isCompactedAway(); - - /** - * @return true if the file is still used in reads - */ - boolean isReferencedInReads(); - /** * Return the timestamp at which this bulk load file was generated. */ @@ -167,50 +123,18 @@ public interface StoreFile { */ HDFSBlocksDistribution getHDFSBlockDistribution(); - /** - * Initialize the reader used for pread. - */ - void initReader() throws IOException; - - /** - * 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() - */ - StoreFileReader getReader(); - - /** - * @param evictOnClose whether to evict blocks belonging to this file - * @throws IOException - */ - void closeReader(boolean evictOnClose) throws IOException; - - /** - * Marks the status of the file as compactedAway. - */ - void markCompactedAway(); - - /** - * Delete this file - * @throws IOException - */ - void deleteReader() throws IOException; - /** * @return a length description of this StoreFile, suitable for debug output */ String toStringDetailed(); + /** + * Get the min timestamp of all the cells in the store file. + */ OptionalLong getMinimumTimestamp(); + /** + * Get the max timestamp of all the cells in the store file. + */ 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 index e8ec9fdac5a..cd265e2f8e1 100644 --- 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 @@ -26,17 +26,17 @@ import java.util.Comparator; import org.apache.yetus.audience.InterfaceAudience; /** - * Useful comparators for comparing StoreFiles. + * Useful comparators for comparing store files. */ @InterfaceAudience.Private final class StoreFileComparators { /** - * Comparator that compares based on the Sequence Ids of the the StoreFiles. Bulk loads that did + * Comparator that compares based on the Sequence Ids of the the store files. 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 = + 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()), @@ -46,23 +46,23 @@ final class StoreFileComparators { * Comparator for time-aware compaction. SeqId is still the first ordering criterion to maintain * MVCC. */ - public static final Comparator SEQ_ID_MAX_TIMESTAMP = + 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 { + private static class GetSeqId implements Function { @Override - public Long apply(StoreFile sf) { + public Long apply(HStoreFile sf) { return sf.getMaxSequenceId(); } } - private static class GetFileSize implements Function { + private static class GetFileSize implements Function { @Override - public Long apply(StoreFile sf) { + public Long apply(HStoreFile sf) { if (sf.getReader() != null) { return sf.getReader().length(); } else { @@ -73,23 +73,23 @@ final class StoreFileComparators { } } - private static class GetBulkTime implements Function { + private static class GetBulkTime implements Function { @Override - public Long apply(StoreFile sf) { + public Long apply(HStoreFile sf) { return sf.getBulkLoadTimestamp().orElse(Long.MAX_VALUE); } } - private static class GetPathName implements Function { + private static class GetPathName implements Function { @Override - public String apply(StoreFile sf) { + public String apply(HStoreFile sf) { return sf.getPath().getName(); } } - private static class GetMaxTimestamp implements Function { + private static class GetMaxTimestamp implements Function { @Override - public Long apply(StoreFile sf) { + public Long apply(HStoreFile sf) { return sf.getMaximumTimestamp().orElse(Long.MAX_VALUE); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java index c7740806ec5..67ef4de4769 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileManager.java @@ -23,13 +23,14 @@ import java.util.Collection; import java.util.Comparator; import java.util.Iterator; import java.util.List; - -import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection; +import java.util.Optional; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection; + /** * Manages the store files and basic metadata about that that determines the logical structure * (e.g. what files to return for scan, how to determine split point, and such). @@ -45,13 +46,13 @@ public interface StoreFileManager { * Loads the initial store files into empty StoreFileManager. * @param storeFiles The files to load. */ - void loadFiles(List storeFiles); + void loadFiles(List storeFiles); /** * Adds new files, either for from MemStore flush or bulk insert, into the structure. * @param sfs New store files. */ - void insertNewFiles(Collection sfs) throws IOException; + void insertNewFiles(Collection sfs) throws IOException; /** * Adds only the new compaction results into the structure. @@ -59,34 +60,34 @@ public interface StoreFileManager { * @param results The resulting files for the compaction. */ void addCompactionResults( - Collection compactedFiles, Collection results) throws IOException; + Collection compactedFiles, Collection results) throws IOException; /** * Remove the compacted files * @param compactedFiles the list of compacted files * @throws IOException */ - void removeCompactedFiles(Collection compactedFiles) throws IOException; + void removeCompactedFiles(Collection compactedFiles) throws IOException; /** * Clears all the files currently in use and returns them. * @return The files previously in use. */ - ImmutableCollection clearFiles(); + ImmutableCollection clearFiles(); /** * Clears all the compacted files and returns them. This method is expected to be * accessed single threaded. * @return The files compacted previously. */ - Collection clearCompactedFiles(); + Collection clearCompactedFiles(); /** * Gets the snapshot of the store files currently in use. Can be used for things like metrics * and checks; should not assume anything about relations between store files in the list. * @return The list of StoreFiles. */ - Collection getStorefiles(); + Collection getStorefiles(); /** * List of compacted files inside this store that needs to be excluded in reads @@ -95,7 +96,7 @@ public interface StoreFileManager { * compacted files are done. * @return the list of compacted files */ - Collection getCompactedfiles(); + Collection getCompactedfiles(); /** * Returns the number of files currently in use. @@ -115,7 +116,7 @@ public interface StoreFileManager { * @param stopRow Stop row of the request. * @return The list of files that are to be read for this request. */ - Collection getFilesForScan(byte[] startRow, boolean includeStartRow, byte[] stopRow, + Collection getFilesForScan(byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow); /** @@ -124,9 +125,7 @@ public interface StoreFileManager { * @return The files that may have the key less than or equal to targetKey, in reverse * order of new-ness, and preference for target key. */ - Iterator getCandidateFilesForRowKeyBefore( - KeyValue targetKey - ); + Iterator getCandidateFilesForRowKeyBefore(KeyValue targetKey); /** * Updates the candidate list for finding row key before. Based on the list of candidates @@ -139,17 +138,16 @@ public interface StoreFileManager { * @param candidate The current best candidate found. * @return The list to replace candidateFiles. */ - Iterator updateCandidateFilesForRowKeyBefore( - Iterator candidateFiles, KeyValue targetKey, Cell candidate - ); + Iterator updateCandidateFilesForRowKeyBefore(Iterator candidateFiles, + KeyValue targetKey, Cell candidate); /** * Gets the split point for the split of this set of store files (approx. middle). - * @return The mid-point, or null if no split is possible. + * @return The mid-point if possible. * @throws IOException */ - byte[] getSplitPoint() throws IOException; + Optional getSplitPoint() throws IOException; /** * @return The store compaction priority. @@ -161,7 +159,7 @@ public interface StoreFileManager { * @param filesCompacting Files that are currently compacting. * @return The files which don't have any necessary data according to TTL and other criteria. */ - Collection getUnneededFiles(long maxTs, List filesCompacting); + Collection getUnneededFiles(long maxTs, List filesCompacting); /** * @return the compaction pressure used for compaction throughput tuning. @@ -171,7 +169,7 @@ public interface StoreFileManager { /** * @return the comparator used to sort storefiles. Usually, the - * {@link StoreFile#getMaxSequenceId()} is the first priority. + * {@link HStoreFile#getMaxSequenceId()} is the first priority. */ - Comparator getStoreFileComparator(); + Comparator getStoreFileComparator(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java index e3f97a25fe0..67b8fbd589b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileReader.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,9 +17,14 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY; + import java.io.DataInput; import java.io.IOException; import java.util.Map; +import java.util.Optional; import java.util.SortedSet; import java.util.concurrent.atomic.AtomicInteger; @@ -414,40 +418,40 @@ public class StoreFileReader { * @return true if there is overlap, false otherwise */ public boolean passesKeyRangeFilter(Scan scan) { - if (this.getFirstKey() == null || this.getLastKey() == null) { + Optional firstKeyKV = this.getFirstKey(); + Optional lastKeyKV = this.getLastKey(); + if (!firstKeyKV.isPresent() || !lastKeyKV.isPresent()) { // the file is empty return false; } - if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW) - && Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) { + if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW) && + Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) { return true; } byte[] smallestScanRow = scan.isReversed() ? scan.getStopRow() : scan.getStartRow(); byte[] largestScanRow = scan.isReversed() ? scan.getStartRow() : scan.getStopRow(); - Cell firstKeyKV = this.getFirstKey(); - Cell lastKeyKV = this.getLastKey(); - boolean nonOverLapping = (getComparator().compareRows(firstKeyKV, - largestScanRow, 0, largestScanRow.length) > 0 - && !Bytes - .equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(), - HConstants.EMPTY_END_ROW)) - || getComparator().compareRows(lastKeyKV, smallestScanRow, 0, smallestScanRow.length) < 0; + boolean nonOverLapping = (getComparator() + .compareRows(firstKeyKV.get(), largestScanRow, 0, largestScanRow.length) > 0 && + !Bytes.equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(), + HConstants.EMPTY_END_ROW)) || + getComparator().compareRows(lastKeyKV.get(), smallestScanRow, 0, + smallestScanRow.length) < 0; return !nonOverLapping; } public Map loadFileInfo() throws IOException { Map fi = reader.loadFileInfo(); - byte[] b = fi.get(StoreFile.BLOOM_FILTER_TYPE_KEY); + byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY); if (b != null) { bloomFilterType = BloomType.valueOf(Bytes.toString(b)); } - lastBloomKey = fi.get(StoreFile.LAST_BLOOM_KEY); + lastBloomKey = fi.get(LAST_BLOOM_KEY); if(bloomFilterType == BloomType.ROWCOL) { lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length); } - byte[] cnt = fi.get(StoreFile.DELETE_FAMILY_COUNT); + byte[] cnt = fi.get(DELETE_FAMILY_COUNT); if (cnt != null) { deleteFamilyCnt = Bytes.toLong(cnt); } @@ -537,16 +541,16 @@ public class StoreFileReader { this.deleteFamilyBloomFilter = null; } - public Cell getLastKey() { + public Optional getLastKey() { return reader.getLastKey(); } - public byte[] getLastRowKey() { + public Optional getLastRowKey() { return reader.getLastRowKey(); } - public Cell midkey() throws IOException { - return reader.midkey(); + public Optional midKey() throws IOException { + return reader.midKey(); } public long length() { @@ -565,7 +569,7 @@ public class StoreFileReader { return deleteFamilyCnt; } - public Cell getFirstKey() { + public Optional getFirstKey() { return reader.getFirstKey(); } 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 08111dc3890..f21b30b4fbb 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.Optional; import java.util.PriorityQueue; import java.util.concurrent.atomic.LongAdder; @@ -102,7 +103,7 @@ public class StoreFileScanner implements KeyValueScanner { /** * Return an array of scanners corresponding to the given set of store files. */ - public static List getScannersForStoreFiles(Collection files, + public static List getScannersForStoreFiles(Collection files, boolean cacheBlocks, boolean usePread, long readPt) throws IOException { return getScannersForStoreFiles(files, cacheBlocks, usePread, false, false, readPt); } @@ -110,7 +111,7 @@ public class StoreFileScanner implements KeyValueScanner { /** * Return an array of scanners corresponding to the given set of store files. */ - public static List getScannersForStoreFiles(Collection files, + public static List getScannersForStoreFiles(Collection files, boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean useDropBehind, long readPt) throws IOException { return getScannersForStoreFiles(files, cacheBlocks, usePread, isCompaction, useDropBehind, null, @@ -121,7 +122,7 @@ public class StoreFileScanner implements KeyValueScanner { * Return an array of scanners corresponding to the given set of store files, And set the * ScanQueryMatcher for each store file scanner for further optimization */ - public static List getScannersForStoreFiles(Collection files, + public static List getScannersForStoreFiles(Collection files, boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop, ScanQueryMatcher matcher, long readPt) throws IOException { if (files.isEmpty()) { @@ -129,15 +130,15 @@ public class StoreFileScanner implements KeyValueScanner { } List scanners = new ArrayList<>(files.size()); boolean canOptimizeForNonNullColumn = matcher != null ? !matcher.hasNullColumnInQuery() : false; - PriorityQueue sortedFiles = + PriorityQueue sortedFiles = new PriorityQueue<>(files.size(), StoreFileComparators.SEQ_ID); - for (StoreFile file : files) { + for (HStoreFile 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(); + HStoreFile sf = sortedFiles.remove(); StoreFileScanner scanner; if (usePread) { scanner = sf.getPreadScanner(cacheBlocks, readPt, i, canOptimizeForNonNullColumn); @@ -154,10 +155,10 @@ public class StoreFileScanner implements KeyValueScanner { * Get scanners for compaction. We will create a separated reader for each store file to avoid * contention with normal read request. */ - public static List getScannersForCompaction(Collection files, + public static List getScannersForCompaction(Collection files, boolean canUseDropBehind, long readPt) throws IOException { List scanners = new ArrayList<>(files.size()); - List sortedFiles = new ArrayList<>(files); + List sortedFiles = new ArrayList<>(files); Collections.sort(sortedFiles, StoreFileComparators.SEQ_ID); boolean succ = false; try { @@ -537,12 +538,11 @@ public class StoreFileScanner implements KeyValueScanner { @Override public boolean seekToLastRow() throws IOException { - byte[] lastRow = reader.getLastRowKey(); - if (lastRow == null) { + Optional lastRow = reader.getLastRowKey(); + if (!lastRow.isPresent()) { return false; } - Cell seekKey = CellUtil - .createFirstOnRow(lastRow, 0, (short) lastRow.length); + Cell seekKey = CellUtil.createFirstOnRow(lastRow.get()); if (seek(seekKey)) { return true; } else { 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 25b9aa1faef..4dbe280ed8d 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 @@ -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,7 +17,13 @@ */ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; import java.io.IOException; import java.net.InetSocketAddress; @@ -33,10 +38,9 @@ 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.HColumnDescriptor; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFileContext; @@ -48,6 +52,9 @@ import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.RowBloomContext; import org.apache.hadoop.hbase.util.RowColBloomContext; import org.apache.hadoop.io.WritableUtils; +import org.apache.yetus.audience.InterfaceAudience; + +import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; /** * A StoreFile writer. Use this to read/write HBase Store Files. It is package @@ -185,10 +192,9 @@ public class StoreFileWriter implements CellSink, ShipperListener { * @throws IOException problem writing to FS */ public void appendMetadata(final long maxSequenceId, final boolean majorCompaction) - throws IOException { - writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId)); - writer.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY, - Bytes.toBytes(majorCompaction)); + throws IOException { + writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId)); + writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction)); appendTrackedTimestampsToMetadata(); } @@ -202,9 +208,9 @@ public class StoreFileWriter implements CellSink, ShipperListener { */ public void appendMetadata(final long maxSequenceId, final boolean majorCompaction, final long mobCellsCount) throws IOException { - writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId)); - writer.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction)); - writer.appendFileInfo(StoreFile.MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount)); + writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId)); + writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction)); + writer.appendFileInfo(MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount)); appendTrackedTimestampsToMetadata(); } @@ -212,8 +218,8 @@ public class StoreFileWriter implements CellSink, ShipperListener { * Add TimestampRange and earliest put timestamp to Metadata */ public void appendTrackedTimestampsToMetadata() throws IOException { - appendFileInfo(StoreFile.TIMERANGE_KEY, WritableUtils.toByteArray(timeRangeTracker)); - appendFileInfo(StoreFile.EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs)); + appendFileInfo(TIMERANGE_KEY, WritableUtils.toByteArray(timeRangeTracker)); + appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs)); } /** @@ -310,8 +316,7 @@ public class StoreFileWriter implements CellSink, ShipperListener { // add the general Bloom filter writer and append file info if (hasGeneralBloom) { writer.addGeneralBloomFilter(generalBloomFilterWriter); - writer.appendFileInfo(StoreFile.BLOOM_FILTER_TYPE_KEY, - Bytes.toBytes(bloomType.toString())); + writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString())); bloomContext.addLastBloomKey(writer); } return hasGeneralBloom; @@ -327,8 +332,7 @@ public class StoreFileWriter implements CellSink, ShipperListener { // append file info about the number of delete family kvs // even if there is no delete family Bloom. - writer.appendFileInfo(StoreFile.DELETE_FAMILY_COUNT, - Bytes.toBytes(this.deleteFamilyCnt)); + writer.appendFileInfo(DELETE_FAMILY_COUNT, Bytes.toBytes(this.deleteFamilyCnt)); return hasDeleteFamilyBloom; } @@ -501,7 +505,7 @@ public class StoreFileWriter implements CellSink, ShipperListener { } // set block storage policy for temp path - String policyName = this.conf.get(HColumnDescriptor.STORAGE_POLICY); + String policyName = this.conf.get(ColumnFamilyDescriptorBuilder.STORAGE_POLICY); if (null == policyName) { policyName = this.conf.get(HStore.BLOCK_STORAGE_POLICY_KEY); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java index f670ade33e0..bc5a7cf5bb9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFlusher.java @@ -29,10 +29,10 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; +import org.apache.yetus.audience.InterfaceAudience; /** * Store flusher interface. Turns a snapshot of memstore into a set of store files (usually one). @@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; @InterfaceAudience.Private abstract class StoreFlusher { protected Configuration conf; - protected Store store; + protected HStore store; - public StoreFlusher(Configuration conf, Store store) { + public StoreFlusher(Configuration conf, HStore store) { this.conf = conf; this.store = store; } 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 3a9847924e8..dd68d28bdbc 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 @@ -16,7 +16,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import java.io.IOException; @@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.IsolationLevel; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.executor.ExecutorService; @@ -51,6 +49,7 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher; import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher; import org.apache.hadoop.hbase.util.CollectionUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; @@ -67,7 +66,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner implements KeyValueScanner, InternalScanner, ChangedReadersObserver { private static final Log LOG = LogFactory.getLog(StoreScanner.class); // In unit tests, the store could be null - protected final Optional store; + protected final Optional store; private ScanQueryMatcher matcher; protected KeyValueHeap heap; private boolean cacheBlocks; @@ -147,7 +146,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // Indicates whether there was flush during the course of the scan private volatile boolean flushed = false; // generally we get one file from a flush - private final List flushedStoreFiles = new ArrayList<>(1); + private final List flushedStoreFiles = new ArrayList<>(1); // Since CompactingMemstore is now default, we get three memstore scanners from a flush private final List memStoreScannersAfterFlush = new ArrayList<>(3); // The current list of scanners @@ -160,7 +159,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner private boolean topChanged = false; /** An internal constructor. */ - private StoreScanner(Optional store, Scan scan, ScanInfo scanInfo, + private StoreScanner(Optional store, Scan scan, ScanInfo scanInfo, int numColumns, long readPt, boolean cacheBlocks, ScanType scanType) { this.readPt = readPt; this.store = store; @@ -223,7 +222,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner * @param columns which columns we are scanning * @throws IOException */ - public StoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet columns, + public StoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet columns, long readPt) throws IOException { this(Optional.of(store), scan, scanInfo, columns != null ? columns.size() : 0, readPt, scan.getCacheBlocks(), ScanType.USER_SCAN); @@ -275,7 +274,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner * @param scanners ancillary scanners * @param smallestReadPoint the readPoint that we should use for tracking versions */ - public StoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions, + public StoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions, List scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException { this(store, scanInfo, maxVersions, scanners, scanType, smallestReadPoint, earliestPutTs, null, @@ -292,20 +291,20 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner * @param dropDeletesFromRow The inclusive left bound of the range; can be EMPTY_START_ROW. * @param dropDeletesToRow The exclusive right bound of the range; can be EMPTY_END_ROW. */ - public StoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions, + public StoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions, List scanners, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException { this(store, scanInfo, maxVersions, scanners, ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint, earliestPutTs, dropDeletesFromRow, dropDeletesToRow); } - private StoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions, + private StoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions, List scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException { this(Optional.of(store), maxVersions.isPresent() ? new Scan().readVersions(maxVersions.getAsInt()) : SCAN_FOR_COMPACTION, - scanInfo, 0, ((HStore) store).getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED), + scanInfo, 0, store.getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED), false, scanType); assert scanType != ScanType.USER_SCAN; matcher = @@ -844,9 +843,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner // Implementation of ChangedReadersObserver @Override - public void updateReaders(List sfs, List memStoreScanners) throws IOException { - if (CollectionUtils.isEmpty(sfs) - && CollectionUtils.isEmpty(memStoreScanners)) { + public void updateReaders(List sfs, List memStoreScanners) + throws IOException { + if (CollectionUtils.isEmpty(sfs) && CollectionUtils.isEmpty(memStoreScanners)) { return; } flushLock.lock(); @@ -868,7 +867,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner */ protected final boolean reopenAfterFlush() throws IOException { // here we can make sure that we have a Store instance. - Store store = this.store.get(); + HStore store = this.store.get(); Cell lastTop = heap.peek(); // When we have the scan object, should we not pass it to getScanners() to get a limited set of // scanners? We did so in the constructor and we could have done it now by storing the scan @@ -996,7 +995,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner List newCurrentScanners; KeyValueHeap newHeap; // We must have a store instance here - Store store = this.store.get(); + HStore store = this.store.get(); try { // recreate the scanners on the current file scanners fileScanners = store.recreateScanners(scannersToClose, cacheBlocks, false, false, 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 9104546128f..0abaffdf170 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 @@ -41,7 +41,7 @@ public class StoreUtils { /** * Creates a deterministic hash code for store file collection. */ - public static OptionalInt getDeterministicRandomSeed(Collection files) { + public static OptionalInt getDeterministicRandomSeed(Collection files) { return files.stream().mapToInt(f -> f.getPath().getName().hashCode()).findFirst(); } @@ -49,24 +49,17 @@ public class StoreUtils { * Determines whether any files in the collection are references. * @param files The files. */ - public static boolean hasReferences(final Collection files) { - if (files != null) { - for (StoreFile hsf: files) { - if (hsf.isReference()) { - return true; - } - } - } - return false; + public static boolean hasReferences(Collection files) { + // TODO: make sure that we won't pass null here in the future. + return files != null ? files.stream().anyMatch(HStoreFile::isReference) : false; } /** * Gets lowest timestamp from candidate StoreFiles */ - public static long getLowestTimestamp(final Collection candidates) - throws IOException { + public static long getLowestTimestamp(Collection candidates) throws IOException { long minTs = Long.MAX_VALUE; - for (StoreFile storeFile : candidates) { + for (HStoreFile storeFile : candidates) { minTs = Math.min(minTs, storeFile.getModificationTimeStamp()); } return minTs; @@ -77,7 +70,7 @@ public class StoreUtils { * @param candidates The files to choose from. * @return The largest file; null if no file has a reader. */ - static Optional getLargestFile(Collection candidates) { + static Optional getLargestFile(Collection candidates) { return candidates.stream().filter(f -> f.getReader() != null) .max((f1, f2) -> Long.compare(f1.getReader().length(), f2.getReader().length())); } @@ -89,29 +82,19 @@ public class StoreUtils { * @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; + public static long getMaxMemstoreTSInList(Collection sfs) { + return sfs.stream().filter(sf -> !sf.isBulkLoadResult()).mapToLong(HStoreFile::getMaxMemstoreTS) + .max().orElse(0L); } /** - * Return the highest sequence ID found across all storefiles in - * the given list. + * Return the highest sequence ID found across all storefiles in the given list. * @param sfs - * @return 0 if no non-bulk-load files are provided or, this is Store that - * does not yet have any store files. + * @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; + public static long getMaxSequenceIdInList(Collection sfs) { + return sfs.stream().mapToLong(HStoreFile::getMaxSequenceId).max().orElse(0L); } /** @@ -120,7 +103,7 @@ public class StoreUtils { * @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) + static Optional getFileSplitPoint(HStoreFile file, CellComparator comparator) throws IOException { StoreFileReader reader = file.getReader(); if (reader == null) { @@ -130,20 +113,31 @@ public class StoreUtils { // 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)); + Optional optionalMidKey = reader.midKey(); + if (!optionalMidKey.isPresent()) { + return Optional.empty(); } - return Optional.empty(); + Cell midKey = optionalMidKey.get(); + Cell firstKey = reader.getFirstKey().get(); + Cell lastKey = reader.getLastKey().get(); + // 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)); + } + + /** + * Gets the mid point of the largest file passed in as split point. + */ + static Optional getSplitPoint(Collection storefiles, + CellComparator comparator) throws IOException { + Optional largestFile = StoreUtils.getLargestFile(storefiles); + return largestFile.isPresent() ? StoreUtils.getFileSplitPoint(largestFile.get(), comparator) + : Optional.empty(); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java index b14b0d0cc60..39f142f5ca7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreEngine.java @@ -47,7 +47,7 @@ public class StripeStoreEngine extends StoreEngine filesCompacting) { + public boolean needsCompaction(List filesCompacting) { return this.compactionPolicy.needsCompactions(this.storeFileManager, filesCompacting); } @@ -58,7 +58,7 @@ public class StripeStoreEngine extends StoreEngine preSelect(List filesCompacting) { + public List preSelect(List filesCompacting) { return compactionPolicy.preSelectFilesForCoprocessor(storeFileManager, filesCompacting); } @Override - public boolean select(List filesCompacting, boolean isUserCompaction, + public boolean select(List filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak, boolean forceMajor) throws IOException { this.stripeRequest = compactionPolicy.selectCompaction( storeFileManager, filesCompacting, mayUseOffPeak); 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 8966bd46096..737e1a6df59 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 @@ -39,11 +39,11 @@ 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.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ConcatenatedLists; import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; @@ -100,19 +100,19 @@ public class StripeStoreFileManager * same index, except the last one. Inside each list, the files are in reverse order by * seqNum. Note that the length of this is one higher than that of stripeEndKeys. */ - public ArrayList> stripeFiles = new ArrayList<>(); + public ArrayList> stripeFiles = new ArrayList<>(); /** Level 0. The files are in reverse order by seqNum. */ - public ImmutableList level0Files = ImmutableList.of(); + public ImmutableList level0Files = ImmutableList.of(); /** Cached list of all files in the structure, to return from some calls */ - public ImmutableList allFilesCached = ImmutableList.of(); - private ImmutableList allCompactedFilesCached = ImmutableList.of(); + public ImmutableList allFilesCached = ImmutableList.of(); + private ImmutableList allCompactedFilesCached = ImmutableList.of(); } private State state = null; /** Cached file metadata (or overrides as the case may be) */ - private HashMap fileStarts = new HashMap<>(); - private HashMap fileEnds = new HashMap<>(); + private HashMap fileStarts = new HashMap<>(); + private HashMap fileEnds = new HashMap<>(); /** Normally invalid key is null, but in the map null is the result for "no key"; so use * the following constant value in these maps instead. Note that this is a constant and * we use it to compare by reference when we read from the map. */ @@ -132,17 +132,17 @@ public class StripeStoreFileManager } @Override - public void loadFiles(List storeFiles) { + public void loadFiles(List storeFiles) { loadUnclassifiedStoreFiles(storeFiles); } @Override - public Collection getStorefiles() { + public Collection getStorefiles() { return state.allFilesCached; } @Override - public Collection getCompactedfiles() { + public Collection getCompactedfiles() { return state.allCompactedFilesCached; } @@ -152,7 +152,7 @@ public class StripeStoreFileManager } @Override - public void insertNewFiles(Collection sfs) throws IOException { + public void insertNewFiles(Collection sfs) throws IOException { CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(true); // Passing null does not cause NPE?? cmc.mergeResults(null, sfs); @@ -160,8 +160,8 @@ public class StripeStoreFileManager } @Override - public ImmutableCollection clearFiles() { - ImmutableCollection result = state.allFilesCached; + public ImmutableCollection clearFiles() { + ImmutableCollection result = state.allFilesCached; this.state = new State(); this.fileStarts.clear(); this.fileEnds.clear(); @@ -169,8 +169,8 @@ public class StripeStoreFileManager } @Override - public ImmutableCollection clearCompactedFiles() { - ImmutableCollection result = state.allCompactedFilesCached; + public ImmutableCollection clearCompactedFiles() { + ImmutableCollection result = state.allCompactedFilesCached; this.state = new State(); return result; } @@ -183,7 +183,7 @@ public class StripeStoreFileManager /** See {@link StoreFileManager#getCandidateFilesForRowKeyBefore(KeyValue)} * for details on this methods. */ @Override - public Iterator getCandidateFilesForRowKeyBefore(final KeyValue targetKey) { + public Iterator getCandidateFilesForRowKeyBefore(final KeyValue targetKey) { KeyBeforeConcatenatedLists result = new KeyBeforeConcatenatedLists(); // Order matters for this call. result.addSublist(state.level0Files); @@ -200,14 +200,14 @@ public class StripeStoreFileManager * {@link StoreFileManager#updateCandidateFilesForRowKeyBefore(Iterator, KeyValue, Cell)} * for details on this methods. */ @Override - public Iterator updateCandidateFilesForRowKeyBefore( - Iterator candidateFiles, final KeyValue targetKey, final Cell candidate) { + public Iterator updateCandidateFilesForRowKeyBefore( + Iterator candidateFiles, final KeyValue targetKey, final Cell candidate) { KeyBeforeConcatenatedLists.Iterator original = (KeyBeforeConcatenatedLists.Iterator)candidateFiles; assert original != null; - ArrayList> components = original.getComponents(); + ArrayList> components = original.getComponents(); for (int firstIrrelevant = 0; firstIrrelevant < components.size(); ++firstIrrelevant) { - StoreFile sf = components.get(firstIrrelevant).get(0); + HStoreFile sf = components.get(firstIrrelevant).get(0); byte[] endKey = endOf(sf); // Entries are ordered as such: L0, then stripes in reverse order. We never remove // level 0; we remove the stripe, and all subsequent ones, as soon as we find the @@ -221,12 +221,6 @@ public class StripeStoreFileManager return original; } - 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 @@ -235,8 +229,10 @@ public class StripeStoreFileManager * @return The split point, or null if no split is possible. */ @Override - public byte[] getSplitPoint() throws IOException { - if (this.getStorefileCount() == 0) return null; + public Optional getSplitPoint() throws IOException { + if (this.getStorefileCount() == 0) { + return Optional.empty(); + } if (state.stripeFiles.size() <= 1) { return getSplitPointFromAllFiles(); } @@ -265,7 +261,9 @@ public class StripeStoreFileManager if (ratio < 1) { ratio = 1 / ratio; } - if (config.getMaxSplitImbalance() > ratio) return state.stripeEndRows[leftIndex]; + if (config.getMaxSplitImbalance() > ratio) { + return Optional.of(state.stripeEndRows[leftIndex]); + } // If the difference between the sides is too large, we could get the proportional key on // the a stripe to equalize the difference, but there's no proportional key method at the @@ -278,18 +276,21 @@ public class StripeStoreFileManager if (newRatio < 1) { newRatio = 1 / newRatio; } - if (newRatio >= ratio) return state.stripeEndRows[leftIndex]; + if (newRatio >= ratio) { + return Optional.of(state.stripeEndRows[leftIndex]); + } 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 getSplitPoint(state.stripeFiles.get(isRightLarger ? rightIndex : leftIndex)); + // OK, we may get better ratio, get it. + return StoreUtils.getSplitPoint(state.stripeFiles.get(isRightLarger ? rightIndex : leftIndex), + cellComparator); } - private byte[] getSplitPointFromAllFiles() throws IOException { - ConcatenatedLists sfs = new ConcatenatedLists<>(); + private Optional getSplitPointFromAllFiles() throws IOException { + ConcatenatedLists sfs = new ConcatenatedLists<>(); sfs.addSublist(state.level0Files); sfs.addAllSublists(state.stripeFiles); - return getSplitPoint(sfs); + return StoreUtils.getSplitPoint(sfs, cellComparator); } private double getMidStripeSplitRatio(long smallerSize, long largerSize, long lastLargerSize) { @@ -297,7 +298,7 @@ public class StripeStoreFileManager } @Override - public Collection getFilesForScan(byte[] startRow, boolean includeStartRow, + public Collection getFilesForScan(byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow) { if (state.stripeFiles.isEmpty()) { return state.level0Files; // There's just L0. @@ -313,7 +314,7 @@ public class StripeStoreFileManager return state.allFilesCached; // We need to read all files. } - ConcatenatedLists result = new ConcatenatedLists<>(); + ConcatenatedLists result = new ConcatenatedLists<>(); result.addAllSublists(state.stripeFiles.subList(firstStripe, lastStripe + 1)); result.addSublist(state.level0Files); return result; @@ -321,7 +322,7 @@ public class StripeStoreFileManager @Override public void addCompactionResults( - Collection compactedFiles, Collection results) throws IOException { + Collection compactedFiles, Collection results) throws IOException { // See class comment for the assumptions we make here. LOG.debug("Attempting to merge compaction results: " + compactedFiles.size() + " files replaced by " + results.size()); @@ -337,14 +338,14 @@ public class StripeStoreFileManager // Let a background thread close the actual reader on these compacted files and also // ensure to evict the blocks from block cache so that they are no longer in // cache - private void markCompactedAway(Collection compactedFiles) { - for (StoreFile file : compactedFiles) { + private void markCompactedAway(Collection compactedFiles) { + for (HStoreFile file : compactedFiles) { file.markCompactedAway(); } } @Override - public void removeCompactedFiles(Collection compactedFiles) throws IOException { + public void removeCompactedFiles(Collection compactedFiles) throws IOException { // See class comment for the assumptions we make here. LOG.debug("Attempting to delete compaction results: " + compactedFiles.size()); // In order to be able to fail in the middle of the operation, we'll operate on lazy @@ -378,7 +379,7 @@ public class StripeStoreFileManager */ private long getStripeFilesSize(int stripeIndex) { long result = 0; - for (StoreFile sf : state.stripeFiles.get(stripeIndex)) { + for (HStoreFile sf : state.stripeFiles.get(stripeIndex)) { result += sf.getReader().length(); } return result; @@ -391,13 +392,13 @@ public class StripeStoreFileManager * go to level 0. * @param storeFiles Store files to add. */ - private void loadUnclassifiedStoreFiles(List storeFiles) { + private void loadUnclassifiedStoreFiles(List storeFiles) { LOG.debug("Attempting to load " + storeFiles.size() + " store files."); - TreeMap> candidateStripes = new TreeMap<>(MAP_COMPARATOR); - ArrayList level0Files = new ArrayList<>(); + TreeMap> candidateStripes = new TreeMap<>(MAP_COMPARATOR); + ArrayList level0Files = new ArrayList<>(); // Separate the files into tentative stripes; then validate. Currently, we rely on metadata. // If needed, we could dynamically determine the stripes in future. - for (StoreFile sf : storeFiles) { + for (HStoreFile sf : storeFiles) { byte[] startRow = startOf(sf), endRow = endOf(sf); // Validate the range and put the files into place. if (isInvalid(startRow) || isInvalid(endRow)) { @@ -410,7 +411,7 @@ public class StripeStoreFileManager insertFileIntoStripe(level0Files, sf); // Bad metadata - goes to L0 also. ensureLevel0Metadata(sf); } else { - ArrayList stripe = candidateStripes.get(endRow); + ArrayList stripe = candidateStripes.get(endRow); if (stripe == null) { stripe = new ArrayList<>(); candidateStripes.put(endRow, stripe); @@ -423,14 +424,14 @@ public class StripeStoreFileManager boolean hasOverlaps = false; byte[] expectedStartRow = null; // first stripe can start wherever - Iterator>> entryIter = + Iterator>> entryIter = candidateStripes.entrySet().iterator(); while (entryIter.hasNext()) { - Map.Entry> entry = entryIter.next(); - ArrayList files = entry.getValue(); + Map.Entry> entry = entryIter.next(); + ArrayList files = entry.getValue(); // Validate the file start rows, and remove the bad ones to level 0. for (int i = 0; i < files.size(); ++i) { - StoreFile sf = files.get(i); + HStoreFile sf = files.get(i); byte[] startRow = startOf(sf); if (expectedStartRow == null) { expectedStartRow = startRow; // ensure that first stripe is still consistent @@ -439,7 +440,7 @@ public class StripeStoreFileManager LOG.warn("Store file doesn't fit into the tentative stripes - expected to start at [" + Bytes.toString(expectedStartRow) + "], but starts at [" + Bytes.toString(startRow) + "], to L0 it goes"); - StoreFile badSf = files.remove(i); + HStoreFile badSf = files.remove(i); insertFileIntoStripe(level0Files, badSf); ensureLevel0Metadata(badSf); --i; @@ -459,7 +460,7 @@ public class StripeStoreFileManager // as open keys anyway, and log the message. // If there were errors, we'll play it safe and dump everything into L0. if (!candidateStripes.isEmpty()) { - StoreFile firstFile = candidateStripes.firstEntry().getValue().get(0); + HStoreFile firstFile = candidateStripes.firstEntry().getValue().get(0); boolean isOpen = isOpen(startOf(firstFile)) && isOpen(candidateStripes.lastKey()); if (!isOpen) { LOG.warn("The range of the loaded files does not cover full key space: from [" @@ -470,8 +471,8 @@ public class StripeStoreFileManager ensureEdgeStripeMetadata(candidateStripes.lastEntry().getValue(), false); } else { LOG.warn("Inconsistent files, everything goes to L0."); - for (ArrayList files : candidateStripes.values()) { - for (StoreFile sf : files) { + for (ArrayList files : candidateStripes.values()) { + for (HStoreFile sf : files) { insertFileIntoStripe(level0Files, sf); ensureLevel0Metadata(sf); } @@ -486,9 +487,9 @@ public class StripeStoreFileManager state.level0Files = ImmutableList.copyOf(level0Files); state.stripeFiles = new ArrayList<>(candidateStripes.size()); state.stripeEndRows = new byte[Math.max(0, candidateStripes.size() - 1)][]; - ArrayList newAllFiles = new ArrayList<>(level0Files); + ArrayList newAllFiles = new ArrayList<>(level0Files); int i = candidateStripes.size() - 1; - for (Map.Entry> entry : candidateStripes.entrySet()) { + for (Map.Entry> entry : candidateStripes.entrySet()) { state.stripeFiles.add(ImmutableList.copyOf(entry.getValue())); newAllFiles.addAll(entry.getValue()); if (i > 0) { @@ -501,14 +502,14 @@ public class StripeStoreFileManager debugDumpState("Files loaded"); } - private void ensureEdgeStripeMetadata(ArrayList stripe, boolean isFirst) { - HashMap targetMap = isFirst ? fileStarts : fileEnds; - for (StoreFile sf : stripe) { + private void ensureEdgeStripeMetadata(ArrayList stripe, boolean isFirst) { + HashMap targetMap = isFirst ? fileStarts : fileEnds; + for (HStoreFile sf : stripe) { targetMap.put(sf, OPEN_KEY); } } - private void ensureLevel0Metadata(StoreFile sf) { + private void ensureLevel0Metadata(HStoreFile sf) { if (!isInvalid(startOf(sf))) this.fileStarts.put(sf, INVALID_KEY_IN_MAP); if (!isInvalid(endOf(sf))) this.fileEnds.put(sf, INVALID_KEY_IN_MAP); } @@ -616,7 +617,7 @@ public class StripeStoreFileManager } - private byte[] startOf(StoreFile sf) { + private byte[] startOf(HStoreFile sf) { byte[] result = fileStarts.get(sf); // result and INVALID_KEY_IN_MAP are compared _only_ by reference on purpose here as the latter @@ -627,7 +628,7 @@ public class StripeStoreFileManager : result == INVALID_KEY_IN_MAP ? INVALID_KEY : result; } - private byte[] endOf(StoreFile sf) { + private byte[] endOf(HStoreFile sf) { byte[] result = fileEnds.get(sf); // result and INVALID_KEY_IN_MAP are compared _only_ by reference on purpose here as the latter @@ -643,7 +644,7 @@ public class StripeStoreFileManager * @param stripe Stripe copy to insert into. * @param sf File to insert. */ - private static void insertFileIntoStripe(ArrayList stripe, StoreFile sf) { + private static void insertFileIntoStripe(ArrayList stripe, HStoreFile sf) { // The only operation for which sorting of the files matters is KeyBefore. Therefore, // we will store the file in reverse order by seqNum from the outset. for (int insertBefore = 0; ; ++insertBefore) { @@ -664,20 +665,20 @@ public class StripeStoreFileManager * but will quickly cut down on them as it finds something in the more likely ones; thus, * the above allow us to avoid unnecessary copying of a bunch of lists. */ - private static class KeyBeforeConcatenatedLists extends ConcatenatedLists { + private static class KeyBeforeConcatenatedLists extends ConcatenatedLists { @Override - public java.util.Iterator iterator() { + public java.util.Iterator iterator() { return new Iterator(); } - public class Iterator extends ConcatenatedLists.Iterator { - public ArrayList> getComponents() { + public class Iterator extends ConcatenatedLists.Iterator { + public ArrayList> getComponents() { return components; } public void removeComponents(int startIndex) { - List> subList = components.subList(startIndex, components.size()); - for (List entry : subList) { + List> subList = components.subList(startIndex, components.size()); + for (List entry : subList) { size -= entry.size(); } assert size >= 0; @@ -690,7 +691,7 @@ public class StripeStoreFileManager throw new IllegalStateException("No element to remove"); } this.nextWasCalled = false; - List src = components.get(currentComponent); + List src = components.get(currentComponent); if (src instanceof ImmutableList) { src = new ArrayList<>(src); components.set(currentComponent, src); @@ -711,14 +712,14 @@ public class StripeStoreFileManager * then creates a new state object and puts it in place. */ private class CompactionOrFlushMergeCopy { - private ArrayList> stripeFiles = null; - private ArrayList level0Files = null; + private ArrayList> stripeFiles = null; + private ArrayList level0Files = null; private ArrayList stripeEndRows = null; - private Collection compactedFiles = null; - private Collection results = null; + private Collection compactedFiles = null; + private Collection results = null; - private List l0Results = new ArrayList<>(); + private List l0Results = new ArrayList<>(); private final boolean isFlush; public CompactionOrFlushMergeCopy(boolean isFlush) { @@ -727,14 +728,14 @@ public class StripeStoreFileManager this.isFlush = isFlush; } - private void mergeResults(Collection compactedFiles, Collection results) + private void mergeResults(Collection compactedFiles, Collection results) throws IOException { assert this.compactedFiles == null && this.results == null; this.compactedFiles = compactedFiles; this.results = results; // Do logical processing. if (!isFlush) removeCompactedFiles(); - TreeMap newStripes = processResults(); + TreeMap newStripes = processResults(); if (newStripes != null) { processNewCandidateStripes(newStripes); } @@ -744,7 +745,7 @@ public class StripeStoreFileManager updateMetadataMaps(); } - private void deleteResults(Collection compactedFiles) throws IOException { + private void deleteResults(Collection compactedFiles) throws IOException { this.compactedFiles = compactedFiles; // Create new state and update parent. State state = createNewState(true); @@ -762,13 +763,13 @@ public class StripeStoreFileManager newState.stripeEndRows = (this.stripeEndRows == null) ? oldState.stripeEndRows : this.stripeEndRows.toArray(new byte[this.stripeEndRows.size()][]); newState.stripeFiles = new ArrayList<>(this.stripeFiles.size()); - for (List newStripe : this.stripeFiles) { + for (List newStripe : this.stripeFiles) { newState.stripeFiles.add(newStripe instanceof ImmutableList - ? (ImmutableList)newStripe : ImmutableList.copyOf(newStripe)); + ? (ImmutableList)newStripe : ImmutableList.copyOf(newStripe)); } - List newAllFiles = new ArrayList<>(oldState.allFilesCached); - List newAllCompactedFiles = new ArrayList<>(oldState.allCompactedFilesCached); + List newAllFiles = new ArrayList<>(oldState.allFilesCached); + List newAllCompactedFiles = new ArrayList<>(oldState.allCompactedFilesCached); if (!isFlush) { newAllFiles.removeAll(compactedFiles); if (delCompactedFiles) { @@ -788,13 +789,13 @@ public class StripeStoreFileManager private void updateMetadataMaps() { StripeStoreFileManager parent = StripeStoreFileManager.this; if (!isFlush) { - for (StoreFile sf : this.compactedFiles) { + for (HStoreFile sf : this.compactedFiles) { parent.fileStarts.remove(sf); parent.fileEnds.remove(sf); } } if (this.l0Results != null) { - for (StoreFile sf : this.l0Results) { + for (HStoreFile sf : this.l0Results) { parent.ensureLevel0Metadata(sf); } } @@ -804,14 +805,14 @@ public class StripeStoreFileManager * @param index Index of the stripe we need. * @return A lazy stripe copy from current stripes. */ - private final ArrayList getStripeCopy(int index) { - List stripeCopy = this.stripeFiles.get(index); - ArrayList result = null; + private final ArrayList getStripeCopy(int index) { + List stripeCopy = this.stripeFiles.get(index); + ArrayList result = null; if (stripeCopy instanceof ImmutableList) { result = new ArrayList<>(stripeCopy); this.stripeFiles.set(index, result); } else { - result = (ArrayList)stripeCopy; + result = (ArrayList)stripeCopy; } return result; } @@ -819,7 +820,7 @@ public class StripeStoreFileManager /** * @return A lazy L0 copy from current state. */ - private final ArrayList getLevel0Copy() { + private final ArrayList getLevel0Copy() { if (this.level0Files == null) { this.level0Files = new ArrayList<>(StripeStoreFileManager.this.state.level0Files); } @@ -831,9 +832,9 @@ public class StripeStoreFileManager * or to the list of new candidate stripes. * @return New candidate stripes. */ - private TreeMap processResults() throws IOException { - TreeMap newStripes = null; - for (StoreFile sf : this.results) { + private TreeMap processResults() throws IOException { + TreeMap newStripes = null; + for (HStoreFile sf : this.results) { byte[] startRow = startOf(sf), endRow = endOf(sf); if (isInvalid(endRow) || isInvalid(startRow)) { if (!isFlush) { @@ -856,7 +857,7 @@ public class StripeStoreFileManager if (newStripes == null) { newStripes = new TreeMap<>(MAP_COMPARATOR); } - StoreFile oldSf = newStripes.put(endRow, sf); + HStoreFile oldSf = newStripes.put(endRow, sf); if (oldSf != null) { throw new IOException("Compactor has produced multiple files for the stripe ending in [" + Bytes.toString(endRow) + "], found " + sf.getPath() + " and " + oldSf.getPath()); @@ -870,9 +871,9 @@ public class StripeStoreFileManager * @param compactedFiles Compacted files. */ private void removeCompactedFiles() throws IOException { - for (StoreFile oldFile : this.compactedFiles) { + for (HStoreFile oldFile : this.compactedFiles) { byte[] oldEndRow = endOf(oldFile); - List source = null; + List source = null; if (isInvalid(oldEndRow)) { source = getLevel0Copy(); } else { @@ -895,7 +896,7 @@ public class StripeStoreFileManager * @param newStripes New stripes - files by end row. */ private void processNewCandidateStripes( - TreeMap newStripes) throws IOException { + TreeMap newStripes) throws IOException { // Validate that the removed and added aggregate ranges still make for a full key space. boolean hasStripes = !this.stripeFiles.isEmpty(); this.stripeEndRows = new ArrayList<>(Arrays.asList(StripeStoreFileManager.this.state.stripeEndRows)); @@ -907,7 +908,7 @@ public class StripeStoreFileManager } boolean canAddNewStripes = true; - Collection filesForL0 = null; + Collection filesForL0 = null; if (hasStripes) { // Determine which stripes will need to be removed because they conflict with new stripes. // The new boundaries should match old stripe boundaries, so we should get exact matches. @@ -921,7 +922,7 @@ public class StripeStoreFileManager int removeTo = findStripeIndexByEndRow(lastEndRow); if (removeTo < 0) throw new IOException("Compaction is trying to add a bad range."); // See if there are files in the stripes we are trying to replace. - ArrayList conflictingFiles = new ArrayList<>(); + ArrayList conflictingFiles = new ArrayList<>(); for (int removeIndex = removeTo; removeIndex >= removeFrom; --removeIndex) { conflictingFiles.addAll(this.stripeFiles.get(removeIndex)); } @@ -942,7 +943,7 @@ public class StripeStoreFileManager filesForL0 = conflictingFiles; } if (filesForL0 != null) { - for (StoreFile sf : filesForL0) { + for (HStoreFile sf : filesForL0) { insertFileIntoStripe(getLevel0Copy(), sf); } l0Results.addAll(filesForL0); @@ -966,7 +967,7 @@ public class StripeStoreFileManager // Now, insert new stripes. The total ranges match, so we can insert where we removed. byte[] previousEndRow = null; int insertAt = removeFrom; - for (Map.Entry newStripe : newStripes.entrySet()) { + for (Map.Entry newStripe : newStripes.entrySet()) { if (previousEndRow != null) { // Validate that the ranges are contiguous. assert !isOpen(previousEndRow); @@ -977,7 +978,7 @@ public class StripeStoreFileManager } } // Add the new stripe. - ArrayList tmp = new ArrayList<>(); + ArrayList tmp = new ArrayList<>(); tmp.add(newStripe.getValue()); stripeFiles.add(insertAt, tmp); previousEndRow = newStripe.getKey(); @@ -990,7 +991,7 @@ public class StripeStoreFileManager } @Override - public List getLevel0Files() { + public List getLevel0Files() { return this.state.level0Files; } @@ -1005,7 +1006,7 @@ public class StripeStoreFileManager } @Override - public ArrayList> getStripes() { + public ArrayList> getStripes() { return this.state.stripeFiles; } @@ -1015,22 +1016,22 @@ public class StripeStoreFileManager } @Override - public Collection getUnneededFiles(long maxTs, List filesCompacting) { + public Collection getUnneededFiles(long maxTs, List filesCompacting) { // 1) We can never get rid of the last file which has the maximum seqid in a stripe. // 2) Files that are not the latest can't become one due to (1), so the rest are fair game. State state = this.state; - Collection expiredStoreFiles = null; - for (ImmutableList stripe : state.stripeFiles) { + Collection expiredStoreFiles = null; + for (ImmutableList stripe : state.stripeFiles) { expiredStoreFiles = findExpiredFiles(stripe, maxTs, filesCompacting, expiredStoreFiles); } return findExpiredFiles(state.level0Files, maxTs, filesCompacting, expiredStoreFiles); } - private Collection findExpiredFiles(ImmutableList stripe, long maxTs, - List filesCompacting, Collection expiredStoreFiles) { + private Collection findExpiredFiles(ImmutableList stripe, long maxTs, + List filesCompacting, Collection expiredStoreFiles) { // Order by seqnum is reversed. for (int i = 1; i < stripe.size(); ++i) { - StoreFile sf = stripe.get(i); + HStoreFile sf = stripe.get(i); synchronized (sf) { long fileTs = sf.getReader().getMaxTimestamp(); if (fileTs < maxTs && !filesCompacting.contains(sf)) { @@ -1061,7 +1062,7 @@ public class StripeStoreFileManager // we flush data to stripe directly. int delta = stateLocal.level0Files.isEmpty() ? 0 : 1; double max = 0.0; - for (ImmutableList stripeFile : stateLocal.stripeFiles) { + for (ImmutableList stripeFile : stateLocal.stripeFiles) { int stripeFileCount = stripeFile.size(); double normCount = (double) (stripeFileCount + delta - config.getStripeCompactMinFiles()) @@ -1079,7 +1080,7 @@ public class StripeStoreFileManager } @Override - public Comparator getStoreFileComparator() { + public Comparator getStoreFileComparator() { return StoreFileComparators.SEQ_ID; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java index b11b384fa5d..bc4d624b77c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreFlusher.java @@ -26,13 +26,13 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.CellComparator; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; @@ -47,7 +47,7 @@ public class StripeStoreFlusher extends StoreFlusher { private final StripeCompactionPolicy policy; private final StripeCompactionPolicy.StripeInformationProvider stripes; - public StripeStoreFlusher(Configuration conf, Store store, + public StripeStoreFlusher(Configuration conf, HStore store, StripeCompactionPolicy policy, StripeStoreFileManager stripes) { super(conf, store); this.policy = policy; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java index 1c81d3106ac..c5ef127866e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/AbstractMultiOutputCompactor.java @@ -24,13 +24,13 @@ 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.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter; import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter.WriterFactory; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; import org.apache.hadoop.hbase.regionserver.StoreScanner; +import org.apache.yetus.audience.InterfaceAudience; /** * Base class for implementing a Compactor which will generate multiple output files after @@ -42,7 +42,7 @@ public abstract class AbstractMultiOutputCompactor preSelect(final List filesCompacting); + public abstract List preSelect(List filesCompacting); /** * Called to select files for compaction. Must fill in the request field if successful. @@ -54,9 +54,8 @@ public abstract class CompactionContext { * @param forceMajor Whether to force major compaction. * @return Whether the selection succeeded. Selection may be empty and lead to no compaction. */ - public abstract boolean select( - final List filesCompacting, final boolean isUserCompaction, - final boolean mayUseOffPeak, final boolean forceMajor) throws IOException; + public abstract boolean select(List filesCompacting, boolean isUserCompaction, + boolean mayUseOffPeak, boolean forceMajor) throws IOException; /** * Forces external selection to be applied for this compaction. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionPolicy.java index db4e0f0bb45..a2778d34838 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionPolicy.java @@ -22,10 +22,10 @@ package org.apache.hadoop.hbase.regionserver.compactions; import java.io.IOException; import java.util.Collection; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.yetus.audience.InterfaceAudience; /** * A compaction policy determines how to select files for compaction, @@ -45,8 +45,8 @@ public abstract class CompactionPolicy { * @param filesToCompact Files to compact. Can be null. * @return True if we should run a major compaction. */ - public abstract boolean shouldPerformMajorCompaction( - final Collection filesToCompact) throws IOException; + public abstract boolean shouldPerformMajorCompaction(Collection filesToCompact) + throws IOException; /** * @param compactionSize Total size of some compaction 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 e05c16510d3..da35bfcf353 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 @@ -21,8 +21,8 @@ package org.apache.hadoop.hbase.regionserver.compactions; import java.util.Collection; import java.util.stream.Collectors; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; import org.apache.yetus.audience.InterfaceAudience; @@ -40,7 +40,7 @@ public class CompactionRequest { private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR } private DisplayCompactionType isMajor = DisplayCompactionType.MINOR; private int priority = Store.NO_PRIORITY; - private Collection filesToCompact; + private Collection filesToCompact; // CompactRequest object creation time. private long selectionTime; @@ -51,19 +51,19 @@ public class CompactionRequest { private long totalSize = -1L; private CompactionLifeCycleTracker tracker = CompactionLifeCycleTracker.DUMMY; - public CompactionRequest(Collection files) { + public CompactionRequest(Collection files) { this.selectionTime = EnvironmentEdgeManager.currentTime(); this.timeInNanos = System.nanoTime(); this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null"); recalculateSize(); } - public void updateFiles(Collection files) { + public void updateFiles(Collection files) { this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null"); recalculateSize(); } - public Collection getFiles() { + public Collection getFiles() { return this.filesToCompact; } @@ -152,7 +152,7 @@ public class CompactionRequest { * @param files files that should be included in the compaction */ private void recalculateSize() { - this.totalSize = filesToCompact.stream().map(StoreFile::getReader) + this.totalSize = filesToCompact.stream().map(HStoreFile::getReader) .mapToLong(r -> r != null ? r.length() : 0L).sum(); } } 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 e6d19351b76..2c9a519a486 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,6 +17,9 @@ */ package org.apache.hadoop.hbase.regionserver.compactions; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; + import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; @@ -33,19 +36,17 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeyValueUtil; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.HStore; +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.ScanType; import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.ShipperListener; -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.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; @@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.io.Closeables; @@ -70,7 +72,7 @@ public abstract class Compactor { protected static final long COMPACTION_PROGRESS_LOG_INTERVAL = 60 * 1000; protected volatile CompactionProgress progress; protected final Configuration conf; - protected final Store store; + protected final HStore store; protected final int compactionKVMax; protected final Compression.Algorithm compactionCompression; @@ -88,7 +90,7 @@ public abstract class Compactor { private boolean dropCacheMinor; //TODO: depending on Store is not good but, realistically, all compactors currently do. - Compactor(final Configuration conf, final Store store) { + Compactor(Configuration conf, HStore store) { this.conf = conf; this.store = store; this.compactionKVMax = @@ -137,12 +139,12 @@ public abstract class Compactor { * @return The result. */ protected FileDetails getFileDetails( - Collection filesToCompact, boolean allFiles) throws IOException { + Collection filesToCompact, boolean allFiles) throws IOException { FileDetails fd = new FileDetails(); long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() - (1000L * 60 * 60 * 24 * this.keepSeqIdPeriod); - for (StoreFile file : filesToCompact) { + for (HStoreFile file : filesToCompact) { if(allFiles && (file.getModificationTimeStamp() < oldestHFileTimeStampToKeepMVCC)) { // when isAllFiles is true, all files are compacted so we can calculate the smallest // MVCC value to keep @@ -184,7 +186,7 @@ public abstract class Compactor { // This is used to remove family delete marker during compaction. long earliestPutTs = 0; if (allFiles) { - tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS); + tmp = fileInfo.get(EARLIEST_PUT_TS); if (tmp == null) { // There's a file with no information, must be an old one // assume we have very old puts @@ -194,7 +196,7 @@ public abstract class Compactor { fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs); } } - tmp = fileInfo.get(StoreFile.TIMERANGE_KEY); + tmp = fileInfo.get(TIMERANGE_KEY); TimeRangeTracker trt = TimeRangeTracker.getTimeRangeTracker(tmp); fd.latestPutTs = trt == null? HConstants.LATEST_TIMESTAMP: trt.getMax(); if (LOG.isDebugEnabled()) { @@ -215,7 +217,7 @@ public abstract class Compactor { * @param filesToCompact Files. * @return Scanners. */ - protected List createFileScanners(Collection filesToCompact, + protected List createFileScanners(Collection filesToCompact, long smallestReadPoint, boolean useDropBehind) throws IOException { return StoreFileScanner.getScannersForCompaction(filesToCompact, useDropBehind, smallestReadPoint); @@ -498,7 +500,7 @@ public abstract class Compactor { * @param earliestPutTs Earliest put across all files. * @return A compaction scanner. */ - protected InternalScanner createScanner(Store store, List scanners, + protected InternalScanner createScanner(HStore store, List scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException { return new StoreScanner(store, store.getScanInfo(), OptionalInt.empty(), scanners, scanType, smallestReadPoint, earliestPutTs); @@ -513,7 +515,7 @@ public abstract class Compactor { * @param dropDeletesToRow Drop deletes ending with this row, exclusive. Can be null. * @return A compaction scanner. */ - protected InternalScanner createScanner(Store store, List scanners, + protected InternalScanner createScanner(HStore store, List scanners, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException { return new StoreScanner(store, store.getScanInfo(), OptionalInt.empty(), scanners, 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 6cfe207ca6f..a4cc65c3785 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 @@ -30,14 +30,14 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HDFSBlocksDistribution; -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.ReflectionUtils; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterators; @@ -98,9 +98,9 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { */ @Override @VisibleForTesting - public boolean needsCompaction(final Collection storeFiles, - final List filesCompacting) { - ArrayList candidates = new ArrayList<>(storeFiles); + public boolean needsCompaction(Collection storeFiles, + List filesCompacting) { + ArrayList candidates = new ArrayList<>(storeFiles); try { return !selectMinorCompaction(candidates, false, true).getFiles().isEmpty(); } catch (Exception e) { @@ -109,8 +109,8 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { } } - public boolean shouldPerformMajorCompaction(final Collection filesToCompact) - throws IOException { + public boolean shouldPerformMajorCompaction(Collection filesToCompact) + throws IOException { long mcTime = getNextMajorCompactTime(filesToCompact); if (filesToCompact == null || mcTime == 0) { if (LOG.isDebugEnabled()) { @@ -135,7 +135,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { List boundaries = getCompactBoundariesForMajor(filesToCompact, now); boolean[] filesInWindow = new boolean[boundaries.size()]; - for (StoreFile file: filesToCompact) { + for (HStoreFile file: filesToCompact) { OptionalLong minTimestamp = file.getMinimumTimestamp(); long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE; if (cfTTL != Long.MAX_VALUE && oldest >= cfTTL) { @@ -186,7 +186,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { } @Override - protected CompactionRequest createCompactionRequest(ArrayList candidateSelection, + protected CompactionRequest createCompactionRequest(ArrayList candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck) throws IOException { CompactionRequest result = tryingMajor ? selectMajorCompaction(candidateSelection) : selectMinorCompaction(candidateSelection, mayUseOffPeak, mayBeStuck); @@ -196,7 +196,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { return result; } - public CompactionRequest selectMajorCompaction(ArrayList candidateSelection) { + public CompactionRequest selectMajorCompaction(ArrayList candidateSelection) { long now = EnvironmentEdgeManager.currentTime(); return new DateTieredCompactionRequest(candidateSelection, this.getCompactBoundariesForMajor(candidateSelection, now)); @@ -210,15 +210,15 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { * by seqId and maxTimestamp in descending order and build the time windows. All the out-of-order * data into the same compaction windows, guaranteeing contiguous compaction based on sequence id. */ - public CompactionRequest selectMinorCompaction(ArrayList candidateSelection, + public CompactionRequest selectMinorCompaction(ArrayList candidateSelection, boolean mayUseOffPeak, boolean mayBeStuck) throws IOException { long now = EnvironmentEdgeManager.currentTime(); long oldestToCompact = getOldestToCompact(comConf.getDateTieredMaxStoreFileAgeMillis(), now); - List> storefileMaxTimestampPairs = + List> storefileMaxTimestampPairs = Lists.newArrayListWithCapacity(candidateSelection.size()); long maxTimestampSeen = Long.MIN_VALUE; - for (StoreFile storeFile : candidateSelection) { + for (HStoreFile storeFile : candidateSelection) { // if there is out-of-order data, // we put them in the same window as the last file in increasing order maxTimestampSeen = @@ -229,7 +229,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { CompactionWindow window = getIncomingWindow(now); int minThreshold = comConf.getDateTieredIncomingWindowMin(); - PeekingIterator> it = + PeekingIterator> it = Iterators.peekingIterator(storefileMaxTimestampPairs.iterator()); while (it.hasNext()) { if (window.compareToTimestamp(oldestToCompact) < 0) { @@ -242,7 +242,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { minThreshold = comConf.getMinFilesToCompact(); } else { // The file is within the target window - ArrayList fileList = Lists.newArrayList(); + ArrayList fileList = Lists.newArrayList(); // Add all files in the same window. For incoming window // we tolerate files with future data although it is sub-optimal while (it.hasNext() && window.compareToTimestamp(it.peek().getSecond()) <= 0) { @@ -261,10 +261,10 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { } } // A non-null file list is expected by HStore - return new CompactionRequest(Collections. emptyList()); + return new CompactionRequest(Collections.emptyList()); } - private DateTieredCompactionRequest generateCompactionRequest(ArrayList storeFiles, + private DateTieredCompactionRequest generateCompactionRequest(ArrayList storeFiles, CompactionWindow window, boolean mayUseOffPeak, boolean mayBeStuck, int minThreshold) throws IOException { // The files has to be in ascending order for ratio-based compaction to work right @@ -273,7 +273,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { // Compact everything in the window if have more files than comConf.maxBlockingFiles compactionPolicyPerWindow.setMinThreshold(minThreshold); - ArrayList storeFileSelection = mayBeStuck ? storeFiles + ArrayList storeFileSelection = mayBeStuck ? storeFiles : compactionPolicyPerWindow.applyCompactionPolicy(storeFiles, mayUseOffPeak, false); if (storeFileSelection != null && !storeFileSelection.isEmpty()) { // If there is any file in the window excluded from compaction, @@ -291,7 +291,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { /** * Return a list of boundaries for multiple compaction output in ascending order. */ - private List getCompactBoundariesForMajor(Collection filesToCompact, long now) { + private List getCompactBoundariesForMajor(Collection filesToCompact, long now) { long minTimestamp = filesToCompact.stream().mapToLong(f -> f.getMinimumTimestamp().orElse(Long.MAX_VALUE)).min() .orElse(Long.MAX_VALUE); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java index b33663fe4e7..5fddf33ebff 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionRequest.java @@ -21,14 +21,14 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_DOESNT_OVERRIDE_EQUALS", justification="It is intended to use the same equal method as superclass") public class DateTieredCompactionRequest extends CompactionRequest { private List boundaries; - public DateTieredCompactionRequest(Collection files, List boundaryList) { + public DateTieredCompactionRequest(Collection files, List boundaryList) { super(files); boundaries = boundaryList; } 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 9c3f576a382..ba0caa4f515 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 @@ -24,13 +24,13 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; +import org.apache.yetus.audience.InterfaceAudience; /** * This compactor will generate StoreFile for different time ranges. @@ -40,7 +40,7 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor { private static final Log LOG = LogFactory.getLog(DefaultCompactor.class); - public DefaultCompactor(final Configuration conf, final Store store) { + public DefaultCompactor(Configuration conf, HStore store) { super(conf, store); } @@ -76,7 +76,7 @@ public class DefaultCompactor extends Compactor { * made it through the compaction. * @throws IOException */ - public List compactForTesting(final Collection filesToCompact, boolean isMajor) + public List compactForTesting(Collection filesToCompact, boolean isMajor) throws IOException { CompactionRequest cr = new CompactionRequest(filesToCompact); cr.setIsMajor(isMajor, isMajor); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java index 842d6e8860b..b0942f60893 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/ExploringCompactionPolicy.java @@ -26,9 +26,9 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.yetus.audience.InterfaceAudience; /** * Class to pick which files if any to compact together. @@ -51,21 +51,20 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy { } @Override - protected final ArrayList applyCompactionPolicy(final ArrayList candidates, - final boolean mayUseOffPeak, final boolean mightBeStuck) throws IOException { - return new ArrayList<>(applyCompactionPolicy(candidates, mightBeStuck, - mayUseOffPeak, comConf.getMinFilesToCompact(), comConf.getMaxFilesToCompact())); + protected final ArrayList applyCompactionPolicy(ArrayList candidates, + boolean mayUseOffPeak, boolean mightBeStuck) throws IOException { + return new ArrayList<>(applyCompactionPolicy(candidates, mightBeStuck, mayUseOffPeak, + comConf.getMinFilesToCompact(), comConf.getMaxFilesToCompact())); } - public List applyCompactionPolicy(final List candidates, - boolean mightBeStuck, boolean mayUseOffPeak, int minFiles, int maxFiles) { - + public List applyCompactionPolicy(List candidates, boolean mightBeStuck, + boolean mayUseOffPeak, int minFiles, int maxFiles) { final double currentRatio = mayUseOffPeak ? comConf.getCompactionRatioOffPeak() : comConf.getCompactionRatio(); // Start off choosing nothing. - List bestSelection = new ArrayList<>(0); - List smallest = mightBeStuck ? new ArrayList<>(0) : null; + List bestSelection = new ArrayList<>(0); + List smallest = mightBeStuck ? new ArrayList<>(0) : null; long bestSize = 0; long smallestSize = Long.MAX_VALUE; @@ -75,7 +74,7 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy { // Consider every different sub list permutation in between start and end with min files. for (int currentEnd = start + minFiles - 1; currentEnd < candidates.size(); currentEnd++) { - List potentialMatchFiles = candidates.subList(start, currentEnd + 1); + List potentialMatchFiles = candidates.subList(start, currentEnd + 1); // Sanity checks if (potentialMatchFiles.size() < minFiles) { @@ -125,8 +124,8 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy { return new ArrayList<>(bestSelection); } - private boolean isBetterSelection(List bestSelection, - long bestSize, List selection, long size, boolean mightBeStuck) { + private boolean isBetterSelection(List bestSelection, long bestSize, + List selection, long size, boolean mightBeStuck) { if (mightBeStuck && bestSize > 0 && size > 0) { // Keep the selection that removes most files for least size. That penaltizes adding // large files to compaction, but not small files, so we don't become totally inefficient @@ -146,13 +145,8 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy { * @param potentialMatchFiles StoreFile list. * @return Sum of StoreFile.getReader().length(); */ - private long getTotalStoreSize(final List potentialMatchFiles) { - long size = 0; - - for (StoreFile s:potentialMatchFiles) { - size += s.getReader().length(); - } - return size; + private long getTotalStoreSize(List potentialMatchFiles) { + return potentialMatchFiles.stream().mapToLong(sf -> sf.getReader().length()).sum(); } /** @@ -163,14 +157,14 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy { * @param currentRatio The ratio to use. * @return a boolean if these files satisfy the ratio constraints. */ - private boolean filesInRatio(final List files, final double currentRatio) { + private boolean filesInRatio(List files, double currentRatio) { if (files.size() < 2) { return true; } long totalFileSize = getTotalStoreSize(files); - for (StoreFile file : files) { + for (HStoreFile file : files) { long singleFileSize = file.getReader().length(); long sumAllOtherFileSizes = totalFileSize - singleFileSize; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java index 5eb15cf4479..a0609bc6930 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/FIFOCompactionPolicy.java @@ -26,11 +26,11 @@ import java.util.List; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; /** * @@ -55,10 +55,9 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy { } @Override - public CompactionRequest selectCompaction(Collection candidateFiles, - List filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak, + public CompactionRequest selectCompaction(Collection candidateFiles, + List filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak, boolean forceMajor) throws IOException { - if(forceMajor){ LOG.warn("Major compaction is not supported for FIFO compaction policy. Ignore the flag."); } @@ -70,13 +69,13 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy { } // Nothing to compact - Collection toCompact = getExpiredStores(candidateFiles, filesCompacting); + Collection toCompact = getExpiredStores(candidateFiles, filesCompacting); CompactionRequest result = new CompactionRequest(toCompact); return result; } @Override - public boolean shouldPerformMajorCompaction(Collection filesToCompact) + public boolean shouldPerformMajorCompaction(Collection filesToCompact) throws IOException { boolean isAfterSplit = StoreUtils.hasReferences(filesToCompact); if(isAfterSplit){ @@ -87,8 +86,8 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy { } @Override - public boolean needsCompaction(Collection storeFiles, - List filesCompacting) { + public boolean needsCompaction(Collection storeFiles, + List filesCompacting) { boolean isAfterSplit = StoreUtils.hasReferences(storeFiles); if(isAfterSplit){ LOG.info("Split detected, delegate to the parent policy."); @@ -97,9 +96,9 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy { return hasExpiredStores(storeFiles); } - private boolean hasExpiredStores(Collection files) { + private boolean hasExpiredStores(Collection files) { long currentTime = EnvironmentEdgeManager.currentTime(); - for(StoreFile sf: files){ + for(HStoreFile sf: files){ // Check MIN_VERSIONS is in HStore removeUnneededFiles long maxTs = sf.getReader().getMaxTimestamp(); long maxTtl = storeConfigInfo.getStoreFileTtl(); @@ -113,11 +112,11 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy { return false; } - private Collection getExpiredStores(Collection files, - Collection filesCompacting) { + private Collection getExpiredStores(Collection files, + Collection filesCompacting) { long currentTime = EnvironmentEdgeManager.currentTime(); - Collection expiredStores = new ArrayList<>(); - for(StoreFile sf: files){ + Collection expiredStores = new ArrayList<>(); + for(HStoreFile sf: files){ // Check MIN_VERSIONS is in HStore removeUnneededFiles long maxTs = sf.getReader().getMaxTimestamp(); long maxTtl = storeConfigInfo.getStoreFileTtl(); 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 9e7f8694b64..172f17a2dfa 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 @@ -28,13 +28,13 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; /** * The default algorithm for selecting files for compaction. @@ -55,7 +55,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy { * @return True if we should run a major compaction. */ @Override - public boolean shouldPerformMajorCompaction(final Collection filesToCompact) + public boolean shouldPerformMajorCompaction(Collection filesToCompact) throws IOException { boolean result = false; long mcTime = getNextMajorCompactTime(filesToCompact); @@ -79,7 +79,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy { } if (filesToCompact.size() == 1) { // Single file - StoreFile sf = filesToCompact.iterator().next(); + HStoreFile sf = filesToCompact.iterator().next(); OptionalLong minTimestamp = sf.getMinimumTimestamp(); long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE; if (sf.isMajorCompactionResult() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) { @@ -113,7 +113,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy { } @Override - protected CompactionRequest createCompactionRequest(ArrayList + protected CompactionRequest createCompactionRequest(ArrayList candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck) throws IOException { if (!tryingMajor) { @@ -155,7 +155,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy { * @param candidates pre-filtrate * @return filtered subset */ - protected ArrayList applyCompactionPolicy(ArrayList candidates, + protected ArrayList applyCompactionPolicy(ArrayList candidates, boolean mayUseOffPeak, boolean mayBeStuck) throws IOException { if (candidates.isEmpty()) { return candidates; @@ -174,7 +174,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy { long[] fileSizes = new long[countOfFiles]; long[] sumSize = new long[countOfFiles]; for (int i = countOfFiles - 1; i >= 0; --i) { - StoreFile file = candidates.get(i); + HStoreFile file = candidates.get(i); fileSizes[i] = file.getReader().length(); // calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo int tooFar = i + comConf.getMaxFilesToCompact() - 1; @@ -209,8 +209,8 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy { * @param filesCompacting files being scheduled to compact. * @return true to schedule a request. */ - public boolean needsCompaction(final Collection storeFiles, - final List filesCompacting) { + public boolean needsCompaction(Collection storeFiles, + List filesCompacting) { int numCandidates = storeFiles.size() - filesCompacting.size(); return numCandidates >= comConf.getMinFilesToCompact(); } 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 5f22d1d3ae9..facc161f608 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 @@ -20,10 +20,10 @@ import java.util.Random; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreUtils; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate; @@ -42,8 +42,8 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { super(conf, storeConfigInfo); } - public List preSelectCompactionForCoprocessor(final Collection candidates, - final List filesCompacting) { + public List preSelectCompactionForCoprocessor(Collection candidates, + List filesCompacting) { return getCurrentEligibleFiles(new ArrayList<>(candidates), filesCompacting); } @@ -53,11 +53,11 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { * on seqId for data consistency. * @return subset copy of candidate list that meets compaction criteria */ - public CompactionRequest selectCompaction(Collection candidateFiles, - final List filesCompacting, final boolean isUserCompaction, - final boolean mayUseOffPeak, final boolean forceMajor) throws IOException { + public CompactionRequest selectCompaction(Collection candidateFiles, + List filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak, + boolean forceMajor) throws IOException { // Preliminary compaction subject to filters - ArrayList candidateSelection = new ArrayList<>(candidateFiles); + ArrayList candidateSelection = new ArrayList<>(candidateFiles); // Stuck and not compacting enough (estimate). It is not guaranteed that we will be // able to compact more if stuck and compacting, because ratio policy excludes some // non-compacting files from consideration during compaction (see getCurrentEligibleFiles). @@ -88,7 +88,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { CompactionRequest result = createCompactionRequest(candidateSelection, isTryingMajor || isAfterSplit, mayUseOffPeak, mayBeStuck); - ArrayList filesToCompact = Lists.newArrayList(result.getFiles()); + ArrayList filesToCompact = Lists.newArrayList(result.getFiles()); removeExcessFiles(filesToCompact, isUserCompaction, isTryingMajor); result.updateFiles(filesToCompact); @@ -99,16 +99,16 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { return result; } - protected abstract CompactionRequest createCompactionRequest(ArrayList - candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck) - throws IOException; + protected abstract CompactionRequest createCompactionRequest( + ArrayList candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, + boolean mayBeStuck) throws IOException; - /* + /** * @param filesToCompact Files to compact. Can be null. * @return True if we should run a major compaction. */ - public abstract boolean shouldPerformMajorCompaction(final Collection filesToCompact) - throws IOException; + public abstract boolean shouldPerformMajorCompaction(Collection filesToCompact) + throws IOException; /** * Used calculation jitter @@ -119,7 +119,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { * @param filesToCompact * @return When to run next major compaction */ - public long getNextMajorCompactTime(Collection filesToCompact) { + public long getNextMajorCompactTime(Collection filesToCompact) { // default = 24hrs long period = comConf.getMajorCompactionPeriod(); if (period <= 0) { @@ -154,16 +154,16 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { return compactionSize > comConf.getThrottlePoint(); } - public abstract boolean needsCompaction(final Collection storeFiles, - final List filesCompacting); + public abstract boolean needsCompaction(Collection storeFiles, + List filesCompacting); - protected ArrayList getCurrentEligibleFiles(ArrayList candidateFiles, - final List filesCompacting) { + protected ArrayList getCurrentEligibleFiles(ArrayList candidateFiles, + final List filesCompacting) { // candidates = all storefiles not already in compaction queue if (!filesCompacting.isEmpty()) { // exclude all files older than the newest file we're currently // compacting. this allows us to preserve contiguity (HBASE-2856) - StoreFile last = filesCompacting.get(filesCompacting.size() - 1); + HStoreFile last = filesCompacting.get(filesCompacting.size() - 1); int idx = candidateFiles.indexOf(last); Preconditions.checkArgument(idx != -1); candidateFiles.subList(0, idx + 1).clear(); @@ -176,7 +176,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { * @return filtered subset exclude all files above maxCompactSize * Also save all references. We MUST compact them */ - protected ArrayList skipLargeFiles(ArrayList candidates, + protected ArrayList skipLargeFiles(ArrayList candidates, boolean mayUseOffpeak) { int pos = 0; while (pos < candidates.size() && !candidates.get(pos).isReference() @@ -195,10 +195,10 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { * @param candidates pre-filtrate * @return filtered subset exclude all bulk load files if configured */ - protected ArrayList filterBulk(ArrayList candidates) { - candidates.removeAll(Collections2.filter(candidates, new Predicate() { + protected ArrayList filterBulk(ArrayList candidates) { + candidates.removeAll(Collections2.filter(candidates, new Predicate() { @Override - public boolean apply(StoreFile input) { + public boolean apply(HStoreFile input) { return input.excludeFromMinorCompaction(); } })); @@ -208,7 +208,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { /** * @param candidates pre-filtrate */ - protected void removeExcessFiles(ArrayList candidates, + protected void removeExcessFiles(ArrayList candidates, boolean isUserCompaction, boolean isMajorCompaction) { int excess = candidates.size() - comConf.getMaxFilesToCompact(); if (excess > 0) { @@ -227,12 +227,12 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy { * @param candidates pre-filtrate * @return filtered subset forget the compactionSelection if we don't have enough files */ - protected ArrayList checkMinFilesCriteria(ArrayList candidates, - int minFiles) { + protected ArrayList checkMinFilesCriteria(ArrayList candidates, + int minFiles) { if (candidates.size() < minFiles) { if (LOG.isDebugEnabled()) { - LOG.debug("Not compacting files because we only have " + candidates.size() - + " files ready for compaction. Need " + minFiles + " to initiate."); + LOG.debug("Not compacting files because we only have " + candidates.size() + + " files ready for compaction. Need " + minFiles + " to initiate."); } candidates.clear(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java index e677ed17f15..9dc8246f261 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactionPolicy.java @@ -30,9 +30,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparator; -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.regionserver.StripeStoreConfig; import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher; @@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.ConcatenatedLists; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.Pair; +import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; @@ -63,12 +63,12 @@ public class StripeCompactionPolicy extends CompactionPolicy { stripePolicy = new ExploringCompactionPolicy(conf, storeConfigInfo); } - public List preSelectFilesForCoprocessor(StripeInformationProvider si, - List filesCompacting) { + public List preSelectFilesForCoprocessor(StripeInformationProvider si, + List filesCompacting) { // We sincerely hope nobody is messing with us with their coprocessors. // If they do, they are very likely to shoot themselves in the foot. // We'll just exclude all the filesCompacting from the list. - ArrayList candidateFiles = new ArrayList<>(si.getStorefiles()); + ArrayList candidateFiles = new ArrayList<>(si.getStorefiles()); candidateFiles.removeAll(filesCompacting); return candidateFiles; } @@ -102,7 +102,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { } public StripeCompactionRequest selectCompaction(StripeInformationProvider si, - List filesCompacting, boolean isOffpeak) throws IOException { + List filesCompacting, boolean isOffpeak) throws IOException { // TODO: first cut - no parallel compactions. To have more fine grained control we // probably need structure more sophisticated than a list. if (!filesCompacting.isEmpty()) { @@ -116,7 +116,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { // This can happen due to region split. We can skip it later; for now preserve // compact-all-things behavior. - Collection allFiles = si.getStorefiles(); + Collection allFiles = si.getStorefiles(); if (StoreUtils.hasReferences(allFiles)) { LOG.debug("There are references in the store; compacting all files"); long targetKvs = estimateTargetKvs(allFiles, config.getInitialCount()).getFirst(); @@ -127,7 +127,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { } int stripeCount = si.getStripeCount(); - List l0Files = si.getLevel0Files(); + List l0Files = si.getLevel0Files(); // See if we need to make new stripes. boolean shouldCompactL0 = (this.config.getLevel0MinFiles() <= l0Files.size()); @@ -157,7 +157,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { return selectSingleStripeCompaction(si, false, canDropDeletesNoL0, isOffpeak); } - public boolean needsCompactions(StripeInformationProvider si, List filesCompacting) { + public boolean needsCompactions(StripeInformationProvider si, List filesCompacting) { // Approximation on whether we need compaction. return filesCompacting.isEmpty() && (StoreUtils.hasReferences(si.getStorefiles()) @@ -166,7 +166,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { } @Override - public boolean shouldPerformMajorCompaction(Collection filesToCompact) + public boolean shouldPerformMajorCompaction(Collection filesToCompact) throws IOException { return false; // there's never a major compaction! } @@ -182,7 +182,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { */ protected boolean needsSingleStripeCompaction(StripeInformationProvider si) { int minFiles = this.config.getStripeCompactMinFiles(); - for (List stripe : si.getStripes()) { + for (List stripe : si.getStripes()) { if (stripe.size() >= minFiles) return true; } return false; @@ -190,20 +190,20 @@ public class StripeCompactionPolicy extends CompactionPolicy { protected StripeCompactionRequest selectSingleStripeCompaction(StripeInformationProvider si, boolean includeL0, boolean canDropDeletesWithoutL0, boolean isOffpeak) throws IOException { - ArrayList> stripes = si.getStripes(); + ArrayList> stripes = si.getStripes(); int bqIndex = -1; - List bqSelection = null; + List bqSelection = null; int stripeCount = stripes.size(); long bqTotalSize = -1; for (int i = 0; i < stripeCount; ++i) { // If we want to compact L0 to drop deletes, we only want whole-stripe compactions. // So, pass includeL0 as 2nd parameter to indicate that. - List selection = selectSimpleCompaction(stripes.get(i), + List selection = selectSimpleCompaction(stripes.get(i), !canDropDeletesWithoutL0 && includeL0, isOffpeak); if (selection.isEmpty()) continue; long size = 0; - for (StoreFile sf : selection) { + for (HStoreFile sf : selection) { size += sf.getReader().length(); } if (bqSelection == null || selection.size() > bqSelection.size() || @@ -217,7 +217,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { LOG.debug("No good compaction is possible in any stripe"); return null; } - List filesToCompact = new ArrayList<>(bqSelection); + List filesToCompact = new ArrayList<>(bqSelection); // See if we can, and need to, split this stripe. int targetCount = 1; long targetKvs = Long.MAX_VALUE; @@ -244,9 +244,9 @@ public class StripeCompactionPolicy extends CompactionPolicy { StripeCompactionRequest req; if (includeL0) { assert hasAllFiles; - List l0Files = si.getLevel0Files(); + List l0Files = si.getLevel0Files(); LOG.debug("Adding " + l0Files.size() + " files to compaction to be able to drop deletes"); - ConcatenatedLists sfs = new ConcatenatedLists<>(); + ConcatenatedLists sfs = new ConcatenatedLists<>(); sfs.addSublist(filesToCompact); sfs.addSublist(l0Files); req = new BoundaryStripeCompactionRequest(sfs, si.getStripeBoundaries()); @@ -267,33 +267,16 @@ public class StripeCompactionPolicy extends CompactionPolicy { * @param allFilesOnly Whether a compaction of all-or-none files is needed. * @return The resulting selection. */ - private List selectSimpleCompaction( - List sfs, boolean allFilesOnly, boolean isOffpeak) { + private List selectSimpleCompaction( + List sfs, boolean allFilesOnly, boolean isOffpeak) { int minFilesLocal = Math.max( allFilesOnly ? sfs.size() : 0, this.config.getStripeCompactMinFiles()); int maxFilesLocal = Math.max(this.config.getStripeCompactMaxFiles(), minFilesLocal); return stripePolicy.applyCompactionPolicy(sfs, false, isOffpeak, minFilesLocal, maxFilesLocal); } - /** - * Selects the compaction that compacts all files (to be removed later). - * @param si StoreFileManager. - * @param targetStripeCount Target stripe count. - * @param targetSize Target stripe size. - * @return The compaction. - */ - private StripeCompactionRequest selectCompactionOfAllFiles(StripeInformationProvider si, - int targetStripeCount, long targetSize) { - Collection allFiles = si.getStorefiles(); - SplitStripeCompactionRequest request = new SplitStripeCompactionRequest( - allFiles, OPEN_KEY, OPEN_KEY, targetStripeCount, targetSize); - request.setMajorRangeFull(); - LOG.debug("Selecting a compaction that includes all " + allFiles.size() + " files"); - return request; - } - private StripeCompactionRequest selectNewStripesCompaction(StripeInformationProvider si) { - List l0Files = si.getLevel0Files(); + List l0Files = si.getLevel0Files(); Pair kvsAndCount = estimateTargetKvs(l0Files, config.getInitialCount()); LOG.debug("Creating " + kvsAndCount.getSecond() + " initial stripes with " + kvsAndCount.getFirst() + " kvs each via L0 compaction of " + l0Files.size() + " files"); @@ -312,9 +295,9 @@ public class StripeCompactionPolicy extends CompactionPolicy { long timestampCutoff = EnvironmentEdgeManager.currentTime() - cfTtl; // Merge the longest sequence of stripes where all files have expired, if any. int start = -1, bestStart = -1, length = 0, bestLength = 0; - ArrayList> stripes = si.getStripes(); + ArrayList> stripes = si.getStripes(); OUTER: for (int i = 0; i < stripes.size(); ++i) { - for (StoreFile storeFile : stripes.get(i)) { + for (HStoreFile storeFile : stripes.get(i)) { if (storeFile.getReader().getMaxTimestamp() < timestampCutoff) continue; // Found non-expired file, this stripe has to stay. if (length > bestLength) { @@ -345,7 +328,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { } LOG.debug("Merging " + bestLength + " stripes to delete expired store files"); int endIndex = bestStart + bestLength - 1; - ConcatenatedLists sfs = new ConcatenatedLists<>(); + ConcatenatedLists sfs = new ConcatenatedLists<>(); sfs.addAllSublists(stripes.subList(bestStart, endIndex + 1)); SplitStripeCompactionRequest result = new SplitStripeCompactionRequest(sfs, si.getStartRow(bestStart), si.getEndRow(endIndex), 1, Long.MAX_VALUE); @@ -355,23 +338,23 @@ public class StripeCompactionPolicy extends CompactionPolicy { return result; } - private static long getTotalKvCount(final Collection candidates) { + private static long getTotalKvCount(final Collection candidates) { long totalSize = 0; - for (StoreFile storeFile : candidates) { + for (HStoreFile storeFile : candidates) { totalSize += storeFile.getReader().getEntries(); } return totalSize; } - public static long getTotalFileSize(final Collection candidates) { + public static long getTotalFileSize(final Collection candidates) { long totalSize = 0; - for (StoreFile storeFile : candidates) { + for (HStoreFile storeFile : candidates) { totalSize += storeFile.getReader().length(); } return totalSize; } - private Pair estimateTargetKvs(Collection files, double splitCount) { + private Pair estimateTargetKvs(Collection files, double splitCount) { // If the size is larger than what we target, we don't want to split into proportionally // larger parts and then have to split again very soon. So, we will increase the multiplier // by one until we get small enough parts. E.g. 5Gb stripe that should have been split into @@ -452,7 +435,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { this.targetBoundaries = targetBoundaries; } - public BoundaryStripeCompactionRequest(Collection files, + public BoundaryStripeCompactionRequest(Collection files, List targetBoundaries) { this(new CompactionRequest(files), targetBoundaries); } @@ -494,16 +477,11 @@ public class StripeCompactionPolicy extends CompactionPolicy { } public SplitStripeCompactionRequest( - CompactionRequest request, byte[] startRow, byte[] endRow, long targetKvs) { - this(request, startRow, endRow, Integer.MAX_VALUE, targetKvs); - } - - public SplitStripeCompactionRequest( - Collection files, byte[] startRow, byte[] endRow, long targetKvs) { + Collection files, byte[] startRow, byte[] endRow, long targetKvs) { this(files, startRow, endRow, Integer.MAX_VALUE, targetKvs); } - public SplitStripeCompactionRequest(Collection files, + public SplitStripeCompactionRequest(Collection files, byte[] startRow, byte[] endRow, int targetCount, long targetKvs) { this(new CompactionRequest(files), startRow, endRow, targetCount, targetKvs); } @@ -524,7 +502,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { /** The information about stripes that the policy needs to do its stuff */ public static interface StripeInformationProvider { - public Collection getStorefiles(); + public Collection getStorefiles(); /** * Gets the start row for a given stripe. @@ -543,7 +521,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { /** * @return Level 0 files. */ - public List getLevel0Files(); + public List getLevel0Files(); /** * @return All stripe boundaries; including the open ones on both ends. @@ -553,7 +531,7 @@ public class StripeCompactionPolicy extends CompactionPolicy { /** * @return The stripes. */ - public ArrayList> getStripes(); + public ArrayList> getStripes(); /** * @return Stripe count. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java index c75b24c9a5f..f552f9684e4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/StripeCompactor.java @@ -24,15 +24,15 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; /** * This is the placeholder for stripe compactor. The implementation, as well as the proper javadoc, @@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.util.Bytes; public class StripeCompactor extends AbstractMultiOutputCompactor { private static final Log LOG = LogFactory.getLog(StripeCompactor.class); - public StripeCompactor(Configuration conf, Store store) { + public StripeCompactor(Configuration conf, HStore store) { super(conf, store); } 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 615d492d2b8..b371b3eedb1 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 @@ -37,16 +37,21 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare; import org.apache.hadoop.hbase.mob.MobUtils; 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.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSTableDescriptors; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.yetus.audience.InterfaceAudience; + import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream; import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException; @@ -54,10 +59,6 @@ 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; -import org.apache.hadoop.hbase.util.Threads; /** * Utility class to help read/write the Snapshot Manifest. @@ -228,20 +229,20 @@ public final class SnapshotManifest { // 2. iterate through all the stores in the region LOG.debug("Creating references for hfiles"); - for (Store store : region.getStores()) { + for (HStore store : region.getStores()) { // 2.1. build the snapshot reference for the store Object familyData = visitor.familyOpen(regionData, store.getColumnFamilyDescriptor().getName()); monitor.rethrowException(); - List storeFiles = new ArrayList<>(store.getStorefiles()); + List storeFiles = new ArrayList<>(store.getStorefiles()); if (LOG.isDebugEnabled()) { LOG.debug("Adding snapshot references for " + storeFiles + " hfiles"); } // 2.2. iterate through all the store's files and create "references". for (int i = 0, sz = storeFiles.size(); i < sz; i++) { - StoreFile storeFile = storeFiles.get(i); + HStoreFile storeFile = storeFiles.get(i); monitor.rethrowException(); // create "reference" to this store file. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java index 08a1512824a..c457e224dac 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/LoadIncrementalHFiles.java @@ -34,6 +34,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -702,7 +703,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { Multimap regionGroups, final LoadQueueItem item, final Table table, final Pair startEndKeys) throws IOException { Path hfilePath = item.getFilePath(); - byte[] first, last; + Optional first, last; try (HFile.Reader hfr = HFile.createReader(hfilePath.getFileSystem(getConf()), hfilePath, new CacheConfig(getConf()), true, getConf())) { hfr.loadFileInfo(); @@ -713,19 +714,19 @@ public class LoadIncrementalHFiles extends Configured implements Tool { return new Pair<>(null, hfilePath.getName()); } - LOG.info("Trying to load hfile=" + hfilePath + " first=" + Bytes.toStringBinary(first) + - " last=" + Bytes.toStringBinary(last)); - if (first == null || last == null) { - assert first == null && last == null; + LOG.info("Trying to load hfile=" + hfilePath + " first=" + first.map(Bytes::toStringBinary) + + " last=" + last.map(Bytes::toStringBinary)); + if (!first.isPresent() || !last.isPresent()) { + assert !first.isPresent() && !last.isPresent(); // TODO what if this is due to a bad HFile? LOG.info("hfile " + hfilePath + " has no entries, skipping"); return null; } - if (Bytes.compareTo(first, last) > 0) { - throw new IllegalArgumentException( - "Invalid range: " + Bytes.toStringBinary(first) + " > " + Bytes.toStringBinary(last)); + if (Bytes.compareTo(first.get(), last.get()) > 0) { + throw new IllegalArgumentException("Invalid range: " + Bytes.toStringBinary(first.get()) + + " > " + Bytes.toStringBinary(last.get())); } - int idx = Arrays.binarySearch(startEndKeys.getFirst(), first, Bytes.BYTES_COMPARATOR); + int idx = Arrays.binarySearch(startEndKeys.getFirst(), first.get(), Bytes.BYTES_COMPARATOR); if (idx < 0) { // not on boundary, returns -(insertion index). Calculate region it // would be in. @@ -753,7 +754,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool { "Please use hbck tool to fix it first."); } - boolean lastKeyInRange = Bytes.compareTo(last, startEndKeys.getSecond()[idx]) < 0 || + boolean lastKeyInRange = Bytes.compareTo(last.get(), startEndKeys.getSecond()[idx]) < 0 || Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY); if (!lastKeyInRange) { List lqis = splitStoreFile(item, table, @@ -834,8 +835,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool { " for family " + builder.getNameAsString()); } reader.loadFileInfo(); - byte[] first = reader.getFirstRowKey(); - byte[] last = reader.getLastRowKey(); + byte[] first = reader.getFirstRowKey().get(); + byte[] last = reader.getLastRowKey().get(); LOG.info("Trying to figure out region boundaries hfile=" + hfile + " first=" + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last)); 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 d4c62542366..e8ee3ffdc6e 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 @@ -1,5 +1,4 @@ -/* - * +/** * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information @@ -16,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.util; import java.io.DataInput; @@ -26,17 +24,17 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; -import org.apache.yetus.audience.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; import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.yetus.audience.InterfaceAudience; /** - * Handles Bloom filter initialization based on configuration and serialized - * metadata in the reader and writer of {@link org.apache.hadoop.hbase.regionserver.StoreFile}. + * Handles Bloom filter initialization based on configuration and serialized metadata in the reader + * and writer of {@link org.apache.hadoop.hbase.regionserver.HStoreFile}. */ @InterfaceAudience.Private public final class BloomFilterFactory { @@ -155,7 +153,7 @@ public final class BloomFilterFactory { /** * Creates a new general (Row or RowCol) Bloom filter at the time of - * {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing. + * {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing. * * @param conf * @param cacheConf @@ -203,7 +201,7 @@ public final class BloomFilterFactory { /** * Creates a new Delete Family Bloom filter at the time of - * {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing. + * {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing. * @param conf * @param cacheConf * @param maxKeys an estimate of the number of keys we expect to insert. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 53160e6182d..0c95e7ec8ec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -50,6 +50,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.SortedSet; @@ -858,13 +859,13 @@ public class HBaseFsck extends Configured implements Closeable { new CacheConfig(getConf()), true, getConf()); if ((reader.getFirstKey() != null) && ((storeFirstKey == null) || (comparator.compare(storeFirstKey, - ((KeyValue.KeyOnlyKeyValue) reader.getFirstKey()).getKey()) > 0))) { - storeFirstKey = ((KeyValue.KeyOnlyKeyValue)reader.getFirstKey()).getKey(); + ((KeyValue.KeyOnlyKeyValue) reader.getFirstKey().get()).getKey()) > 0))) { + storeFirstKey = ((KeyValue.KeyOnlyKeyValue)reader.getFirstKey().get()).getKey(); } if ((reader.getLastKey() != null) && ((storeLastKey == null) || (comparator.compare(storeLastKey, - ((KeyValue.KeyOnlyKeyValue)reader.getLastKey()).getKey())) < 0)) { - storeLastKey = ((KeyValue.KeyOnlyKeyValue)reader.getLastKey()).getKey(); + ((KeyValue.KeyOnlyKeyValue)reader.getLastKey().get()).getKey())) < 0)) { + storeLastKey = ((KeyValue.KeyOnlyKeyValue)reader.getLastKey().get()).getKey(); } reader.close(); } @@ -961,10 +962,10 @@ public class HBaseFsck extends Configured implements Closeable { CacheConfig cacheConf = new CacheConfig(getConf()); hf = HFile.createReader(fs, hfile.getPath(), cacheConf, true, getConf()); hf.loadFileInfo(); - Cell startKv = hf.getFirstKey(); - start = CellUtil.cloneRow(startKv); - Cell endKv = hf.getLastKey(); - end = CellUtil.cloneRow(endKv); + Optional startKv = hf.getFirstKey(); + start = CellUtil.cloneRow(startKv.get()); + Optional endKv = hf.getLastKey(); + end = CellUtil.cloneRow(endKv.get()); } catch (IOException ioe) { LOG.warn("Problem reading orphan file " + hfile + ", skipping"); continue; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java index 34fd6f71de2..369aed768f3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowBloomContext.java @@ -17,14 +17,15 @@ */ package org.apache.hadoop.hbase.util; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY; + import java.io.IOException; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.HFile.Writer; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.yetus.audience.InterfaceAudience; /** * Handles ROW bloom related context. It works with both ByteBufferedCell and byte[] backed cells @@ -39,7 +40,7 @@ public class RowBloomContext extends BloomContext { public void addLastBloomKey(Writer writer) throws IOException { if (this.getLastCell() != null) { byte[] key = CellUtil.copyRow(this.getLastCell()); - writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key); + writer.appendFileInfo(LAST_BLOOM_KEY, key); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java index 9ead570cf72..90cbcb05988 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/RowColBloomContext.java @@ -17,14 +17,15 @@ */ package org.apache.hadoop.hbase.util; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY; + import java.io.IOException; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CellUtil; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.io.hfile.HFile.Writer; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.yetus.audience.InterfaceAudience; /** * Handles ROWCOL bloom related context. It can work with both BytebufferdCells @@ -43,7 +44,7 @@ public class RowColBloomContext extends BloomContext { Cell firstOnRow = CellUtil.createFirstOnRowCol(this.getLastCell()); // This copy happens only once when the writer is closed byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(firstOnRow); - writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key); + writer.appendFileInfo(LAST_BLOOM_KEY, key); } } diff --git a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp index 04aafa712a1..4b25635b0f6 100644 --- a/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp +++ b/hbase-server/src/main/resources/hbase-webapps/regionserver/region.jsp @@ -95,7 +95,7 @@ List stores = region.getStores(); for (Store store : stores) { String cf = store.getColumnFamilyName(); - Collection storeFiles = store.getStorefiles(); %> + Collection storeFiles = store.getStorefiles(); %>

Column Family: <%= cf %>

diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java index 85f65e895c9..6e7bf7d3dc7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestIOFencing.java @@ -20,8 +20,6 @@ package org.apache.hadoop.hbase; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; - import java.io.IOException; import java.util.Collection; import java.util.List; @@ -41,16 +39,14 @@ import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionServerServices; import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.MiscTests; import org.apache.hadoop.hbase.util.Bytes; @@ -59,6 +55,10 @@ import org.apache.hadoop.hbase.wal.WAL; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; + /** * Test for the case where a regionserver going down has enough cycles to do damage to regions that * have actually been assigned elsehwere. @@ -201,7 +201,7 @@ public class TestIOFencing { } @Override - protected void completeCompaction(Collection compactedFiles) throws IOException { + protected void completeCompaction(Collection compactedFiles) throws IOException { try { r.compactionsWaiting.countDown(); r.compactionsBlocked.await(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java index dfdd11eb6f6..9c100a257b6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin1.java @@ -49,12 +49,10 @@ import org.apache.hadoop.hbase.TableNotDisabledException; import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.exceptions.MergeRegionException; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.master.LoadBalancer; -import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest; +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.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -69,6 +67,9 @@ import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TestName; +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest; + /** * Class to test HBaseAdmin. * Spins up the minicluster once at test start and then takes it down afterward. @@ -1320,8 +1321,8 @@ public class TestAdmin1 { List regions = TEST_UTIL.getMiniHBaseCluster().getRegions(tableName); for (HRegion r : regions) { - Store store = r.getStore(Bytes.toBytes(fn)); - for (StoreFile sf : store.getStorefiles()) { + HStore store = r.getStore(Bytes.toBytes(fn)); + for (HStoreFile sf : store.getStorefiles()) { assertTrue(sf.toString().contains(fn)); assertTrue("Column family " + fn + " should have 3 copies", FSUtils.getDefaultReplication(TEST_UTIL.getTestFileSystem(), sf.getPath()) == (sf @@ -1329,7 +1330,7 @@ public class TestAdmin1 { } store = r.getStore(Bytes.toBytes(fn1)); - for (StoreFile sf : store.getStorefiles()) { + for (HStoreFile sf : store.getStorefiles()) { assertTrue(sf.toString().contains(fn1)); assertTrue("Column family " + fn1 + " should have only 1 copy", 1 == sf.getFileInfo() .getFileStatus().getReplication()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java index ac404bb6adf..061d0973453 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAvoidCellReferencesIntoShippedBlocks.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.CachedBlock; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.Region; @@ -253,14 +254,16 @@ public class TestAvoidCellReferencesIntoShippedBlocks { } public static class CompactorRegionObserver implements RegionObserver { + @Override public InternalScanner preCompactScannerOpen(ObserverContext c, - Store store, List scanners, ScanType scanType, long earliestPutTs, - InternalScanner s, CompactionLifeCycleTracker request, long readPoint) throws IOException { - return createCompactorScanner(store, scanners, scanType, earliestPutTs); + Store store, List scanners, ScanType scanType, + long earliestPutTs, InternalScanner s, CompactionLifeCycleTracker request, long readPoint) + throws IOException { + return createCompactorScanner((HStore) store, scanners, scanType, earliestPutTs); } - private InternalScanner createCompactorScanner(Store store, + private InternalScanner createCompactorScanner(HStore store, List scanners, ScanType scanType, long earliestPutTs) throws IOException { return new CompactorStoreScanner(store, store.getScanInfo(), OptionalInt.empty(), scanners, @@ -270,7 +273,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks { private static class CompactorStoreScanner extends StoreScanner { - public CompactorStoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions, + public CompactorStoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions, List scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException { super(store, scanInfo, maxVersions, scanners, scanType, smallestReadPoint, earliestPutTs); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java index ae4f8a87c8b..aaddd34a365 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java @@ -54,6 +54,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.ClusterStatus.Option; import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -69,7 +70,6 @@ import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.ClusterStatus.Option; import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; @@ -105,6 +105,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateR import org.apache.hadoop.hbase.regionserver.DelegatingKeyValueScanner; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.regionserver.Region; @@ -554,7 +555,7 @@ public class TestFromClientSide { } class MyStoreScanner extends StoreScanner { - public MyStoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet columns, + public MyStoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet columns, long readPt) throws IOException { super(store, scanInfo, scan, columns, readPt); } @@ -588,7 +589,7 @@ public class TestFromClientSide { public KeyValueScanner preStoreScannerOpen(ObserverContext c, Store store, Scan scan, NavigableSet targetCols, KeyValueScanner s, final long readPt) throws IOException { - return new MyStoreScanner(store, store.getScanInfo(), scan, targetCols, readPt); + return new MyStoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, readPt); } } 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 9938c1894cc..ba1e222057e 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 @@ -19,6 +19,11 @@ package org.apache.hadoop.hbase.coprocessor; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + import java.io.IOException; import java.util.List; import java.util.Map; @@ -57,16 +62,12 @@ 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.regionserver.compactions.CompactionLifeCycleTracker; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; /** * A sample region observer that tests the RegionObserver interface. @@ -202,13 +203,13 @@ public class SimpleRegionObserver implements RegionObserver { @Override public void preCompactSelection(ObserverContext c, Store store, - List candidates, CompactionLifeCycleTracker tracker) throws IOException { + List candidates, CompactionLifeCycleTracker tracker) throws IOException { ctPreCompactSelect.incrementAndGet(); } @Override public void postCompactSelection(ObserverContext c, Store store, - ImmutableList selected, CompactionLifeCycleTracker tracker) { + ImmutableList selected, CompactionLifeCycleTracker tracker) { ctPostCompactSelect.incrementAndGet(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java index 9c06c3ee02d..afeb7637249 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverScannerOpenHook.java @@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.CoprocessorTests; @@ -116,7 +115,7 @@ public class TestRegionObserverScannerOpenHook { Store store, Scan scan, NavigableSet targetCols, KeyValueScanner s, long readPt) throws IOException { scan.setFilter(new NoDataFilter()); - return new StoreScanner(store, store.getScanInfo(), scan, targetCols, readPt); + return new StoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, readPt); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java index 0fd3cdb357a..37d6b8fa876 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestHalfStoreFileReader.java @@ -96,7 +96,7 @@ public class TestHalfStoreFileReader { HFile.Reader r = HFile.createReader(fs, p, cacheConf, true, conf); r.loadFileInfo(); - Cell midKV = r.midkey(); + Cell midKV = r.midKey().get(); byte[] midkey = CellUtil.cloneRow(midKV); // System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey)); @@ -155,7 +155,7 @@ public class TestHalfStoreFileReader { HFile.Reader r = HFile.createReader(fs, p, cacheConf, true, conf); r.loadFileInfo(); - Cell midKV = r.midkey(); + Cell midKV = r.midKey().get(); byte[] midkey = CellUtil.cloneRow(midKV); Reference bottom = new Reference(midkey, Reference.Range.bottom); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java index 62a7c489b3f..13589fb097a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java @@ -164,8 +164,8 @@ public class TestHFile { w.close(); Reader r = HFile.createReader(fs, f, cacheConf, true, conf); r.loadFileInfo(); - assertNull(r.getFirstKey()); - assertNull(r.getLastKey()); + assertFalse(r.getFirstKey().isPresent()); + assertFalse(r.getLastKey().isPresent()); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java index 3d1af90ade0..82c0ecaacc0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlockIndex.java @@ -571,7 +571,7 @@ public class TestHFileBlockIndex { boolean hasArrayIndexOutOfBoundsException = false; try { // get the mid-key. - reader.midkey(); + reader.midKey(); } catch (ArrayIndexOutOfBoundsException e) { hasArrayIndexOutOfBoundsException = true; } finally { @@ -649,8 +649,8 @@ public class TestHFileBlockIndex { assertEquals(expectedNumLevels, reader.getTrailer().getNumDataIndexLevels()); - assertTrue(Bytes.equals(keys[0], ((KeyValue)reader.getFirstKey()).getKey())); - assertTrue(Bytes.equals(keys[NUM_KV - 1], ((KeyValue)reader.getLastKey()).getKey())); + assertTrue(Bytes.equals(keys[0], ((KeyValue)reader.getFirstKey().get()).getKey())); + assertTrue(Bytes.equals(keys[NUM_KV - 1], ((KeyValue)reader.getLastKey().get()).getKey())); LOG.info("Last key: " + Bytes.toStringBinary(keys[NUM_KV - 1])); for (boolean pread : new boolean[] { false, true }) { @@ -706,7 +706,7 @@ public class TestHFileBlockIndex { // Validate the mid-key. assertEquals( Bytes.toStringBinary(blockKeys.get((blockKeys.size() - 1) / 2)), - reader.midkey()); + reader.midKey()); assertEquals(UNCOMPRESSED_INDEX_SIZES[testI], reader.getTrailer().getUncompressedDataIndexSize()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java index 5cc25803f9b..5f5cb7415b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileSeek.java @@ -186,8 +186,8 @@ public class TestHFileSeek extends TestCase { Reader reader = HFile.createReaderFromStream(path, fsdis, fs.getFileStatus(path).getLen(), new CacheConfig(conf), conf); reader.loadFileInfo(); - KeySampler kSampler = new KeySampler(rng, ((KeyValue) reader.getFirstKey()).getKey(), - ((KeyValue) reader.getLastKey()).getKey(), keyLenGen); + KeySampler kSampler = new KeySampler(rng, ((KeyValue) reader.getFirstKey().get()).getKey(), + ((KeyValue) reader.getLastKey().get()).getKey(), keyLenGen); HFileScanner scanner = reader.getScanner(false, USE_PREAD); BytesWritable key = new BytesWritable(); timer.reset(); 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 12aed50f70d..8f0c5d6e283 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 @@ -723,8 +723,8 @@ public class TestMobCompactor { @Override public void preCompactSelection(ObserverContext c, Store store, - List candidates, CompactionLifeCycleTracker tracker) throws IOException { - + List candidates, CompactionLifeCycleTracker tracker) + throws IOException { int count = candidates.size(); if (count >= 2) { for (int i = 0; i < count - 2; i++) { @@ -815,7 +815,7 @@ public class TestMobCompactor { Assert.assertTrue(hasFiles); Path path = files[0].getPath(); CacheConfig cacheConf = new CacheConfig(conf); - StoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf, + HStoreFile 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 b8e1204028c..6681a969b72 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,6 +18,7 @@ */ package org.apache.hadoop.hbase.mob.compactions; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; @@ -71,7 +72,6 @@ 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; @@ -81,7 +81,6 @@ 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 org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; @@ -494,12 +493,12 @@ public class TestPartitionedMobCompactor { PartitionedMobCompactionRequest request = select(files, isForceAllFiles); // Make sure that there is no del Partitions - Assert.assertTrue(request.getDelPartitions().size() == 0); + assertTrue(request.getDelPartitions().size() == 0); // Make sure that when there is no startKey/endKey for partition. for (CompactionPartition p : request.getCompactionPartitions()) { - Assert.assertTrue(p.getStartKey() == null); - Assert.assertTrue(p.getEndKey() == null); + assertTrue(p.getStartKey() == null); + assertTrue(p.getEndKey() == null); } return null; } @@ -530,18 +529,18 @@ public class TestPartitionedMobCompactor { } PartitionedMobCompactionRequest request = select(files, isForceAllFiles); - Assert.assertTrue(request.getDelPartitions().size() == delPartitionSize); + assertTrue(request.getDelPartitions().size() == delPartitionSize); if (request.getDelPartitions().size() > 0) { for (CompactionPartition p : request.getCompactionPartitions()) { - Assert.assertTrue(p.getStartKey() != null); - Assert.assertTrue(p.getEndKey() != null); + assertTrue(p.getStartKey() != null); + assertTrue(p.getEndKey() != null); } } try { for (CompactionDelPartition delPartition : request.getDelPartitions()) { for (Path newDelPath : delPartition.listDelFiles()) { - StoreFile sf = + HStoreFile sf = 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. @@ -553,11 +552,11 @@ public class TestPartitionedMobCompactor { // Make sure that CompactionDelPartitions does not overlap CompactionDelPartition prevDelP = null; for (CompactionDelPartition delP : request.getDelPartitions()) { - Assert.assertTrue( + assertTrue( Bytes.compareTo(delP.getId().getStartKey(), delP.getId().getEndKey()) <= 0); if (prevDelP != null) { - Assert.assertTrue( + assertTrue( Bytes.compareTo(prevDelP.getId().getEndKey(), delP.getId().getStartKey()) < 0); } } @@ -567,7 +566,7 @@ public class TestPartitionedMobCompactor { // Make sure that only del files within key range for a partition is included in compaction. // compact the mob files by partitions in parallel. for (CompactionPartition partition : request.getCompactionPartitions()) { - List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); + List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); if (!request.getDelPartitions().isEmpty()) { if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(), partition.getEndKey()) > 0) || (Bytes.compareTo( @@ -575,23 +574,23 @@ public class TestPartitionedMobCompactor { .getEndKey(), partition.getStartKey()) < 0))) { if (delFiles.size() > 0) { - Assert.assertTrue(delFiles.size() == 1); + assertTrue(delFiles.size() == 1); affectedPartitions += delFiles.size(); - Assert.assertTrue(Bytes.compareTo(partition.getStartKey(), - CellUtil.cloneRow(delFiles.get(0).getLastKey())) <= 0); - Assert.assertTrue(Bytes.compareTo(partition.getEndKey(), - CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey())) >= 0); + assertTrue(Bytes.compareTo(partition.getStartKey(), + CellUtil.cloneRow(delFiles.get(0).getLastKey().get())) <= 0); + assertTrue(Bytes.compareTo(partition.getEndKey(), + CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey().get())) >= 0); } } } } // The del file is only included in one partition - Assert.assertTrue(affectedPartitions == PartitionsIncludeDelFiles); + assertTrue(affectedPartitions == PartitionsIncludeDelFiles); } finally { for (CompactionDelPartition delPartition : request.getDelPartitions()) { - for (StoreFile storeFile : delPartition.getStoreFiles()) { + for (HStoreFile storeFile : delPartition.getStoreFiles()) { try { - storeFile.closeReader(true); + storeFile.closeStoreFile(true); } catch (IOException e) { LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e); } @@ -679,19 +678,19 @@ public class TestPartitionedMobCompactor { // Make sure that when there is no del files, there will be no startKey/endKey for partition. if (request.getDelPartitions().size() == 0) { for (CompactionPartition p : request.getCompactionPartitions()) { - Assert.assertTrue(p.getStartKey() == null); - Assert.assertTrue(p.getEndKey() == null); + assertTrue(p.getStartKey() == null); + assertTrue(p.getEndKey() == null); } } // Make sure that CompactionDelPartitions does not overlap CompactionDelPartition prevDelP = null; for (CompactionDelPartition delP : request.getDelPartitions()) { - Assert.assertTrue(Bytes.compareTo(delP.getId().getStartKey(), + assertTrue(Bytes.compareTo(delP.getId().getStartKey(), delP.getId().getEndKey()) <= 0); if (prevDelP != null) { - Assert.assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(), + assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(), delP.getId().getStartKey()) < 0); } } @@ -699,25 +698,24 @@ public class TestPartitionedMobCompactor { // Make sure that only del files within key range for a partition is included in compaction. // compact the mob files by partitions in parallel. for (CompactionPartition partition : request.getCompactionPartitions()) { - List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); + List delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions()); if (!request.getDelPartitions().isEmpty()) { if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(), partition.getEndKey()) > 0) || (Bytes.compareTo( request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId() .getEndKey(), partition.getStartKey()) < 0))) { if (delFiles.size() > 0) { - Assert.assertTrue(Bytes - .compareTo(partition.getStartKey(), delFiles.get(0).getFirstKey().getRowArray()) - >= 0); - Assert.assertTrue(Bytes.compareTo(partition.getEndKey(), - delFiles.get(delFiles.size() - 1).getLastKey().getRowArray()) <= 0); + assertTrue(Bytes.compareTo(partition.getStartKey(), + delFiles.get(0).getFirstKey().get().getRowArray()) >= 0); + assertTrue(Bytes.compareTo(partition.getEndKey(), + delFiles.get(delFiles.size() - 1).getLastKey().get().getRowArray()) <= 0); } } } } // assert the compaction type - Assert.assertEquals(type, request.type); + assertEquals(type, request.type); // assert get the right partitions compareCompactedPartitions(expected, request.compactionPartitions); // assert get the right del files @@ -750,8 +748,8 @@ public class TestPartitionedMobCompactor { } List newDelPaths = compactDelFiles(request, delFilePaths); // assert the del files are merged. - Assert.assertEquals(expectedFileCount, newDelPaths.size()); - Assert.assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths)); + assertEquals(expectedFileCount, newDelPaths.size()); + assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths)); return null; } }; @@ -784,9 +782,9 @@ public class TestPartitionedMobCompactor { } Collections.sort(expected); Collections.sort(actualKeys); - Assert.assertEquals(expected.size(), actualKeys.size()); + assertEquals(expected.size(), actualKeys.size()); for (int i = 0; i < expected.size(); i++) { - Assert.assertEquals(expected.get(i), actualKeys.get(i)); + assertEquals(expected.get(i), actualKeys.get(i)); } } @@ -802,7 +800,7 @@ public class TestPartitionedMobCompactor { } } for (Path f : delFiles) { - Assert.assertTrue(delMap.containsKey(f)); + assertTrue(delMap.containsKey(f)); } } @@ -874,10 +872,10 @@ public class TestPartitionedMobCompactor { * @return the cell size */ private int countDelCellsInDelFiles(List paths) throws IOException { - List sfs = new ArrayList<>(); + List sfs = new ArrayList<>(); int size = 0; for (Path path : paths) { - StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); + HStoreFile 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/quotas/SpaceQuotaHelperForTests.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java index 68c5d195c5e..86df39fe529 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/quotas/SpaceQuotaHelperForTests.java @@ -38,16 +38,15 @@ import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter.Predicate; -import org.apache.yetus.audience.InterfaceAudience; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; import org.junit.rules.TestName; import org.apache.hadoop.hbase.shaded.com.google.common.collect.HashMultimap; @@ -435,10 +434,9 @@ public class SpaceQuotaHelperForTests { @Override public boolean evaluate() throws Exception { for (HRegion region : cluster.getRegions(tn)) { - for (Store store : region.getStores()) { - HStore hstore = (HStore) store; - Collection files = - hstore.getStoreEngine().getStoreFileManager().getCompactedfiles(); + for (HStore store : region.getStores()) { + Collection files = + store.getStoreEngine().getStoreFileManager().getCompactedfiles(); if (null != files && !files.isEmpty()) { LOG.debug(region.getRegionInfo().getEncodedName() + " still has compacted files"); return false; 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 b33b45d0cf7..58691c420c2 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 @@ -19,9 +19,6 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -32,9 +29,12 @@ import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; + public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy { - protected ArrayList sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes) + protected ArrayList sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes) throws IOException { ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge(); EnvironmentEdgeManager.injectEdge(timeMachine); @@ -45,17 +45,17 @@ public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy ageInDisk.add(0L); } - ArrayList ret = Lists.newArrayList(); + ArrayList ret = Lists.newArrayList(); for (int i = 0; i < sizes.length; i++) { - MockStoreFile msf = - new MockStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i); + MockHStoreFile msf = + new MockHStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i); msf.setTimeRangeTracker(new TimeRangeTracker(minTimestamps[i], maxTimestamps[i])); ret.add(msf); } return ret; } - protected void compactEquals(long now, ArrayList candidates, long[] expectedFileSizes, + protected void compactEquals(long now, ArrayList candidates, long[] expectedFileSizes, long[] expectedBoundaries, boolean isMajor, boolean toCompact) throws IOException { ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge(); EnvironmentEdgeManager.injectEdge(timeMachine); @@ -64,17 +64,17 @@ public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy DateTieredCompactionPolicy policy = (DateTieredCompactionPolicy) store.storeEngine.getCompactionPolicy(); if (isMajor) { - for (StoreFile file : candidates) { - ((MockStoreFile) file).setIsMajor(true); + for (HStoreFile file : candidates) { + ((MockHStoreFile) file).setIsMajor(true); } assertEquals(toCompact, policy.shouldPerformMajorCompaction(candidates)); request = (DateTieredCompactionRequest) policy.selectMajorCompaction(candidates); } else { - assertEquals(toCompact, policy.needsCompaction(candidates, ImmutableList. of())); + assertEquals(toCompact, policy.needsCompaction(candidates, ImmutableList.of())); request = (DateTieredCompactionRequest) policy.selectMinorCompaction(candidates, false, false); } - List actual = Lists.newArrayList(request.getFiles()); + List actual = Lists.newArrayList(request.getFiles()); assertEquals(Arrays.toString(expectedFileSizes), Arrays.toString(getSizes(actual))); assertEquals(Arrays.toString(expectedBoundaries), Arrays.toString(request.getBoundaries().toArray())); 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 2635e2d3169..5f858264ad8 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 HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true); + HStoreFile 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 bde09348734..82e17555bbc 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 HStoreFile(testingUtility.getTestFileSystem(), + HStoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(), path, configuration, cacheConf, BloomType.NONE, true); storeFile.initReader(); StoreFileReader reader = storeFile.getReader(); @@ -71,7 +71,7 @@ public class EncodedSeekPerformanceTest { allKeyValues.add(current); } - storeFile.closeReader(cacheConf.shouldEvictOnClose()); + storeFile.closeStoreFile(cacheConf.shouldEvictOnClose()); // pick seeks by random List seeks = new ArrayList<>(); @@ -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 HStoreFile(testingUtility.getTestFileSystem(), + HStoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(), path, configuration, cacheConf, BloomType.NONE, true); storeFile.initReader(); long totalSize = 0; @@ -132,7 +132,7 @@ public class EncodedSeekPerformanceTest { double seeksPerSec = (seeks.size() * NANOSEC_IN_SEC) / (finishSeeksTime - startSeeksTime); - storeFile.closeReader(cacheConf.shouldEvictOnClose()); + storeFile.closeStoreFile(cacheConf.shouldEvictOnClose()); clearBlockCache(); System.out.println(blockEncoding); 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/MockHStoreFile.java similarity index 82% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockStoreFile.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MockHStoreFile.java index 6fa951ee5cd..78b1ef668c2 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/MockHStoreFile.java @@ -15,26 +15,30 @@ * 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.Arrays; import java.util.Map; +import java.util.Optional; import java.util.OptionalLong; import java.util.TreeMap; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.CellBuilderFactory; +import org.apache.hadoop.hbase.CellBuilderType; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HDFSBlocksDistribution; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; /** A mock used so our tests don't deal with actual StoreFiles */ -public class MockStoreFile extends HStoreFile { +@InterfaceAudience.Private +public class MockHStoreFile extends HStoreFile { long length = 0; boolean isRef = false; long ageInDisk; @@ -48,7 +52,7 @@ public class MockStoreFile extends HStoreFile { long modificationTime; boolean compactedAway; - MockStoreFile(HBaseTestingUtility testUtil, Path testPath, + MockHStoreFile(HBaseTestingUtility testUtil, Path testPath, long length, long ageInDisk, boolean isRef, long sequenceid) throws IOException { super(testUtil.getTestFileSystem(), testPath, testUtil.getConfiguration(), new CacheConfig(testUtil.getConfiguration()), BloomType.NONE, true); @@ -184,29 +188,34 @@ public class MockStoreFile extends HStoreFile { } @Override - public Cell getLastKey() { + public Optional getLastKey() { if (splitPoint != null) { - return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length + 1)); + return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY) + .setType(KeyValue.Type.Put.getCode()) + .setRow(Arrays.copyOf(splitPoint, splitPoint.length + 1)).build()); } else { - return null; + return Optional.empty(); } } @Override - public Cell midkey() throws IOException { + public Optional midKey() throws IOException { if (splitPoint != null) { - return CellUtil.createCell(splitPoint); + return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY) + .setType(KeyValue.Type.Put.getCode()).setRow(splitPoint).build()); } else { - return null; + return Optional.empty(); } } @Override - public Cell getFirstKey() { + public Optional getFirstKey() { if (splitPoint != null) { - return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length - 1)); + return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY) + .setType(KeyValue.Type.Put.getCode()).setRow(splitPoint, 0, splitPoint.length - 1) + .build()); } else { - return null; + return Optional.empty(); } } }; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java index 36c2e19f485..eecc06921c9 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/NoOpScanPolicyObserver.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; /** * RegionObserver that just reimplements the default behavior, @@ -51,7 +50,7 @@ public class NoOpScanPolicyObserver implements RegionObserver { ScanInfo oldSI = store.getScanInfo(); ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); - return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, + return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP); } @@ -67,7 +66,7 @@ public class NoOpScanPolicyObserver implements RegionObserver { ScanInfo oldSI = store.getScanInfo(); ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); - return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, scanType, + return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, scanType, store.getSmallestReadPoint(), earliestPutTs); } @@ -76,10 +75,10 @@ public class NoOpScanPolicyObserver implements RegionObserver { Store store, Scan scan, NavigableSet targetCols, KeyValueScanner s, long readPoint) throws IOException { Region r = c.getEnvironment().getRegion(); - return scan.isReversed() ? new ReversedStoreScanner(store, - store.getScanInfo(), scan, targetCols, r.getReadPoint(scan - .getIsolationLevel())) : new StoreScanner(store, - store.getScanInfo(), scan, targetCols, r.getReadPoint(scan - .getIsolationLevel())); + return scan.isReversed() + ? new ReversedStoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, + r.getReadPoint(scan.getIsolationLevel())) + : new StoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, + r.getReadPoint(scan.getIsolationLevel())); } } 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 8fad1576dc8..2d08e50c4b1 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 HStoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true); + HStoreFile 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/TestCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java index 364982394f5..356054ef455 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java @@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY; import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES; import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.Matchers.any; @@ -226,7 +227,7 @@ public class TestCompaction { private int count() throws IOException { int count = 0; - for (StoreFile f: this.r.stores. + for (HStoreFile f: this.r.stores. get(COLUMN_FAMILY_TEXT).getStorefiles()) { HFileScanner scanner = f.getReader().getScanner(false, false); if (!scanner.seekTo()) { @@ -255,9 +256,9 @@ public class TestCompaction { for (int i = 0; i < nfiles; i++) { createStoreFile(r); } - HStore store = (HStore) r.getStore(COLUMN_FAMILY); + HStore store = r.getStore(COLUMN_FAMILY); - Collection storeFiles = store.getStorefiles(); + Collection storeFiles = store.getStorefiles(); DefaultCompactor tool = (DefaultCompactor)store.storeEngine.getCompactor(); tool.compactForTesting(storeFiles, false); @@ -276,8 +277,8 @@ public class TestCompaction { } catch (Exception e) { // The complete compaction should fail and the corrupt file should remain // in the 'tmp' directory; - assert (fs.exists(origPath)); - assert (!fs.exists(dstPath)); + assertTrue(fs.exists(origPath)); + assertFalse(fs.exists(dstPath)); System.out.println("testCompactionWithCorruptResult Passed"); return; } @@ -389,8 +390,8 @@ public class TestCompaction { } private class StoreMockMaker extends StatefulStoreMockMaker { - public ArrayList compacting = new ArrayList<>(); - public ArrayList notCompacting = new ArrayList<>(); + public ArrayList compacting = new ArrayList<>(); + public ArrayList notCompacting = new ArrayList<>(); private ArrayList results; public StoreMockMaker(ArrayList results) { @@ -398,19 +399,21 @@ public class TestCompaction { } public class TestCompactionContext extends CompactionContext { - private List selectedFiles; - public TestCompactionContext(List selectedFiles) { + + private List selectedFiles; + + public TestCompactionContext(List selectedFiles) { super(); this.selectedFiles = selectedFiles; } @Override - public List preSelect(List filesCompacting) { + public List preSelect(List filesCompacting) { return new ArrayList<>(); } @Override - public boolean select(List filesCompacting, boolean isUserCompaction, + public boolean select(List filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak, boolean forceMajor) throws IOException { this.request = new CompactionRequest(selectedFiles); this.request.setPriority(getPriority()); @@ -445,7 +448,7 @@ public class TestCompaction { notCompacting.addAll(ctx.selectedFiles); } - public synchronized void finishCompaction(List sfs) { + public synchronized void finishCompaction(List sfs) { if (sfs.isEmpty()) return; synchronized (results) { results.add(sfs.size()); @@ -466,7 +469,9 @@ public class TestCompaction { public volatile boolean isInCompact = false; public void unblock() { - synchronized (this) { this.notifyAll(); } + synchronized (this) { + this.notifyAll(); + } } @Override @@ -484,12 +489,12 @@ public class TestCompaction { } @Override - public List preSelect(List filesCompacting) { + public List preSelect(List filesCompacting) { return new ArrayList<>(); } @Override - public boolean select(List f, boolean i, boolean m, boolean e) + public boolean select(List f, boolean i, boolean m, boolean e) throws IOException { this.request = new CompactionRequest(new ArrayList<>()); return true; @@ -673,14 +678,14 @@ public class TestCompaction { } public static class DummyCompactor extends DefaultCompactor { - public DummyCompactor(Configuration conf, Store store) { + public DummyCompactor(Configuration conf, HStore store) { super(conf, store); this.keepSeqIdPeriod = 0; } } - private static StoreFile createFile() throws Exception { - StoreFile sf = mock(StoreFile.class); + private static HStoreFile createFile() throws Exception { + HStoreFile sf = mock(HStoreFile.class); when(sf.getPath()).thenReturn(new Path("file")); StoreFileReader r = mock(StoreFileReader.class); when(r.length()).thenReturn(10L); 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 d68f07e6e10..2a2602c98c2 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 @@ -1,4 +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 @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; @@ -89,7 +88,7 @@ public class TestCompactionArchiveConcurrentClose { HTableDescriptor htd = new HTableDescriptor(tableName); htd.addFamily(new HColumnDescriptor(fam)); HRegionInfo info = new HRegionInfo(tableName, null, null, false); - Region region = initHRegion(htd, info); + HRegion region = initHRegion(htd, info); RegionServerServices rss = mock(RegionServerServices.class); List regions = new ArrayList<>(); regions.add(region); @@ -112,12 +111,12 @@ public class TestCompactionArchiveConcurrentClose { region.flush(true); } - Store store = region.getStore(fam); + HStore store = region.getStore(fam); assertEquals(fileCount, store.getStorefilesCount()); - Collection storefiles = store.getStorefiles(); + Collection storefiles = store.getStorefiles(); // None of the files should be in compacted state. - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { assertFalse(file.isCompactedAway()); } // Do compaction @@ -157,7 +156,7 @@ public class TestCompactionArchiveConcurrentClose { } } - private Region initHRegion(HTableDescriptor htd, HRegionInfo info) + private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info) throws IOException { Configuration conf = testUtil.getConfiguration(); Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName()); @@ -185,7 +184,7 @@ public class TestCompactionArchiveConcurrentClose { } @Override - public void removeStoreFiles(String familyName, Collection storeFiles) + public void removeStoreFiles(String familyName, Collection storeFiles) throws IOException { super.removeStoreFiles(familyName, storeFiles); archived.set(true); 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 3f1613c8a3b..1d976e91678 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 @@ -120,24 +120,24 @@ public class TestCompactionArchiveIOException { region.flush(true); } - HStore store = (HStore) region.getStore(fam); + HStore store = region.getStore(fam); assertEquals(fileCount, store.getStorefilesCount()); - Collection storefiles = store.getStorefiles(); + Collection storefiles = store.getStorefiles(); // None of the files should be in compacted state. - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { assertFalse(file.isCompactedAway()); } StoreFileManager fileManager = store.getStoreEngine().getStoreFileManager(); - Collection initialCompactedFiles = fileManager.getCompactedfiles(); + Collection initialCompactedFiles = fileManager.getCompactedfiles(); assertTrue(initialCompactedFiles == null || initialCompactedFiles.isEmpty()); // Do compaction region.compact(true); // all prior store files should now be compacted - Collection compactedFilesPreClean = fileManager.getCompactedfiles(); + Collection compactedFilesPreClean = fileManager.getCompactedfiles(); assertNotNull(compactedFilesPreClean); assertTrue(compactedFilesPreClean.size() > 0); @@ -148,17 +148,17 @@ public class TestCompactionArchiveIOException { out.writeInt(1); out.close(); - StoreFile errStoreFile = new MockStoreFile(testUtil, errFile, 1, 0, false, 1); + HStoreFile errStoreFile = new MockHStoreFile(testUtil, errFile, 1, 0, false, 1); fileManager.addCompactionResults( - ImmutableList.of(errStoreFile), ImmutableList.of()); + ImmutableList.of(errStoreFile), ImmutableList.of()); // cleanup compacted files cleaner.chore(); // make sure the compacted files are cleared - Collection compactedFilesPostClean = fileManager.getCompactedfiles(); + Collection compactedFilesPostClean = fileManager.getCompactedfiles(); assertEquals(1, compactedFilesPostClean.size()); - for (StoreFile origFile : compactedFilesPreClean) { + for (HStoreFile origFile : compactedFilesPreClean) { assertFalse(compactedFilesPostClean.contains(origFile)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java index 0e4c4f9ecf2..6ae10ec33de 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionPolicy.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -46,6 +44,8 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; + public class TestCompactionPolicy { private final static Log LOG = LogFactory.getLog(TestCompactionPolicy.class); protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); @@ -141,7 +141,7 @@ public class TestCompactionPolicy { return result; } - List sfCreate(long... sizes) throws IOException { + List sfCreate(long... sizes) throws IOException { ArrayList ageInDisk = new ArrayList<>(); for (int i = 0; i < sizes.length; i++) { ageInDisk.add(0L); @@ -149,11 +149,11 @@ public class TestCompactionPolicy { return sfCreate(toArrayList(sizes), ageInDisk); } - List sfCreate(ArrayList sizes, ArrayList ageInDisk) throws IOException { + List sfCreate(ArrayList sizes, ArrayList ageInDisk) throws IOException { return sfCreate(false, sizes, ageInDisk); } - List sfCreate(boolean isReference, long... sizes) throws IOException { + List sfCreate(boolean isReference, long... sizes) throws IOException { ArrayList ageInDisk = new ArrayList<>(sizes.length); for (int i = 0; i < sizes.length; i++) { ageInDisk.add(0L); @@ -161,17 +161,17 @@ public class TestCompactionPolicy { return sfCreate(isReference, toArrayList(sizes), ageInDisk); } - List sfCreate(boolean isReference, ArrayList sizes, ArrayList ageInDisk) + List sfCreate(boolean isReference, ArrayList sizes, ArrayList ageInDisk) throws IOException { - List ret = Lists.newArrayList(); + List ret = Lists.newArrayList(); for (int i = 0; i < sizes.size(); i++) { - ret.add(new MockStoreFile(TEST_UTIL, TEST_FILE, sizes.get(i), ageInDisk.get(i), isReference, + ret.add(new MockHStoreFile(TEST_UTIL, TEST_FILE, sizes.get(i), ageInDisk.get(i), isReference, i)); } return ret; } - long[] getSizes(List sfList) { + long[] getSizes(List sfList) { long[] aNums = new long[sfList.size()]; for (int i = 0; i < sfList.size(); ++i) { aNums[i] = sfList.get(i).getReader().length(); @@ -179,23 +179,23 @@ public class TestCompactionPolicy { return aNums; } - void compactEquals(List candidates, long... expected) throws IOException { + void compactEquals(List candidates, long... expected) throws IOException { compactEquals(candidates, false, false, expected); } - void compactEquals(List candidates, boolean forcemajor, long... expected) + void compactEquals(List candidates, boolean forcemajor, long... expected) throws IOException { compactEquals(candidates, forcemajor, false, expected); } - void compactEquals(List candidates, boolean forcemajor, boolean isOffPeak, + void compactEquals(List candidates, boolean forcemajor, boolean isOffPeak, long... expected) throws IOException { store.forceMajor = forcemajor; // Test Default compactions CompactionRequest result = ((RatioBasedCompactionPolicy) store.storeEngine.getCompactionPolicy()).selectCompaction( candidates, new ArrayList<>(), false, isOffPeak, forcemajor); - List actual = new ArrayList<>(result.getFiles()); + List actual = new ArrayList<>(result.getFiles()); if (isOffPeak && !forcemajor) { Assert.assertTrue(result.isOffPeak()); } 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 26172f5562a..9c33d289cca 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 @@ -200,7 +200,7 @@ public class TestCompoundBloomFilter { private void readStoreFile(int t, BloomType bt, List kvs, Path sfPath) throws IOException { - StoreFile sf = new HStoreFile(fs, sfPath, conf, cacheConf, bt, true); + HStoreFile 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/TestDefaultCompactSelection.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java index 059b850b827..3689cf79a93 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultCompactSelection.java @@ -98,7 +98,7 @@ public class TestDefaultCompactSelection extends TestCompactionPolicy { // timestamp a bit to make sure that now - lowestModTime is greater than major compaction // period(1ms). // trigger an aged major compaction - List candidates = sfCreate(50, 25, 12, 12); + List candidates = sfCreate(50, 25, 12, 12); edge.increment(2); compactEquals(candidates, 50, 25, 12, 12); // major sure exceeding maxCompactSize also downgrades aged minors @@ -164,10 +164,10 @@ public class TestDefaultCompactSelection extends TestCompactionPolicy { oldScanInfo.getTimeToPurgeDeletes(), oldScanInfo.getComparator(), oldScanInfo.isNewVersionBehavior()); store.setScanInfo(newScanInfo); // Do not compact empty store file - List candidates = sfCreate(0); - for (StoreFile file : candidates) { - if (file instanceof MockStoreFile) { - MockStoreFile mockFile = (MockStoreFile) file; + List candidates = sfCreate(0); + for (HStoreFile file : candidates) { + if (file instanceof MockHStoreFile) { + MockHStoreFile mockFile = (MockHStoreFile) file; mockFile.setTimeRangeTracker(new TimeRangeTracker(-1, -1)); mockFile.setEntries(0); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java index b9982aa731f..df5e97a6942 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java @@ -22,10 +22,10 @@ package org.apache.hadoop.hbase.regionserver; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; +import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; -import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy; -import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -34,13 +34,13 @@ import org.mockito.Mockito; @Category({RegionServerTests.class, SmallTests.class}) public class TestDefaultStoreEngine { public static class DummyStoreFlusher extends DefaultStoreFlusher { - public DummyStoreFlusher(Configuration conf, Store store) { + public DummyStoreFlusher(Configuration conf, HStore store) { super(conf, store); } } public static class DummyCompactor extends DefaultCompactor { - public DummyCompactor(Configuration conf, Store store) { + public DummyCompactor(Configuration conf, HStore store) { super(conf, store); } } @@ -59,7 +59,7 @@ public class TestDefaultStoreEngine { DummyCompactionPolicy.class.getName()); conf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY, DummyStoreFlusher.class.getName()); - Store mockStore = Mockito.mock(Store.class); + HStore mockStore = Mockito.mock(HStore.class); StoreEngine se = StoreEngine.create(mockStore, conf, CellComparator.COMPARATOR); Assert.assertTrue(se instanceof DefaultStoreEngine); Assert.assertTrue(se.getCompactionPolicy() instanceof DummyCompactionPolicy); 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 9acf24479b0..f58d19a9482 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 @@ -210,14 +210,14 @@ public class TestEncryptionKeyRotation { boolean compacted = false; for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName) .getOnlineRegions(tableName)) { - for (Store store : region.getStores()) { + for (HStore store : ((HRegion) region).getStores()) { compacted = false; while (!compacted) { if (store.getStorefiles() != null) { while (store.getStorefilesCount() != 1) { Thread.sleep(100); } - for (StoreFile storefile : store.getStorefiles()) { + for (HStoreFile storefile : store.getStorefiles()) { if (!storefile.isCompactedAway()) { compacted = true; break; @@ -234,10 +234,10 @@ public class TestEncryptionKeyRotation { private static List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList<>(); - for (Region region: - TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(tableName)) { - for (Store store: region.getStores()) { - for (StoreFile storefile: store.getStorefiles()) { + for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName) + .getOnlineRegions(tableName)) { + for (HStore store : ((HRegion) region).getStores()) { + for (HStoreFile storefile : store.getStorefiles()) { paths.add(storefile.getPath()); } } @@ -247,13 +247,13 @@ public class TestEncryptionKeyRotation { private static List findCompactedStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList<>(); - for (Region region: - TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(tableName)) { - for (Store store : region.getStores()) { - Collection compactedfiles = - ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); + for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName) + .getOnlineRegions(tableName)) { + for (HStore store : ((HRegion) region).getStores()) { + Collection compactedfiles = + store.getStoreEngine().getStoreFileManager().getCompactedfiles(); if (compactedfiles != null) { - for (StoreFile storefile : compactedfiles) { + for (HStoreFile storefile : compactedfiles) { paths.add(storefile.getPath()); } } 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 3837e940111..75c752cd3ba 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 @@ -55,8 +55,8 @@ public class TestEncryptionRandomKeying { List paths = new ArrayList<>(); for (Region region: TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { - for (Store store: region.getStores()) { - for (StoreFile storefile: store.getStorefiles()) { + for (HStore store : ((HRegion) region).getStores()) { + for (HStoreFile storefile : store.getStorefiles()) { paths.add(storefile.getPath()); } } 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 daddb5c94ea..48081bdbb29 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 @@ -94,7 +94,7 @@ public class TestFSErrorsExposed { TestHStoreFile.writeStoreFile( writer, Bytes.toBytes("cf"), Bytes.toBytes("qual")); - StoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf, + HStoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf, BloomType.NONE, true); sf.initReader(); StoreFileReader reader = sf.getReader(); @@ -144,12 +144,12 @@ public class TestFSErrorsExposed { TestHStoreFile.writeStoreFile( writer, Bytes.toBytes("cf"), Bytes.toBytes("qual")); - StoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf, + HStoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf, BloomType.NONE, true); List scanners = StoreFileScanner.getScannersForStoreFiles( Collections.singletonList(sf), false, true, false, false, - // 0 is passed as readpoint because this test operates on StoreFile directly + // 0 is passed as readpoint because this test operates on HStoreFile directly 0); KeyValueScanner scanner = scanners.get(0); 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 603203ac2f9..fc0659f811c 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 @@ -518,7 +518,7 @@ public class TestHMobStore { this.store.add(new KeyValue(row, family, qf6, 1, value), null); flush(2); - Collection storefiles = this.store.getStorefiles(); + Collection storefiles = this.store.getStorefiles(); checkMobHFileEncrytption(storefiles); // Scan the values @@ -547,8 +547,8 @@ public class TestHMobStore { checkMobHFileEncrytption(this.store.getStorefiles()); } - private void checkMobHFileEncrytption(Collection storefiles) { - StoreFile storeFile = storefiles.iterator().next(); + private void checkMobHFileEncrytption(Collection storefiles) { + HStoreFile storeFile = storefiles.iterator().next(); HFile.Reader reader = storeFile.getReader().getHFileReader(); byte[] encryptionKey = reader.getTrailer().getEncryptionKey(); Assert.assertTrue(null != encryptionKey); 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 334df17e779..0b0d651b853 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 @@ -18,6 +18,51 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS; +import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; +import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2; +import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; +import static org.mockito.Matchers.anyLong; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.security.PrivilegedExceptionAction; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -84,25 +129,14 @@ import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; import org.apache.hadoop.hbase.regionserver.Region.RowLock; -import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem; +import org.apache.hadoop.hbase.regionserver.TestHStore.FaultyFileSystem; import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler; import org.apache.hadoop.hbase.regionserver.wal.FSHLog; import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL; import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps; -import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor; -import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; import org.apache.hadoop.hbase.test.MetricsAssertHelper; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; @@ -116,6 +150,7 @@ import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.FaultyFSLog; import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.wal.WALProvider; @@ -136,50 +171,16 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.io.InterruptedIOException; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; -import java.util.UUID; -import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; - -import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS; -import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; -import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2; -import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor; +import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; /** * Basic stand-alone testing of HRegion. No clusters! @@ -897,7 +898,7 @@ public class TestHRegion { // this will create a region with 3 files assertEquals(3, region.getStore(family).getStorefilesCount()); List storeFiles = new ArrayList<>(3); - for (StoreFile sf : region.getStore(family).getStorefiles()) { + for (HStoreFile sf : region.getStore(family).getStorefiles()) { storeFiles.add(sf.getPath()); } @@ -958,8 +959,8 @@ public class TestHRegion { } // now check whether we have only one store file, the compacted one - Collection sfs = region.getStore(family).getStorefiles(); - for (StoreFile sf : sfs) { + Collection sfs = region.getStore(family).getStorefiles(); + for (HStoreFile sf : sfs) { LOG.info(sf.getPath()); } if (!mismatchedRegionName) { @@ -1011,7 +1012,7 @@ public class TestHRegion { // this will create a region with 3 files from flush assertEquals(3, region.getStore(family).getStorefilesCount()); List storeFiles = new ArrayList<>(3); - for (StoreFile sf : region.getStore(family).getStorefiles()) { + for (HStoreFile sf : region.getStore(family).getStorefiles()) { storeFiles.add(sf.getPath().getName()); } @@ -4052,8 +4053,8 @@ public class TestHRegion { } // before compaction HStore store = (HStore) region.getStore(fam1); - Collection storeFiles = store.getStorefiles(); - for (StoreFile storefile : storeFiles) { + Collection storeFiles = store.getStorefiles(); + for (HStoreFile storefile : storeFiles) { StoreFileReader reader = storefile.getReader(); reader.loadFileInfo(); reader.loadBloomfilter(); @@ -4065,7 +4066,7 @@ public class TestHRegion { // after compaction storeFiles = store.getStorefiles(); - for (StoreFile storefile : storeFiles) { + for (HStoreFile storefile : storeFiles) { StoreFileReader reader = storefile.getReader(); reader.loadFileInfo(); reader.loadBloomfilter(); @@ -4814,7 +4815,7 @@ public class TestHRegion { secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF); // move the file of the primary region to the archive, simulating a compaction - Collection storeFiles = primaryRegion.getStore(families[0]).getStorefiles(); + Collection storeFiles = primaryRegion.getStore(families[0]).getStorefiles(); primaryRegion.getRegionFileSystem().removeStoreFiles(Bytes.toString(families[0]), storeFiles); Collection storeFileInfos = primaryRegion.getRegionFileSystem() .getStoreFiles(families[0]); @@ -5843,9 +5844,9 @@ public class TestHRegion { put.addColumn(fam1, qual1, Bytes.toBytes("c1-value")); region.put(put); region.flush(true); - Store store = region.getStore(fam1); - Collection storefiles = store.getStorefiles(); - for (StoreFile sf : storefiles) { + HStore store = region.getStore(fam1); + Collection storefiles = store.getStorefiles(); + for (HStoreFile sf : storefiles) { assertFalse("Tags should not be present " ,sf.getReader().getHFileReader().getFileContext().isIncludesTags()); } 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 62ada8fb65a..ab98ec9b4f7 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 @@ -238,9 +238,9 @@ public class TestHRegionReplayEvents { verifyData(secondaryRegion, 0, 1000, cq, families); // close the region, and inspect that it has not flushed - Map> files = secondaryRegion.close(false); + Map> files = secondaryRegion.close(false); // assert that there are no files (due to flush) - for (List f : files.values()) { + for (List f : files.values()) { assertTrue(f.isEmpty()); } } @@ -1524,8 +1524,8 @@ public class TestHRegionReplayEvents { storeFileName.addAll(storeDesc.getStoreFileList()); } // assert that the bulk loaded files are picked - for (Store s : secondaryRegion.getStores()) { - for (StoreFile sf : s.getStorefiles()) { + for (HStore s : secondaryRegion.getStores()) { + for (HStoreFile sf : s.getStorefiles()) { storeFileName.remove(sf.getPath().getName()); } } 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 97f8ce3007d..af64be69a1f 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 @@ -17,11 +17,10 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertThat; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; - import java.io.IOException; import java.io.InterruptedIOException; import java.util.ArrayList; @@ -68,17 +67,13 @@ 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.compactions.CompactionLifeCycleTracker; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener; -import org.apache.hadoop.hbase.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.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.WAL; +import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.wal.WALKey; import org.junit.BeforeClass; import org.junit.Test; @@ -87,6 +82,11 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameters; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; +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; + /** * Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of * the region server's bullkLoad functionality. @@ -160,7 +160,7 @@ public class TestHRegionServerBulkLoad { KeyValue kv = new KeyValue(rowkey(i), family, qualifier, now, value); writer.append(kv); } - writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(now)); + writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(now)); } finally { writer.close(); } 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/TestHStore.java similarity index 93% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStore.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java index 2095dcdb5fc..b6851157b49 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/TestHStore.java @@ -21,7 +21,9 @@ package org.apache.hadoop.hbase.regionserver; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -86,7 +88,6 @@ import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor; import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; import org.apache.hadoop.hbase.security.User; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -98,7 +99,6 @@ import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WALFactory; import org.apache.hadoop.util.Progressable; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -106,13 +106,16 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.mockito.Mockito; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; + /** * Test class for the Store */ -@Category({RegionServerTests.class, MediumTests.class}) -public class TestStore { - private static final Log LOG = LogFactory.getLog(TestStore.class); - @Rule public TestName name = new TestName(); +@Category({ RegionServerTests.class, MediumTests.class }) +public class TestHStore { + private static final Log LOG = LogFactory.getLog(TestHStore.class); + @Rule + public TestName name = new TestName(); HStore store; byte [] table = Bytes.toBytes("table"); @@ -161,8 +164,7 @@ public class TestStore { init(methodName, TEST_UTIL.getConfiguration()); } - private Store init(String methodName, Configuration conf) - throws IOException { + private Store init(String methodName, Configuration conf) throws IOException { HColumnDescriptor hcd = new HColumnDescriptor(family); // some of the tests write 4 versions and then flush // (with HBASE-4241, lower versions are collected on flush) @@ -170,24 +172,24 @@ public class TestStore { return init(methodName, conf, hcd); } - private Store init(String methodName, Configuration conf, - HColumnDescriptor hcd) throws IOException { + private HStore init(String methodName, Configuration conf, HColumnDescriptor hcd) + throws IOException { HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table)); return init(methodName, conf, htd, hcd); } - private Store init(String methodName, Configuration conf, HTableDescriptor htd, + private HStore init(String methodName, Configuration conf, HTableDescriptor htd, HColumnDescriptor hcd) throws IOException { return init(methodName, conf, htd, hcd, null); } @SuppressWarnings("deprecation") - private Store init(String methodName, Configuration conf, HTableDescriptor htd, + private HStore init(String methodName, Configuration conf, HTableDescriptor htd, HColumnDescriptor hcd, MyStoreHook hook) throws IOException { return init(methodName, conf, htd, hcd, hook, false); } @SuppressWarnings("deprecation") - private Store init(String methodName, Configuration conf, HTableDescriptor htd, + private HStore init(String methodName, Configuration conf, HTableDescriptor htd, HColumnDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException { //Setting up a Store Path basedir = new Path(DIR+methodName); @@ -240,45 +242,45 @@ public class TestStore { public Object run() throws Exception { // Make sure it worked (above is sensitive to caching details in hadoop core) FileSystem fs = FileSystem.get(conf); - Assert.assertEquals(FaultyFileSystem.class, fs.getClass()); + assertEquals(FaultyFileSystem.class, fs.getClass()); FaultyFileSystem ffs = (FaultyFileSystem)fs; // Initialize region init(name.getMethodName(), conf); MemstoreSize size = store.memstore.getFlushableSize(); - Assert.assertEquals(0, size.getDataSize()); + assertEquals(0, size.getDataSize()); LOG.info("Adding some data"); MemstoreSize kvSize = new MemstoreSize(); store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), kvSize); // add the heap size of active (mutable) segment kvSize.incMemstoreSize(0, MutableSegment.DEEP_OVERHEAD); size = store.memstore.getFlushableSize(); - Assert.assertEquals(kvSize, size); + assertEquals(kvSize, size); // Flush. Bug #1 from HBASE-10466. Make sure size calculation on failed flush is right. try { LOG.info("Flushing"); flushStore(store, id++); - Assert.fail("Didn't bubble up IOE!"); + fail("Didn't bubble up IOE!"); } catch (IOException ioe) { - Assert.assertTrue(ioe.getMessage().contains("Fault injected")); + assertTrue(ioe.getMessage().contains("Fault injected")); } // due to snapshot, change mutable to immutable segment kvSize.incMemstoreSize(0, CSLMImmutableSegment.DEEP_OVERHEAD_CSLM-MutableSegment.DEEP_OVERHEAD); size = store.memstore.getFlushableSize(); - Assert.assertEquals(kvSize, size); + assertEquals(kvSize, size); MemstoreSize kvSize2 = new MemstoreSize(); store.add(new KeyValue(row, family, qf2, 2, (byte[])null), kvSize2); kvSize2.incMemstoreSize(0, MutableSegment.DEEP_OVERHEAD); // Even though we add a new kv, we expect the flushable size to be 'same' since we have // not yet cleared the snapshot -- the above flush failed. - Assert.assertEquals(kvSize, size); + assertEquals(kvSize, size); ffs.fault.set(false); flushStore(store, id++); size = store.memstore.getFlushableSize(); // Size should be the foreground kv size. - Assert.assertEquals(kvSize2, size); + assertEquals(kvSize2, size); flushStore(store, id++); size = store.memstore.getFlushableSize(); assertEquals(0, size.getDataSize()); @@ -313,8 +315,8 @@ public class TestStore { // Verify that compression and encoding settings are respected HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), true, conf); - Assert.assertEquals(hcd.getCompressionType(), reader.getCompressionAlgorithm()); - Assert.assertEquals(hcd.getDataBlockEncoding(), reader.getDataBlockEncoding()); + assertEquals(hcd.getCompressionType(), reader.getCompressionAlgorithm()); + assertEquals(hcd.getDataBlockEncoding(), reader.getDataBlockEncoding()); reader.close(); } @@ -360,19 +362,19 @@ public class TestStore { } // Verify the total number of store files - Assert.assertEquals(storeFileNum, this.store.getStorefiles().size()); + assertEquals(storeFileNum, this.store.getStorefiles().size()); // Each call will find one expired store file and delete it before compaction happens. // There will be no compaction due to threshold above. Last file will not be replaced. for (int i = 1; i <= storeFileNum - 1; i++) { // verify the expired store file. assertFalse(this.store.requestCompaction().isPresent()); - Collection sfs = this.store.getStorefiles(); + Collection sfs = this.store.getStorefiles(); // Ensure i files are gone. if (minVersions == 0) { assertEquals(storeFileNum - i, sfs.size()); // Ensure only non-expired files remain. - for (StoreFile sf : sfs) { + for (HStoreFile sf : sfs) { assertTrue(sf.getReader().getMaxTimestamp() >= (edge.currentTime() - storeTtl)); } } else { @@ -383,7 +385,7 @@ public class TestStore { } assertFalse(this.store.requestCompaction().isPresent()); - Collection sfs = this.store.getStorefiles(); + Collection sfs = this.store.getStorefiles(); // Assert the last expired file is not removed. if (minVersions == 0) { assertEquals(1, sfs.size()); @@ -391,8 +393,8 @@ public class TestStore { long ts = sfs.iterator().next().getReader().getMaxTimestamp(); assertTrue(ts < (edge.currentTime() - storeTtl)); - for (StoreFile sf : sfs) { - sf.closeReader(true); + for (HStoreFile sf : sfs) { + sf.closeStoreFile(true); } } @@ -414,24 +416,24 @@ public class TestStore { // after flush; check the lowest time stamp long lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles()); long lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles()); - Assert.assertEquals(lowestTimeStampFromManager,lowestTimeStampFromFS); + assertEquals(lowestTimeStampFromManager,lowestTimeStampFromFS); // after compact; check the lowest time stamp store.compact(store.requestCompaction().get(), NoLimitThroughputController.INSTANCE, null); lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles()); lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles()); - Assert.assertEquals(lowestTimeStampFromManager, lowestTimeStampFromFS); + assertEquals(lowestTimeStampFromManager, lowestTimeStampFromFS); } private static long getLowestTimeStampFromFS(FileSystem fs, - final Collection candidates) throws IOException { + final Collection candidates) throws IOException { long minTs = Long.MAX_VALUE; if (candidates.isEmpty()) { return minTs; } Path[] p = new Path[candidates.size()]; int i = 0; - for (StoreFile sf : candidates) { + for (HStoreFile sf : candidates) { p[i] = sf.getPath(); ++i; } @@ -464,7 +466,7 @@ public class TestStore { flush(1); // Now put in place an empty store file. Its a little tricky. Have to // do manually with hacked in sequence id. - StoreFile f = this.store.getStorefiles().iterator().next(); + HStoreFile f = this.store.getStorefiles().iterator().next(); Path storedir = f.getPath().getParent(); long seqid = f.getMaxSequenceId(); Configuration c = HBaseConfiguration.create(); @@ -480,12 +482,12 @@ public class TestStore { this.store.close(); // Reopen it... should pick up two files this.store = new HStore(this.store.getHRegion(), this.store.getColumnFamilyDescriptor(), c); - Assert.assertEquals(2, this.store.getStorefilesCount()); + assertEquals(2, this.store.getStorefilesCount()); result = HBaseTestingUtility.getFromStoreFile(store, get.getRow(), qualifiers); - Assert.assertEquals(1, result.size()); + assertEquals(1, result.size()); } /** @@ -589,14 +591,14 @@ public class TestStore { private void flush(int storeFilessize) throws IOException{ this.store.snapshot(); flushStore(store, id++); - Assert.assertEquals(storeFilessize, this.store.getStorefiles().size()); - Assert.assertEquals(0, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount()); + assertEquals(storeFilessize, this.store.getStorefiles().size()); + assertEquals(0, ((AbstractMemStore)this.store.memstore).getActive().getCellsCount()); } private void assertCheck() { - Assert.assertEquals(expected.size(), result.size()); + assertEquals(expected.size(), result.size()); for(int i=0; i files = store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName()); - Assert.assertEquals(0, files != null ? files.size() : 0); + assertEquals(0, files != null ? files.size() : 0); //flush try { LOG.info("Flushing"); flush(1); - Assert.fail("Didn't bubble up IOE!"); + fail("Didn't bubble up IOE!"); } catch (IOException ioe) { - Assert.assertTrue(ioe.getMessage().contains("Fault injected")); + assertTrue(ioe.getMessage().contains("Fault injected")); } LOG.info("After failed flush, we should still have no files!"); files = store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName()); - Assert.assertEquals(0, files != null ? files.size() : 0); + assertEquals(0, files != null ? files.size() : 0); store.getHRegion().getWAL().close(); return null; } @@ -775,27 +777,27 @@ public class TestStore { get.setTimeRange(0,15); result = HBaseTestingUtility.getFromStoreFile(store, get); - Assert.assertTrue(result.size()>0); + assertTrue(result.size()>0); get.setTimeRange(40,90); result = HBaseTestingUtility.getFromStoreFile(store, get); - Assert.assertTrue(result.size()>0); + assertTrue(result.size()>0); get.setTimeRange(10,45); result = HBaseTestingUtility.getFromStoreFile(store, get); - Assert.assertTrue(result.size()>0); + assertTrue(result.size()>0); get.setTimeRange(80,145); result = HBaseTestingUtility.getFromStoreFile(store, get); - Assert.assertTrue(result.size()>0); + assertTrue(result.size()>0); get.setTimeRange(1,2); result = HBaseTestingUtility.getFromStoreFile(store, get); - Assert.assertTrue(result.size()>0); + assertTrue(result.size()>0); get.setTimeRange(90,200); result = HBaseTestingUtility.getFromStoreFile(store, get); - Assert.assertTrue(result.size()==0); + assertTrue(result.size()==0); } /** @@ -806,9 +808,9 @@ public class TestStore { @Test public void testSplitWithEmptyColFam() throws IOException { init(this.name.getMethodName()); - Assert.assertNull(store.getSplitPoint()); + assertFalse(store.getSplitPoint().isPresent()); store.getHRegion().forceSplit(null); - Assert.assertNull(store.getSplitPoint()); + assertFalse(store.getSplitPoint().isPresent()); store.getHRegion().clearSplit(); } @@ -823,8 +825,8 @@ public class TestStore { Configuration conf = HBaseConfiguration.create(); conf.setLong(CONFIG_KEY, anyValue); init(name.getMethodName() + "-xml", conf); - Assert.assertTrue(store.throttleCompaction(anyValue + 1)); - Assert.assertFalse(store.throttleCompaction(anyValue)); + assertTrue(store.throttleCompaction(anyValue + 1)); + assertFalse(store.throttleCompaction(anyValue)); // HTD overrides XML. --anyValue; @@ -832,22 +834,22 @@ public class TestStore { HColumnDescriptor hcd = new HColumnDescriptor(family); htd.setConfiguration(CONFIG_KEY, Long.toString(anyValue)); init(name.getMethodName() + "-htd", conf, htd, hcd); - Assert.assertTrue(store.throttleCompaction(anyValue + 1)); - Assert.assertFalse(store.throttleCompaction(anyValue)); + assertTrue(store.throttleCompaction(anyValue + 1)); + assertFalse(store.throttleCompaction(anyValue)); // HCD overrides them both. --anyValue; hcd.setConfiguration(CONFIG_KEY, Long.toString(anyValue)); init(name.getMethodName() + "-hcd", conf, htd, hcd); - Assert.assertTrue(store.throttleCompaction(anyValue + 1)); - Assert.assertFalse(store.throttleCompaction(anyValue)); + assertTrue(store.throttleCompaction(anyValue + 1)); + assertFalse(store.throttleCompaction(anyValue)); } public static class DummyStoreEngine extends DefaultStoreEngine { public static DefaultCompactor lastCreatedCompactor = null; @Override - protected void createComponents(Configuration conf, Store store, CellComparator comparator) + protected void createComponents(Configuration conf, HStore store, CellComparator comparator) throws IOException { super.createComponents(conf, store, comparator); lastCreatedCompactor = this.compactor; @@ -859,12 +861,12 @@ public class TestStore { Configuration conf = HBaseConfiguration.create(); conf.set(StoreEngine.STORE_ENGINE_CLASS_KEY, DummyStoreEngine.class.getName()); init(this.name.getMethodName(), conf); - Assert.assertEquals(DummyStoreEngine.lastCreatedCompactor, + assertEquals(DummyStoreEngine.lastCreatedCompactor, this.store.storeEngine.getCompactor()); } private void addStoreFile() throws IOException { - StoreFile f = this.store.getStorefiles().iterator().next(); + HStoreFile f = this.store.getStorefiles().iterator().next(); Path storedir = f.getPath().getParent(); long seqid = this.store.getMaxSequenceId(); Configuration c = TEST_UTIL.getConfiguration(); @@ -881,9 +883,9 @@ public class TestStore { } private void archiveStoreFile(int index) throws IOException { - Collection files = this.store.getStorefiles(); - StoreFile sf = null; - Iterator it = files.iterator(); + Collection files = this.store.getStorefiles(); + HStoreFile sf = null; + Iterator it = files.iterator(); for (int i = 0; i <= index; i++) { sf = it.next(); } @@ -891,14 +893,14 @@ public class TestStore { } private void closeCompactedFile(int index) throws IOException { - Collection files = + Collection files = this.store.getStoreEngine().getStoreFileManager().getCompactedfiles(); - StoreFile sf = null; - Iterator it = files.iterator(); + HStoreFile sf = null; + Iterator it = files.iterator(); for (int i = 0; i <= index; i++) { sf = it.next(); } - sf.closeReader(true); + sf.closeStoreFile(true); store.getStoreEngine().getStoreFileManager().removeCompactedFiles(Lists.newArrayList(sf)); } @@ -1461,7 +1463,7 @@ public class TestStore { } @Override - public List getScanners(List files, boolean cacheBlocks, + public List getScanners(List files, boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner) throws IOException { @@ -1469,6 +1471,7 @@ public class TestStore { return super.getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, false, readPt, includeMemstoreScanner); } + @Override public long getSmallestReadPoint() { return hook.getSmallestReadPoint(this); @@ -1520,8 +1523,8 @@ public class TestStore { ScanInfo scanInfo = store.getScanInfo(); Scan scan = new Scan(); scan.addFamily(family); - Collection storefiles2 = store.getStorefiles(); - ArrayList actualStorefiles = Lists.newArrayList(storefiles2); + Collection storefiles2 = store.getStorefiles(); + ArrayList actualStorefiles = Lists.newArrayList(storefiles2); StoreScanner storeScanner = (StoreScanner) store.getScanner(scan, scan.getFamilyMap().get(family), Long.MAX_VALUE); // get the current heap @@ -1541,7 +1544,7 @@ public class TestStore { // flush them flushStore(store, seqID); storefiles2 = store.getStorefiles(); - ArrayList actualStorefiles1 = Lists.newArrayList(storefiles2); + ArrayList actualStorefiles1 = Lists.newArrayList(storefiles2); actualStorefiles1.removeAll(actualStorefiles); // Do compaction List exceptions = new ArrayList(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java index e74e9393665..b20cae86e44 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStoreFile.java @@ -176,11 +176,9 @@ public class TestHStoreFile extends HBaseTestCase { // Split on a row, not in middle of row. Midkey returned by reader // may be in middle of row. Create new one with empty column and // timestamp. - Cell kv = reader.midkey(); - byte [] midRow = CellUtil.cloneRow(kv); - kv = reader.getLastKey(); - byte [] finalRow = CellUtil.cloneRow(kv); - hsf.closeReader(true); + byte [] midRow = CellUtil.cloneRow(reader.midKey().get()); + byte [] finalRow = CellUtil.cloneRow(reader.getLastKey().get()); + hsf.closeStoreFile(true); // Make a reference HRegionInfo splitHri = new HRegionInfo(hri.getTable(), null, midRow); @@ -190,7 +188,8 @@ public class TestHStoreFile extends HBaseTestCase { // Now confirm that I can read from the reference and that it only gets // keys from top half of the file. HFileScanner s = refHsf.getReader().getScanner(false, false); - for(boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) { + Cell kv = null; + for (boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) { ByteBuffer bb = ByteBuffer.wrap(((KeyValue) s.getKey()).getKey()); kv = KeyValueUtil.createKeyValueFromKey(bb); if (first) { @@ -301,7 +300,7 @@ public class TestHStoreFile extends HBaseTestCase { f.initReader(); Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top Path pathB = splitStoreFile(cloneRegionFs, splitHriB, TEST_FAMILY, f, SPLITKEY, false);// bottom - f.closeReader(true); + f.closeStoreFile(true); // OK test the thing FSUtils.logFileSystemState(fs, testDir, LOG); @@ -342,7 +341,7 @@ public class TestHStoreFile extends HBaseTestCase { private void checkHalfHFile(final HRegionFileSystem regionFs, final HStoreFile f) throws IOException { f.initReader(); - Cell midkey = f.getReader().midkey(); + Cell midkey = f.getReader().midKey().get(); KeyValue midKV = (KeyValue)midkey; byte [] midRow = CellUtil.cloneRow(midKV); // Create top split. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java index 707540a7c0c..0c33bdb338f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMajorCompaction.java @@ -84,7 +84,7 @@ public class TestMajorCompaction { private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU(); protected Configuration conf = UTIL.getConfiguration(); - private Region r = null; + private HRegion r = null; private HTableDescriptor htd = null; private static final byte [] COLUMN_FAMILY = fam1; private final byte [] STARTROW = Bytes.toBytes(START_KEY); @@ -328,7 +328,7 @@ public class TestMajorCompaction { // ensure that major compaction time is deterministic RatioBasedCompactionPolicy c = (RatioBasedCompactionPolicy)s.storeEngine.getCompactionPolicy(); - Collection storeFiles = s.getStorefiles(); + Collection storeFiles = s.getStorefiles(); long mcTime = c.getNextMajorCompactTime(storeFiles); for (int i = 0; i < 10; ++i) { assertEquals(mcTime, c.getNextMajorCompactTime(storeFiles)); @@ -358,7 +358,7 @@ public class TestMajorCompaction { private void verifyCounts(int countRow1, int countRow2) throws Exception { int count1 = 0; int count2 = 0; - for (StoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) { + for (HStoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) { HFileScanner scanner = f.getReader().getScanner(false, false); scanner.seekTo(); do { @@ -377,7 +377,7 @@ public class TestMajorCompaction { private int count() throws IOException { int count = 0; - for (StoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) { + for (HStoreFile f: r.getStore(COLUMN_FAMILY_TEXT).getStorefiles()) { HFileScanner scanner = f.getReader().getScanner(false, false); if (!scanner.seekTo()) { continue; 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 9ab1440642e..c08bd718443 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 @@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver; import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY; import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -318,10 +320,10 @@ public class TestMobStoreCompaction { if (fs.exists(mobDirPath)) { FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath); for (FileStatus file : files) { - StoreFile sf = new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true); + HStoreFile 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); + byte[] count = fileInfo.get(MOB_CELLS_COUNT); assertTrue(count != null); mobCellsCount += Bytes.toLong(count); } @@ -349,7 +351,7 @@ public class TestMobStoreCompaction { Bytes.toBytes("colX"), now, dummyData); writer.append(kv); } finally { - writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())); + writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())); writer.close(); } } @@ -428,20 +430,20 @@ public class TestMobStoreCompaction { copyOfConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0f); CacheConfig cacheConfig = new CacheConfig(copyOfConf); Path mobDirPath = MobUtils.getMobFamilyPath(conf, htd.getTableName(), hcd.getNameAsString()); - List sfs = new ArrayList<>(); + List sfs = new ArrayList<>(); int numDelfiles = 0; int size = 0; if (fs.exists(mobDirPath)) { for (FileStatus f : fs.listStatus(mobDirPath)) { - StoreFile sf = new HStoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true); + HStoreFile 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, - HConstants.LATEST_TIMESTAMP); + List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, + false, false, HConstants.LATEST_TIMESTAMP); long timeToPurgeDeletes = Math.max(conf.getLong("hbase.hstore.time.to.purge.deletes", 0), 0); long ttl = HStore.determineTTLFromFamily(hcd); ScanInfo scanInfo = new ScanInfo(copyOfConf, hcd, ttl, timeToPurgeDeletes, 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 72a968cb673..86fe5af602b 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 @@ -45,9 +45,6 @@ import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.io.hfile.HFileScanner; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -55,9 +52,17 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.util.StringUtils; -import org.junit.*; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; import org.junit.experimental.categories.Category; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; + /** * Tests for region replicas. Sad that we cannot isolate these without bringing up a whole * cluster. See {@link TestRegionServerNoMaster}. @@ -472,7 +477,7 @@ public class TestRegionReplicas { // should be able to deal with it giving us all the result we expect. int keys = 0; int sum = 0; - for (StoreFile sf: secondaryRegion.getStore(f).getStorefiles()) { + for (HStoreFile sf : ((HStore) secondaryRegion.getStore(f)).getStorefiles()) { // Our file does not exist anymore. was moved by the compaction above. LOG.debug(getRS().getFileSystem().exists(sf.getPath())); Assert.assertFalse(getRS().getFileSystem().exists(sf.getPath())); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java index cad060e5e19..06c0bfd5116 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionSplitPolicy.java @@ -25,13 +25,14 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.util.Bytes; @@ -246,7 +247,7 @@ public class TestRegionSplitPolicy { HStore mockStore = Mockito.mock(HStore.class); Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); - Mockito.doReturn(Bytes.toBytes("abcd")).when(mockStore).getSplitPoint(); + Mockito.doReturn(Optional.of(Bytes.toBytes("abcd"))).when(mockStore).getSplitPoint(); stores.add(mockStore); KeyPrefixRegionSplitPolicy policy = (KeyPrefixRegionSplitPolicy) RegionSplitPolicy @@ -322,8 +323,7 @@ public class TestRegionSplitPolicy { HStore mockStore = Mockito.mock(HStore.class); Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); - Mockito.doReturn(Bytes.toBytes("store 1 split")) - .when(mockStore).getSplitPoint(); + Mockito.doReturn(Optional.of(Bytes.toBytes("store 1 split"))).when(mockStore).getSplitPoint(); stores.add(mockStore); assertEquals("store 1 split", @@ -333,8 +333,7 @@ public class TestRegionSplitPolicy { HStore mockStore2 = Mockito.mock(HStore.class); Mockito.doReturn(4000L).when(mockStore2).getSize(); Mockito.doReturn(true).when(mockStore2).canSplit(); - Mockito.doReturn(Bytes.toBytes("store 2 split")) - .when(mockStore2).getSplitPoint(); + Mockito.doReturn(Optional.of(Bytes.toBytes("store 2 split"))).when(mockStore2).getSplitPoint(); stores.add(mockStore2); assertEquals("store 2 split", @@ -355,7 +354,7 @@ public class TestRegionSplitPolicy { HStore mockStore = Mockito.mock(HStore.class); Mockito.doReturn(2000L).when(mockStore).getSize(); Mockito.doReturn(true).when(mockStore).canSplit(); - Mockito.doReturn(Bytes.toBytes("ab,cd")).when(mockStore).getSplitPoint(); + Mockito.doReturn(Optional.of(Bytes.toBytes("ab,cd"))).when(mockStore).getSplitPoint(); stores.add(mockStore); DelimitedKeyPrefixRegionSplitPolicy policy = (DelimitedKeyPrefixRegionSplitPolicy) RegionSplitPolicy 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 8b34a2f15c7..dbf3be015ac 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 @@ -113,7 +113,7 @@ public class TestReversibleScanners { .withFileContext(hFileContext).build(); writeStoreFile(writer); - StoreFile sf = new HStoreFile(fs, writer.getPath(), TEST_UTIL.getConfiguration(), cacheConf, + HStoreFile 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 HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf, + HStoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE, true); - StoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf, + HStoreFile 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 HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf, + HStoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE, true); - StoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf, + HStoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf, BloomType.NONE, true); ScanInfo scanInfo = @@ -418,19 +418,15 @@ public class TestReversibleScanners { verifyCountAndOrder(scanner, expectedRowNum * 2 * 2, expectedRowNum, false); } - private StoreScanner getReversibleStoreScanner(MemStore memstore, - StoreFile sf1, StoreFile sf2, Scan scan, - ScanInfo scanInfo, int readPoint) throws IOException { - List scanners = getScanners(memstore, sf1, sf2, null, - false, readPoint); + private StoreScanner getReversibleStoreScanner(MemStore memstore, HStoreFile sf1, HStoreFile sf2, + Scan scan, ScanInfo scanInfo, int readPoint) throws IOException { + List scanners = getScanners(memstore, sf1, sf2, null, false, readPoint); NavigableSet columns = null; - for (Map.Entry> entry : scan.getFamilyMap() - .entrySet()) { + for (Map.Entry> entry : scan.getFamilyMap().entrySet()) { // Should only one family columns = entry.getValue(); } - StoreScanner storeScanner = new ReversedStoreScanner(scan, scanInfo, - columns, scanners); + StoreScanner storeScanner = new ReversedStoreScanner(scan, scanInfo, columns, scanners); return storeScanner; } @@ -487,22 +483,17 @@ public class TestReversibleScanners { assertEquals(null, kvHeap.peek()); } - private ReversedKeyValueHeap getReversibleKeyValueHeap(MemStore memstore, - StoreFile sf1, StoreFile sf2, byte[] startRow, int readPoint) - throws IOException { - List scanners = getScanners(memstore, sf1, sf2, startRow, - true, readPoint); - ReversedKeyValueHeap kvHeap = new ReversedKeyValueHeap(scanners, - CellComparator.COMPARATOR); + private ReversedKeyValueHeap getReversibleKeyValueHeap(MemStore memstore, HStoreFile sf1, + HStoreFile sf2, byte[] startRow, int readPoint) throws IOException { + List scanners = getScanners(memstore, sf1, sf2, startRow, true, readPoint); + ReversedKeyValueHeap kvHeap = new ReversedKeyValueHeap(scanners, CellComparator.COMPARATOR); return kvHeap; } - private List getScanners(MemStore memstore, StoreFile sf1, - StoreFile sf2, byte[] startRow, boolean doSeek, int readPoint) - throws IOException { - List fileScanners = StoreFileScanner - .getScannersForStoreFiles(Lists.newArrayList(sf1, sf2), false, true, - false, false, readPoint); + private List getScanners(MemStore memstore, HStoreFile sf1, HStoreFile sf2, + byte[] startRow, boolean doSeek, int readPoint) throws IOException { + List fileScanners = StoreFileScanner.getScannersForStoreFiles( + Lists.newArrayList(sf1, sf2), false, true, false, false, readPoint); List memScanners = memstore.getScanners(readPoint); List scanners = new ArrayList<>(fileScanners.size() + 1); scanners.addAll(fileScanners); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java index 5dce4adea03..0c014fd5e7a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestScannerWithBulkload.java @@ -1,4 +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 @@ -17,6 +17,9 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY; + import java.io.IOException; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -171,10 +174,10 @@ public class TestScannerWithBulkload { // Set a big MAX_SEQ_ID_KEY. Scan should not look at this seq id in a bulk loaded file. // Scan should only look at the seq id appended at the bulk load time, and not skip its // kv. - writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(new Long(9999999))); + writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(new Long(9999999))); } else { - writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())); + writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())); } writer.close(); return hfilePath; 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 3ba22997089..c2c317120f3 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 @@ -82,12 +82,6 @@ import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; -import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController; -import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; -import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; -import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.util.Bytes; @@ -109,6 +103,13 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.junit.rules.TestRule; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController; +import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; +import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest; +import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse; + /** * The below tests are testing split region against a running cluster */ @@ -750,8 +751,8 @@ public class TestSplitTransactionOnCluster { region.put(p); } region.flush(true); - Store store = region.getStore(Bytes.toBytes("f")); - Collection storefiles = store.getStorefiles(); + HStore store = region.getStore(Bytes.toBytes("f")); + Collection storefiles = store.getStorefiles(); assertEquals(storefiles.size(), 1); assertFalse(region.hasReferences()); Path referencePath = 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 8c460a6c56e..a562af86979 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 @@ -82,8 +82,8 @@ public class TestStripeStoreEngine { .thenReturn(new ArrayList<>()); // Produce 3 L0 files. - StoreFile sf = createFile(); - ArrayList compactUs = al(sf, createFile(), createFile()); + HStoreFile sf = createFile(); + ArrayList compactUs = al(sf, createFile(), createFile()); se.getStoreFileManager().loadFiles(compactUs); // Create a compaction that would want to split the stripe. CompactionContext compaction = se.createCompaction(); @@ -103,8 +103,8 @@ public class TestStripeStoreEngine { NoLimitThroughputController.INSTANCE, null); } - private static StoreFile createFile() throws Exception { - StoreFile sf = mock(StoreFile.class); + private static HStoreFile createFile() throws Exception { + HStoreFile sf = mock(HStoreFile.class); when(sf.getMetadataValue(any(byte[].class))) .thenReturn(StripeStoreFileManager.INVALID_KEY); when(sf.getReader()).thenReturn(mock(StoreFileReader.class)); @@ -114,12 +114,12 @@ public class TestStripeStoreEngine { } private static TestStoreEngine createEngine(Configuration conf) throws Exception { - Store store = mock(Store.class); + HStore store = mock(HStore.class); CellComparator kvComparator = mock(CellComparator.class); return (TestStoreEngine)StoreEngine.create(store, conf, kvComparator); } - private static ArrayList al(StoreFile... sfs) { + private static ArrayList al(HStoreFile... sfs) { return new ArrayList<>(Arrays.asList(sfs)); } } 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 76959c6c13c..6e5aeed101c 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 @@ -22,7 +22,6 @@ import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_K import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -87,10 +86,10 @@ public class TestStripeStoreFileManager { @Test public void testInsertFilesIntoL0() throws Exception { StripeStoreFileManager manager = createManager(); - MockStoreFile sf = createFile(); + MockHStoreFile sf = createFile(); manager.insertNewFiles(al(sf)); assertEquals(1, manager.getStorefileCount()); - Collection filesForGet = manager.getFilesForScan(KEY_A, true, KEY_A, true); + Collection filesForGet = manager.getFilesForScan(KEY_A, true, KEY_A, true); assertEquals(1, filesForGet.size()); assertTrue(filesForGet.contains(sf)); @@ -109,14 +108,14 @@ public class TestStripeStoreFileManager { manager.addCompactionResults(al(), al(createFile(OPEN_KEY, KEY_B), createFile(KEY_B, OPEN_KEY))); assertEquals(4, manager.getStorefileCount()); - Collection allFiles = manager.clearFiles(); + Collection allFiles = manager.clearFiles(); assertEquals(4, allFiles.size()); assertEquals(0, manager.getStorefileCount()); assertEquals(0, manager.getStorefiles().size()); } - private static ArrayList dumpIterator(Iterator iter) { - ArrayList result = new ArrayList<>(); + private static ArrayList dumpIterator(Iterator iter) { + ArrayList result = new ArrayList<>(); for (; iter.hasNext(); result.add(iter.next())); return result; } @@ -124,23 +123,23 @@ public class TestStripeStoreFileManager { @Test public void testRowKeyBefore() throws Exception { StripeStoreFileManager manager = createManager(); - StoreFile l0File = createFile(), l0File2 = createFile(); + HStoreFile l0File = createFile(), l0File2 = createFile(); manager.insertNewFiles(al(l0File)); manager.insertNewFiles(al(l0File2)); // Get candidate files. - Iterator sfs = manager.getCandidateFilesForRowKeyBefore(KV_B); + Iterator sfs = manager.getCandidateFilesForRowKeyBefore(KV_B); sfs.next(); sfs.remove(); // Suppose we found a candidate in this file... make sure L0 file remaining is not removed. sfs = manager.updateCandidateFilesForRowKeyBefore(sfs, KV_B, KV_A); assertTrue(sfs.hasNext()); // Now add some stripes (remove L0 file too) - MockStoreFile stripe0a = createFile(0, 100, OPEN_KEY, KEY_B), + MockHStoreFile stripe0a = createFile(0, 100, OPEN_KEY, KEY_B), stripe1 = createFile(KEY_B, OPEN_KEY); manager.addCompactionResults(al(l0File), al(stripe0a, stripe1)); manager.removeCompactedFiles(al(l0File)); // If we want a key <= KEY_A, we should get everything except stripe1. - ArrayList sfsDump = dumpIterator(manager.getCandidateFilesForRowKeyBefore(KV_A)); + ArrayList sfsDump = dumpIterator(manager.getCandidateFilesForRowKeyBefore(KV_A)); assertEquals(2, sfsDump.size()); assertTrue(sfsDump.contains(stripe0a)); assertFalse(sfsDump.contains(stripe1)); @@ -162,7 +161,7 @@ public class TestStripeStoreFileManager { // Add one more, later, file to stripe0, remove the last annoying L0 file. // This file should be returned in preference to older L0 file; also, after we get // a candidate from the first file, the old one should not be removed. - StoreFile stripe0b = createFile(0, 101, OPEN_KEY, KEY_B); + HStoreFile stripe0b = createFile(0, 101, OPEN_KEY, KEY_B); manager.addCompactionResults(al(l0File2), al(stripe0b)); manager.removeCompactedFiles(al(l0File2)); sfs = manager.getCandidateFilesForRowKeyBefore(KV_A); @@ -176,24 +175,24 @@ public class TestStripeStoreFileManager { public void testGetSplitPointEdgeCases() throws Exception { StripeStoreFileManager manager = createManager(); // No files => no split. - assertNull(manager.getSplitPoint()); + assertFalse(manager.getSplitPoint().isPresent()); // If there are no stripes, should pick midpoint from the biggest file in L0. - MockStoreFile sf5 = createFile(5, 0); + MockHStoreFile sf5 = createFile(5, 0); sf5.splitPoint = new byte[] { 1 }; manager.insertNewFiles(al(sf5)); manager.insertNewFiles(al(createFile(1, 0))); - assertArrayEquals(sf5.splitPoint, manager.getSplitPoint()); + assertArrayEquals(sf5.splitPoint, manager.getSplitPoint().get()); // 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))); - assertArrayEquals(sf5.splitPoint, manager.getSplitPoint()); + assertArrayEquals(sf5.splitPoint, manager.getSplitPoint().get()); // If the biggest file is in the stripe, should get from it. - MockStoreFile sf6 = createFile(6, 0, OPEN_KEY, OPEN_KEY); + MockHStoreFile sf6 = createFile(6, 0, OPEN_KEY, OPEN_KEY); sf6.splitPoint = new byte[] { 2 }; manager.addCompactionResults(al(), al(sf6)); - assertArrayEquals(sf6.splitPoint, manager.getSplitPoint()); + assertArrayEquals(sf6.splitPoint, manager.getSplitPoint().get()); } @Test @@ -234,11 +233,11 @@ public class TestStripeStoreFileManager { private void verifySplitPointScenario(int splitPointAfter, boolean shouldSplitStripe, float splitRatioToVerify, int... sizes) throws Exception { assertTrue(sizes.length > 1); - ArrayList sfs = new ArrayList<>(); + ArrayList sfs = new ArrayList<>(); for (int sizeIx = 0; sizeIx < sizes.length; ++sizeIx) { byte[] startKey = (sizeIx == 0) ? OPEN_KEY : Bytes.toBytes(sizeIx - 1); byte[] endKey = (sizeIx == sizes.length - 1) ? OPEN_KEY : Bytes.toBytes(sizeIx); - MockStoreFile sf = createFile(sizes[sizeIx], 0, startKey, endKey); + MockHStoreFile sf = createFile(sizes[sizeIx], 0, startKey, endKey); sf.splitPoint = Bytes.toBytes(-sizeIx); // set split point to the negative index sfs.add(sf); } @@ -249,7 +248,7 @@ public class TestStripeStoreFileManager { } StripeStoreFileManager manager = createManager(al(), conf); manager.addCompactionResults(al(), sfs); - int result = Bytes.toInt(manager.getSplitPoint()); + int result = Bytes.toInt(manager.getSplitPoint().get()); // Either end key and thus positive index, or "middle" of the file and thus negative index. assertEquals(splitPointAfter * (shouldSplitStripe ? -1 : 1), result); } @@ -265,7 +264,7 @@ public class TestStripeStoreFileManager { verifyGetAndScanScenario(manager, KEY_B, KEY_C); // Populate one L0 file. - MockStoreFile sf0 = createFile(); + MockHStoreFile sf0 = createFile(); manager.insertNewFiles(al(sf0)); verifyGetAndScanScenario(manager, null, null, sf0); verifyGetAndScanScenario(manager, null, KEY_C, sf0); @@ -273,11 +272,11 @@ public class TestStripeStoreFileManager { verifyGetAndScanScenario(manager, KEY_B, KEY_C, sf0); // Populate a bunch of files for stripes, keep L0. - MockStoreFile sfA = createFile(OPEN_KEY, KEY_A); - MockStoreFile sfB = createFile(KEY_A, KEY_B); - MockStoreFile sfC = createFile(KEY_B, KEY_C); - MockStoreFile sfD = createFile(KEY_C, KEY_D); - MockStoreFile sfE = createFile(KEY_D, OPEN_KEY); + MockHStoreFile sfA = createFile(OPEN_KEY, KEY_A); + MockHStoreFile sfB = createFile(KEY_A, KEY_B); + MockHStoreFile sfC = createFile(KEY_B, KEY_C); + MockHStoreFile sfD = createFile(KEY_C, KEY_D); + MockHStoreFile sfE = createFile(KEY_D, OPEN_KEY); manager.addCompactionResults(al(), al(sfA, sfB, sfC, sfD, sfE)); verifyGetAndScanScenario(manager, null, null, sf0, sfA, sfB, sfC, sfD, sfE); @@ -292,7 +291,7 @@ public class TestStripeStoreFileManager { } private void verifyGetAndScanScenario(StripeStoreFileManager manager, byte[] start, byte[] end, - StoreFile... results) throws Exception { + HStoreFile... results) throws Exception { verifyGetOrScanScenario(manager, start, end, results); } @@ -302,18 +301,18 @@ public class TestStripeStoreFileManager { // In L0, there will be file w/o metadata (real L0, 3 files with invalid metadata, and 3 // files that overlap valid stripes in various ways). Note that the 4th way to overlap the // stripes will cause the structure to be mostly scraped, and is tested separately. - ArrayList validStripeFiles = al(createFile(OPEN_KEY, KEY_B), + ArrayList validStripeFiles = al(createFile(OPEN_KEY, KEY_B), createFile(KEY_B, KEY_C), createFile(KEY_C, OPEN_KEY), createFile(KEY_C, OPEN_KEY)); - ArrayList filesToGoToL0 = al(createFile(), createFile(null, KEY_A), + ArrayList filesToGoToL0 = al(createFile(), createFile(null, KEY_A), createFile(KEY_D, null), createFile(KEY_D, KEY_A), createFile(keyAfter(KEY_A), KEY_C), createFile(OPEN_KEY, KEY_D), createFile(KEY_D, keyAfter(KEY_D))); - ArrayList allFilesToGo = flattenLists(validStripeFiles, filesToGoToL0); + ArrayList allFilesToGo = flattenLists(validStripeFiles, filesToGoToL0); Collections.shuffle(allFilesToGo); StripeStoreFileManager manager = createManager(allFilesToGo); - List l0Files = manager.getLevel0Files(); + List l0Files = manager.getLevel0Files(); assertEquals(filesToGoToL0.size(), l0Files.size()); - for (StoreFile sf : filesToGoToL0) { + for (HStoreFile sf : filesToGoToL0) { assertTrue(l0Files.contains(sf)); } verifyAllFiles(manager, allFilesToGo); @@ -323,7 +322,7 @@ public class TestStripeStoreFileManager { public void testLoadFilesWithBadStripe() throws Exception { // Current "algorithm" will see the after-B key before C key, add it as valid stripe, // and then fail all other stripes. So everything would end up in L0. - ArrayList allFilesToGo = al(createFile(OPEN_KEY, KEY_B), + ArrayList allFilesToGo = al(createFile(OPEN_KEY, KEY_B), createFile(KEY_B, KEY_C), createFile(KEY_C, OPEN_KEY), createFile(KEY_B, keyAfter(KEY_B))); Collections.shuffle(allFilesToGo); @@ -346,7 +345,7 @@ public class TestStripeStoreFileManager { @Test public void testLoadFilesAfterSplit() throws Exception { // If stripes are good but have non-open ends, they must be treated as open ends. - MockStoreFile sf = createFile(KEY_B, KEY_C); + MockHStoreFile sf = createFile(KEY_B, KEY_C); StripeStoreFileManager manager = createManager(al(createFile(OPEN_KEY, KEY_B), sf)); assertEquals(0, manager.getLevel0Files().size()); // Here, [B, C] is logically [B, inf), so we should be able to compact it to that only. @@ -367,7 +366,7 @@ public class TestStripeStoreFileManager { public void testAddingCompactionResults() throws Exception { StripeStoreFileManager manager = createManager(); // First, add some L0 files and "compact" one with new stripe creation. - StoreFile sf_L0_0a = createFile(), sf_L0_0b = createFile(); + HStoreFile sf_L0_0a = createFile(), sf_L0_0b = createFile(); manager.insertNewFiles(al(sf_L0_0a, sf_L0_0b)); // Try compacting with invalid new branches (gaps, overlaps) - no effect. @@ -379,24 +378,24 @@ public class TestStripeStoreFileManager { verifyInvalidCompactionScenario(manager, al(sf_L0_0a), al(createFile(OPEN_KEY, KEY_B), createFile(KEY_A, KEY_B), createFile(KEY_B, OPEN_KEY))); - StoreFile sf_i2B_0 = createFile(OPEN_KEY, KEY_B); - StoreFile sf_B2C_0 = createFile(KEY_B, KEY_C); - StoreFile sf_C2i_0 = createFile(KEY_C, OPEN_KEY); + HStoreFile sf_i2B_0 = createFile(OPEN_KEY, KEY_B); + HStoreFile sf_B2C_0 = createFile(KEY_B, KEY_C); + HStoreFile sf_C2i_0 = createFile(KEY_C, OPEN_KEY); manager.addCompactionResults(al(sf_L0_0a), al(sf_i2B_0, sf_B2C_0, sf_C2i_0)); manager.removeCompactedFiles(al(sf_L0_0a)); verifyAllFiles(manager, al(sf_L0_0b, sf_i2B_0, sf_B2C_0, sf_C2i_0)); // Add another l0 file, "compact" both L0 into two stripes - StoreFile sf_L0_1 = createFile(); - StoreFile sf_i2B_1 = createFile(OPEN_KEY, KEY_B); - StoreFile sf_B2C_1 = createFile(KEY_B, KEY_C); + HStoreFile sf_L0_1 = createFile(); + HStoreFile sf_i2B_1 = createFile(OPEN_KEY, KEY_B); + HStoreFile sf_B2C_1 = createFile(KEY_B, KEY_C); manager.insertNewFiles(al(sf_L0_1)); manager.addCompactionResults(al(sf_L0_0b, sf_L0_1), al(sf_i2B_1, sf_B2C_1)); manager.removeCompactedFiles(al(sf_L0_0b, sf_L0_1)); verifyAllFiles(manager, al(sf_i2B_0, sf_B2C_0, sf_C2i_0, sf_i2B_1, sf_B2C_1)); // Try compacting with invalid file (no metadata) - should add files to L0. - StoreFile sf_L0_2 = createFile(null, null); + HStoreFile sf_L0_2 = createFile(null, null); manager.addCompactionResults(al(), al(sf_L0_2)); manager.removeCompactedFiles(al()); verifyAllFiles(manager, al(sf_i2B_0, sf_B2C_0, sf_C2i_0, sf_i2B_1, sf_B2C_1, sf_L0_2)); @@ -405,46 +404,46 @@ public class TestStripeStoreFileManager { manager.removeCompactedFiles(al(sf_L0_2)); // Do regular compaction in the first stripe. - StoreFile sf_i2B_3 = createFile(OPEN_KEY, KEY_B); + HStoreFile sf_i2B_3 = createFile(OPEN_KEY, KEY_B); manager.addCompactionResults(al(sf_i2B_0, sf_i2B_1), al(sf_i2B_3)); manager.removeCompactedFiles(al(sf_i2B_0, sf_i2B_1)); verifyAllFiles(manager, al(sf_B2C_0, sf_C2i_0, sf_B2C_1, sf_i2B_3)); // Rebalance two stripes. - StoreFile sf_B2D_4 = createFile(KEY_B, KEY_D); - StoreFile sf_D2i_4 = createFile(KEY_D, OPEN_KEY); + HStoreFile sf_B2D_4 = createFile(KEY_B, KEY_D); + HStoreFile sf_D2i_4 = createFile(KEY_D, OPEN_KEY); manager.addCompactionResults(al(sf_B2C_0, sf_C2i_0, sf_B2C_1), al(sf_B2D_4, sf_D2i_4)); manager.removeCompactedFiles(al(sf_B2C_0, sf_C2i_0, sf_B2C_1)); verifyAllFiles(manager, al(sf_i2B_3, sf_B2D_4, sf_D2i_4)); // Split the first stripe. - StoreFile sf_i2A_5 = createFile(OPEN_KEY, KEY_A); - StoreFile sf_A2B_5 = createFile(KEY_A, KEY_B); + HStoreFile sf_i2A_5 = createFile(OPEN_KEY, KEY_A); + HStoreFile sf_A2B_5 = createFile(KEY_A, KEY_B); manager.addCompactionResults(al(sf_i2B_3), al(sf_i2A_5, sf_A2B_5)); manager.removeCompactedFiles(al(sf_i2B_3)); verifyAllFiles(manager, al(sf_B2D_4, sf_D2i_4, sf_i2A_5, sf_A2B_5)); // Split the middle stripe. - StoreFile sf_B2C_6 = createFile(KEY_B, KEY_C); - StoreFile sf_C2D_6 = createFile(KEY_C, KEY_D); + HStoreFile sf_B2C_6 = createFile(KEY_B, KEY_C); + HStoreFile sf_C2D_6 = createFile(KEY_C, KEY_D); manager.addCompactionResults(al(sf_B2D_4), al(sf_B2C_6, sf_C2D_6)); manager.removeCompactedFiles(al(sf_B2D_4)); verifyAllFiles(manager, al(sf_D2i_4, sf_i2A_5, sf_A2B_5, sf_B2C_6, sf_C2D_6)); // Merge two different middle stripes. - StoreFile sf_A2C_7 = createFile(KEY_A, KEY_C); + HStoreFile sf_A2C_7 = createFile(KEY_A, KEY_C); manager.addCompactionResults(al(sf_A2B_5, sf_B2C_6), al(sf_A2C_7)); manager.removeCompactedFiles(al(sf_A2B_5, sf_B2C_6)); verifyAllFiles(manager, al(sf_D2i_4, sf_i2A_5, sf_C2D_6, sf_A2C_7)); // Merge lower half. - StoreFile sf_i2C_8 = createFile(OPEN_KEY, KEY_C); + HStoreFile sf_i2C_8 = createFile(OPEN_KEY, KEY_C); manager.addCompactionResults(al(sf_i2A_5, sf_A2C_7), al(sf_i2C_8)); manager.removeCompactedFiles(al(sf_i2A_5, sf_A2C_7)); verifyAllFiles(manager, al(sf_D2i_4, sf_C2D_6, sf_i2C_8)); // Merge all. - StoreFile sf_i2i_9 = createFile(OPEN_KEY, OPEN_KEY); + HStoreFile sf_i2i_9 = createFile(OPEN_KEY, OPEN_KEY); manager.addCompactionResults(al(sf_D2i_4, sf_C2D_6, sf_i2C_8), al(sf_i2i_9)); manager.removeCompactedFiles(al(sf_D2i_4, sf_C2D_6, sf_i2C_8)); verifyAllFiles(manager, al(sf_i2i_9)); @@ -455,11 +454,11 @@ public class TestStripeStoreFileManager { // Add file flush into stripes StripeStoreFileManager sfm = createManager(); assertEquals(0, sfm.getStripeCount()); - StoreFile sf_i2c = createFile(OPEN_KEY, KEY_C), sf_c2i = createFile(KEY_C, OPEN_KEY); + HStoreFile sf_i2c = createFile(OPEN_KEY, KEY_C), sf_c2i = createFile(KEY_C, OPEN_KEY); sfm.insertNewFiles(al(sf_i2c, sf_c2i)); assertEquals(2, sfm.getStripeCount()); // Now try to add conflicting flush - should throw. - StoreFile sf_i2d = createFile(OPEN_KEY, KEY_D), sf_d2i = createFile(KEY_D, OPEN_KEY); + HStoreFile sf_i2d = createFile(OPEN_KEY, KEY_D), sf_d2i = createFile(KEY_D, OPEN_KEY); sfm.insertNewFiles(al(sf_i2d, sf_d2i)); assertEquals(2, sfm.getStripeCount()); assertEquals(2, sfm.getLevel0Files().size()); @@ -470,7 +469,7 @@ public class TestStripeStoreFileManager { assertEquals(0, sfm.getLevel0Files().size()); // Add another file to stripe; then "rebalance" stripes w/o it - the file, which was // presumably flushed during compaction, should go to L0. - StoreFile sf_i2c_2 = createFile(OPEN_KEY, KEY_C); + HStoreFile sf_i2c_2 = createFile(OPEN_KEY, KEY_C); sfm.insertNewFiles(al(sf_i2c_2)); sfm.addCompactionResults(al(sf_i2c, sf_c2i), al(sf_i2d, sf_d2i)); sfm.removeCompactedFiles(al(sf_i2c, sf_c2i)); @@ -482,16 +481,16 @@ public class TestStripeStoreFileManager { public void testEmptyResultsForStripes() throws Exception { // Test that we can compact L0 into a subset of stripes. StripeStoreFileManager manager = createManager(); - StoreFile sf0a = createFile(); - StoreFile sf0b = createFile(); + HStoreFile sf0a = createFile(); + HStoreFile sf0b = createFile(); manager.insertNewFiles(al(sf0a)); manager.insertNewFiles(al(sf0b)); - ArrayList compacted = al(createFile(OPEN_KEY, KEY_B), + ArrayList compacted = al(createFile(OPEN_KEY, KEY_B), createFile(KEY_B, KEY_C), createFile(KEY_C, OPEN_KEY)); manager.addCompactionResults(al(sf0a), compacted); manager.removeCompactedFiles(al(sf0a)); // Next L0 compaction only produces file for the first and last stripe. - ArrayList compacted2 = al(createFile(OPEN_KEY, KEY_B), createFile(KEY_C, OPEN_KEY)); + ArrayList compacted2 = al(createFile(OPEN_KEY, KEY_B), createFile(KEY_C, OPEN_KEY)); manager.addCompactionResults(al(sf0b), compacted2); manager.removeCompactedFiles(al(sf0b)); compacted.addAll(compacted2); @@ -526,7 +525,7 @@ public class TestStripeStoreFileManager { sfm.insertNewFiles(al(createFile())); } for (int i = 0; i < filesInStripe; ++i) { - ArrayList stripe = new ArrayList<>(); + ArrayList stripe = new ArrayList<>(); for (int j = 0; j < stripes; ++j) { stripe.add(createFile( (j == 0) ? OPEN_KEY : keys[j - 1], (j == stripes - 1) ? OPEN_KEY : keys[j])); @@ -537,8 +536,8 @@ public class TestStripeStoreFileManager { } private void verifyInvalidCompactionScenario(StripeStoreFileManager manager, - ArrayList filesToCompact, ArrayList filesToInsert) throws Exception { - Collection allFiles = manager.getStorefiles(); + ArrayList filesToCompact, ArrayList filesToInsert) throws Exception { + Collection allFiles = manager.getStorefiles(); try { manager.addCompactionResults(filesToCompact, filesToInsert); fail("Should have thrown"); @@ -549,33 +548,33 @@ public class TestStripeStoreFileManager { } private void verifyGetOrScanScenario(StripeStoreFileManager manager, byte[] start, byte[] end, - StoreFile... results) throws Exception { + HStoreFile... results) throws Exception { verifyGetOrScanScenario(manager, start, end, Arrays.asList(results)); } private void verifyGetOrScanScenario(StripeStoreFileManager manager, byte[] start, byte[] end, - Collection results) throws Exception { + Collection results) throws Exception { start = start != null ? start : HConstants.EMPTY_START_ROW; end = end != null ? end : HConstants.EMPTY_END_ROW; - Collection sfs = manager.getFilesForScan(start, true, end, false); + Collection sfs = manager.getFilesForScan(start, true, end, false); assertEquals(results.size(), sfs.size()); - for (StoreFile result : results) { + for (HStoreFile result : results) { assertTrue(sfs.contains(result)); } } private void verifyAllFiles( - StripeStoreFileManager manager, Collection results) throws Exception { + StripeStoreFileManager manager, Collection results) throws Exception { verifyGetOrScanScenario(manager, null, null, results); } // TODO: replace with Mockito? - private static MockStoreFile createFile( + private static MockHStoreFile createFile( long size, long seqNum, byte[] startKey, byte[] endKey) throws Exception { FileSystem fs = TEST_UTIL.getTestFileSystem(); Path testFilePath = StoreFileWriter.getUniqueFile(fs, CFDIR); fs.create(testFilePath).close(); - MockStoreFile sf = new MockStoreFile(TEST_UTIL, testFilePath, size, 0, false, seqNum); + MockHStoreFile sf = new MockHStoreFile(TEST_UTIL, testFilePath, size, 0, false, seqNum); if (startKey != null) { sf.setMetadataValue(StripeStoreFileManager.STRIPE_START_KEY, startKey); } @@ -585,15 +584,15 @@ public class TestStripeStoreFileManager { return sf; } - private static MockStoreFile createFile(long size, long seqNum) throws Exception { + private static MockHStoreFile createFile(long size, long seqNum) throws Exception { return createFile(size, seqNum, null, null); } - private static MockStoreFile createFile(byte[] startKey, byte[] endKey) throws Exception { + private static MockHStoreFile createFile(byte[] startKey, byte[] endKey) throws Exception { return createFile(0, 0, startKey, endKey); } - private static MockStoreFile createFile() throws Exception { + private static MockHStoreFile createFile() throws Exception { return createFile(null, null); } @@ -601,12 +600,12 @@ public class TestStripeStoreFileManager { return createManager(new ArrayList<>()); } - private static StripeStoreFileManager createManager(ArrayList sfs) throws Exception { + private static StripeStoreFileManager createManager(ArrayList sfs) throws Exception { return createManager(sfs, TEST_UTIL.getConfiguration()); } private static StripeStoreFileManager createManager( - ArrayList sfs, Configuration conf) throws Exception { + ArrayList sfs, Configuration conf) throws Exception { StripeStoreConfig config = new StripeStoreConfig( conf, Mockito.mock(StoreConfigInformation.class)); StripeStoreFileManager result = new StripeStoreFileManager(CellComparator.COMPARATOR, conf, @@ -615,13 +614,13 @@ public class TestStripeStoreFileManager { return result; } - private static ArrayList al(StoreFile... sfs) { + private static ArrayList al(HStoreFile... sfs) { return new ArrayList<>(Arrays.asList(sfs)); } - private static ArrayList flattenLists(ArrayList... sfls) { - ArrayList result = new ArrayList<>(); - for (ArrayList sfl : sfls) { + private static ArrayList flattenLists(ArrayList... sfls) { + ArrayList result = new ArrayList<>(); + for (ArrayList sfl : sfls) { result.addAll(sfl); } return result; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java index 767ad2e6cea..730696c9b75 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestSwitchToStreamRead.java @@ -28,12 +28,12 @@ import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -67,7 +67,10 @@ public class TestSwitchToStreamRead { } VALUE_PREFIX = sb.append("-").toString(); REGION = UTIL.createLocalHRegion( - new HTableDescriptor(TABLE_NAME).addFamily(new HColumnDescriptor(FAMILY).setBlocksize(1024)), + TableDescriptorBuilder.newBuilder(TABLE_NAME) + .addColumnFamily( + ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).setBlocksize(1024).build()) + .build(), null, null); for (int i = 0; i < 900; i++) { REGION @@ -122,7 +125,7 @@ public class TestSwitchToStreamRead { } } // make sure all scanners are closed. - for (StoreFile sf : REGION.getStore(FAMILY).getStorefiles()) { + for (HStoreFile sf : REGION.getStore(FAMILY).getStorefiles()) { assertFalse(sf.isReferencedInReads()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java index 5014b414153..2e47566a9cb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ConstantSizeFileListGenerator.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.compactions; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; /** * Class to generate several lists of StoreFiles that are all the same size. @@ -37,8 +36,8 @@ class ConstantSizeFileListGenerator extends StoreFileListGenerator { } @Override - public final Iterator> iterator() { - return new Iterator>() { + public final Iterator> iterator() { + return new Iterator>() { private int count = 0; @Override @@ -47,9 +46,9 @@ class ConstantSizeFileListGenerator extends StoreFileListGenerator { } @Override - public List next() { + public List next() { count += 1; - ArrayList files = new ArrayList<>(NUM_FILES_GEN); + ArrayList files = new ArrayList<>(NUM_FILES_GEN); for (int i = 0; i < NUM_FILES_GEN; i++) { files.add(createMockStoreFile(FILESIZE)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/EverythingPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/EverythingPolicy.java index 46bb639182a..ca65bf1178b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/EverythingPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/EverythingPolicy.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.compactions; @@ -23,8 +22,8 @@ import java.io.IOException; import java.util.ArrayList; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; -import org.apache.hadoop.hbase.regionserver.StoreFile; /** * Test Policy to compact everything every time. @@ -36,19 +35,16 @@ public class EverythingPolicy extends RatioBasedCompactionPolicy { * @param conf The Conf. * @param storeConfigInfo Info about the store. */ - public EverythingPolicy(final Configuration conf, - final StoreConfigInformation storeConfigInfo) { + public EverythingPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) { super(conf, storeConfigInfo); } @Override - protected final ArrayList applyCompactionPolicy(final ArrayList candidates, - final boolean mayUseOffPeak, final boolean mayBeStuck) throws IOException { - + protected final ArrayList applyCompactionPolicy(ArrayList candidates, + boolean mayUseOffPeak, boolean mayBeStuck) throws IOException { if (candidates.size() < comConf.getMinFilesToCompact()) { return new ArrayList<>(0); } - return new ArrayList<>(candidates); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java index 24302b81b68..e14f69618b1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/ExplicitFileListGenerator.java @@ -15,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.compactions; import java.util.Iterator; import java.util.List; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; /** * Class to create list of mock storefiles of specified length. @@ -53,8 +52,8 @@ class ExplicitFileListGenerator extends StoreFileListGenerator { } @Override - public final Iterator> iterator() { - return new Iterator>() { + public final Iterator> iterator() { + return new Iterator>() { private int nextIndex = 0; @Override public boolean hasNext() { @@ -62,8 +61,8 @@ class ExplicitFileListGenerator extends StoreFileListGenerator { } @Override - public List next() { - List files = createStoreFileList(fileSizes[nextIndex]); + public List next() { + List files = createStoreFileList(fileSizes[nextIndex]); nextIndex += 1; return files; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java index a2964fff185..fe5f9040988 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/GaussianFileListGenerator.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.compactions; import java.util.ArrayList; @@ -24,7 +23,7 @@ import java.util.List; import org.apache.commons.math3.random.GaussianRandomGenerator; import org.apache.commons.math3.random.MersenneTwister; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; class GaussianFileListGenerator extends StoreFileListGenerator { @@ -33,8 +32,8 @@ class GaussianFileListGenerator extends StoreFileListGenerator { } @Override - public Iterator> iterator() { - return new Iterator>() { + public Iterator> iterator() { + return new Iterator>() { private GaussianRandomGenerator gen = new GaussianRandomGenerator(new MersenneTwister(random.nextInt())); private int count = 0; @@ -45,9 +44,9 @@ class GaussianFileListGenerator extends StoreFileListGenerator { } @Override - public List next() { + public List next() { count += 1; - ArrayList files = new ArrayList<>(NUM_FILES_GEN); + ArrayList files = new ArrayList<>(NUM_FILES_GEN); for (int i = 0; i < NUM_FILES_GEN; i++) { files.add(createMockStoreFile( (int) Math.ceil(Math.max(0, gen.nextNormalizedDouble() * 32 + 32))) 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 85350cae740..21cee1379f6 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 @@ -21,18 +21,18 @@ package org.apache.hadoop.hbase.regionserver.compactions; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import org.apache.hadoop.hbase.shaded.com.google.common.base.MoreObjects; - import java.util.LinkedList; import java.util.List; import java.util.Random; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileReader; import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.hbase.shaded.com.google.common.base.MoreObjects; + /** * Base class of objects that can create mock store files with a given size. */ @@ -42,31 +42,31 @@ class MockStoreFileGenerator { /** The random number generator. */ protected Random random; - MockStoreFileGenerator(Class klass) { + MockStoreFileGenerator(Class klass) { random = new Random(klass.getSimpleName().hashCode()); } - protected List createStoreFileList(final int[] fs) { - List storeFiles = new LinkedList<>(); + protected List createStoreFileList(final int[] fs) { + List storeFiles = new LinkedList<>(); for (int fileSize : fs) { storeFiles.add(createMockStoreFile(fileSize)); } return storeFiles; } - protected StoreFile createMockStoreFile(final long size) { + protected HStoreFile createMockStoreFile(final long size) { return createMockStoreFile(size * 1024 * 1024, -1L); } - protected StoreFile createMockStoreFileBytes(final long size) { + protected HStoreFile createMockStoreFileBytes(final long size) { return createMockStoreFile(size, -1L); } - protected StoreFile createMockStoreFile(final long sizeInBytes, final long seqId) { - StoreFile mockSf = mock(StoreFile.class); + protected HStoreFile createMockStoreFile(final long sizeInBytes, final long seqId) { + HStoreFile mockSf = mock(HStoreFile.class); StoreFileReader reader = mock(StoreFileReader.class); - String stringPath = "/hbase/testTable/regionA/" - + RandomStringUtils.random(FILENAME_LENGTH, 0, 0, true, true, null, random); + String stringPath = "/hbase/testTable/regionA/" + + RandomStringUtils.random(FILENAME_LENGTH, 0, 0, true, true, null, random); Path path = new Path(stringPath); 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 2dbc26f097b..55546833ca8 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 @@ -26,13 +26,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -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.HStoreFile; 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; @@ -45,9 +43,6 @@ import org.junit.runners.Parameterized; @RunWith(Parameterized.class) public class PerfTestCompactionPolicies extends MockStoreFileGenerator { - - private static final Log LOG = LogFactory.getLog(PerfTestCompactionPolicies.class); - private final RatioBasedCompactionPolicy cp; private final StoreFileListGenerator generator; private final HStore store; @@ -62,13 +57,13 @@ public class PerfTestCompactionPolicies extends MockStoreFileGenerator { - Class[] policyClasses = new Class[]{ + Class[] policyClasses = new Class[]{ EverythingPolicy.class, RatioBasedCompactionPolicy.class, ExploringCompactionPolicy.class, }; - Class[] fileListGenClasses = new Class[]{ + Class[] fileListGenClasses = new Class[]{ ExplicitFileListGenerator.class, ConstantSizeFileListGenerator.class, SemiConstantSizeFileListGenerator.class, @@ -88,12 +83,12 @@ public class PerfTestCompactionPolicies extends MockStoreFileGenerator { * policyClasses.length); - for (Class policyClass : policyClasses) { - for (Class genClass: fileListGenClasses) { - for (int maxFile:maxFileValues) { - for (int minFile:minFilesValues) { - for (float ratio:ratioValues) { - params.add(new Object[] {policyClass, genClass, maxFile, minFile, ratio}); + for (Class policyClass : policyClasses) { + for (Class genClass : fileListGenClasses) { + for (int maxFile : maxFileValues) { + for (int minFile : minFilesValues) { + for (float ratio : ratioValues) { + params.add(new Object[] { policyClass, genClass, maxFile, minFile, ratio }); } } } @@ -150,9 +145,9 @@ public class PerfTestCompactionPolicies extends MockStoreFileGenerator { @Test public final void testSelection() throws Exception { long fileDiff = 0; - for (List storeFileList : generator) { - List currentFiles = new ArrayList<>(18); - for (StoreFile file : storeFileList) { + for (List storeFileList : generator) { + List currentFiles = new ArrayList<>(18); + for (HStoreFile file : storeFileList) { currentFiles.add(file); currentFiles = runIteration(currentFiles); } @@ -172,21 +167,20 @@ public class PerfTestCompactionPolicies extends MockStoreFileGenerator { } - private List runIteration(List startingStoreFiles) throws IOException { - - List storeFiles = new ArrayList<>(startingStoreFiles); + private List runIteration(List startingStoreFiles) throws IOException { + List storeFiles = new ArrayList<>(startingStoreFiles); CompactionRequest req = cp.selectCompaction( storeFiles, new ArrayList<>(), false, false, false); long newFileSize = 0; - Collection filesToCompact = req.getFiles(); + Collection filesToCompact = req.getFiles(); if (!filesToCompact.isEmpty()) { storeFiles = new ArrayList<>(storeFiles); storeFiles.removeAll(filesToCompact); - for (StoreFile storeFile : filesToCompact) { + for (HStoreFile storeFile : filesToCompact) { newFileSize += storeFile.getReader().length(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java index 5fe47f3444f..c89f136c86f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SemiConstantSizeFileListGenerator.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.compactions; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; class SemiConstantSizeFileListGenerator extends StoreFileListGenerator { SemiConstantSizeFileListGenerator() { @@ -30,8 +29,8 @@ class SemiConstantSizeFileListGenerator extends StoreFileListGenerator { } @Override - public Iterator> iterator() { - return new Iterator>() { + public Iterator> iterator() { + return new Iterator>() { private int count = 0; @Override @@ -40,9 +39,9 @@ class SemiConstantSizeFileListGenerator extends StoreFileListGenerator { } @Override - public List next() { + public List next() { count += 1; - ArrayList files = new ArrayList<>(NUM_FILES_GEN); + ArrayList files = new ArrayList<>(NUM_FILES_GEN); for (int i = 0; i < NUM_FILES_GEN; i++) { files.add(createMockStoreFile(random.nextInt(5) + 30)); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java index f5f36ac84d8..d270da74d27 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SinusoidalFileListGenerator.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.compactions; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; class SinusoidalFileListGenerator extends StoreFileListGenerator { SinusoidalFileListGenerator() { @@ -30,10 +29,8 @@ class SinusoidalFileListGenerator extends StoreFileListGenerator { } @Override - public Iterator> iterator() { - - - return new Iterator>() { + public Iterator> iterator() { + return new Iterator>() { private int count = 0; @Override public boolean hasNext() { @@ -41,9 +38,9 @@ class SinusoidalFileListGenerator extends StoreFileListGenerator { } @Override - public List next() { + public List next() { count += 1; - ArrayList files = new ArrayList<>(NUM_FILES_GEN); + ArrayList files = new ArrayList<>(NUM_FILES_GEN); for (int x = 0; x < NUM_FILES_GEN; x++) { int fileSize = (int) Math.abs(64 * Math.sin((Math.PI * x) / 50.0)) + 1; files.add(createMockStoreFile(fileSize)); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java index 5201eb790ed..bed1342a03a 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/SpikyFileListGenerator.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.compactions; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; class SpikyFileListGenerator extends StoreFileListGenerator { @@ -31,8 +30,8 @@ class SpikyFileListGenerator extends StoreFileListGenerator { } @Override - public Iterator> iterator() { - return new Iterator>() { + public Iterator> iterator() { + return new Iterator>() { private int count = 0; @Override @@ -41,9 +40,9 @@ class SpikyFileListGenerator extends StoreFileListGenerator { } @Override - public List next() { + public List next() { count += 1; - ArrayList files = new ArrayList<>(NUM_FILES_GEN); + ArrayList files = new ArrayList<>(NUM_FILES_GEN); for (int x = 0; x < NUM_FILES_GEN; x++) { int fileSize = random.nextInt(5) + 1; if ( x % 10 == 0) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java index 643f7714fd2..88546766fa7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/StoreFileListGenerator.java @@ -15,20 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.regionserver.compactions; import java.util.List; -import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.HStoreFile; -public abstract class StoreFileListGenerator - extends MockStoreFileGenerator implements Iterable> { +public abstract class StoreFileListGenerator extends MockStoreFileGenerator + implements Iterable> { public static final int MAX_FILE_GEN_ITERS = 10; public static final int NUM_FILES_GEN = 1000; - StoreFileListGenerator(final Class klass) { + StoreFileListGenerator(Class klass) { super(klass); } } 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 a71e76607e1..e30383e3414 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 @@ -47,8 +47,6 @@ 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; -import org.apache.hadoop.hbase.regionserver.Store; -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.Bytes; @@ -60,7 +58,7 @@ import org.junit.experimental.categories.Category; @Category({ MediumTests.class, RegionServerTests.class }) public class TestCompactedHFilesDischarger { private final HBaseTestingUtility testUtil = new HBaseTestingUtility(); - private Region region; + private HRegion region; private final static byte[] fam = Bytes.toBytes("cf_1"); private final static byte[] qual1 = Bytes.toBytes("qf_1"); private final static byte[] val = Bytes.toBytes("val"); @@ -120,21 +118,21 @@ public class TestCompactedHFilesDischarger { // flush them region.flush(true); - Store store = region.getStore(fam); + HStore store = region.getStore(fam); assertEquals(3, store.getStorefilesCount()); - Collection storefiles = store.getStorefiles(); - Collection compactedfiles = - ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); + Collection storefiles = store.getStorefiles(); + Collection compactedfiles = + store.getStoreEngine().getStoreFileManager().getCompactedfiles(); // None of the files should be in compacted state. - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { assertFalse(file.isCompactedAway()); } // Try to run the cleaner without compaction. there should not be any change cleaner.chore(); storefiles = store.getStorefiles(); // None of the files should be in compacted state. - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { assertFalse(file.isCompactedAway()); } // now do some compaction @@ -149,7 +147,7 @@ public class TestCompactedHFilesDischarger { cleaner.chore(); assertEquals(1, store.getStorefilesCount()); storefiles = store.getStorefiles(); - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { // Should not be in compacted state assertFalse(file.isCompactedAway()); } @@ -186,14 +184,14 @@ public class TestCompactedHFilesDischarger { // flush them region.flush(true); - Store store = region.getStore(fam); + HStore store = region.getStore(fam); assertEquals(3, store.getStorefilesCount()); - Collection storefiles = store.getStorefiles(); - Collection compactedfiles = - ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); + Collection storefiles = store.getStorefiles(); + Collection compactedfiles = + store.getStoreEngine().getStoreFileManager().getCompactedfiles(); // None of the files should be in compacted state. - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { assertFalse(file.isCompactedAway()); } // Do compaction @@ -203,13 +201,13 @@ public class TestCompactedHFilesDischarger { storefiles = store.getStorefiles(); int usedReaderCount = 0; int unusedReaderCount = 0; - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { if (((HStoreFile) file).getRefCount() == 3) { usedReaderCount++; } } compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); - for(StoreFile file : compactedfiles) { + for(HStoreFile file : compactedfiles) { assertEquals("Refcount should be 3", 0, ((HStoreFile) file).getRefCount()); unusedReaderCount++; } @@ -221,7 +219,7 @@ public class TestCompactedHFilesDischarger { countDown(); assertEquals(1, store.getStorefilesCount()); storefiles = store.getStorefiles(); - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { // Should not be in compacted state assertFalse(file.isCompactedAway()); } @@ -257,14 +255,14 @@ public class TestCompactedHFilesDischarger { // flush them region.flush(true); - Store store = region.getStore(fam); + HStore store = region.getStore(fam); assertEquals(3, store.getStorefilesCount()); - Collection storefiles = store.getStorefiles(); - Collection compactedfiles = - ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); + Collection storefiles = store.getStorefiles(); + Collection compactedfiles = + store.getStoreEngine().getStoreFileManager().getCompactedfiles(); // None of the files should be in compacted state. - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { assertFalse(file.isCompactedAway()); } startScannerThreads(); @@ -274,14 +272,13 @@ public class TestCompactedHFilesDischarger { storefiles = store.getStorefiles(); int usedReaderCount = 0; int unusedReaderCount = 0; - for (StoreFile file : storefiles) { - if (((HStoreFile) file).getRefCount() == 0) { + for (HStoreFile file : storefiles) { + if (file.getRefCount() == 0) { unusedReaderCount++; } } - compactedfiles = - ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); - for(StoreFile file : compactedfiles) { + compactedfiles = store.getStoreEngine().getStoreFileManager().getCompactedfiles(); + for(HStoreFile file : compactedfiles) { assertEquals("Refcount should be 3", 3, ((HStoreFile) file).getRefCount()); usedReaderCount++; } @@ -307,14 +304,14 @@ public class TestCompactedHFilesDischarger { storefiles = store.getStorefiles(); usedReaderCount = 0; unusedReaderCount = 0; - for (StoreFile file : storefiles) { - if (((HStoreFile) file).getRefCount() == 3) { + for (HStoreFile file : storefiles) { + if (file.getRefCount() == 3) { usedReaderCount++; } } compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles(); - for(StoreFile file : compactedfiles) { - assertEquals("Refcount should be 0", 0, ((HStoreFile) file).getRefCount()); + for (HStoreFile file : compactedfiles) { + assertEquals("Refcount should be 0", 0, file.getRefCount()); unusedReaderCount++; } // Though there are files we are not using them for reads @@ -329,7 +326,7 @@ public class TestCompactedHFilesDischarger { // Now the cleaner should be able to clear it up because there are no active readers assertEquals(1, store.getStorefilesCount()); storefiles = store.getStorefiles(); - for (StoreFile file : storefiles) { + for (HStoreFile file : storefiles) { // Should not be in compacted state assertFalse(file.isCompactedAway()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java index 170fba250e9..eb4801a8261 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCompactor.java @@ -43,9 +43,9 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.regionserver.BloomType; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileReader; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StoreFileWriter; @@ -56,10 +56,10 @@ import org.mockito.stubbing.Answer; public class TestCompactor { - public static StoreFile createDummyStoreFile(long maxSequenceId) throws Exception { + public static HStoreFile createDummyStoreFile(long maxSequenceId) throws Exception { // "Files" are totally unused, it's Scanner class below that gives compactor fake KVs. // But compaction depends on everything under the sun, so stub everything with dummies. - StoreFile sf = mock(StoreFile.class); + HStoreFile sf = mock(HStoreFile.class); StoreFileReader r = mock(StoreFileReader.class); when(r.length()).thenReturn(1L); when(r.getBloomFilterType()).thenReturn(BloomType.NONE); 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 ca7c0ae7584..ef0c93185c2 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 @@ -42,11 +42,11 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; 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; @@ -87,14 +87,14 @@ public class TestDateTieredCompactor { public boolean usePrivateReaders; private DateTieredCompactor createCompactor(StoreFileWritersCapture writers, - final KeyValue[] input, List storefiles) throws Exception { + final KeyValue[] input, List storefiles) throws Exception { Configuration conf = HBaseConfiguration.create(); conf.setBoolean("hbase.regionserver.compaction.private.readers", usePrivateReaders); final Scanner scanner = new Scanner(input); // Create store mock that is satisfactory for compactor. HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS); ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparator.COMPARATOR); - final Store store = mock(Store.class); + HStore store = mock(HStore.class); when(store.getStorefiles()).thenReturn(storefiles); when(store.getColumnFamilyDescriptor()).thenReturn(col); when(store.getScanInfo()).thenReturn(si); @@ -109,14 +109,14 @@ public class TestDateTieredCompactor { return new DateTieredCompactor(conf, store) { @Override - protected InternalScanner createScanner(Store store, List scanners, + protected InternalScanner createScanner(HStore store, List scanners, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException { return scanner; } @Override - protected InternalScanner createScanner(Store store, List scanners, + protected InternalScanner createScanner(HStore store, List scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException { return scanner; } @@ -126,8 +126,8 @@ public class TestDateTieredCompactor { private void verify(KeyValue[] input, List boundaries, KeyValue[][] output, boolean allFiles) throws Exception { StoreFileWritersCapture writers = new StoreFileWritersCapture(); - StoreFile sf1 = createDummyStoreFile(1L); - StoreFile sf2 = createDummyStoreFile(2L); + HStoreFile sf1 = createDummyStoreFile(1L); + HStoreFile sf2 = createDummyStoreFile(2L); DateTieredCompactor dtc = createCompactor(writers, input, Arrays.asList(sf1, sf2)); List paths = dtc.compact(new CompactionRequest(Arrays.asList(sf1)), boundaries.subList(0, boundaries.size() - 1), NoLimitThroughputController.INSTANCE, null); 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 49c3cba120e..1249fee6ced 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,9 +36,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -58,12 +55,12 @@ import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.io.compress.Compression; import org.apache.hadoop.hbase.io.hfile.HFile; 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.InternalScanner; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.ScannerContext; -import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFileReader; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter; @@ -88,6 +85,9 @@ import org.junit.runners.Parameterized.Parameter; import org.junit.runners.Parameterized.Parameters; import org.mockito.ArgumentMatcher; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList; +import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists; + @RunWith(Parameterized.class) @Category({RegionServerTests.class, SmallTests.class}) public class TestStripeCompactionPolicy { @@ -163,14 +163,14 @@ public class TestStripeCompactionPolicy { StripeCompactionPolicy policy = new StripeCompactionPolicy(conf, sci, ssc) { @Override public StripeCompactionRequest selectCompaction(StripeInformationProvider si, - List filesCompacting, boolean isOffpeak) throws IOException { + List filesCompacting, boolean isOffpeak) throws IOException { if (!filesCompacting.isEmpty()) return null; return selectSingleStripeCompaction(si, false, false, isOffpeak); } @Override public boolean needsCompactions( - StripeInformationProvider si, List filesCompacting) { + StripeInformationProvider si, List filesCompacting) { if (!filesCompacting.isEmpty()) return false; return needsSingleStripeCompaction(si); } @@ -202,7 +202,7 @@ public class TestStripeCompactionPolicy { verifySingleStripeCompaction(policy, si, 1, null); // Verify max count is respected. si = createStripesWithSizes(0, 0, new Long[] { 5L }, new Long[] { 5L, 4L, 4L, 4L, 4L }); - List sfs = si.getStripes().get(1).subList(1, 5); + List sfs = si.getStripes().get(1).subList(1, 5); verifyCompaction(policy, si, sfs, null, 1, null, si.getStartRow(1), si.getEndRow(1), true); // Verify ratio is applied. si = createStripesWithSizes(0, 0, new Long[] { 5L }, new Long[] { 50L, 4L, 4L, 4L, 4L }); @@ -222,10 +222,10 @@ public class TestStripeCompactionPolicy { public void testWithReferences() throws Exception { StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create()); StripeCompactor sc = mock(StripeCompactor.class); - StoreFile ref = createFile(); + HStoreFile ref = createFile(); when(ref.isReference()).thenReturn(true); StripeInformationProvider si = mock(StripeInformationProvider.class); - Collection sfs = al(ref, createFile()); + Collection sfs = al(ref, createFile()); when(si.getStorefiles()).thenReturn(sfs); assertTrue(policy.needsCompactions(si, al())); @@ -349,12 +349,12 @@ public class TestStripeCompactionPolicy { edge.setValue(now); EnvironmentEdgeManager.injectEdge(edge); try { - StoreFile expiredFile = createFile(), notExpiredFile = createFile(); + HStoreFile expiredFile = createFile(), notExpiredFile = createFile(); when(expiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl - 1); when(notExpiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl + 1); - List expired = Lists.newArrayList(expiredFile, expiredFile); - List notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile); - List mixed = Lists.newArrayList(expiredFile, notExpiredFile); + List expired = Lists.newArrayList(expiredFile, expiredFile); + List notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile); + List mixed = Lists.newArrayList(expiredFile, notExpiredFile); StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create(), defaultSplitSize, defaultSplitCount, defaultInitialCount, true); @@ -389,11 +389,11 @@ public class TestStripeCompactionPolicy { edge.setValue(now); EnvironmentEdgeManager.injectEdge(edge); try { - StoreFile expiredFile = createFile(), notExpiredFile = createFile(); + HStoreFile expiredFile = createFile(), notExpiredFile = createFile(); when(expiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl - 1); when(notExpiredFile.getReader().getMaxTimestamp()).thenReturn(now - defaultTtl + 1); - List expired = Lists.newArrayList(expiredFile, expiredFile); - List notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile); + List expired = Lists.newArrayList(expiredFile, expiredFile); + List notExpired = Lists.newArrayList(notExpiredFile, notExpiredFile); StripeCompactionPolicy policy = createPolicy(HBaseConfiguration.create(), defaultSplitSize, defaultSplitCount, @@ -414,7 +414,7 @@ public class TestStripeCompactionPolicy { @SuppressWarnings("unchecked") private static StripeCompactionPolicy.StripeInformationProvider createStripesWithFiles( - List... stripeFiles) throws Exception { + List... stripeFiles) throws Exception { return createStripesWithFiles(createBoundaries(stripeFiles.length), Lists.newArrayList(stripeFiles), new ArrayList<>()); } @@ -434,7 +434,7 @@ public class TestStripeCompactionPolicy { verifySingleStripeCompaction(policy, si, 0, false); // Unless there are enough to cause L0 compaction. si = createStripesWithSizes(6, 2, stripes); - ConcatenatedLists sfs = new ConcatenatedLists<>(); + ConcatenatedLists sfs = new ConcatenatedLists<>(); sfs.addSublist(si.getLevel0Files()); sfs.addSublist(si.getStripes().get(0)); verifyCompaction( @@ -447,12 +447,13 @@ public class TestStripeCompactionPolicy { // if all files of stripe aren't selected, delete must not be dropped. stripes = new Long[][] { new Long[] { 100L, 3L, 2L, 2L, 2L }, new Long[] { 6L } }; si = createStripesWithSizes(0, 0, stripes); - List compact_file = new ArrayList<>(); - Iterator iter = si.getStripes().get(0).listIterator(1); + List compactFile = new ArrayList<>(); + Iterator iter = si.getStripes().get(0).listIterator(1); while (iter.hasNext()) { - compact_file.add(iter.next()); + compactFile.add(iter.next()); } - verifyCompaction(policy, si, compact_file, false, 1, null, si.getStartRow(0), si.getEndRow(0), true); + verifyCompaction(policy, si, compactFile, false, 1, null, si.getStartRow(0), si.getEndRow(0), + true); } /********* HELPER METHODS ************/ @@ -472,14 +473,14 @@ public class TestStripeCompactionPolicy { return new StripeCompactionPolicy(conf, sci, ssc); } - private static ArrayList al(StoreFile... sfs) { + private static ArrayList al(HStoreFile... sfs) { return new ArrayList<>(Arrays.asList(sfs)); } private void verifyMergeCompatcion(StripeCompactionPolicy policy, StripeInformationProvider si, int from, int to) throws Exception { StripeCompactionPolicy.StripeCompactionRequest scr = policy.selectCompaction(si, al(), false); - Collection sfs = getAllFiles(si, from, to); + Collection sfs = getAllFiles(si, from, to); verifyCollectionsEqual(sfs, scr.getRequest().getFiles()); // All the Stripes are expired, so the Compactor will not create any Writers. We need to create @@ -538,7 +539,7 @@ public class TestStripeCompactionPolicy { * @param boundaries Expected target stripe boundaries. */ private void verifyCompaction(StripeCompactionPolicy policy, StripeInformationProvider si, - Collection sfs, byte[] dropDeletesFrom, byte[] dropDeletesTo, + Collection sfs, byte[] dropDeletesFrom, byte[] dropDeletesTo, final List boundaries) throws Exception { StripeCompactor sc = mock(StripeCompactor.class); assertTrue(policy.needsCompactions(si, al())); @@ -573,7 +574,7 @@ public class TestStripeCompactionPolicy { * @param righr Right boundary of the compaction. */ private void verifyCompaction(StripeCompactionPolicy policy, StripeInformationProvider si, - Collection sfs, Boolean dropDeletes, Integer count, Long size, + Collection sfs, Boolean dropDeletes, Integer count, Long size, byte[] start, byte[] end, boolean needsCompaction) throws IOException { StripeCompactor sc = mock(StripeCompactor.class); assertTrue(!needsCompaction || policy.needsCompactions(si, al())); @@ -612,15 +613,15 @@ public class TestStripeCompactionPolicy { : (dropDeletes.booleanValue() ? aryEq(value) : isNull(byte[].class)); } - private void verifyCollectionsEqual(Collection sfs, Collection scr) { + private void verifyCollectionsEqual(Collection sfs, Collection scr) { // Dumb. assertEquals(sfs.size(), scr.size()); assertTrue(scr.containsAll(sfs)); } - private static List getAllFiles( + private static List getAllFiles( StripeInformationProvider si, int fromStripe, int toStripe) { - ArrayList expected = new ArrayList<>(); + ArrayList expected = new ArrayList<>(); for (int i = fromStripe; i <= toStripe; ++i) { expected.addAll(si.getStripes().get(i)); } @@ -694,15 +695,15 @@ public class TestStripeCompactionPolicy { private static StripeInformationProvider createStripes(List boundaries, List> stripeSizes, List l0Sizes) throws Exception { - List> stripeFiles = new ArrayList<>(stripeSizes.size()); + List> stripeFiles = new ArrayList<>(stripeSizes.size()); for (List sizes : stripeSizes) { - List sfs = new ArrayList<>(sizes.size()); + List sfs = new ArrayList<>(sizes.size()); for (Long size : sizes) { sfs.add(createFile(size)); } stripeFiles.add(sfs); } - List l0Files = new ArrayList<>(); + List l0Files = new ArrayList<>(); for (Long size : l0Sizes) { l0Files.add(createFile(size)); } @@ -713,8 +714,8 @@ public class TestStripeCompactionPolicy { * This method actually does all the work. */ private static StripeInformationProvider createStripesWithFiles(List boundaries, - List> stripeFiles, List l0Files) throws Exception { - ArrayList> stripes = new ArrayList<>(); + List> stripeFiles, List l0Files) throws Exception { + ArrayList> stripes = new ArrayList<>(); ArrayList boundariesList = new ArrayList<>(); StripeInformationProvider si = mock(StripeInformationProvider.class); if (!stripeFiles.isEmpty()) { @@ -724,7 +725,7 @@ public class TestStripeCompactionPolicy { byte[] startKey = ((i == 0) ? OPEN_KEY : boundaries.get(i - 1)); byte[] endKey = ((i == boundaries.size()) ? OPEN_KEY : boundaries.get(i)); boundariesList.add(endKey); - for (StoreFile sf : stripeFiles.get(i)) { + for (HStoreFile sf : stripeFiles.get(i)) { setFileStripe(sf, startKey, endKey); } stripes.add(ImmutableList.copyOf(stripeFiles.get(i))); @@ -732,7 +733,7 @@ public class TestStripeCompactionPolicy { when(si.getEndRow(eq(i))).thenReturn(endKey); } } - ConcatenatedLists sfs = new ConcatenatedLists<>(); + ConcatenatedLists sfs = new ConcatenatedLists<>(); sfs.addAllSublists(stripes); sfs.addSublist(l0Files); when(si.getStorefiles()).thenReturn(sfs); @@ -743,8 +744,8 @@ public class TestStripeCompactionPolicy { return si; } - private static StoreFile createFile(long size) throws Exception { - StoreFile sf = mock(StoreFile.class); + private static HStoreFile createFile(long size) throws Exception { + HStoreFile sf = mock(HStoreFile.class); when(sf.getPath()).thenReturn(new Path("moo")); StoreFileReader r = mock(StoreFileReader.class); when(r.getEntries()).thenReturn(size); @@ -758,11 +759,11 @@ public class TestStripeCompactionPolicy { return sf; } - private static StoreFile createFile() throws Exception { + private static HStoreFile createFile() throws Exception { return createFile(0); } - private static void setFileStripe(StoreFile sf, byte[] startKey, byte[] endKey) { + private static void setFileStripe(HStoreFile sf, byte[] startKey, byte[] endKey) { when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_START_KEY)).thenReturn(startKey); when(sf.getMetadataValue(StripeStoreFileManager.STRIPE_END_KEY)).thenReturn(endKey); } @@ -770,7 +771,7 @@ public class TestStripeCompactionPolicy { private StripeCompactor createCompactor() throws Exception { HColumnDescriptor col = new HColumnDescriptor(Bytes.toBytes("foo")); StoreFileWritersCapture writers = new StoreFileWritersCapture(); - Store store = mock(Store.class); + HStore store = mock(HStore.class); HRegionInfo info = mock(HRegionInfo.class); when(info.getRegionNameAsString()).thenReturn("testRegion"); when(store.getColumnFamilyDescriptor()).thenReturn(col); @@ -784,14 +785,14 @@ public class TestStripeCompactionPolicy { final Scanner scanner = new Scanner(); return new StripeCompactor(conf, store) { @Override - protected InternalScanner createScanner(Store store, List scanners, + protected InternalScanner createScanner(HStore store, List scanners, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException { return scanner; } @Override - protected InternalScanner createScanner(Store store, List scanners, + protected InternalScanner createScanner(HStore store, List scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException { return scanner; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java index e51a675adeb..bd3a80376f5 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactor.java @@ -42,10 +42,10 @@ import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.io.compress.Compression; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreFileScanner; import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner; import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture; @@ -195,7 +195,7 @@ public class TestStripeCompactor { // Create store mock that is satisfactory for compactor. HColumnDescriptor col = new HColumnDescriptor(NAME_OF_THINGS); ScanInfo si = new ScanInfo(conf, col, Long.MAX_VALUE, 0, CellComparator.COMPARATOR); - Store store = mock(Store.class); + HStore store = mock(HStore.class); when(store.getColumnFamilyDescriptor()).thenReturn(col); when(store.getScanInfo()).thenReturn(si); when(store.areWritesEnabled()).thenReturn(true); @@ -207,14 +207,14 @@ public class TestStripeCompactor { return new StripeCompactor(conf, store) { @Override - protected InternalScanner createScanner(Store store, List scanners, + protected InternalScanner createScanner(HStore store, List scanners, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException { return scanner; } @Override - protected InternalScanner createScanner(Store store, List scanners, + protected InternalScanner createScanner(HStore store, List scanners, ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException { return scanner; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java index d25829d3ba4..391155e19eb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java @@ -51,7 +51,17 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; -import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Put; @@ -60,7 +70,6 @@ import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.monitoring.MonitoredTask; -import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine; import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher; import org.apache.hadoop.hbase.regionserver.FlushRequestListener; @@ -101,6 +110,8 @@ import org.mockito.Mockito; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; + /** * Test replay of edits out of a WAL split. */ @@ -636,7 +647,7 @@ public abstract class AbstractTestWALReplay { // Switch between throw and not throw exception in flush static final AtomicBoolean throwExceptionWhenFlushing = new AtomicBoolean(false); - public CustomStoreFlusher(Configuration conf, Store store) { + public CustomStoreFlusher(Configuration conf, HStore store) { super(conf, store); } @Override 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 2b5c78caab2..9dc13d97cde 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 @@ -62,9 +62,9 @@ import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.Visibil import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.Region; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; @@ -843,10 +843,10 @@ public abstract class TestVisibilityLabels { } TEST_UTIL.getAdmin().flush(tableName); List regions = TEST_UTIL.getHBaseCluster().getRegions(tableName); - Store store = regions.get(0).getStore(fam); - Collection storefiles = store.getStorefiles(); + HStore store = regions.get(0).getStore(fam); + Collection storefiles = store.getStorefiles(); assertTrue(storefiles.size() > 0); - for (StoreFile storeFile : storefiles) { + for (HStoreFile storeFile : storefiles) { assertTrue(storeFile.getReader().getHFileReader().getFileContext().isIncludesTags()); } } 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 0487bf4bd68..acf555b79df 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,7 @@ */ package org.apache.hadoop.hbase.util; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.fail; @@ -46,7 +47,6 @@ 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.mob.MobUtils; -import org.apache.hadoop.hbase.regionserver.StoreFile; /** * Utility class for HFile-related testing. @@ -144,7 +144,7 @@ public class HFileTestUtil { writer.append(kv); } } finally { - writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())); + writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())); writer.close(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java index d4d22b1ea09..14c22bf575d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestCoprocessorScanPolicy.java @@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; +import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.KeyValueScanner; import org.apache.hadoop.hbase.regionserver.ScanInfo; @@ -58,10 +59,9 @@ import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.StoreScanner; import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; -import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; -import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MiscTests; +import org.apache.hadoop.hbase.wal.WALEdit; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -254,7 +254,7 @@ public class TestCoprocessorScanPolicy { family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions, newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(), family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), family.isNewVersionBehavior()); - return new StoreScanner(store, scanInfo, + return new StoreScanner((HStore) store, scanInfo, newVersions == null ? OptionalInt.empty() : OptionalInt.of(newVersions.intValue()), scanners, ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP); @@ -274,7 +274,7 @@ public class TestCoprocessorScanPolicy { newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(), family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), family.isNewVersionBehavior()); - return new StoreScanner(store, scanInfo, + return new StoreScanner((HStore) store, scanInfo, newVersions == null ? OptionalInt.empty() : OptionalInt.of(newVersions.intValue()), scanners, scanType, store.getSmallestReadPoint(), earliestPutTs); } @@ -292,8 +292,9 @@ public class TestCoprocessorScanPolicy { ScanInfo scanInfo = new ScanInfo(TEST_UTIL.getConfiguration(), family.getName(), family.getMinVersions(), newVersions == null ? family.getMaxVersions() : newVersions, newTtl == null ? oldSI.getTtl() : newTtl, family.getKeepDeletedCells(), - family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), family.isNewVersionBehavior()); - return new StoreScanner(store, scanInfo, scan, targetCols, readPt); + family.getBlocksize(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator(), + family.isNewVersionBehavior()); + return new StoreScanner((HStore) store, scanInfo, scan, targetCols, readPt); } else { return s; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java index b9efe5e9ee4..a2b9c1aaed0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsckEncryption.java @@ -42,9 +42,10 @@ import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.io.crypto.aes.AES; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; +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.Store; -import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.security.EncryptionUtil; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.testclassification.LargeTests; @@ -140,10 +141,10 @@ public class TestHBaseFsckEncryption { private List findStorefilePaths(TableName tableName) throws Exception { List paths = new ArrayList<>(); - for (Region region: - TEST_UTIL.getRSForFirstRegionInTable(tableName).getOnlineRegions(htd.getTableName())) { - for (Store store: region.getStores()) { - for (StoreFile storefile: store.getStorefiles()) { + for (Region region : TEST_UTIL.getRSForFirstRegionInTable(tableName) + .getOnlineRegions(htd.getTableName())) { + for (HStore store : ((HRegion) region).getStores()) { + for (HStoreFile storefile : store.getStorefiles()) { paths.add(storefile.getPath()); } } diff --git a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala index 07f34b55b05..d7e3f4f6ee0 100644 --- a/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala +++ b/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.io.compress.Compression import org.apache.hadoop.hbase.io.compress.Compression.Algorithm import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding import org.apache.hadoop.hbase.io.hfile.{HFile, CacheConfig, HFileContextBuilder, HFileWriterImpl} -import org.apache.hadoop.hbase.regionserver.{HStore, StoreFile, StoreFileWriter, BloomType} +import org.apache.hadoop.hbase.regionserver.{HStore, HStoreFile, StoreFileWriter, BloomType} import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.mapred.JobConf import org.apache.spark.broadcast.Broadcast @@ -1075,13 +1075,13 @@ class HBaseContext(@transient sc: SparkContext, previousRow: Array[Byte], compactionExclude: Boolean): Unit = { if (w != null) { - w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, + w.appendFileInfo(HStoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())) - w.appendFileInfo(StoreFile.BULKLOAD_TASK_KEY, + w.appendFileInfo(HStoreFile.BULKLOAD_TASK_KEY, Bytes.toBytes(regionSplitPartitioner.getPartition(previousRow))) - w.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY, + w.appendFileInfo(HStoreFile.MAJOR_COMPACTION_KEY, Bytes.toBytes(true)) - w.appendFileInfo(StoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY, + w.appendFileInfo(HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY, Bytes.toBytes(compactionExclude)) w.appendTrackedTimestampsToMetadata() w.close()