From 5f20c0fcdf61c91ca247aad981029afbc55360a9 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Tue, 5 Nov 2024 13:50:42 +0000 Subject: [PATCH 01/11] HBASE-29412 Extend date tiered compaction to allow for tiering by values other than cell timestamp Change-Id: I9574d01c5cb88d1ba35db3b0970d2d710207fdf4 --- .../CustomCellTieredStoreEngine.java | 53 ++++++++++++ .../regionserver/DateTieredStoreEngine.java | 17 +++- .../regionserver/compactions/Compactor.java | 5 ++ .../CustomCellDateTieredCompactionPolicy.java | 84 +++++++++++++++++++ .../CustomCellTieredCompactor.java | 27 ++++++ .../DateTieredCompactionPolicy.java | 2 +- 6 files changed, 186 insertions(+), 2 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java new file mode 100644 index 000000000000..fb940f41155c --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactor; +import org.apache.yetus.audience.InterfaceAudience; +import java.io.IOException; +import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY; + +/** + * Extension of {@link DateTieredStoreEngine} that uses a pluggable value provider for + * extracting the value to be used for comparison in this tiered compaction. + * + * Differently from the existing Date Tiered Compaction, this doesn't yield multiple tiers + * or files, but rather provides two tiers based on a configurable “cut-off” age. + * All rows with the cell tiering value older than this “cut-off” age would be placed together + * in an “old” tier, whilst younger rows would go to a separate, “young” tier file. + */ +@InterfaceAudience.Private +public class CustomCellTieredStoreEngine extends DateTieredStoreEngine { + + @Override + protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator) + throws IOException { + conf = new Configuration(conf); + conf.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, + CustomCellDateTieredCompactionPolicy.class.getName()); + createCompactionPolicy(conf, store); + this.storeFileManager = new DefaultStoreFileManager(kvComparator, + StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, conf, compactionPolicy.getConf()); + this.storeFlusher = new DefaultStoreFlusher(conf, store); + this.compactor = new CustomCellTieredCompactor(conf, 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 26437ab11242..88eb59f69e80 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 @@ -29,7 +29,9 @@ import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor; 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; +import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY; /** * HBASE-15400 This store engine allows us to store data in date tiered layout with exponential @@ -44,6 +46,19 @@ public class DateTieredStoreEngine extends StoreEngine filesCompacting) { return compactionPolicy.needsCompaction(storeFileManager.getStoreFiles(), filesCompacting); @@ -57,7 +72,7 @@ public CompactionContext createCompaction() throws IOException { @Override protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator) throws IOException { - this.compactionPolicy = new DateTieredCompactionPolicy(conf, store); + createCompactionPolicy(conf, store); this.storeFileManager = new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, conf, compactionPolicy.getConf()); this.storeFlusher = new DefaultStoreFlusher(conf, store); 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 055ad85e5a39..95a123c0a86e 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 @@ -403,6 +403,10 @@ protected abstract List commitWriter(T writer, FileDetails fd, protected abstract void abortWriter(T writer) throws IOException; + protected void decorateCells(List cells) { + //no op + } + /** * Performs the compaction. * @param fd FileDetails of cell sink writer @@ -459,6 +463,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel // output to writer: Cell lastCleanCell = null; long lastCleanCellSeqId = 0; + decorateCells(cells); for (ExtendedCell c : cells) { if (cleanSeqId && c.getSequenceId() <= smallestReadPoint) { lastCleanCell = c; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java new file mode 100644 index 000000000000..6421147c6c09 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java @@ -0,0 +1,84 @@ +package org.apache.hadoop.hbase.regionserver.compactions; + +import org.apache.commons.lang3.mutable.MutableLong; +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.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.yetus.audience.InterfaceAudience; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * Custom implementation of DateTieredCompactionPolicy that calculates compaction boundaries based + * on the hbase.hstore.compaction.date.tiered.custom.age.limit.millis configuration property + * and the TIERING_CELL_MIN/TIERING_CELL_MAX stored on metadata of each store file. + * + * This policy would produce either one or two tiers: + * - One tier if either all files data age are older than the configured age limit or all files + * data age are younger than the configured age limit. + * - Two tiers if files have both younger and older data than the configured age limit. + * + */ +@InterfaceAudience.Private +public class CustomCellDateTieredCompactionPolicy extends DateTieredCompactionPolicy { + + public static final String AGE_LIMIT_MILLIS = + "hbase.hstore.compaction.date.tiered.custom.age.limit.millis"; + + public static final String TIERING_CELL_MIN = "TIERING_CELL_MIN"; + + public static final String TIERING_CELL_MAX = "TIERING_CELL_MAX"; + + private long cutOffTimestamp; + + public CustomCellDateTieredCompactionPolicy(Configuration conf, + StoreConfigInformation storeConfigInfo) throws IOException { + super(conf, storeConfigInfo); + cutOffTimestamp = EnvironmentEdgeManager.currentTime() - + conf.getLong(AGE_LIMIT_MILLIS, (long) (10*365.25*24*60*60*1000)); + + } + + @Override + protected List getCompactBoundariesForMajor(Collection filesToCompact, long now) { + MutableLong min = new MutableLong(Long.MAX_VALUE); + MutableLong max = new MutableLong(0); + filesToCompact.forEach(f -> { + byte[] fileMin = f.getMetadataValue(Bytes.toBytes(TIERING_CELL_MIN)); + byte[] fileMax = f.getMetadataValue(Bytes.toBytes(TIERING_CELL_MAX)); + if (fileMin != null) { + long minCurrent = Bytes.toLong(fileMin); + if(min.getValue() < minCurrent) { + min.setValue(minCurrent); + } + } else { + min.setValue(0); + } + if (fileMax != null) { + long maxCurrent = Bytes.toLong(fileMax); + if(max.getValue() > maxCurrent) { + max.setValue(maxCurrent); + } + } else { + max.setValue(Long.MAX_VALUE); + } + }); + + List boundaries = new ArrayList<>(); + if (min.getValue() < cutOffTimestamp) { + boundaries.add(min.getValue()); + if (max.getValue() > cutOffTimestamp) { + boundaries.add(cutOffTimestamp); + } + } + boundaries.add(Long.MIN_VALUE); + Collections.reverse(boundaries); + return boundaries; + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java new file mode 100644 index 000000000000..8e1afee52e47 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java @@ -0,0 +1,27 @@ +package org.apache.hadoop.hbase.regionserver.compactions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.yetus.audience.InterfaceAudience; +import java.util.List; + +/** + * An extension of DateTieredCompactor, overriding the decorateCells method to allow for custom + * values to be used for the different file tiers during compaction. + */ +@InterfaceAudience.Private +public class CustomCellTieredCompactor extends DateTieredCompactor { + public CustomCellTieredCompactor(Configuration conf, HStore store) { + super(conf, store); + } + + @Override + protected void decorateCells(List cells) { + //TODO + } + +} 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 9dbe9aae9cf2..a4de078f6858 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 @@ -296,7 +296,7 @@ private DateTieredCompactionRequest generateCompactionRequest(ArrayList getCompactBoundariesForMajor(Collection filesToCompact, long now) { + protected List getCompactBoundariesForMajor(Collection filesToCompact, long now) { long minTimestamp = filesToCompact.stream() .mapToLong(f -> f.getMinimumTimestamp().orElse(Long.MAX_VALUE)).min().orElse(Long.MAX_VALUE); From edddf3ef465211f838a89851c515289b7db596aa Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Fri, 8 Nov 2024 10:07:00 +0000 Subject: [PATCH 02/11] HBASE-29413 Implement a custom qualifier tiered compaction Change-Id: I6eff2199afbc8c02426b07b696c18af258bcb36c --- .../java/org/apache/hadoop/hbase/TagType.java | 2 + .../hbase/io/hfile/HFileWriterImpl.java | 12 +- .../CustomCellTieredStoreEngine.java | 9 ++ .../CustomTieringMultiFileWriter.java | 73 +++++++++ .../DateTieredMultiFileWriter.java | 21 ++- .../regionserver/compactions/Compactor.java | 5 +- .../CustomCellDateTieredCompactionPolicy.java | 67 +++++--- .../CustomCellTieredCompactor.java | 80 +++++++++- .../compactions/DateTieredCompactor.java | 16 +- .../TestCustomCellTieredCompactor.java | 150 ++++++++++++++++++ 10 files changed, 396 insertions(+), 39 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java index eb9a7f3eccc9..811710991eb5 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java @@ -36,4 +36,6 @@ public final class TagType { // String based tag type used in replication public static final byte STRING_VIS_TAG_TYPE = (byte) 7; public static final byte TTL_TAG_TYPE = (byte) 8; + //tag with the custom cell tiering value for the row + public static final byte CELL_VALUE_TIERING_TAG_TYPE = (byte) 9; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 96bfe42f1fda..8d58032799d8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -29,6 +29,8 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.function.Function; +import java.util.function.Supplier; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -127,6 +129,12 @@ public class HFileWriterImpl implements HFile.Writer { /** Cache configuration for caching data on write. */ protected final CacheConfig cacheConf; + public void setTimeRangeToTrack(Supplier timeRangeToTrack) { + this.timeRangeToTrack = timeRangeToTrack; + } + + private Supplier timeRangeToTrack; + /** * Name for this object used when logging or in toString. Is either the result of a toString on * stream or else name of passed file Path. @@ -186,7 +194,9 @@ public HFileWriterImpl(final Configuration conf, CacheConfig cacheConf, Path pat this.path = path; this.name = path != null ? path.getName() : outputStream.toString(); this.hFileContext = fileContext; + //TODO: Move this back to upper layer this.timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); + this.timeRangeToTrack = () -> this.timeRangeTracker; DataBlockEncoding encoding = hFileContext.getDataBlockEncoding(); if (encoding != DataBlockEncoding.NONE) { this.blockEncoder = new HFileDataBlockEncoderImpl(encoding); @@ -588,7 +598,7 @@ private BlockCacheKey buildCacheBlockKey(long offset, BlockType blockType) { } private boolean shouldCacheBlock(BlockCache cache, BlockCacheKey key) { - Optional result = cache.shouldCacheBlock(key, timeRangeTracker, conf); + Optional result = cache.shouldCacheBlock(key, timeRangeToTrack.get(), conf); return result.orElse(true); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java index fb940f41155c..4f061736612b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java @@ -19,11 +19,19 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine; +import org.apache.hadoop.hbase.regionserver.DefaultStoreFileManager; +import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.StoreFileComparators; import org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactor; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor; +import org.apache.yetus.audience.InterfaceAudience; import java.io.IOException; import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy.TIERING_CELL_QUALIFIER; /** * Extension of {@link DateTieredStoreEngine} that uses a pluggable value provider for @@ -41,6 +49,7 @@ public class CustomCellTieredStoreEngine extends DateTieredStoreEngine { protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator) throws IOException { conf = new Configuration(conf); + conf.set(TIERING_CELL_QUALIFIER, store.conf.get(TIERING_CELL_QUALIFIER)); conf.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, CustomCellDateTieredCompactionPolicy.class.getName()); createCompactionPolicy(conf, store); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java new file mode 100644 index 000000000000..a09c45f837ea --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java @@ -0,0 +1,73 @@ +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.function.Function; + +@InterfaceAudience.Private +public class CustomTieringMultiFileWriter extends DateTieredMultiFileWriter { + + public static final byte[] TIERING_CELL_TIME_RANGE = + Bytes.toBytes("TIERING_CELL_TIME_RANGE"); + + private NavigableMap lowerBoundary2TimeRanger = new TreeMap<>(); + + public CustomTieringMultiFileWriter(List lowerBoundaries, + Map lowerBoundariesPolicies, boolean needEmptyFile, + Function tieringFunction) { + super(lowerBoundaries, lowerBoundariesPolicies, needEmptyFile, tieringFunction); + for (Long lowerBoundary : lowerBoundaries) { + lowerBoundary2TimeRanger.put(lowerBoundary, null); + } + } + + @Override + public void append(ExtendedCell cell) throws IOException { + super.append(cell); + long tieringValue = tieringFunction.apply(cell); + Map.Entry entry = + lowerBoundary2TimeRanger.floorEntry(tieringValue); + if(entry.getValue()==null) { + TimeRangeTracker timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); + timeRangeTracker.setMin(tieringValue); + timeRangeTracker.setMax(tieringValue); + lowerBoundary2TimeRanger.put(entry.getKey(), timeRangeTracker); + ((HFileWriterImpl)lowerBoundary2Writer.get(entry.getKey()).getLiveFileWriter()) + .setTimeRangeToTrack(()->timeRangeTracker); + } else { + TimeRangeTracker timeRangeTracker = entry.getValue(); + if(timeRangeTracker.getMin() > tieringValue) { + timeRangeTracker.setMin(tieringValue); + } + if(timeRangeTracker.getMax() < tieringValue) { + timeRangeTracker.setMax(tieringValue); + } + } + } + + @Override + public List commitWriters(long maxSeqId, boolean majorCompaction, + Collection storeFiles) throws IOException { + for(Map.Entry entry : this.lowerBoundary2Writer.entrySet()){ + StoreFileWriter writer = entry.getValue(); + if(writer!=null) { + writer.appendFileInfo(TIERING_CELL_TIME_RANGE, + TimeRangeTracker.toByteArray(lowerBoundary2TimeRanger.get(entry.getKey()))); + } + } + return super.commitWriters(maxSeqId, majorCompaction, storeFiles); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java index b800178e8a28..5796d7c890e8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java @@ -24,6 +24,8 @@ import java.util.NavigableMap; import java.util.TreeMap; import org.apache.hadoop.hbase.ExtendedCell; +import java.util.function.Function; +import org.apache.hadoop.hbase.Cell; import org.apache.yetus.audience.InterfaceAudience; /** @@ -33,12 +35,14 @@ @InterfaceAudience.Private public class DateTieredMultiFileWriter extends AbstractMultiFileWriter { - private final NavigableMap lowerBoundary2Writer = new TreeMap<>(); + protected final NavigableMap lowerBoundary2Writer = new TreeMap<>(); private final boolean needEmptyFile; private final Map lowerBoundariesPolicies; + protected Function tieringFunction; + /** * @param lowerBoundariesPolicies each window to storage policy map. * @param needEmptyFile whether need to create an empty store file if we haven't written @@ -46,16 +50,29 @@ public class DateTieredMultiFileWriter extends AbstractMultiFileWriter { */ public DateTieredMultiFileWriter(List lowerBoundaries, Map lowerBoundariesPolicies, boolean needEmptyFile) { + this(lowerBoundaries, lowerBoundariesPolicies, needEmptyFile, c -> c.getTimestamp()); + } + + /** + * @param lowerBoundariesPolicies each window to storage policy map. + * @param needEmptyFile whether need to create an empty store file if we haven't written + * out anything. + */ + public DateTieredMultiFileWriter(List lowerBoundaries, + Map lowerBoundariesPolicies, boolean needEmptyFile, + Function tieringFunction) { for (Long lowerBoundary : lowerBoundaries) { lowerBoundary2Writer.put(lowerBoundary, null); } this.needEmptyFile = needEmptyFile; this.lowerBoundariesPolicies = lowerBoundariesPolicies; + this.tieringFunction = tieringFunction; } @Override public void append(ExtendedCell cell) throws IOException { - Map.Entry entry = lowerBoundary2Writer.floorEntry(cell.getTimestamp()); + Map.Entry entry = + lowerBoundary2Writer.floorEntry(tieringFunction.apply(cell)); StoreFileWriter writer = entry.getValue(); if (writer == null) { String lowerBoundaryStoragePolicy = lowerBoundariesPolicies.get(entry.getKey()); 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 95a123c0a86e..2e679e1a84dc 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 @@ -403,8 +403,9 @@ protected abstract List commitWriter(T writer, FileDetails fd, protected abstract void abortWriter(T writer) throws IOException; - protected void decorateCells(List cells) { + protected List decorateCells(List cells) { //no op + return cells; } /** @@ -463,7 +464,7 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel // output to writer: Cell lastCleanCell = null; long lastCleanCellSeqId = 0; - decorateCells(cells); + cells = decorateCells(cells); for (ExtendedCell c : cells) { if (cleanSeqId && c.getSequenceId() <= smallestReadPoint) { lastCleanCell = c; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java index 6421147c6c09..3a5a0834e875 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java @@ -1,17 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hbase.regionserver.compactions; import org.apache.commons.lang3.mutable.MutableLong; 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.util.Bytes; +import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.List; +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; /** * Custom implementation of DateTieredCompactionPolicy that calculates compaction boundaries based @@ -30,9 +47,7 @@ public class CustomCellDateTieredCompactionPolicy extends DateTieredCompactionPo public static final String AGE_LIMIT_MILLIS = "hbase.hstore.compaction.date.tiered.custom.age.limit.millis"; - public static final String TIERING_CELL_MIN = "TIERING_CELL_MIN"; - - public static final String TIERING_CELL_MAX = "TIERING_CELL_MAX"; + public static final String TIERING_CELL_QUALIFIER = "TIERING_CELL_QUALIFIER"; private long cutOffTimestamp; @@ -40,7 +55,7 @@ public CustomCellDateTieredCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) throws IOException { super(conf, storeConfigInfo); cutOffTimestamp = EnvironmentEdgeManager.currentTime() - - conf.getLong(AGE_LIMIT_MILLIS, (long) (10*365.25*24*60*60*1000)); + conf.getLong(AGE_LIMIT_MILLIS, (long) (10L*365.25*24L*60L*60L*1000L)); } @@ -49,35 +64,35 @@ protected List getCompactBoundariesForMajor(Collection filesTo MutableLong min = new MutableLong(Long.MAX_VALUE); MutableLong max = new MutableLong(0); filesToCompact.forEach(f -> { - byte[] fileMin = f.getMetadataValue(Bytes.toBytes(TIERING_CELL_MIN)); - byte[] fileMax = f.getMetadataValue(Bytes.toBytes(TIERING_CELL_MAX)); - if (fileMin != null) { - long minCurrent = Bytes.toLong(fileMin); - if(min.getValue() < minCurrent) { - min.setValue(minCurrent); - } - } else { - min.setValue(0); - } - if (fileMax != null) { - long maxCurrent = Bytes.toLong(fileMax); - if(max.getValue() > maxCurrent) { - max.setValue(maxCurrent); - } - } else { - max.setValue(Long.MAX_VALUE); + byte[] timeRangeBytes = f.getMetadataValue(TIERING_CELL_TIME_RANGE); + long minCurrent = Long.MAX_VALUE; + long maxCurrent = 0; + if(timeRangeBytes!=null) { + try { + TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(timeRangeBytes); + timeRangeTracker.getMin(); + minCurrent = timeRangeTracker.getMin(); + maxCurrent = timeRangeTracker.getMax(); + } catch (IOException e) { + //TODO debug this } - }); + } + if(minCurrent < min.getValue()) { + min.setValue(minCurrent); + } + if(maxCurrent > max.getValue()) { + max.setValue(maxCurrent); + } + }); List boundaries = new ArrayList<>(); + boundaries.add(Long.MIN_VALUE); if (min.getValue() < cutOffTimestamp) { boundaries.add(min.getValue()); if (max.getValue() > cutOffTimestamp) { boundaries.add(cutOffTimestamp); } } - boundaries.add(Long.MIN_VALUE); - Collections.reverse(boundaries); return boundaries; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java index 8e1afee52e47..21b98b5611b2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java @@ -1,13 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hbase.regionserver.compactions; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.ExtendedCell; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.PrivateCellUtil; +import org.apache.hadoop.hbase.Tag; +import org.apache.hadoop.hbase.TagType; +import org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter; +import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter; import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; +import java.util.Map; +import java.util.Optional; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy.TIERING_CELL_QUALIFIER; /** * An extension of DateTieredCompactor, overriding the decorateCells method to allow for custom @@ -15,13 +42,60 @@ */ @InterfaceAudience.Private public class CustomCellTieredCompactor extends DateTieredCompactor { + + private byte[] tieringQualifier; + public CustomCellTieredCompactor(Configuration conf, HStore store) { super(conf, store); + tieringQualifier = Bytes.toBytes(conf.get(TIERING_CELL_QUALIFIER)); + } + + @Override + protected List decorateCells(List cells) { + //if no tiering qualifier properly set, skips the whole flow + if(tieringQualifier!=null) { + byte[] tieringValue = null; + //first iterates through the cells within a row, to find the tiering value for the row + for (Cell cell : cells) { + byte[] qualifier = new byte[cell.getQualifierLength()]; + System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), qualifier, 0, cell.getQualifierLength()); + if (Arrays.equals(qualifier, tieringQualifier)) { + tieringValue = new byte[cell.getValueLength()]; + System.arraycopy(cell.getValueArray(), cell.getValueOffset(), tieringValue, 0, cell.getValueLength()); + break; + } + } + if(tieringValue==null){ + tieringValue = Bytes.toBytes(Long.MAX_VALUE); + } + //now apply the tiering value as a tag to all cells within the row + Tag tieringValueTag = new ArrayBackedTag(TagType.CELL_VALUE_TIERING_TAG_TYPE, tieringValue); + List newCells = new ArrayList<>(cells.size()); + for(ExtendedCell cell : cells) { + List tags = PrivateCellUtil.getTags(cell); + tags.add(tieringValueTag); + newCells.add(PrivateCellUtil.createCell(cell, tags)); + } + return newCells; + } else { + return cells; + } + } + + private long getTieringValue(ExtendedCell cell) { + Optional tagOptional = PrivateCellUtil.getTag(cell, TagType.CELL_VALUE_TIERING_TAG_TYPE); + if(tagOptional.isPresent()) { + Tag tag = tagOptional.get(); + return Bytes.toLong(tag.getValueByteBuffer().array(), tag.getValueOffset(), tag.getValueLength()); + } + return Long.MAX_VALUE; } @Override - protected void decorateCells(List cells) { - //TODO + protected DateTieredMultiFileWriter createMultiWriter(final CompactionRequestImpl request, + final List lowerBoundaries, final Map lowerBoundariesPolicies) { + return new CustomTieringMultiFileWriter(lowerBoundaries, lowerBoundariesPolicies, + needEmptyFile(request), CustomCellTieredCompactor.this::getTieringValue); } } 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 b5911b0cec46..5dacf63ab6ce 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,6 +24,7 @@ import java.util.function.Consumer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -46,7 +47,7 @@ public DateTieredCompactor(Configuration conf, HStore store) { super(conf, store); } - private boolean needEmptyFile(CompactionRequestImpl request) { + protected boolean needEmptyFile(CompactionRequestImpl request) { // if we are going to compact the last N files, then we need to emit an empty file to retain the // maxSeqId if we haven't written out anything. OptionalLong maxSeqId = StoreUtils.getMaxSequenceIdInList(request.getFiles()); @@ -68,16 +69,21 @@ public List compact(final CompactionRequestImpl request, final List @Override public DateTieredMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd, - boolean shouldDropBehind, boolean major, Consumer writerCreationTracker) - throws IOException { - DateTieredMultiFileWriter writer = new DateTieredMultiFileWriter(lowerBoundaries, - lowerBoundariesPolicies, needEmptyFile(request)); + boolean shouldDropBehind, boolean major, Consumer writerCreationTracker) throws IOException { + DateTieredMultiFileWriter writer = + createMultiWriter(request, lowerBoundaries, lowerBoundariesPolicies); initMultiWriter(writer, scanner, fd, shouldDropBehind, major, writerCreationTracker); return writer; } }, throughputController, user); } + protected DateTieredMultiFileWriter createMultiWriter(final CompactionRequestImpl request, + final List lowerBoundaries, final Map lowerBoundariesPolicies) { + return new DateTieredMultiFileWriter(lowerBoundaries, + lowerBoundariesPolicies, needEmptyFile(request), c -> c.getTimestamp()); + } + @Override protected List commitWriter(DateTieredMultiFileWriter writer, FileDetails fd, CompactionRequestImpl request) throws IOException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java new file mode 100644 index 000000000000..f76b6857a41c --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.compactions; + +import org.apache.hadoop.hbase.HBaseClassTestRule; + +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.Connection; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Table; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.regionserver.CustomCellTieredStoreEngine; +import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy.TIERING_CELL_QUALIFIER; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestCustomCellTieredCompactor { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCustomCellTieredCompactor.class); + + public static final byte[] FAMILY = Bytes.toBytes("cf"); + + protected HBaseTestingUtil utility; + + protected Admin admin; + + @Before + public void setUp() throws Exception { + utility = new HBaseTestingUtil(); + utility.getConfiguration().setInt("hbase.hfile.compaction.discharger.interval", 10); + utility.startMiniCluster(); + } + + @After + public void tearDown() throws Exception { + utility.shutdownMiniCluster(); + } + + @Test + public void testCustomCellTieredCompactor() throws Exception { + ColumnFamilyDescriptorBuilder clmBuilder = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY); + clmBuilder.setValue("hbase.hstore.engine.class", CustomCellTieredStoreEngine.class.getName()); + clmBuilder.setValue(TIERING_CELL_QUALIFIER, "date"); + TableName tableName = TableName.valueOf("testCustomCellTieredCompactor"); + TableDescriptorBuilder tblBuilder = TableDescriptorBuilder.newBuilder(tableName); + tblBuilder.setColumnFamily(clmBuilder.build()); + utility.getAdmin().createTable(tblBuilder.build()); + utility.waitTableAvailable(tableName); + Connection connection = utility.getConnection(); + Table table = connection.getTable(tableName); + long recordTime = System.currentTimeMillis(); + // write data and flush multiple store files: + for (int i = 0; i < 6; i++) { + List puts = new ArrayList<>(2); + Put put = new Put(Bytes.toBytes(i)); + put.addColumn(FAMILY, Bytes.toBytes("val"), Bytes.toBytes("v" + i)); + put.addColumn(FAMILY, Bytes.toBytes("date"), + Bytes.toBytes(recordTime - (11L*366L*24L*60L*60L*1000L))); + puts.add(put); + put = new Put(Bytes.toBytes(i+1000)); + put.addColumn(FAMILY, Bytes.toBytes("val"), Bytes.toBytes("v" + (i + 1000))); + put.addColumn(FAMILY, Bytes.toBytes("date"), + Bytes.toBytes(recordTime)); + puts.add(put); + table.put(puts); + utility.flush(tableName); + } + table.close(); + long firstCompactionTime = System.currentTimeMillis(); + utility.getAdmin().majorCompact(tableName); + Waiter.waitFor(utility.getConfiguration(), 5000, + () -> utility.getMiniHBaseCluster().getMaster() + .getLastMajorCompactionTimestamp(tableName) > firstCompactionTime); + long numHFiles = utility.getNumHFiles(tableName, FAMILY); + //The first major compaction would have no means to detect more than one tier, + // because without the min/max values available in the file info portion of the selected files + // for compaction, CustomCellDateTieredCompactionPolicy has no means + // to calculate the proper boundaries. + assertEquals(1, numHFiles); + utility.getMiniHBaseCluster().getRegions(tableName).get(0).getStore(FAMILY).getStorefiles().forEach( + file -> { + byte[] rangeBytes = file.getMetadataValue(TIERING_CELL_TIME_RANGE); + assertNotNull(rangeBytes); + try { + TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(rangeBytes); + assertEquals((recordTime - (11L*366L*24L*60L*60L*1000L)), timeRangeTracker.getMin()); + assertEquals(recordTime, timeRangeTracker.getMax()); + } catch (IOException e) { + fail(e.getMessage()); + } + } + ); + //now do major compaction again, to make sure we write two separate files + long secondCompactionTime = System.currentTimeMillis(); + utility.getAdmin().majorCompact(tableName); + Waiter.waitFor(utility.getConfiguration(), 5000, + () -> utility.getMiniHBaseCluster().getMaster() + .getLastMajorCompactionTimestamp(tableName) > secondCompactionTime); + numHFiles = utility.getNumHFiles(tableName, FAMILY); + assertEquals(2, numHFiles); + utility.getMiniHBaseCluster().getRegions(tableName).get(0).getStore(FAMILY).getStorefiles().forEach( + file -> { + byte[] rangeBytes = file.getMetadataValue(TIERING_CELL_TIME_RANGE); + assertNotNull(rangeBytes); + try { + TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(rangeBytes); + assertEquals(timeRangeTracker.getMin(), timeRangeTracker.getMax()); + } catch (IOException e) { + fail(e.getMessage()); + } + } + ); + } +} From 27937664538a092df3444272cab2a7ffb9e67691 Mon Sep 17 00:00:00 2001 From: Janardhan Hungund Date: Wed, 13 Nov 2024 18:20:21 +0530 Subject: [PATCH 03/11] HBASE-29414 Refactor DataTieringManager to make priority logic pluggable Co-authored-by: Wellington Ramos Chevreuil --- .../hadoop/hbase/io/hfile/BlockCache.java | 2 +- .../hbase/io/hfile/CombinedBlockCache.java | 6 +- .../apache/hadoop/hbase/io/hfile/HFile.java | 7 + .../hbase/io/hfile/HFileReaderImpl.java | 2 +- .../hbase/io/hfile/HFileWriterImpl.java | 10 +- .../hbase/io/hfile/bucket/BucketCache.java | 5 +- .../hbase/regionserver/CellTSTiering.java | 37 + .../regionserver/CustomCellValueTiering.java | 36 + .../hbase/regionserver/DataTiering.java | 12 + .../regionserver/DataTieringManager.java | 76 +- .../hbase/regionserver/DataTieringType.java | 11 +- .../hbase/regionserver/StoreFileWriter.java | 15 +- .../TestCustomCellDataTieringManager.java | 865 ++++++++++++++++++ .../regionserver/TestDataTieringManager.java | 12 +- 14 files changed, 1030 insertions(+), 66 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java index cef5a6488fa6..71121c85991a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java @@ -220,7 +220,7 @@ default Optional shouldCacheFile(HFileInfo hFileInfo, Configuration con * @return An empty Optional if this method is not supported; otherwise, the returned Optional * contains the boolean value indicating if the block should be cached. */ - default Optional shouldCacheBlock(BlockCacheKey key, TimeRangeTracker timeRangeTracker, + default Optional shouldCacheBlock(BlockCacheKey key, long maxTimeStamp, Configuration conf) { return Optional.empty(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java index 672a7bc1e72c..ae763e1d5bee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java @@ -494,10 +494,10 @@ public Optional shouldCacheFile(HFileInfo hFileInfo, Configuration conf } @Override - public Optional shouldCacheBlock(BlockCacheKey key, TimeRangeTracker timeRangeTracker, + public Optional shouldCacheBlock(BlockCacheKey key, long maxTimeStamp, Configuration conf) { - return combineCacheResults(l1Cache.shouldCacheBlock(key, timeRangeTracker, conf), - l2Cache.shouldCacheBlock(key, timeRangeTracker, conf)); + return combineCacheResults(l1Cache.shouldCacheBlock(key, maxTimeStamp, conf), + l2Cache.shouldCacheBlock(key, maxTimeStamp, conf)); } private Optional combineCacheResults(Optional result1, 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 d8dffce59e85..a99eac4085e4 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 @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.regionserver.CellSink; import org.apache.hadoop.hbase.regionserver.ShipperListener; +import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.BloomFilterWriter; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; @@ -217,6 +218,12 @@ public interface Writer extends Closeable, CellSink, ShipperListener { */ void appendTrackedTimestampsToMetadata() throws IOException; + /** + * Add Custom cell timestamp to Metadata + */ + public void appendCustomCellTimestampsToMetadata(TimeRangeTracker timeRangeTracker) + throws IOException; + /** Returns the path to this {@link HFile} */ Path getPath(); 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 4b60ef662c25..972e8070e1cc 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 @@ -1380,7 +1380,7 @@ public HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize, final bo HFileBlock unpackedNoChecksum = BlockCacheUtil.getBlockForCaching(cacheConf, unpacked); // Cache the block if necessary cacheConf.getBlockCache().ifPresent(cache -> { - if (cacheBlock && cacheConf.shouldCacheBlockOnRead(category)) { + if (cacheBlock && cacheOnRead) { // Using the wait on cache during compaction and prefetching. cache.cacheBlock(cacheKey, cacheCompressed diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 8d58032799d8..5e51be72d2d4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.io.hfile; import static org.apache.hadoop.hbase.io.hfile.BlockCompressedSizePredicator.MAX_BLOCK_SIZE_UNCOMPRESSED; +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS; import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; @@ -598,7 +599,7 @@ private BlockCacheKey buildCacheBlockKey(long offset, BlockType blockType) { } private boolean shouldCacheBlock(BlockCache cache, BlockCacheKey key) { - Optional result = cache.shouldCacheBlock(key, timeRangeToTrack.get(), conf); + Optional result = cache.shouldCacheBlock(key, timeRangeToTrack.get().getMax(), conf); return result.orElse(true); } @@ -909,6 +910,13 @@ public void appendTrackedTimestampsToMetadata() throws IOException { appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs)); } + public void appendCustomCellTimestampsToMetadata(TimeRangeTracker timeRangeTracker) + throws IOException { + // TODO: The StoreFileReader always converts the byte[] to TimeRange + // via TimeRangeTracker, so we should write the serialization data of TimeRange directly. + appendFileInfo(TIERING_CELL_TIME_RANGE, TimeRangeTracker.toByteArray(timeRangeTracker)); + } + /** * Record the earliest Put timestamp. If the timeRangeTracker is not set, update TimeRangeTracker * to include the timestamp of this key diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index 2af46a49d89a..21956e6d40a5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -86,7 +86,6 @@ import org.apache.hadoop.hbase.regionserver.DataTieringManager; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.StoreFileInfo; -import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.IdReadWriteLock; @@ -2458,10 +2457,10 @@ public Optional shouldCacheFile(HFileInfo hFileInfo, Configuration conf } @Override - public Optional shouldCacheBlock(BlockCacheKey key, TimeRangeTracker timeRangeTracker, + public Optional shouldCacheBlock(BlockCacheKey key, long maxTimestamp, Configuration conf) { DataTieringManager dataTieringManager = DataTieringManager.getInstance(); - if (dataTieringManager != null && !dataTieringManager.isHotData(timeRangeTracker, conf)) { + if (dataTieringManager != null && !dataTieringManager.isHotData(maxTimestamp, conf)) { LOG.debug("Data tiering is enabled for file: '{}' and it is not hot data", key.getHfileName()); return Optional.of(false); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java new file mode 100644 index 000000000000..0e66a5466107 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java @@ -0,0 +1,37 @@ +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.hbase.io.hfile.HFileInfo; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.OptionalLong; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; + +@InterfaceAudience.Private +public class CellTSTiering implements DataTiering { + private static final Logger LOG = LoggerFactory.getLogger(CellTSTiering.class); + public long getTimestamp(HStoreFile hStoreFile) { + OptionalLong maxTimestamp = hStoreFile.getMaximumTimestamp(); + if (!maxTimestamp.isPresent()) { + LOG.info("Maximum timestamp not present for {}", hStoreFile.getPath()); + return Long.MAX_VALUE; + } + return maxTimestamp.getAsLong(); + } + public long getTimestamp(HFileInfo hFileInfo) { + try { + byte[] hFileTimeRange = hFileInfo.get(TIMERANGE_KEY); + if (hFileTimeRange == null) { + LOG.info("Timestamp information not found for file: {}", + hFileInfo.getHFileContext().getHFileName()); + return Long.MAX_VALUE; + } + return TimeRangeTracker.parseFrom(hFileTimeRange).getMax(); + } catch (IOException e) { + LOG.error("Error occurred while reading the timestamp metadata of file: {}", + hFileInfo.getHFileContext().getHFileName(), e); + return Long.MAX_VALUE; + } + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java new file mode 100644 index 000000000000..9d367c655e35 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java @@ -0,0 +1,36 @@ +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.hfile.HFileInfo; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.OptionalLong; +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; + +@InterfaceAudience.Private +public class CustomCellValueTiering implements DataTiering { + private static final Logger LOG = LoggerFactory.getLogger(CustomCellValueTiering.class); + private long getMaxTSFromTimeRange(byte[] hFileTimeRange, String hFileName) { + try { + if (hFileTimeRange == null) { + LOG.info("Custom cell-based timestamp information not found for file: {}", hFileName); + return Long.MAX_VALUE; + } + return TimeRangeTracker.parseFrom(hFileTimeRange).getMax(); + } catch (IOException e) { + LOG.error("Error occurred while reading the Custom cell-based timestamp metadata of file: {}", + hFileName, e); + return Long.MAX_VALUE; + } + } + public long getTimestamp(HStoreFile hStoreFile) { + return getMaxTSFromTimeRange(hStoreFile.getMetadataValue(TIERING_CELL_TIME_RANGE), + hStoreFile.getPath().getName()); + } + public long getTimestamp(HFileInfo hFileInfo) { + return getMaxTSFromTimeRange(hFileInfo.get(TIERING_CELL_TIME_RANGE), + hFileInfo.getHFileContext().getHFileName()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java new file mode 100644 index 000000000000..a1a0984836dd --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java @@ -0,0 +1,12 @@ +package org.apache.hadoop.hbase.regionserver; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.io.hfile.HFileInfo; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public interface DataTiering { + long getTimestamp(HStoreFile hFile); + long getTimestamp(HFileInfo hFileInfo); + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java index aa56e3f64445..ee32327a1e38 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.NULL_VALUE; import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; import java.io.IOException; @@ -25,6 +26,7 @@ import java.util.Map; import java.util.OptionalLong; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; @@ -136,17 +138,18 @@ public boolean isHotData(BlockCacheKey key) throws DataTieringException { * the data tiering type is set to {@link DataTieringType#TIME_RANGE}, it uses the maximum * timestamp from the time range tracker to determine if the data is hot. Otherwise, it considers * the data as hot by default. - * @param timeRangeTracker the time range tracker containing the timestamps + * @param maxTimestamp the maximum timestamp associated with the data. * @param conf The configuration object to use for determining hot data criteria. * @return {@code true} if the data is hot, {@code false} otherwise */ - public boolean isHotData(TimeRangeTracker timeRangeTracker, Configuration conf) { + public boolean isHotData(long maxTimestamp, Configuration conf) { DataTieringType dataTieringType = getDataTieringType(conf); + if ( - dataTieringType.equals(DataTieringType.TIME_RANGE) - && timeRangeTracker.getMax() != TimeRangeTracker.INITIAL_MAX_TIMESTAMP + !dataTieringType.equals(DataTieringType.NONE) + && maxTimestamp != TimeRangeTracker.INITIAL_MAX_TIMESTAMP ) { - return hotDataValidator(timeRangeTracker.getMax(), getDataTieringHotDataAge(conf)); + return hotDataValidator(maxTimestamp, getDataTieringHotDataAge(conf)); } // DataTieringType.NONE or other types are considered hot by default return true; @@ -165,8 +168,13 @@ public boolean isHotData(Path hFilePath) throws DataTieringException { Configuration configuration = getConfiguration(hFilePath); DataTieringType dataTieringType = getDataTieringType(configuration); - if (dataTieringType.equals(DataTieringType.TIME_RANGE)) { - return hotDataValidator(getMaxTimestamp(hFilePath), getDataTieringHotDataAge(configuration)); + if (!dataTieringType.equals(DataTieringType.NONE)) { + HStoreFile hStoreFile = getHStoreFile(hFilePath); + if (hStoreFile == null) { + throw new DataTieringException("Store file corresponding to " + hFilePath + " doesn't exist"); + } + return hotDataValidator(dataTieringType.instance.getTimestamp(getHStoreFile(hFilePath)), + getDataTieringHotDataAge(configuration)); } // DataTieringType.NONE or other types are considered hot by default return true; @@ -181,17 +189,18 @@ public boolean isHotData(Path hFilePath) throws DataTieringException { * @param maxTimestamp the maximum timestamp to validate against * @return {@code true} if the data is hot, {@code false} otherwise * @throws DataTieringException if there is an error retrieving data tiering information - */ + public boolean isHotData(Path hFilePath, long maxTimestamp) throws DataTieringException { Configuration configuration = getConfiguration(hFilePath); DataTieringType dataTieringType = getDataTieringType(configuration); - if (dataTieringType.equals(DataTieringType.TIME_RANGE)) { + if (!dataTieringType.equals(DataTieringType.NONE)) { return hotDataValidator(maxTimestamp, getDataTieringHotDataAge(configuration)); } // DataTieringType.NONE or other types are considered hot by default return true; } + */ /** * Determines whether the data in the HFile being read is considered hot based on the configured @@ -204,8 +213,9 @@ public boolean isHotData(Path hFilePath, long maxTimestamp) throws DataTieringEx */ public boolean isHotData(HFileInfo hFileInfo, Configuration configuration) { DataTieringType dataTieringType = getDataTieringType(configuration); - if (dataTieringType.equals(DataTieringType.TIME_RANGE)) { - return hotDataValidator(getMaxTimestamp(hFileInfo), getDataTieringHotDataAge(configuration)); + if (hFileInfo != null && !dataTieringType.equals(DataTieringType.NONE)) { + return hotDataValidator(dataTieringType.instance.getTimestamp(hFileInfo), + getDataTieringHotDataAge(configuration)); } // DataTieringType.NONE or other types are considered hot by default return true; @@ -217,36 +227,6 @@ private boolean hotDataValidator(long maxTimestamp, long hotDataAge) { return diff <= hotDataAge; } - private long getMaxTimestamp(Path hFilePath) throws DataTieringException { - HStoreFile hStoreFile = getHStoreFile(hFilePath); - if (hStoreFile == null) { - LOG.error("HStoreFile corresponding to {} doesn't exist", hFilePath); - return Long.MAX_VALUE; - } - OptionalLong maxTimestamp = hStoreFile.getMaximumTimestamp(); - if (!maxTimestamp.isPresent()) { - LOG.error("Maximum timestamp not present for {}", hFilePath); - return Long.MAX_VALUE; - } - return maxTimestamp.getAsLong(); - } - - private long getMaxTimestamp(HFileInfo hFileInfo) { - try { - byte[] hFileTimeRange = hFileInfo.get(TIMERANGE_KEY); - if (hFileTimeRange == null) { - LOG.error("Timestamp information not found for file: {}", - hFileInfo.getHFileContext().getHFileName()); - return Long.MAX_VALUE; - } - return TimeRangeTracker.parseFrom(hFileTimeRange).getMax(); - } catch (IOException e) { - LOG.error("Error occurred while reading the timestamp metadata of file: {}", - hFileInfo.getHFileContext().getHFileName(), e); - return Long.MAX_VALUE; - } - } - private long getCurrentTimestamp() { return EnvironmentEdgeManager.getDelegate().currentTime(); } @@ -299,7 +279,7 @@ private HStore getHStore(Path hFilePath) throws DataTieringException { private HStoreFile getHStoreFile(Path hFilePath) throws DataTieringException { HStore hStore = getHStore(hFilePath); for (HStoreFile file : hStore.getStorefiles()) { - if (file.getPath().equals(hFilePath)) { + if (file.getPath().toUri().getPath().toString().equals(hFilePath.toString())) { return file; } } @@ -330,7 +310,8 @@ public Map getColdFilesList() { for (HRegion r : this.onlineRegions.values()) { for (HStore hStore : r.getStores()) { Configuration conf = hStore.getReadOnlyConfiguration(); - if (getDataTieringType(conf) != DataTieringType.TIME_RANGE) { + DataTieringType dataTieringType = getDataTieringType(conf); + if (dataTieringType == DataTieringType.NONE) { // Data-Tiering not enabled for the store. Just skip it. continue; } @@ -339,14 +320,9 @@ public Map getColdFilesList() { for (HStoreFile hStoreFile : hStore.getStorefiles()) { String hFileName = hStoreFile.getFileInfo().getHFileInfo().getHFileContext().getHFileName(); - OptionalLong maxTimestamp = hStoreFile.getMaximumTimestamp(); - if (!maxTimestamp.isPresent()) { - LOG.warn("maxTimestamp missing for file: {}", - hStoreFile.getFileInfo().getActiveFileName()); - continue; - } + long maxTimeStamp = dataTieringType.instance.getTimestamp(hStoreFile); long currentTimestamp = EnvironmentEdgeManager.getDelegate().currentTime(); - long fileAge = currentTimestamp - maxTimestamp.getAsLong(); + long fileAge = currentTimestamp - maxTimeStamp; if (fileAge > hotDataAge) { // Values do not matter. coldFiles.put(hFileName, null); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java index ee54576a6487..88c03a9e6102 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java @@ -17,10 +17,17 @@ */ package org.apache.hadoop.hbase.regionserver; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Public public enum DataTieringType { - NONE, - TIME_RANGE + NONE(null), + TIME_RANGE(new CellTSTiering()), + CUSTOM_CELL_VALUE(new CustomCellValueTiering()); + + final DataTiering instance; + DataTieringType(DataTiering instance) { + this.instance = instance; + } } 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 569b9d3faa67..58d7fdf1778e 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 @@ -255,6 +255,14 @@ public void appendTrackedTimestampsToMetadata() throws IOException { } } + public void appendCustomCellTimestampsToMetadata(TimeRangeTracker timeRangeTracker) + throws IOException { + liveFileWriter.appendCustomCellTimestampsToMetadata(timeRangeTracker); + if (historicalFileWriter != null) { + historicalFileWriter.appendCustomCellTimestampsToMetadata(timeRangeTracker); + } + } + @Override public void beforeShipped() throws IOException { liveFileWriter.beforeShipped(); @@ -663,7 +671,12 @@ private void appendTrackedTimestampsToMetadata() throws IOException { writer.appendTrackedTimestampsToMetadata(); } - private void appendGeneralBloomfilter(final ExtendedCell cell) throws IOException { + public void appendCustomCellTimestampsToMetadata(TimeRangeTracker timeRangeTracker) + throws IOException { + writer.appendCustomCellTimestampsToMetadata(timeRangeTracker); + } + + private void appendGeneralBloomfilter(final ExtendedCell cell) throws IOException { if (this.generalBloomFilterWriter != null) { /* * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java new file mode 100644 index 000000000000..9bf8138933a0 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java @@ -0,0 +1,865 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_SIZE_KEY; +import static org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.DEFAULT_ERROR_TOLERATION_DURATION; +import static org.apache.hadoop.hbase.regionserver.TestDataTieringManager.createHStoreFile; +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 java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.Set; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.Waiter; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.client.RegionInfoBuilder; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptorBuilder; +import org.apache.hadoop.hbase.fs.HFileSystem; +import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; +import org.apache.hadoop.hbase.io.hfile.BlockCache; +import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory; +import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; +import org.apache.hadoop.hbase.io.hfile.BlockType; +import org.apache.hadoop.hbase.io.hfile.BlockType.BlockCategory; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; +import org.apache.hadoop.hbase.io.hfile.CacheTestUtils; +import org.apache.hadoop.hbase.io.hfile.HFileBlock; +import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.CommonFSUtils; +import org.apache.hadoop.hbase.util.Pair; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is used to test the functionality of the DataTieringManager. + * + * The mock online regions are stored in {@link TestCustomCellDataTieringManager#testOnlineRegions}. + * For all tests, the setup of {@link TestCustomCellDataTieringManager#testOnlineRegions} occurs only once. + * Please refer to {@link TestCustomCellDataTieringManager#setupOnlineRegions()} for the structure. + * Additionally, a list of all store files is maintained in {@link TestCustomCellDataTieringManager#hStoreFiles}. + * The characteristics of these store files are listed below: + * @formatter:off ## HStoreFile Information + * + * | HStoreFile | Region | Store | DataTiering | isHot | + * |------------------|--------------------|---------------------|-----------------------|-------| + * | hStoreFile0 | region1 | hStore11 | CUSTOM_CELL_VALUE | true | + * | hStoreFile1 | region1 | hStore12 | NONE | true | + * | hStoreFile2 | region2 | hStore21 | CUSTOM_CELL_VALUE | true | + * | hStoreFile3 | region2 | hStore22 | CUSTOM_CELL_VALUE | false | + * @formatter:on + */ + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestCustomCellDataTieringManager { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCustomCellDataTieringManager.class); + + private static final Logger LOG = LoggerFactory.getLogger(TestCustomCellDataTieringManager.class); + private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + private static final long DAY = 24 * 60 * 60 * 1000; + private static Configuration defaultConf; + private static FileSystem fs; + private static BlockCache blockCache; + private static CacheConfig cacheConf; + private static Path testDir; + private static final Map testOnlineRegions = new HashMap<>(); + + private static DataTieringManager dataTieringManager; + private static final List hStoreFiles = new ArrayList<>(); + + /** + * Represents the current lexicographically increasing string used as a row key when writing + * HFiles. It is incremented each time {@link #nextString()} is called to generate unique row + * keys. + */ + private static String rowKeyString; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + testDir = TEST_UTIL.getDataTestDir(TestCustomCellDataTieringManager.class.getSimpleName()); + defaultConf = TEST_UTIL.getConfiguration(); + updateCommonConfigurations(); + assertTrue(DataTieringManager.instantiate(defaultConf, testOnlineRegions)); + dataTieringManager = DataTieringManager.getInstance(); + rowKeyString = ""; + } + + private static void updateCommonConfigurations() { + defaultConf.setBoolean(DataTieringManager.GLOBAL_DATA_TIERING_ENABLED_KEY, true); + defaultConf.setStrings(HConstants.BUCKET_CACHE_IOENGINE_KEY, "offheap"); + defaultConf.setLong(BUCKET_CACHE_SIZE_KEY, 32); + } + + @FunctionalInterface + interface DataTieringMethodCallerWithPath { + boolean call(DataTieringManager manager, Path path) throws DataTieringException; + } + + @FunctionalInterface + interface DataTieringMethodCallerWithKey { + boolean call(DataTieringManager manager, BlockCacheKey key) throws DataTieringException; + } + + @Test + public void testDataTieringEnabledWithKey() throws IOException { + initializeTestEnvironment(); + DataTieringMethodCallerWithKey methodCallerWithKey = DataTieringManager::isDataTieringEnabled; + + // Test with valid key + BlockCacheKey key = new BlockCacheKey(hStoreFiles.get(0).getPath(), 0, true, BlockType.DATA); + testDataTieringMethodWithKeyNoException(methodCallerWithKey, key, true); + + // Test with another valid key + key = new BlockCacheKey(hStoreFiles.get(1).getPath(), 0, true, BlockType.DATA); + testDataTieringMethodWithKeyNoException(methodCallerWithKey, key, false); + + // Test with valid key with no HFile Path + key = new BlockCacheKey(hStoreFiles.get(0).getPath().getName(), 0); + testDataTieringMethodWithKeyExpectingException(methodCallerWithKey, key, + new DataTieringException("BlockCacheKey Doesn't Contain HFile Path")); + } + + @Test + public void testDataTieringEnabledWithPath() throws IOException { + initializeTestEnvironment(); + DataTieringMethodCallerWithPath methodCallerWithPath = DataTieringManager::isDataTieringEnabled; + + // Test with valid path + Path hFilePath = hStoreFiles.get(1).getPath(); + testDataTieringMethodWithPathNoException(methodCallerWithPath, hFilePath, false); + + // Test with another valid path + hFilePath = hStoreFiles.get(3).getPath(); + testDataTieringMethodWithPathNoException(methodCallerWithPath, hFilePath, true); + + // Test with an incorrect path + hFilePath = new Path("incorrectPath"); + testDataTieringMethodWithPathExpectingException(methodCallerWithPath, hFilePath, + new DataTieringException("Incorrect HFile Path: " + hFilePath)); + + // Test with a non-existing HRegion path + Path basePath = hStoreFiles.get(0).getPath().getParent().getParent().getParent(); + hFilePath = new Path(basePath, "incorrectRegion/cf1/filename"); + testDataTieringMethodWithPathExpectingException(methodCallerWithPath, hFilePath, + new DataTieringException("HRegion corresponding to " + hFilePath + " doesn't exist")); + + // Test with a non-existing HStore path + basePath = hStoreFiles.get(0).getPath().getParent().getParent(); + hFilePath = new Path(basePath, "incorrectCf/filename"); + testDataTieringMethodWithPathExpectingException(methodCallerWithPath, hFilePath, + new DataTieringException("HStore corresponding to " + hFilePath + " doesn't exist")); + } + + @Test + public void testHotDataWithKey() throws IOException { + initializeTestEnvironment(); + DataTieringMethodCallerWithKey methodCallerWithKey = DataTieringManager::isHotData; + + // Test with valid key + BlockCacheKey key = new BlockCacheKey(hStoreFiles.get(0).getPath(), 0, true, BlockType.DATA); + testDataTieringMethodWithKeyNoException(methodCallerWithKey, key, true); + + // Test with another valid key + key = new BlockCacheKey(hStoreFiles.get(3).getPath(), 0, true, BlockType.DATA); + testDataTieringMethodWithKeyNoException(methodCallerWithKey, key, false); + } + + @Test + public void testHotDataWithPath() throws IOException { + initializeTestEnvironment(); + DataTieringMethodCallerWithPath methodCallerWithPath = DataTieringManager::isHotData; + + // Test with valid path + Path hFilePath = hStoreFiles.get(2).getPath(); + testDataTieringMethodWithPathNoException(methodCallerWithPath, hFilePath, true); + + // Test with another valid path + hFilePath = hStoreFiles.get(3).getPath(); + testDataTieringMethodWithPathNoException(methodCallerWithPath, hFilePath, false); + + // Test with a filename where corresponding HStoreFile in not present + hFilePath = new Path(hStoreFiles.get(0).getPath().getParent(), "incorrectFileName"); + testDataTieringMethodWithPathExpectingException(methodCallerWithPath, hFilePath, + new DataTieringException("Store file corresponding to " + hFilePath + " doesn't exist")); + } + + @Test + public void testPrefetchWhenDataTieringEnabled() throws IOException { + setPrefetchBlocksOnOpen(); + initializeTestEnvironment(); + // Evict blocks from cache by closing the files and passing evict on close. + // Then initialize the reader again. Since Prefetch on open is set to true, it should prefetch + // those blocks. + for (HStoreFile file : hStoreFiles) { + file.closeStoreFile(true); + file.initReader(); + } + + // Since we have one cold file among four files, only three should get prefetched. + Optional>> fullyCachedFiles = blockCache.getFullyCachedFiles(); + assertTrue("We should get the fully cached files from the cache", fullyCachedFiles.isPresent()); + Waiter.waitFor(defaultConf, 10000, () -> fullyCachedFiles.get().size() == 3); + assertEquals("Number of fully cached files are incorrect", 3, fullyCachedFiles.get().size()); + } + + private void setPrefetchBlocksOnOpen() { + defaultConf.setBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, true); + } + + @Test + public void testColdDataFiles() throws IOException { + initializeTestEnvironment(); + Set allCachedBlocks = new HashSet<>(); + for (HStoreFile file : hStoreFiles) { + allCachedBlocks.add(new BlockCacheKey(file.getPath(), 0, true, BlockType.DATA)); + } + + // Verify hStoreFile3 is identified as cold data + DataTieringMethodCallerWithPath methodCallerWithPath = DataTieringManager::isHotData; + Path hFilePath = hStoreFiles.get(3).getPath(); + testDataTieringMethodWithPathNoException(methodCallerWithPath, hFilePath, false); + + // Verify all the other files in hStoreFiles are hot data + for (int i = 0; i < hStoreFiles.size() - 1; i++) { + hFilePath = hStoreFiles.get(i).getPath(); + testDataTieringMethodWithPathNoException(methodCallerWithPath, hFilePath, true); + } + + try { + Set coldFilePaths = dataTieringManager.getColdDataFiles(allCachedBlocks); + assertEquals(1, coldFilePaths.size()); + } catch (DataTieringException e) { + fail("Unexpected DataTieringException: " + e.getMessage()); + } + } + + @Test + public void testCacheCompactedBlocksOnWriteDataTieringDisabled() throws IOException { + setCacheCompactBlocksOnWrite(); + initializeTestEnvironment(); + + HRegion region = createHRegion("table3"); + testCacheCompactedBlocksOnWrite(region, true); + } + + @Test + public void testCacheCompactedBlocksOnWriteWithHotData() throws IOException { + setCacheCompactBlocksOnWrite(); + initializeTestEnvironment(); + + HRegion region = createHRegion("table3", getConfWithCustomCellDataTieringEnabled(5 * DAY)); + testCacheCompactedBlocksOnWrite(region, true); + } + + @Test + public void testCacheCompactedBlocksOnWriteWithColdData() throws IOException { + setCacheCompactBlocksOnWrite(); + initializeTestEnvironment(); + + HRegion region = createHRegion("table3", getConfWithCustomCellDataTieringEnabled(DAY)); + testCacheCompactedBlocksOnWrite(region, false); + } + + private void setCacheCompactBlocksOnWrite() { + defaultConf.setBoolean(CacheConfig.CACHE_COMPACTED_BLOCKS_ON_WRITE_KEY, true); + } + + private void testCacheCompactedBlocksOnWrite(HRegion region, boolean expectDataBlocksCached) + throws IOException { + HStore hStore = createHStore(region, "cf1"); + createTestFilesForCompaction(hStore); + hStore.refreshStoreFiles(); + + region.stores.put(Bytes.toBytes("cf1"), hStore); + testOnlineRegions.put(region.getRegionInfo().getEncodedName(), region); + + long initialStoreFilesCount = hStore.getStorefilesCount(); + long initialCacheDataBlockCount = blockCache.getDataBlockCount(); + assertEquals(3, initialStoreFilesCount); + assertEquals(0, initialCacheDataBlockCount); + + region.compact(true); + + long compactedStoreFilesCount = hStore.getStorefilesCount(); + long compactedCacheDataBlockCount = blockCache.getDataBlockCount(); + assertEquals(1, compactedStoreFilesCount); + assertEquals(expectDataBlocksCached, compactedCacheDataBlockCount > 0); + } + + private void createTestFilesForCompaction(HStore hStore) throws IOException { + long currentTime = System.currentTimeMillis(); + Path storeDir = hStore.getStoreContext().getFamilyStoreDirectoryPath(); + Configuration configuration = hStore.getReadOnlyConfiguration(); + + HRegionFileSystem regionFS = hStore.getHRegion().getRegionFileSystem(); + + createHStoreFile(storeDir, configuration, currentTime - 2 * DAY, regionFS); + createHStoreFile(storeDir, configuration, currentTime - 3 * DAY, regionFS); + createHStoreFile(storeDir, configuration, currentTime - 4 * DAY, regionFS); + } + + @Test + public void testPickColdDataFiles() throws IOException { + initializeTestEnvironment(); + Map coldDataFiles = dataTieringManager.getColdFilesList(); + assertEquals(1, coldDataFiles.size()); + // hStoreFiles[3] is the cold file. + assert (coldDataFiles.containsKey(hStoreFiles.get(3).getFileInfo().getActiveFileName())); + } + + /* + * Verify that two cold blocks(both) are evicted when bucket reaches its capacity. The hot file + * remains in the cache. + */ + @Test + public void testBlockEvictions() throws Exception { + initializeTestEnvironment(); + long capacitySize = 40 * 1024; + int writeThreads = 3; + int writerQLen = 64; + int[] bucketSizes = new int[] { 8 * 1024 + 1024 }; + + // Setup: Create a bucket cache with lower capacity + BucketCache bucketCache = + new BucketCache("file:" + testDir + "/bucket.cache", capacitySize, 8192, bucketSizes, + writeThreads, writerQLen, null, DEFAULT_ERROR_TOLERATION_DURATION, defaultConf); + + // Create three Cache keys with cold data files and a block with hot data. + // hStoreFiles.get(3) is a cold data file, while hStoreFiles.get(0) is a hot file. + Set cacheKeys = new HashSet<>(); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(3).getPath(), 0, true, BlockType.DATA)); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(3).getPath(), 8192, true, BlockType.DATA)); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(0).getPath(), 0, true, BlockType.DATA)); + + // Create dummy data to be cached and fill the cache completely. + CacheTestUtils.HFileBlockPair[] blocks = CacheTestUtils.generateHFileBlocks(8192, 3); + + int blocksIter = 0; + for (BlockCacheKey key : cacheKeys) { + bucketCache.cacheBlock(key, blocks[blocksIter++].getBlock()); + // Ensure that the block is persisted to the file. + Waiter.waitFor(defaultConf, 10000, 100, () -> (bucketCache.getBackingMap().containsKey(key))); + } + + // Verify that the bucket cache contains 3 blocks. + assertEquals(3, bucketCache.getBackingMap().keySet().size()); + + // Add an additional block into cache with hot data which should trigger the eviction + BlockCacheKey newKey = new BlockCacheKey(hStoreFiles.get(2).getPath(), 0, true, BlockType.DATA); + CacheTestUtils.HFileBlockPair[] newBlock = CacheTestUtils.generateHFileBlocks(8192, 1); + + bucketCache.cacheBlock(newKey, newBlock[0].getBlock()); + Waiter.waitFor(defaultConf, 10000, 100, + () -> (bucketCache.getBackingMap().containsKey(newKey))); + + // Verify that the bucket cache now contains 2 hot blocks blocks only. + // Both cold blocks of 8KB will be evicted to make room for 1 block of 8KB + an additional + // space. + validateBlocks(bucketCache.getBackingMap().keySet(), 2, 2, 0); + } + + /* + * Verify that two cold blocks(both) are evicted when bucket reaches its capacity, but one cold + * block remains in the cache since the required space is freed. + */ + @Test + public void testBlockEvictionsAllColdBlocks() throws Exception { + initializeTestEnvironment(); + long capacitySize = 40 * 1024; + int writeThreads = 3; + int writerQLen = 64; + int[] bucketSizes = new int[] { 8 * 1024 + 1024 }; + + // Setup: Create a bucket cache with lower capacity + BucketCache bucketCache = + new BucketCache("file:" + testDir + "/bucket.cache", capacitySize, 8192, bucketSizes, + writeThreads, writerQLen, null, DEFAULT_ERROR_TOLERATION_DURATION, defaultConf); + + // Create three Cache keys with three cold data blocks. + // hStoreFiles.get(3) is a cold data file. + Set cacheKeys = new HashSet<>(); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(3).getPath(), 0, true, BlockType.DATA)); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(3).getPath(), 8192, true, BlockType.DATA)); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(3).getPath(), 16384, true, BlockType.DATA)); + + // Create dummy data to be cached and fill the cache completely. + CacheTestUtils.HFileBlockPair[] blocks = CacheTestUtils.generateHFileBlocks(8192, 3); + + int blocksIter = 0; + for (BlockCacheKey key : cacheKeys) { + bucketCache.cacheBlock(key, blocks[blocksIter++].getBlock()); + // Ensure that the block is persisted to the file. + Waiter.waitFor(defaultConf, 10000, 100, () -> (bucketCache.getBackingMap().containsKey(key))); + } + + // Verify that the bucket cache contains 3 blocks. + assertEquals(3, bucketCache.getBackingMap().keySet().size()); + + // Add an additional block into cache with hot data which should trigger the eviction + BlockCacheKey newKey = new BlockCacheKey(hStoreFiles.get(2).getPath(), 0, true, BlockType.DATA); + CacheTestUtils.HFileBlockPair[] newBlock = CacheTestUtils.generateHFileBlocks(8192, 1); + + bucketCache.cacheBlock(newKey, newBlock[0].getBlock()); + Waiter.waitFor(defaultConf, 10000, 100, + () -> (bucketCache.getBackingMap().containsKey(newKey))); + + // Verify that the bucket cache now contains 1 cold block and a newly added hot block. + validateBlocks(bucketCache.getBackingMap().keySet(), 2, 1, 1); + } + + /* + * Verify that a hot block evicted along with a cold block when bucket reaches its capacity. + */ + @Test + public void testBlockEvictionsHotBlocks() throws Exception { + initializeTestEnvironment(); + long capacitySize = 40 * 1024; + int writeThreads = 3; + int writerQLen = 64; + int[] bucketSizes = new int[] { 8 * 1024 + 1024 }; + + // Setup: Create a bucket cache with lower capacity + BucketCache bucketCache = + new BucketCache("file:" + testDir + "/bucket.cache", capacitySize, 8192, bucketSizes, + writeThreads, writerQLen, null, DEFAULT_ERROR_TOLERATION_DURATION, defaultConf); + + // Create three Cache keys with two hot data blocks and one cold data block + // hStoreFiles.get(0) is a hot data file and hStoreFiles.get(3) is a cold data file. + Set cacheKeys = new HashSet<>(); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(0).getPath(), 0, true, BlockType.DATA)); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(0).getPath(), 8192, true, BlockType.DATA)); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(3).getPath(), 0, true, BlockType.DATA)); + + // Create dummy data to be cached and fill the cache completely. + CacheTestUtils.HFileBlockPair[] blocks = CacheTestUtils.generateHFileBlocks(8192, 3); + + int blocksIter = 0; + for (BlockCacheKey key : cacheKeys) { + bucketCache.cacheBlock(key, blocks[blocksIter++].getBlock()); + // Ensure that the block is persisted to the file. + Waiter.waitFor(defaultConf, 10000, 100, () -> (bucketCache.getBackingMap().containsKey(key))); + } + + // Verify that the bucket cache contains 3 blocks. + assertEquals(3, bucketCache.getBackingMap().keySet().size()); + + // Add an additional block which should evict the only cold block with an additional hot block. + BlockCacheKey newKey = new BlockCacheKey(hStoreFiles.get(2).getPath(), 0, true, BlockType.DATA); + CacheTestUtils.HFileBlockPair[] newBlock = CacheTestUtils.generateHFileBlocks(8192, 1); + + bucketCache.cacheBlock(newKey, newBlock[0].getBlock()); + Waiter.waitFor(defaultConf, 10000, 100, + () -> (bucketCache.getBackingMap().containsKey(newKey))); + + // Verify that the bucket cache now contains 2 hot blocks. + // Only one of the older hot blocks is retained and other one is the newly added hot block. + validateBlocks(bucketCache.getBackingMap().keySet(), 2, 2, 0); + } + + @Test + public void testFeatureKeyDisabled() throws Exception { + DataTieringManager.resetForTestingOnly(); + defaultConf.setBoolean(DataTieringManager.GLOBAL_DATA_TIERING_ENABLED_KEY, false); + initializeTestEnvironment(); + + try { + assertFalse(DataTieringManager.instantiate(defaultConf, testOnlineRegions)); + // Verify that the DataaTieringManager instance is not instantiated in the + // instantiate call above. + assertNull(DataTieringManager.getInstance()); + + // Also validate that data temperature is not honoured. + long capacitySize = 40 * 1024; + int writeThreads = 3; + int writerQLen = 64; + int[] bucketSizes = new int[] { 8 * 1024 + 1024 }; + + // Setup: Create a bucket cache with lower capacity + BucketCache bucketCache = + new BucketCache("file:" + testDir + "/bucket.cache", capacitySize, 8192, bucketSizes, + writeThreads, writerQLen, null, DEFAULT_ERROR_TOLERATION_DURATION, defaultConf); + + // Create three Cache keys with two hot data blocks and one cold data block + // hStoreFiles.get(0) is a hot data file and hStoreFiles.get(3) is a cold data file. + List cacheKeys = new ArrayList<>(); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(0).getPath(), 0, true, BlockType.DATA)); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(0).getPath(), 8192, true, BlockType.DATA)); + cacheKeys.add(new BlockCacheKey(hStoreFiles.get(3).getPath(), 0, true, BlockType.DATA)); + + // Create dummy data to be cached and fill the cache completely. + CacheTestUtils.HFileBlockPair[] blocks = CacheTestUtils.generateHFileBlocks(8192, 3); + + int blocksIter = 0; + for (BlockCacheKey key : cacheKeys) { + LOG.info("Adding {}", key); + bucketCache.cacheBlock(key, blocks[blocksIter++].getBlock()); + // Ensure that the block is persisted to the file. + Waiter.waitFor(defaultConf, 10000, 100, + () -> (bucketCache.getBackingMap().containsKey(key))); + } + + // Verify that the bucket cache contains 3 blocks. + assertEquals(3, bucketCache.getBackingMap().keySet().size()); + + // Add an additional hot block, which triggers eviction. + BlockCacheKey newKey = + new BlockCacheKey(hStoreFiles.get(2).getPath(), 0, true, BlockType.DATA); + CacheTestUtils.HFileBlockPair[] newBlock = CacheTestUtils.generateHFileBlocks(8192, 1); + + bucketCache.cacheBlock(newKey, newBlock[0].getBlock()); + Waiter.waitFor(defaultConf, 10000, 100, + () -> (bucketCache.getBackingMap().containsKey(newKey))); + + // Verify that the bucket still contains the only cold block and one newly added hot block. + // The older hot blocks are evicted and data-tiering mechanism does not kick in to evict + // the cold block. + validateBlocks(bucketCache.getBackingMap().keySet(), 2, 1, 1); + } finally { + DataTieringManager.resetForTestingOnly(); + defaultConf.setBoolean(DataTieringManager.GLOBAL_DATA_TIERING_ENABLED_KEY, true); + assertTrue(DataTieringManager.instantiate(defaultConf, testOnlineRegions)); + } + } + + @Test + public void testCacheConfigShouldCacheFile() throws Exception { + initializeTestEnvironment(); + // Verify that the API shouldCacheFileBlock returns the result correctly. + // hStoreFiles[0], hStoreFiles[1], hStoreFiles[2] are hot files. + // hStoreFiles[3] is a cold file. + assertTrue(cacheConf.shouldCacheBlockOnRead(BlockCategory.DATA, + hStoreFiles.get(0).getFileInfo().getHFileInfo(), hStoreFiles.get(0).getFileInfo().getConf())); + assertTrue(cacheConf.shouldCacheBlockOnRead(BlockCategory.DATA, + hStoreFiles.get(1).getFileInfo().getHFileInfo(), hStoreFiles.get(1).getFileInfo().getConf())); + assertTrue(cacheConf.shouldCacheBlockOnRead(BlockCategory.DATA, + hStoreFiles.get(2).getFileInfo().getHFileInfo(), hStoreFiles.get(2).getFileInfo().getConf())); + assertFalse(cacheConf.shouldCacheBlockOnRead(BlockCategory.DATA, + hStoreFiles.get(3).getFileInfo().getHFileInfo(), hStoreFiles.get(3).getFileInfo().getConf())); + } + + @Test + public void testCacheOnReadColdFile() throws Exception { + initializeTestEnvironment(); + // hStoreFiles[3] is a cold file. the blocks should not get loaded after a readBlock call. + HStoreFile hStoreFile = hStoreFiles.get(3); + BlockCacheKey cacheKey = new BlockCacheKey(hStoreFile.getPath(), 0, true, BlockType.DATA); + testCacheOnRead(hStoreFile, cacheKey, -1, false); + } + + @Test + public void testCacheOnReadHotFile() throws Exception { + initializeTestEnvironment(); + // hStoreFiles[0] is a hot file. the blocks should get loaded after a readBlock call. + HStoreFile hStoreFile = hStoreFiles.get(0); + BlockCacheKey cacheKey = + new BlockCacheKey(hStoreFiles.get(0).getPath(), 0, true, BlockType.DATA); + testCacheOnRead(hStoreFile, cacheKey, -1, true); + } + + private void testCacheOnRead(HStoreFile hStoreFile, BlockCacheKey key, long onDiskBlockSize, + boolean expectedCached) throws Exception { + // Execute the read block API which will try to cache the block if the block is a hot block. + hStoreFile.getReader().getHFileReader().readBlock(key.getOffset(), onDiskBlockSize, true, false, + false, false, key.getBlockType(), DataBlockEncoding.NONE); + // Validate that the hot block gets cached and cold block is not cached. + HFileBlock block = (HFileBlock) blockCache.getBlock(key, false, false, false); + if (expectedCached) { + assertNotNull(block); + } else { + assertNull(block); + } + } + + private void validateBlocks(Set keys, int expectedTotalKeys, int expectedHotBlocks, + int expectedColdBlocks) { + int numHotBlocks = 0, numColdBlocks = 0; + + Waiter.waitFor(defaultConf, 10000, 100, () -> (expectedTotalKeys == keys.size())); + int iter = 0; + for (BlockCacheKey key : keys) { + try { + if (dataTieringManager.isHotData(key)) { + numHotBlocks++; + } else { + numColdBlocks++; + } + } catch (Exception e) { + LOG.debug("Error validating priority for key {}",key, e); + fail(e.getMessage()); + } + } + assertEquals(expectedHotBlocks, numHotBlocks); + assertEquals(expectedColdBlocks, numColdBlocks); + } + + private void testDataTieringMethodWithPath(DataTieringMethodCallerWithPath caller, Path path, + boolean expectedResult, DataTieringException exception) { + try { + boolean value = caller.call(dataTieringManager, path); + if (exception != null) { + fail("Expected DataTieringException to be thrown"); + } + assertEquals(expectedResult, value); + } catch (DataTieringException e) { + if (exception == null) { + fail("Unexpected DataTieringException: " + e.getMessage()); + } + assertEquals(exception.getMessage(), e.getMessage()); + } + } + + private void testDataTieringMethodWithKey(DataTieringMethodCallerWithKey caller, + BlockCacheKey key, boolean expectedResult, DataTieringException exception) { + try { + boolean value = caller.call(dataTieringManager, key); + if (exception != null) { + fail("Expected DataTieringException to be thrown"); + } + assertEquals(expectedResult, value); + } catch (DataTieringException e) { + if (exception == null) { + fail("Unexpected DataTieringException: " + e.getMessage()); + } + assertEquals(exception.getMessage(), e.getMessage()); + } + } + + private void testDataTieringMethodWithPathExpectingException( + DataTieringMethodCallerWithPath caller, Path path, DataTieringException exception) { + testDataTieringMethodWithPath(caller, path, false, exception); + } + + private void testDataTieringMethodWithPathNoException(DataTieringMethodCallerWithPath caller, + Path path, boolean expectedResult) { + testDataTieringMethodWithPath(caller, path, expectedResult, null); + } + + private void testDataTieringMethodWithKeyExpectingException(DataTieringMethodCallerWithKey caller, + BlockCacheKey key, DataTieringException exception) { + testDataTieringMethodWithKey(caller, key, false, exception); + } + + private void testDataTieringMethodWithKeyNoException(DataTieringMethodCallerWithKey caller, + BlockCacheKey key, boolean expectedResult) { + testDataTieringMethodWithKey(caller, key, expectedResult, null); + } + + private static void initializeTestEnvironment() throws IOException { + setupFileSystemAndCache(); + setupOnlineRegions(); + } + + private static void setupFileSystemAndCache() throws IOException { + fs = HFileSystem.get(defaultConf); + blockCache = BlockCacheFactory.createBlockCache(defaultConf); + cacheConf = new CacheConfig(defaultConf, blockCache); + } + + private static void setupOnlineRegions() throws IOException { + testOnlineRegions.clear(); + hStoreFiles.clear(); + long day = 24 * 60 * 60 * 1000; + long currentTime = System.currentTimeMillis(); + + HRegion region1 = createHRegion("table1"); + + HStore hStore11 = createHStore(region1, "cf1", getConfWithCustomCellDataTieringEnabled(day)); + hStoreFiles.add(createHStoreFile(hStore11.getStoreContext().getFamilyStoreDirectoryPath(), + hStore11.getReadOnlyConfiguration(), currentTime, region1.getRegionFileSystem())); + hStore11.refreshStoreFiles(); + HStore hStore12 = createHStore(region1, "cf2"); + hStoreFiles.add(createHStoreFile(hStore12.getStoreContext().getFamilyStoreDirectoryPath(), + hStore12.getReadOnlyConfiguration(), currentTime - day, region1.getRegionFileSystem())); + hStore12.refreshStoreFiles(); + + region1.stores.put(Bytes.toBytes("cf1"), hStore11); + region1.stores.put(Bytes.toBytes("cf2"), hStore12); + + HRegion region2 = + createHRegion("table2", getConfWithCustomCellDataTieringEnabled((long) (2.5 * day))); + + HStore hStore21 = createHStore(region2, "cf1"); + hStoreFiles.add(createHStoreFile(hStore21.getStoreContext().getFamilyStoreDirectoryPath(), + hStore21.getReadOnlyConfiguration(), currentTime - 2 * day, + region2.getRegionFileSystem())); + hStore21.refreshStoreFiles(); + HStore hStore22 = createHStore(region2, "cf2"); + hStoreFiles.add(createHStoreFile(hStore22.getStoreContext().getFamilyStoreDirectoryPath(), + hStore22.getReadOnlyConfiguration(), currentTime - 3 * day, + region2.getRegionFileSystem())); + hStore22.refreshStoreFiles(); + + region2.stores.put(Bytes.toBytes("cf1"), hStore21); + region2.stores.put(Bytes.toBytes("cf2"), hStore22); + + for (HStoreFile file : hStoreFiles) { + file.initReader(); + } + + testOnlineRegions.put(region1.getRegionInfo().getEncodedName(), region1); + testOnlineRegions.put(region2.getRegionInfo().getEncodedName(), region2); + } + + private static HRegion createHRegion(String table) throws IOException { + return createHRegion(table, defaultConf); + } + + private static HRegion createHRegion(String table, Configuration conf) throws IOException { + TableName tableName = TableName.valueOf(table); + + TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName) + .setValue(DataTieringManager.DATATIERING_KEY, conf.get(DataTieringManager.DATATIERING_KEY)) + .setValue(DataTieringManager.DATATIERING_HOT_DATA_AGE_KEY, + conf.get(DataTieringManager.DATATIERING_HOT_DATA_AGE_KEY)) + .build(); + RegionInfo hri = RegionInfoBuilder.newBuilder(tableName).build(); + + Configuration testConf = new Configuration(conf); + CommonFSUtils.setRootDir(testConf, testDir); + HRegionFileSystem regionFs = HRegionFileSystem.createRegionOnFileSystem(testConf, fs, + CommonFSUtils.getTableDir(testDir, hri.getTable()), hri); + + HRegion region = new HRegion(regionFs, null, conf, htd, null); + // Manually sets the BlockCache for the HRegion instance. + // This is necessary because the region server is not started within this method, + // and therefore the BlockCache needs to be explicitly configured. + region.setBlockCache(blockCache); + return region; + } + + private static HStore createHStore(HRegion region, String columnFamily) throws IOException { + return createHStore(region, columnFamily, defaultConf); + } + + private static HStore createHStore(HRegion region, String columnFamily, Configuration conf) + throws IOException { + ColumnFamilyDescriptor columnFamilyDescriptor = + ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(columnFamily)) + .setValue(DataTieringManager.DATATIERING_KEY, conf.get(DataTieringManager.DATATIERING_KEY)) + .setValue(DataTieringManager.DATATIERING_HOT_DATA_AGE_KEY, + conf.get(DataTieringManager.DATATIERING_HOT_DATA_AGE_KEY)) + .build(); + + return new HStore(region, columnFamilyDescriptor, conf, false); + } + + + private static HStoreFile createHStoreFile(Path storeDir, Configuration conf, long timestamp, + HRegionFileSystem regionFs) throws IOException { + String columnFamily = storeDir.getName(); + + StoreFileWriter storeFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs) + .withOutputDir(storeDir).withFileContext(new HFileContextBuilder().build()).build(); + + writeStoreFileRandomData(storeFileWriter, Bytes.toBytes(columnFamily), timestamp); + + StoreContext storeContext = + StoreContext.getBuilder().withRegionFileSystem(regionFs).build(); + StoreFileTracker sft = StoreFileTrackerFactory.create(conf, true, storeContext); + return new HStoreFile(fs, storeFileWriter.getPath(), conf, cacheConf, BloomType.NONE, true, sft); + } + + private static Configuration getConfWithCustomCellDataTieringEnabled(long hotDataAge) { + Configuration conf = new Configuration(defaultConf); + conf.set(DataTieringManager.DATATIERING_KEY, DataTieringType.CUSTOM_CELL_VALUE.name()); + conf.set(DataTieringManager.DATATIERING_HOT_DATA_AGE_KEY, String.valueOf(hotDataAge)); + return conf; + } + + /** + * Writes random data to a store file with rows arranged in lexicographically increasing order. + * Each row is generated using the {@link #nextString()} method, ensuring that each subsequent row + * is lexicographically larger than the previous one. + */ + private static void writeStoreFileRandomData(final StoreFileWriter writer, byte[] columnFamily, + long timestamp) throws IOException { + int cellsPerFile = 10; + byte[] qualifier = Bytes.toBytes("qualifier"); + byte[] value = generateRandomBytes(4 * 1024); + try { + for (int i = 0; i < cellsPerFile; i++) { + byte[] row = Bytes.toBytes(nextString()); + writer.append(new KeyValue(row, columnFamily, qualifier, timestamp, value)); + } + } finally { + writer.appendTrackedTimestampsToMetadata(); + TimeRangeTracker timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); + timeRangeTracker.setMin(timestamp); + timeRangeTracker.setMax(timestamp); + writer.appendCustomCellTimestampsToMetadata(timeRangeTracker); + writer.close(); + } + } + + private static byte[] generateRandomBytes(int sizeInBytes) { + Random random = new Random(); + byte[] randomBytes = new byte[sizeInBytes]; + random.nextBytes(randomBytes); + return randomBytes; + } + + /** + * Returns the lexicographically larger string every time it's called. + */ + private static String nextString() { + if (rowKeyString == null || rowKeyString.isEmpty()) { + rowKeyString = "a"; + } + char lastChar = rowKeyString.charAt(rowKeyString.length() - 1); + if (lastChar < 'z') { + rowKeyString = rowKeyString.substring(0, rowKeyString.length() - 1) + (char) (lastChar + 1); + } else { + rowKeyString = rowKeyString + "a"; + } + return rowKeyString; + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java index 37e0fe98e7d0..ebe027591eca 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java @@ -61,6 +61,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheTestUtils; import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; +import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; @@ -227,7 +228,8 @@ public void testHotDataWithPath() throws IOException { // Test with a filename where corresponding HStoreFile in not present hFilePath = new Path(hStoreFiles.get(0).getPath().getParent(), "incorrectFileName"); - testDataTieringMethodWithPathNoException(methodCallerWithPath, hFilePath, true); + testDataTieringMethodWithPathExpectingException(methodCallerWithPath, hFilePath, + new DataTieringException("Store file corresponding to " + hFilePath + " doesn't exist")); } @Test @@ -600,19 +602,21 @@ public void testCacheConfigShouldCacheFile() throws Exception { @Test public void testCacheOnReadColdFile() throws Exception { + initializeTestEnvironment(); // hStoreFiles[3] is a cold file. the blocks should not get loaded after a readBlock call. HStoreFile hStoreFile = hStoreFiles.get(3); BlockCacheKey cacheKey = new BlockCacheKey(hStoreFile.getPath(), 0, true, BlockType.DATA); - testCacheOnRead(hStoreFile, cacheKey, 23025, false); + testCacheOnRead(hStoreFile, cacheKey, -1, false); } @Test public void testCacheOnReadHotFile() throws Exception { + initializeTestEnvironment(); // hStoreFiles[0] is a hot file. the blocks should get loaded after a readBlock call. HStoreFile hStoreFile = hStoreFiles.get(0); BlockCacheKey cacheKey = new BlockCacheKey(hStoreFiles.get(0).getPath(), 0, true, BlockType.DATA); - testCacheOnRead(hStoreFile, cacheKey, 23025, true); + testCacheOnRead(hStoreFile, cacheKey, -1, true); } private void testCacheOnRead(HStoreFile hStoreFile, BlockCacheKey key, long onDiskBlockSize, @@ -806,7 +810,7 @@ private static Configuration getConfWithTimeRangeDataTieringEnabled(long hotData return conf; } - private static HStoreFile createHStoreFile(Path storeDir, Configuration conf, long timestamp, + static HStoreFile createHStoreFile(Path storeDir, Configuration conf, long timestamp, HRegionFileSystem regionFs) throws IOException { String columnFamily = storeDir.getName(); From 2398e3fbdfdad76c7ecd87a781abbb9b8999311d Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Mon, 25 Nov 2024 21:17:10 +0000 Subject: [PATCH 04/11] HBASE-29422 Implement selectMinorCompation in CustomCellDateTieredCompactionPolicy --- .../CustomCellDateTieredCompactionPolicy.java | 51 ++++ .../DateTieredCompactionPolicy.java | 128 ++++---- .../TestCustomCellTieredCompactionPolicy.java | 278 ++++++++++++++++++ 3 files changed, 403 insertions(+), 54 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java index 3a5a0834e875..07e5376a3898 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java @@ -19,8 +19,10 @@ import org.apache.commons.lang3.mutable.MutableLong; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.regionserver.HStoreFile; import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; +import org.apache.hadoop.hbase.regionserver.StoreUtils; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; @@ -96,4 +98,53 @@ protected List getCompactBoundariesForMajor(Collection filesTo return boundaries; } + @Override + public CompactionRequestImpl selectMinorCompaction(ArrayList candidateSelection, + boolean mayUseOffPeak, boolean mayBeStuck) throws IOException { + ArrayList filteredByPolicy = this.compactionPolicyPerWindow. + applyCompactionPolicy(candidateSelection, mayUseOffPeak, mayBeStuck); + return selectMajorCompaction(filteredByPolicy); + } + + @Override + public boolean shouldPerformMajorCompaction(Collection filesToCompact) + throws IOException{ + long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact); + long now = EnvironmentEdgeManager.currentTime(); + if(isMajorCompactionTime(filesToCompact, now, lowTimestamp)) { + long cfTTL = this.storeConfigInfo.getStoreFileTtl(); + int countLower = 0; + int countHigher = 0; + HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); + for(HStoreFile f : filesToCompact) { + if(checkForTtl(cfTTL, f)){ + return true; + } + if(isMajorOrBulkloadResult(f, now - lowTimestamp)){ + return true; + } + byte[] timeRangeBytes = f.getMetadataValue(TIERING_CELL_TIME_RANGE); + TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(timeRangeBytes); + if(timeRangeTracker.getMin() < cutOffTimestamp) { + if (timeRangeTracker.getMax() > cutOffTimestamp) { + //Found at least one file crossing the cutOffTimestamp + return true; + } else { + countLower++; + } + } else { + countHigher++; + } + hdfsBlocksDistribution.add(f.getHDFSBlockDistribution()); + } + //If we haven't found any files crossing the cutOffTimestamp, we have to check + //if there are at least more than one file on each tier and if so, perform compaction + if( countLower > 1 || countHigher > 1){ + return true; + } + return checkBlockLocality(hdfsBlocksDistribution); + } + return false; + } + } 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 a4de078f6858..64c7678adbc6 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactionPolicy.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.regionserver.compactions; import java.io.IOException; +import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -66,7 +67,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy { private static final Logger LOG = LoggerFactory.getLogger(DateTieredCompactionPolicy.class); - private final RatioBasedCompactionPolicy compactionPolicyPerWindow; + protected final RatioBasedCompactionPolicy compactionPolicyPerWindow; private final CompactionWindowFactory windowFactory; @@ -108,9 +109,8 @@ public boolean needsCompaction(Collection storeFiles, } } - @Override - public boolean shouldPerformMajorCompaction(Collection filesToCompact) - throws IOException { + protected boolean isMajorCompactionTime(Collection filesToCompact, long now, + long lowestModificationTime) throws IOException { long mcTime = getNextMajorCompactTime(filesToCompact); if (filesToCompact == null || mcTime == 0) { if (LOG.isDebugEnabled()) { @@ -118,69 +118,89 @@ public boolean shouldPerformMajorCompaction(Collection filesToCompac } return false; } - // TODO: Use better method for determining stamp of last major (HBASE-2990) - long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact); - long now = EnvironmentEdgeManager.currentTime(); - if (lowTimestamp <= 0L || lowTimestamp >= (now - mcTime)) { + if (lowestModificationTime <= 0L || lowestModificationTime >= (now - mcTime)) { if (LOG.isDebugEnabled()) { - LOG.debug("lowTimestamp: " + lowTimestamp + " lowTimestamp: " + lowTimestamp + " now: " - + now + " mcTime: " + mcTime); + LOG.debug("lowTimestamp: " + lowestModificationTime + " lowTimestamp: " + + lowestModificationTime + " now: " + now + " mcTime: " + mcTime); } return false; } + return true; + } - long cfTTL = this.storeConfigInfo.getStoreFileTtl(); - HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); - List boundaries = getCompactBoundariesForMajor(filesToCompact, now); - boolean[] filesInWindow = new boolean[boundaries.size()]; - - 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) { - LOG.debug("Major compaction triggered on store " + this + "; for TTL maintenance"); - return true; - } - if (!file.isMajorCompactionResult() || file.isBulkLoadResult()) { - LOG.debug("Major compaction triggered on store " + this - + ", because there are new files and time since last major compaction " - + (now - lowTimestamp) + "ms"); - return true; - } - - int lowerWindowIndex = - Collections.binarySearch(boundaries, minTimestamp.orElse(Long.MAX_VALUE)); - int upperWindowIndex = - Collections.binarySearch(boundaries, file.getMaximumTimestamp().orElse(Long.MAX_VALUE)); - // Handle boundary conditions and negative values of binarySearch - lowerWindowIndex = (lowerWindowIndex < 0) ? Math.abs(lowerWindowIndex + 2) : lowerWindowIndex; - upperWindowIndex = (upperWindowIndex < 0) ? Math.abs(upperWindowIndex + 2) : upperWindowIndex; - if (lowerWindowIndex != upperWindowIndex) { - LOG.debug("Major compaction triggered on store " + this + "; because file " + file.getPath() - + " has data with timestamps cross window boundaries"); - return true; - } else if (filesInWindow[upperWindowIndex]) { - LOG.debug("Major compaction triggered on store " + this - + "; because there are more than one file in some windows"); - return true; - } else { - filesInWindow[upperWindowIndex] = true; - } - hdfsBlocksDistribution.add(file.getHDFSBlockDistribution()); + protected boolean checkForTtl(long ttl, HStoreFile file){ + OptionalLong minTimestamp = file.getMinimumTimestamp(); + long oldest = minTimestamp.isPresent() ? + EnvironmentEdgeManager.currentTime() - minTimestamp.getAsLong() : Long.MIN_VALUE; + if (ttl != Long.MAX_VALUE && oldest >= ttl) { + LOG.debug("Major compaction triggered on store " + this + "; for TTL maintenance"); + return true; } + return false; + } + protected boolean isMajorOrBulkloadResult(HStoreFile file, long timeDiff) { + if (!file.isMajorCompactionResult() || file.isBulkLoadResult()) { + LOG.debug("Major compaction triggered on store " + this + ", because there are new files and time since last major compaction " + + timeDiff + "ms"); + return true; + } + return false; + } - float blockLocalityIndex = hdfsBlocksDistribution - .getBlockLocalityIndex(DNS.getHostname(comConf.conf, DNS.ServerType.REGIONSERVER)); + protected boolean checkBlockLocality(HDFSBlocksDistribution hdfsBlocksDistribution) + throws UnknownHostException { + float blockLocalityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(DNS.getHostname(comConf.conf, DNS.ServerType.REGIONSERVER)); if (blockLocalityIndex < comConf.getMinLocalityToForceCompact()) { - LOG.debug("Major compaction triggered on store " + this - + "; to make hdfs blocks local, current blockLocalityIndex is " + blockLocalityIndex + LOG.debug("Major compaction triggered on store " + this + "; to make hdfs blocks local, current blockLocalityIndex is " + blockLocalityIndex + " (min " + comConf.getMinLocalityToForceCompact() + ")"); return true; } + return false; + } - LOG.debug( - "Skipping major compaction of " + this + ", because the files are already major compacted"); + @Override + public boolean shouldPerformMajorCompaction(Collection filesToCompact) + throws IOException { + long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact); + long now = EnvironmentEdgeManager.currentTime(); + if(isMajorCompactionTime(filesToCompact, now, lowTimestamp)) { + long cfTTL = this.storeConfigInfo.getStoreFileTtl(); + HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); + List boundaries = getCompactBoundariesForMajor(filesToCompact, now); + boolean[] filesInWindow = new boolean[boundaries.size()]; + for (HStoreFile file : filesToCompact) { + OptionalLong minTimestamp = file.getMinimumTimestamp(); + if(checkForTtl(cfTTL, file)){ + return true; + } + if(isMajorOrBulkloadResult(file, now - lowTimestamp)){ + return true; + } + int lowerWindowIndex = Collections.binarySearch(boundaries, minTimestamp.orElse(Long.MAX_VALUE)); + int upperWindowIndex = Collections.binarySearch(boundaries, file.getMaximumTimestamp().orElse(Long.MAX_VALUE)); + // Handle boundary conditions and negative values of binarySearch + lowerWindowIndex = + (lowerWindowIndex < 0) ? Math.abs(lowerWindowIndex + 2) : lowerWindowIndex; + upperWindowIndex = + (upperWindowIndex < 0) ? Math.abs(upperWindowIndex + 2) : upperWindowIndex; + if (lowerWindowIndex != upperWindowIndex) { + LOG.debug( + "Major compaction triggered on store " + this + "; because file " + file.getPath() + " has data with timestamps cross window boundaries"); + return true; + } else if (filesInWindow[upperWindowIndex]) { + LOG.debug("Major compaction triggered on store " + this + "; because there are more than one file in some windows"); + return true; + } else { + filesInWindow[upperWindowIndex] = true; + } + hdfsBlocksDistribution.add(file.getHDFSBlockDistribution()); + } + if(checkBlockLocality(hdfsBlocksDistribution)) { + return true; + } + LOG.debug("Skipping major compaction of " + this + ", because the files are already major compacted"); + } return false; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java new file mode 100644 index 000000000000..d50d5a4acb84 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import java.io.IOException; +import java.util.ArrayList; +import java.util.UUID; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtil; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; +import org.apache.hadoop.hbase.client.RegionInfo; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest; +import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerForTest; +import org.apache.hadoop.hbase.testclassification.RegionServerTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.util.ManualEnvironmentEdge; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category({ RegionServerTests.class, SmallTests.class }) +public class TestCustomCellTieredCompactionPolicy { + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestCustomCellTieredCompactionPolicy.class); + + private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); + + public static final byte[] FAMILY = Bytes.toBytes("cf"); + + private HStoreFile createFile(Path file, long minValue, long maxValue, long size, int seqId) throws IOException { + return createFile(mockRegionInfo(), file, minValue, maxValue, size, seqId, 0); + } + + private HStoreFile createFile(RegionInfo regionInfo, Path file, long minValue, long maxValue, long size, int seqId, + long ageInDisk) throws IOException { + FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); + HRegionFileSystem regionFileSystem = new HRegionFileSystem(TEST_UTIL.getConfiguration(),fs, + file, regionInfo); + StoreContext ctx = new StoreContext.Builder() + .withColumnFamilyDescriptor(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()) + .withRegionFileSystem(regionFileSystem).build(); + StoreFileTrackerForTest sftForTest = + new StoreFileTrackerForTest(TEST_UTIL.getConfiguration(), true, ctx); + MockHStoreFile msf = + new MockHStoreFile(TEST_UTIL, file, size, ageInDisk, false, (long) seqId, + sftForTest); + TimeRangeTracker timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); + timeRangeTracker.setMin(minValue); + timeRangeTracker.setMax(maxValue); + msf.setMetadataValue(TIERING_CELL_TIME_RANGE, TimeRangeTracker.toByteArray(timeRangeTracker)); + return msf; + } + + private CustomCellDateTieredCompactionPolicy mockAndCreatePolicy() throws Exception { + RegionInfo mockedRegionInfo = mockRegionInfo(); + return mockAndCreatePolicy(mockedRegionInfo); + } + + private CustomCellDateTieredCompactionPolicy mockAndCreatePolicy(RegionInfo regionInfo) throws Exception { + StoreConfigInformation mockedStoreConfig = mock(StoreConfigInformation.class); + when(mockedStoreConfig.getRegionInfo()).thenReturn(regionInfo); + CustomCellDateTieredCompactionPolicy policy = + new CustomCellDateTieredCompactionPolicy(TEST_UTIL.getConfiguration(), mockedStoreConfig); + return policy; + } + + private RegionInfo mockRegionInfo() { + RegionInfo mockedRegionInfo = mock(RegionInfo.class); + when(mockedRegionInfo.getEncodedName()).thenReturn("1234567890987654321"); + return mockedRegionInfo; + } + + private Path preparePath() throws Exception { + FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); + Path file = new Path(TEST_UTIL.getDataTestDir(), + UUID.randomUUID().toString().replaceAll("-", "")); + fs.create(file); + return file; + } + @Test + public void testGetCompactBoundariesForMajorNoOld() throws Exception { + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 1)); + assertEquals(1, + ((DateTieredCompactionRequest)policy.selectMajorCompaction(files)).getBoundaries().size()); + } + + @Test + public void testGetCompactBoundariesForMajorAllOld() throws Exception { + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + //The default cut off age is 10 years, so any of the min/max value there should get in the old tier + files.add(createFile(file, 0, 1, 1024, 0)); + files.add(createFile(file, 2, 3, 1024, 1)); + assertEquals(2, + ((DateTieredCompactionRequest)policy.selectMajorCompaction(files)).getBoundaries().size()); + } + + @Test + public void testGetCompactBoundariesForMajorOneOnEachSide() throws Exception { + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + files.add(createFile(file, 0, 1, 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), 1024, 1)); + assertEquals(3, + ((DateTieredCompactionRequest)policy.selectMajorCompaction(files)).getBoundaries().size()); + } + + @Test + public void testGetCompactBoundariesForMajorOneCrossing() throws Exception { + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + files.add(createFile(file, 0, EnvironmentEdgeManager.currentTime(), 1024, 0)); + assertEquals(3, + ((DateTieredCompactionRequest)policy.selectMajorCompaction(files)).getBoundaries().size()); + } + + @FunctionalInterface + interface PolicyValidator { + void accept(T t, U u) throws Exception; + } + + private void testShouldPerformMajorCompaction(long min, long max, int numFiles, + PolicyValidator> validation) throws Exception { + RegionInfo mockedRegionInfo = mockRegionInfo(); + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(mockedRegionInfo); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge(); + EnvironmentEdgeManager.injectEdge(timeMachine); + for(int i=0; i assertTrue(p.shouldPerformMajorCompaction(f))); + } + + @Test + public void testShouldPerformMajorCompactionOneFileMinMaxLow() throws Exception { + testShouldPerformMajorCompaction(0, 1, 1, + (p,f) -> assertFalse(p.shouldPerformMajorCompaction(f))); + } + + @Test + public void testShouldPerformMajorCompactionOneFileMinMaxHigh() throws Exception { + long currentTime = EnvironmentEdgeManager.currentTime(); + testShouldPerformMajorCompaction(currentTime, currentTime, 1, + (p,f) -> assertFalse(p.shouldPerformMajorCompaction(f))); + } + + @Test + public void testShouldPerformMajorCompactionTwoFilesMinMaxHigh() throws Exception { + long currentTime = EnvironmentEdgeManager.currentTime(); + testShouldPerformMajorCompaction(currentTime, currentTime, 2, + (p,f) -> assertTrue(p.shouldPerformMajorCompaction(f))); + } + + @Test + public void testSelectMinorCompactionTwoFilesNoOld() throws Exception { + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 1)); + //Shouldn't do minor compaction, as minimum number of files + // for minor compactions is 3 + assertEquals(0, + policy.selectMinorCompaction(files, true, true).getFiles().size()); + } + + @Test + public void testSelectMinorCompactionThreeFilesNoOld() throws Exception { + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 1)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 2)); + assertEquals(3, + policy.selectMinorCompaction(files, true, true).getFiles().size()); + } + + @Test + public void testSelectMinorCompactionThreeFilesAllOld() throws Exception { + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + files.add(createFile(file, 0, 1, + 1024, 0)); + files.add(createFile(file, 1, 2, + 1024, 1)); + files.add(createFile(file, 3, 4, + 1024, 2)); + assertEquals(3, + policy.selectMinorCompaction(files, true, true).getFiles().size()); + } + + @Test + public void testSelectMinorCompactionThreeFilesOneOldTwoNew() throws Exception { + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + files.add(createFile(file, 0, 1, + 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 1)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 2)); + assertEquals(3, + policy.selectMinorCompaction(files, true, true).getFiles().size()); + } + + @Test + public void testSelectMinorCompactionThreeFilesTwoOldOneNew() throws Exception { + CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + Path file = preparePath(); + ArrayList files = new ArrayList<>(); + files.add(createFile(file, 0, 1, + 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 1)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), + 1024, 2)); + assertEquals(3, + policy.selectMinorCompaction(files, true, true).getFiles().size()); + } +} From 9946e36d91affe205ba773ecf5583e12260d66bb Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Tue, 10 Dec 2024 14:57:42 +0000 Subject: [PATCH 05/11] HBASE-29424 Implement configuration validation for custom tiering compactions --- .../procedure/CreateTableProcedure.java | 3 ++ .../procedure/ModifyTableProcedure.java | 3 ++ .../compactions/CustomCellTieringUtils.java | 53 +++++++++++++++++++ 3 files changed, 59 insertions(+) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringUtils.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java index 23ad3b42aef0..de2c9926340a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.ProcedureUtil; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieringUtils; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerValidationUtils; import org.apache.hadoop.hbase.rsgroup.RSGroupInfo; @@ -314,6 +315,8 @@ private boolean prepareCreate(final MasterProcedureEnv env) throws IOException { StoreFileTrackerValidationUtils.checkForCreateTable(env.getMasterConfiguration(), tableDescriptor); + CustomCellTieringUtils.checkForModifyTable(tableDescriptor); + return true; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index 03ad19799cd3..bd3a9fe2fd44 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -51,6 +51,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableState; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieringUtils; + @InterfaceAudience.Private public class ModifyTableProcedure extends AbstractStateMachineTableProcedure { private static final Logger LOG = LoggerFactory.getLogger(ModifyTableProcedure.class); @@ -420,6 +422,7 @@ private void prepareModify(final MasterProcedureEnv env) throws IOException { // check for store file tracker configurations StoreFileTrackerValidationUtils.checkForModifyTable(env.getMasterConfiguration(), unmodifiedTableDescriptor, modifiedTableDescriptor, !isTableEnabled(env)); + CustomCellTieringUtils.checkForModifyTable(modifiedTableDescriptor); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringUtils.java new file mode 100644 index 000000000000..2a8bd3b449be --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringUtils.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.compactions; + +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; +import org.apache.hadoop.hbase.client.TableDescriptor; +import org.apache.hadoop.hbase.regionserver.StoreEngine; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.IOException; +import static org.apache.hadoop.hbase.regionserver.StoreEngine.STORE_ENGINE_CLASS_KEY; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy.TIERING_CELL_QUALIFIER; + +@InterfaceAudience.Private +public class CustomCellTieringUtils { + + private static final Logger LOG = LoggerFactory.getLogger(CustomCellTieringUtils.class); + + public static void checkForModifyTable(TableDescriptor newTable) throws IOException { + for(ColumnFamilyDescriptor descriptor : newTable.getColumnFamilies()) { + String storeEngineClassName = descriptor.getConfigurationValue(STORE_ENGINE_CLASS_KEY); + LOG.info("checking alter table for custom cell tiering on CF: {}. StoreEngine class: {}, " + + "TIERING_CELL_QUALIFIER: {}", Bytes.toString(descriptor.getName()), + storeEngineClassName, descriptor.getConfigurationValue(TIERING_CELL_QUALIFIER)); + if (storeEngineClassName != null && storeEngineClassName. + contains("CustomCellTieredStoreEngine")) { + if( descriptor.getConfigurationValue(TIERING_CELL_QUALIFIER) == null ) { + throw new DoNotRetryIOException("StoreEngine " + storeEngineClassName + + " is missing required " + TIERING_CELL_QUALIFIER + " parameter."); + } + } + } + } + +} From c967603b9d9de8c5177d8098a50e5bdd52c8b7a4 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Mon, 16 Dec 2024 12:49:16 +0000 Subject: [PATCH 06/11] HBASE-29425 Refine and polish code Change-Id: I85b7529da3ba5d0d734f98d2db224740f4ced26f --- .../hadoop/hbase/io/hfile/BlockCache.java | 1 - .../hbase/io/hfile/CombinedBlockCache.java | 1 - .../procedure/CreateTableProcedure.java | 4 +-- .../procedure/ModifyTableProcedure.java | 4 +-- .../hbase/regionserver/CellTSTiering.java | 4 +-- .../CustomCellTieredStoreEngine.java | 13 ++------ .../regionserver/CustomCellValueTiering.java | 4 +-- .../hbase/regionserver/DataTiering.java | 1 - .../regionserver/DataTieringManager.java | 27 ---------------- .../hbase/regionserver/DataTieringType.java | 1 - ... => CustomCellTieredCompactionPolicy.java} | 15 ++++++--- .../CustomCellTieredCompactor.java | 2 +- ...gUtils.java => CustomCellTieredUtils.java} | 13 ++------ .../TestCustomCellTieredCompactionPolicy.java | 32 +++++++++---------- .../TestCustomCellTieredCompactor.java | 2 +- 15 files changed, 41 insertions(+), 83 deletions(-) rename hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/{CustomCellDateTieredCompactionPolicy.java => CustomCellTieredCompactionPolicy.java} (90%) rename hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/{CustomCellTieringUtils.java => CustomCellTieredUtils.java} (76%) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java index 71121c85991a..be3f8beb4547 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java @@ -23,7 +23,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.conf.ConfigurationObserver; -import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java index ae763e1d5bee..e5d52858ab65 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CombinedBlockCache.java @@ -26,7 +26,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.HeapSize; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; -import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.Pair; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java index de2c9926340a..423297f667d3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java @@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; import org.apache.hadoop.hbase.procedure2.ProcedureUtil; -import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieringUtils; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredUtils; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerValidationUtils; import org.apache.hadoop.hbase.rsgroup.RSGroupInfo; @@ -315,7 +315,7 @@ private boolean prepareCreate(final MasterProcedureEnv env) throws IOException { StoreFileTrackerValidationUtils.checkForCreateTable(env.getMasterConfiguration(), tableDescriptor); - CustomCellTieringUtils.checkForModifyTable(tableDescriptor); + CustomCellTieredUtils.checkForModifyTable(tableDescriptor); return true; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index bd3a9fe2fd44..d1799e231d85 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableState; -import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieringUtils; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredUtils; @InterfaceAudience.Private public class ModifyTableProcedure extends AbstractStateMachineTableProcedure { @@ -422,7 +422,7 @@ private void prepareModify(final MasterProcedureEnv env) throws IOException { // check for store file tracker configurations StoreFileTrackerValidationUtils.checkForModifyTable(env.getMasterConfiguration(), unmodifiedTableDescriptor, modifiedTableDescriptor, !isTableEnabled(env)); - CustomCellTieringUtils.checkForModifyTable(modifiedTableDescriptor); + CustomCellTieredUtils.checkForModifyTable(modifiedTableDescriptor); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java index 0e66a5466107..393dea6e7500 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java @@ -14,7 +14,7 @@ public class CellTSTiering implements DataTiering { public long getTimestamp(HStoreFile hStoreFile) { OptionalLong maxTimestamp = hStoreFile.getMaximumTimestamp(); if (!maxTimestamp.isPresent()) { - LOG.info("Maximum timestamp not present for {}", hStoreFile.getPath()); + LOG.debug("Maximum timestamp not present for {}", hStoreFile.getPath()); return Long.MAX_VALUE; } return maxTimestamp.getAsLong(); @@ -23,7 +23,7 @@ public long getTimestamp(HFileInfo hFileInfo) { try { byte[] hFileTimeRange = hFileInfo.get(TIMERANGE_KEY); if (hFileTimeRange == null) { - LOG.info("Timestamp information not found for file: {}", + LOG.debug("Timestamp information not found for file: {}", hFileInfo.getHFileContext().getHFileName()); return Long.MAX_VALUE; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java index 4f061736612b..c0fc6e047173 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java @@ -19,19 +19,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.regionserver.DateTieredStoreEngine; -import org.apache.hadoop.hbase.regionserver.DefaultStoreFileManager; -import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher; -import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.StoreFileComparators; -import org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactor; import org.apache.yetus.audience.InterfaceAudience; -import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactor; -import org.apache.yetus.audience.InterfaceAudience; import java.io.IOException; import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY; -import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy.TIERING_CELL_QUALIFIER; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy.TIERING_CELL_QUALIFIER; /** * Extension of {@link DateTieredStoreEngine} that uses a pluggable value provider for @@ -51,7 +44,7 @@ protected void createComponents(Configuration conf, HStore store, CellComparator conf = new Configuration(conf); conf.set(TIERING_CELL_QUALIFIER, store.conf.get(TIERING_CELL_QUALIFIER)); conf.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, - CustomCellDateTieredCompactionPolicy.class.getName()); + CustomCellTieredCompactionPolicy.class.getName()); createCompactionPolicy(conf, store); this.storeFileManager = new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, conf, compactionPolicy.getConf()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java index 9d367c655e35..ef14ed07dbae 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java @@ -1,12 +1,10 @@ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.hfile.HFileInfo; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.OptionalLong; import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; @InterfaceAudience.Private @@ -15,7 +13,7 @@ public class CustomCellValueTiering implements DataTiering { private long getMaxTSFromTimeRange(byte[] hFileTimeRange, String hFileName) { try { if (hFileTimeRange == null) { - LOG.info("Custom cell-based timestamp information not found for file: {}", hFileName); + LOG.debug("Custom cell-based timestamp information not found for file: {}", hFileName); return Long.MAX_VALUE; } return TimeRangeTracker.parseFrom(hFileTimeRange).getMax(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java index a1a0984836dd..20957d7b26fe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java @@ -1,6 +1,5 @@ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.hfile.HFileInfo; import org.apache.yetus.audience.InterfaceAudience; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java index ee32327a1e38..d143f45ca40d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java @@ -17,16 +17,11 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.apache.hadoop.hbase.regionserver.HStoreFile.NULL_VALUE; -import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; - import java.io.IOException; import java.util.HashMap; import java.util.HashSet; import java.util.Map; -import java.util.OptionalLong; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.io.hfile.BlockCacheKey; @@ -180,28 +175,6 @@ public boolean isHotData(Path hFilePath) throws DataTieringException { return true; } - /** - * Determines whether the data in the HFile at the given path is considered hot based on the - * configured data tiering type and hot data age. If the data tiering type is set to - * {@link DataTieringType#TIME_RANGE}, it validates the data against the provided maximum - * timestamp. - * @param hFilePath the path to the HFile - * @param maxTimestamp the maximum timestamp to validate against - * @return {@code true} if the data is hot, {@code false} otherwise - * @throws DataTieringException if there is an error retrieving data tiering information - - public boolean isHotData(Path hFilePath, long maxTimestamp) throws DataTieringException { - Configuration configuration = getConfiguration(hFilePath); - DataTieringType dataTieringType = getDataTieringType(configuration); - - if (!dataTieringType.equals(DataTieringType.NONE)) { - return hotDataValidator(maxTimestamp, getDataTieringHotDataAge(configuration)); - } - // DataTieringType.NONE or other types are considered hot by default - return true; - } - */ - /** * Determines whether the data in the HFile being read is considered hot based on the configured * data tiering type and hot data age. If the data tiering type is set to diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java index 88c03a9e6102..4e4b6edd25c3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Public diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactionPolicy.java similarity index 90% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactionPolicy.java index 07e5376a3898..e540bf62bf21 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellDateTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactionPolicy.java @@ -26,6 +26,8 @@ import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -44,20 +46,25 @@ * */ @InterfaceAudience.Private -public class CustomCellDateTieredCompactionPolicy extends DateTieredCompactionPolicy { +public class CustomCellTieredCompactionPolicy extends DateTieredCompactionPolicy { public static final String AGE_LIMIT_MILLIS = "hbase.hstore.compaction.date.tiered.custom.age.limit.millis"; + //Defaults to 10 years + public static final long DEFAULT_AGE_LIMIT_MILLIS = (long) (10L*365.25*24L*60L*60L*1000L); + public static final String TIERING_CELL_QUALIFIER = "TIERING_CELL_QUALIFIER"; + private static final Logger LOG = LoggerFactory.getLogger(CustomCellTieredCompactionPolicy.class); + private long cutOffTimestamp; - public CustomCellDateTieredCompactionPolicy(Configuration conf, + public CustomCellTieredCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) throws IOException { super(conf, storeConfigInfo); cutOffTimestamp = EnvironmentEdgeManager.currentTime() - - conf.getLong(AGE_LIMIT_MILLIS, (long) (10L*365.25*24L*60L*60L*1000L)); + conf.getLong(AGE_LIMIT_MILLIS, DEFAULT_AGE_LIMIT_MILLIS); } @@ -76,7 +83,7 @@ protected List getCompactBoundariesForMajor(Collection filesTo minCurrent = timeRangeTracker.getMin(); maxCurrent = timeRangeTracker.getMax(); } catch (IOException e) { - //TODO debug this + LOG.warn("Got TIERING_CELL_TIME_RANGE info from file, but failed to parse it:", e); } } if(minCurrent < min.getValue()) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java index 21b98b5611b2..76948691d89d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java @@ -34,7 +34,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy.TIERING_CELL_QUALIFIER; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy.TIERING_CELL_QUALIFIER; /** * An extension of DateTieredCompactor, overriding the decorateCells method to allow for custom diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java similarity index 76% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringUtils.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java index 2a8bd3b449be..e6ee4757118b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java @@ -20,26 +20,17 @@ import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; -import org.apache.hadoop.hbase.regionserver.StoreEngine; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import static org.apache.hadoop.hbase.regionserver.StoreEngine.STORE_ENGINE_CLASS_KEY; -import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy.TIERING_CELL_QUALIFIER; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy.TIERING_CELL_QUALIFIER; @InterfaceAudience.Private -public class CustomCellTieringUtils { - - private static final Logger LOG = LoggerFactory.getLogger(CustomCellTieringUtils.class); +public class CustomCellTieredUtils { public static void checkForModifyTable(TableDescriptor newTable) throws IOException { for(ColumnFamilyDescriptor descriptor : newTable.getColumnFamilies()) { String storeEngineClassName = descriptor.getConfigurationValue(STORE_ENGINE_CLASS_KEY); - LOG.info("checking alter table for custom cell tiering on CF: {}. StoreEngine class: {}, " - + "TIERING_CELL_QUALIFIER: {}", Bytes.toString(descriptor.getName()), - storeEngineClassName, descriptor.getConfigurationValue(TIERING_CELL_QUALIFIER)); if (storeEngineClassName != null && storeEngineClassName. contains("CustomCellTieredStoreEngine")) { if( descriptor.getConfigurationValue(TIERING_CELL_QUALIFIER) == null ) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java index d50d5a4acb84..18100aca0502 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; -import org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerForTest; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -80,16 +80,16 @@ private HStoreFile createFile(RegionInfo regionInfo, Path file, long minValue, l return msf; } - private CustomCellDateTieredCompactionPolicy mockAndCreatePolicy() throws Exception { + private CustomCellTieredCompactionPolicy mockAndCreatePolicy() throws Exception { RegionInfo mockedRegionInfo = mockRegionInfo(); return mockAndCreatePolicy(mockedRegionInfo); } - private CustomCellDateTieredCompactionPolicy mockAndCreatePolicy(RegionInfo regionInfo) throws Exception { + private CustomCellTieredCompactionPolicy mockAndCreatePolicy(RegionInfo regionInfo) throws Exception { StoreConfigInformation mockedStoreConfig = mock(StoreConfigInformation.class); when(mockedStoreConfig.getRegionInfo()).thenReturn(regionInfo); - CustomCellDateTieredCompactionPolicy policy = - new CustomCellDateTieredCompactionPolicy(TEST_UTIL.getConfiguration(), mockedStoreConfig); + CustomCellTieredCompactionPolicy policy = + new CustomCellTieredCompactionPolicy(TEST_UTIL.getConfiguration(), mockedStoreConfig); return policy; } @@ -108,7 +108,7 @@ private Path preparePath() throws Exception { } @Test public void testGetCompactBoundariesForMajorNoOld() throws Exception { - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), @@ -121,7 +121,7 @@ public void testGetCompactBoundariesForMajorNoOld() throws Exception { @Test public void testGetCompactBoundariesForMajorAllOld() throws Exception { - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); //The default cut off age is 10 years, so any of the min/max value there should get in the old tier @@ -133,7 +133,7 @@ public void testGetCompactBoundariesForMajorAllOld() throws Exception { @Test public void testGetCompactBoundariesForMajorOneOnEachSide() throws Exception { - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, 1, 1024, 0)); @@ -144,7 +144,7 @@ public void testGetCompactBoundariesForMajorOneOnEachSide() throws Exception { @Test public void testGetCompactBoundariesForMajorOneCrossing() throws Exception { - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, EnvironmentEdgeManager.currentTime(), 1024, 0)); @@ -158,9 +158,9 @@ interface PolicyValidator { } private void testShouldPerformMajorCompaction(long min, long max, int numFiles, - PolicyValidator> validation) throws Exception { + PolicyValidator> validation) throws Exception { + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); RegionInfo mockedRegionInfo = mockRegionInfo(); - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(mockedRegionInfo); Path file = preparePath(); ArrayList files = new ArrayList<>(); ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge(); @@ -203,7 +203,7 @@ public void testShouldPerformMajorCompactionTwoFilesMinMaxHigh() throws Exceptio @Test public void testSelectMinorCompactionTwoFilesNoOld() throws Exception { - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), @@ -218,7 +218,7 @@ public void testSelectMinorCompactionTwoFilesNoOld() throws Exception { @Test public void testSelectMinorCompactionThreeFilesNoOld() throws Exception { - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), @@ -233,7 +233,7 @@ public void testSelectMinorCompactionThreeFilesNoOld() throws Exception { @Test public void testSelectMinorCompactionThreeFilesAllOld() throws Exception { - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, 1, @@ -248,7 +248,7 @@ public void testSelectMinorCompactionThreeFilesAllOld() throws Exception { @Test public void testSelectMinorCompactionThreeFilesOneOldTwoNew() throws Exception { - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, 1, @@ -263,7 +263,7 @@ public void testSelectMinorCompactionThreeFilesOneOldTwoNew() throws Exception { @Test public void testSelectMinorCompactionThreeFilesTwoOldOneNew() throws Exception { - CustomCellDateTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, 1, diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java index f76b6857a41c..e44e09758858 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java @@ -42,7 +42,7 @@ import java.util.ArrayList; import java.util.List; import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; -import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellDateTieredCompactionPolicy.TIERING_CELL_QUALIFIER; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy.TIERING_CELL_QUALIFIER; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; From 5a0a3a69b29ef27bd4b231df1cac095758967e89 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Tue, 14 Jan 2025 10:07:18 +0000 Subject: [PATCH 07/11] HBASE-29426 Define a tiering value provider and refactor custom tiered compaction related classes --- .../java/org/apache/hadoop/hbase/TagType.java | 2 +- .../hbase/io/hfile/HFileWriterImpl.java | 7 +- .../hbase/io/hfile/bucket/BucketCache.java | 5 +- .../procedure/ModifyTableProcedure.java | 3 +- .../hbase/regionserver/CellTSTiering.java | 26 ++- .../regionserver/CustomCellValueTiering.java | 34 ---- ...gine.java => CustomTieredStoreEngine.java} | 42 ++--- .../hbase/regionserver/CustomTiering.java | 58 +++++++ .../CustomTieringMultiFileWriter.java | 52 +++--- .../hbase/regionserver/DataTiering.java | 22 ++- .../regionserver/DataTieringManager.java | 15 +- .../hbase/regionserver/DataTieringType.java | 9 +- .../DateTieredMultiFileWriter.java | 3 +- .../regionserver/DateTieredStoreEngine.java | 8 +- .../hbase/regionserver/StoreFileWriter.java | 4 +- .../regionserver/compactions/Compactor.java | 2 +- .../compactions/CustomCellTieredUtils.java | 20 ++- ...va => CustomCellTieringValueProvider.java} | 74 ++++---- ...java => CustomTieredCompactionPolicy.java} | 78 ++++----- .../compactions/CustomTieredCompactor.java | 74 ++++++++ .../DateTieredCompactionPolicy.java | 51 +++--- .../compactions/DateTieredCompactor.java | 8 +- .../TestHFileInlineToRootChunkConversion.java | 3 +- .../TestCustomCellDataTieringManager.java | 73 ++++---- .../TestCustomCellTieredCompactionPolicy.java | 164 +++++++++--------- .../regionserver/TestDataTieringManager.java | 9 +- .../TestCustomCellTieredCompactor.java | 62 ++++--- 27 files changed, 536 insertions(+), 372 deletions(-) delete mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java rename hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/{CustomCellTieredStoreEngine.java => CustomTieredStoreEngine.java} (52%) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTiering.java rename hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/{CustomCellTieredCompactor.java => CustomCellTieringValueProvider.java} (60%) rename hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/{CustomCellTieredCompactionPolicy.java => CustomTieredCompactionPolicy.java} (70%) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactor.java diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java index 811710991eb5..b0df4920e4ed 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagType.java @@ -36,6 +36,6 @@ public final class TagType { // String based tag type used in replication public static final byte STRING_VIS_TAG_TYPE = (byte) 7; public static final byte TTL_TAG_TYPE = (byte) 8; - //tag with the custom cell tiering value for the row + // tag with the custom cell tiering value for the row public static final byte CELL_VALUE_TIERING_TAG_TYPE = (byte) 9; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 5e51be72d2d4..cea5c0361f43 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hbase.io.hfile; import static org.apache.hadoop.hbase.io.hfile.BlockCompressedSizePredicator.MAX_BLOCK_SIZE_UNCOMPRESSED; -import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.CUSTOM_TIERING_TIME_RANGE; import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS; import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; @@ -30,7 +30,6 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import java.util.function.Function; import java.util.function.Supplier; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; @@ -195,7 +194,7 @@ public HFileWriterImpl(final Configuration conf, CacheConfig cacheConf, Path pat this.path = path; this.name = path != null ? path.getName() : outputStream.toString(); this.hFileContext = fileContext; - //TODO: Move this back to upper layer + // TODO: Move this back to upper layer this.timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); this.timeRangeToTrack = () -> this.timeRangeTracker; DataBlockEncoding encoding = hFileContext.getDataBlockEncoding(); @@ -914,7 +913,7 @@ public void appendCustomCellTimestampsToMetadata(TimeRangeTracker timeRangeTrack throws IOException { // TODO: The StoreFileReader always converts the byte[] to TimeRange // via TimeRangeTracker, so we should write the serialization data of TimeRange directly. - appendFileInfo(TIERING_CELL_TIME_RANGE, TimeRangeTracker.toByteArray(timeRangeTracker)); + appendFileInfo(CUSTOM_TIERING_TIME_RANGE, TimeRangeTracker.toByteArray(timeRangeTracker)); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java index 21956e6d40a5..5867fff0861d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/bucket/BucketCache.java @@ -1194,8 +1194,9 @@ void freeSpace(final String why) { } } - if (bytesFreed < bytesToFreeWithExtra && - coldFiles != null && coldFiles.containsKey(bucketEntryWithKey.getKey().getHfileName()) + if ( + bytesFreed < bytesToFreeWithExtra && coldFiles != null + && coldFiles.containsKey(bucketEntryWithKey.getKey().getHfileName()) ) { int freedBlockSize = bucketEntryWithKey.getValue().getLength(); if (evictBlockIfNoRpcReferenced(bucketEntryWithKey.getKey())) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index d1799e231d85..95896838dc2f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.master.MasterCoprocessorHost; import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer; import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; +import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredUtils; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerValidationUtils; import org.apache.hadoop.hbase.rsgroup.RSGroupInfo; import org.apache.hadoop.hbase.util.Bytes; @@ -51,8 +52,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyTableState; -import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredUtils; - @InterfaceAudience.Private public class ModifyTableProcedure extends AbstractStateMachineTableProcedure { private static final Logger LOG = LoggerFactory.getLogger(ModifyTableProcedure.class); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java index 393dea6e7500..ed7dc01ba8d2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellTSTiering.java @@ -1,16 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; + +import java.io.IOException; +import java.util.OptionalLong; import org.apache.hadoop.hbase.io.hfile.HFileInfo; import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.OptionalLong; -import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY; @InterfaceAudience.Private public class CellTSTiering implements DataTiering { private static final Logger LOG = LoggerFactory.getLogger(CellTSTiering.class); + public long getTimestamp(HStoreFile hStoreFile) { OptionalLong maxTimestamp = hStoreFile.getMaximumTimestamp(); if (!maxTimestamp.isPresent()) { @@ -19,6 +38,7 @@ public long getTimestamp(HStoreFile hStoreFile) { } return maxTimestamp.getAsLong(); } + public long getTimestamp(HFileInfo hFileInfo) { try { byte[] hFileTimeRange = hFileInfo.get(TIMERANGE_KEY); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java deleted file mode 100644 index ef14ed07dbae..000000000000 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellValueTiering.java +++ /dev/null @@ -1,34 +0,0 @@ -package org.apache.hadoop.hbase.regionserver; - -import org.apache.hadoop.hbase.io.hfile.HFileInfo; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.IOException; -import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; - -@InterfaceAudience.Private -public class CustomCellValueTiering implements DataTiering { - private static final Logger LOG = LoggerFactory.getLogger(CustomCellValueTiering.class); - private long getMaxTSFromTimeRange(byte[] hFileTimeRange, String hFileName) { - try { - if (hFileTimeRange == null) { - LOG.debug("Custom cell-based timestamp information not found for file: {}", hFileName); - return Long.MAX_VALUE; - } - return TimeRangeTracker.parseFrom(hFileTimeRange).getMax(); - } catch (IOException e) { - LOG.error("Error occurred while reading the Custom cell-based timestamp metadata of file: {}", - hFileName, e); - return Long.MAX_VALUE; - } - } - public long getTimestamp(HStoreFile hStoreFile) { - return getMaxTSFromTimeRange(hStoreFile.getMetadataValue(TIERING_CELL_TIME_RANGE), - hStoreFile.getPath().getName()); - } - public long getTimestamp(HFileInfo hFileInfo) { - return getMaxTSFromTimeRange(hFileInfo.get(TIERING_CELL_TIME_RANGE), - hFileInfo.getHFileContext().getHFileName()); - } -} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java similarity index 52% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java index c0fc6e047173..8ba5a0d5cf1b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomCellTieredStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java @@ -17,39 +17,39 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY; + +import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; -import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy; -import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactor; +import org.apache.hadoop.hbase.CompoundConfiguration; +import org.apache.hadoop.hbase.regionserver.compactions.CustomTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.CustomTieredCompactor; import org.apache.yetus.audience.InterfaceAudience; -import java.io.IOException; -import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY; -import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy.TIERING_CELL_QUALIFIER; /** - * Extension of {@link DateTieredStoreEngine} that uses a pluggable value provider for - * extracting the value to be used for comparison in this tiered compaction. - * - * Differently from the existing Date Tiered Compaction, this doesn't yield multiple tiers - * or files, but rather provides two tiers based on a configurable “cut-off” age. - * All rows with the cell tiering value older than this “cut-off” age would be placed together - * in an “old” tier, whilst younger rows would go to a separate, “young” tier file. + * Extension of {@link DateTieredStoreEngine} that uses a pluggable value provider for extracting + * the value to be used for comparison in this tiered compaction. Differently from the existing Date + * Tiered Compaction, this doesn't yield multiple tiers or files, but rather provides two tiers + * based on a configurable “cut-off” age. All rows with the cell tiering value older than this + * “cut-off” age would be placed together in an “old” tier, whilst younger rows would go to a + * separate, “young” tier file. */ @InterfaceAudience.Private -public class CustomCellTieredStoreEngine extends DateTieredStoreEngine { +public class CustomTieredStoreEngine extends DateTieredStoreEngine { @Override protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator) throws IOException { - conf = new Configuration(conf); - conf.set(TIERING_CELL_QUALIFIER, store.conf.get(TIERING_CELL_QUALIFIER)); - conf.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, - CustomCellTieredCompactionPolicy.class.getName()); - createCompactionPolicy(conf, store); + CompoundConfiguration config = new CompoundConfiguration(); + config.add(conf); + config.add(store.conf); + config.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, CustomTieredCompactionPolicy.class.getName()); + createCompactionPolicy(config, store); this.storeFileManager = new DefaultStoreFileManager(kvComparator, - StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, conf, compactionPolicy.getConf()); - this.storeFlusher = new DefaultStoreFlusher(conf, store); - this.compactor = new CustomCellTieredCompactor(conf, store); + StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, config, compactionPolicy.getConf()); + this.storeFlusher = new DefaultStoreFlusher(config, store); + this.compactor = new CustomTieredCompactor(config, store); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTiering.java new file mode 100644 index 000000000000..7a9914c87d34 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTiering.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver; + +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.CUSTOM_TIERING_TIME_RANGE; + +import java.io.IOException; +import java.util.Date; +import org.apache.hadoop.hbase.io.hfile.HFileInfo; +import org.apache.yetus.audience.InterfaceAudience; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@InterfaceAudience.Private +public class CustomTiering implements DataTiering { + private static final Logger LOG = LoggerFactory.getLogger(CustomTiering.class); + + private long getMaxTSFromTimeRange(byte[] hFileTimeRange, String hFileName) { + try { + if (hFileTimeRange == null) { + LOG.debug("Custom cell-based timestamp information not found for file: {}", hFileName); + return Long.MAX_VALUE; + } + long parsedValue = TimeRangeTracker.parseFrom(hFileTimeRange).getMax(); + LOG.debug("Max TS for file {} is {}", hFileName, new Date(parsedValue)); + return parsedValue; + } catch (IOException e) { + LOG.error("Error occurred while reading the Custom cell-based timestamp metadata of file: {}", + hFileName, e); + return Long.MAX_VALUE; + } + } + + public long getTimestamp(HStoreFile hStoreFile) { + return getMaxTSFromTimeRange(hStoreFile.getMetadataValue(CUSTOM_TIERING_TIME_RANGE), + hStoreFile.getPath().getName()); + } + + public long getTimestamp(HFileInfo hFileInfo) { + return getMaxTSFromTimeRange(hFileInfo.get(CUSTOM_TIERING_TIME_RANGE), + hFileInfo.getHFileContext().getHFileName()); + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java index a09c45f837ea..905b542a1826 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java @@ -1,13 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.ExtendedCell; -import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Collection; import java.util.List; @@ -15,12 +24,16 @@ import java.util.NavigableMap; import java.util.TreeMap; import java.util.function.Function; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.yetus.audience.InterfaceAudience; @InterfaceAudience.Private public class CustomTieringMultiFileWriter extends DateTieredMultiFileWriter { - public static final byte[] TIERING_CELL_TIME_RANGE = - Bytes.toBytes("TIERING_CELL_TIME_RANGE"); + public static final byte[] CUSTOM_TIERING_TIME_RANGE = Bytes.toBytes("CUSTOM_TIERING_TIME_RANGE"); private NavigableMap lowerBoundary2TimeRanger = new TreeMap<>(); @@ -37,21 +50,20 @@ public CustomTieringMultiFileWriter(List lowerBoundaries, public void append(ExtendedCell cell) throws IOException { super.append(cell); long tieringValue = tieringFunction.apply(cell); - Map.Entry entry = - lowerBoundary2TimeRanger.floorEntry(tieringValue); - if(entry.getValue()==null) { + Map.Entry entry = lowerBoundary2TimeRanger.floorEntry(tieringValue); + if (entry.getValue() == null) { TimeRangeTracker timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); timeRangeTracker.setMin(tieringValue); timeRangeTracker.setMax(tieringValue); lowerBoundary2TimeRanger.put(entry.getKey(), timeRangeTracker); - ((HFileWriterImpl)lowerBoundary2Writer.get(entry.getKey()).getLiveFileWriter()) - .setTimeRangeToTrack(()->timeRangeTracker); + ((HFileWriterImpl) lowerBoundary2Writer.get(entry.getKey()).getLiveFileWriter()) + .setTimeRangeToTrack(() -> timeRangeTracker); } else { TimeRangeTracker timeRangeTracker = entry.getValue(); - if(timeRangeTracker.getMin() > tieringValue) { + if (timeRangeTracker.getMin() > tieringValue) { timeRangeTracker.setMin(tieringValue); } - if(timeRangeTracker.getMax() < tieringValue) { + if (timeRangeTracker.getMax() < tieringValue) { timeRangeTracker.setMax(tieringValue); } } @@ -60,10 +72,10 @@ public void append(ExtendedCell cell) throws IOException { @Override public List commitWriters(long maxSeqId, boolean majorCompaction, Collection storeFiles) throws IOException { - for(Map.Entry entry : this.lowerBoundary2Writer.entrySet()){ + for (Map.Entry entry : this.lowerBoundary2Writer.entrySet()) { StoreFileWriter writer = entry.getValue(); - if(writer!=null) { - writer.appendFileInfo(TIERING_CELL_TIME_RANGE, + if (writer != null) { + writer.appendFileInfo(CUSTOM_TIERING_TIME_RANGE, TimeRangeTracker.toByteArray(lowerBoundary2TimeRanger.get(entry.getKey()))); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java index 20957d7b26fe..51e89b0b79d0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTiering.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hbase.regionserver; import org.apache.hadoop.hbase.io.hfile.HFileInfo; @@ -5,7 +22,8 @@ @InterfaceAudience.Private public interface DataTiering { - long getTimestamp(HStoreFile hFile); - long getTimestamp(HFileInfo hFileInfo); + long getTimestamp(HStoreFile hFile); + + long getTimestamp(HFileInfo hFileInfo); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java index d143f45ca40d..2a5e2a5aa39d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringManager.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.regionserver; import java.io.IOException; +import java.util.Date; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -133,8 +134,8 @@ public boolean isHotData(BlockCacheKey key) throws DataTieringException { * the data tiering type is set to {@link DataTieringType#TIME_RANGE}, it uses the maximum * timestamp from the time range tracker to determine if the data is hot. Otherwise, it considers * the data as hot by default. - * @param maxTimestamp the maximum timestamp associated with the data. - * @param conf The configuration object to use for determining hot data criteria. + * @param maxTimestamp the maximum timestamp associated with the data. + * @param conf The configuration object to use for determining hot data criteria. * @return {@code true} if the data is hot, {@code false} otherwise */ public boolean isHotData(long maxTimestamp, Configuration conf) { @@ -166,9 +167,10 @@ public boolean isHotData(Path hFilePath) throws DataTieringException { if (!dataTieringType.equals(DataTieringType.NONE)) { HStoreFile hStoreFile = getHStoreFile(hFilePath); if (hStoreFile == null) { - throw new DataTieringException("Store file corresponding to " + hFilePath + " doesn't exist"); + throw new DataTieringException( + "Store file corresponding to " + hFilePath + " doesn't exist"); } - return hotDataValidator(dataTieringType.instance.getTimestamp(getHStoreFile(hFilePath)), + return hotDataValidator(dataTieringType.getInstance().getTimestamp(getHStoreFile(hFilePath)), getDataTieringHotDataAge(configuration)); } // DataTieringType.NONE or other types are considered hot by default @@ -187,7 +189,7 @@ public boolean isHotData(Path hFilePath) throws DataTieringException { public boolean isHotData(HFileInfo hFileInfo, Configuration configuration) { DataTieringType dataTieringType = getDataTieringType(configuration); if (hFileInfo != null && !dataTieringType.equals(DataTieringType.NONE)) { - return hotDataValidator(dataTieringType.instance.getTimestamp(hFileInfo), + return hotDataValidator(dataTieringType.getInstance().getTimestamp(hFileInfo), getDataTieringHotDataAge(configuration)); } // DataTieringType.NONE or other types are considered hot by default @@ -293,7 +295,8 @@ public Map getColdFilesList() { for (HStoreFile hStoreFile : hStore.getStorefiles()) { String hFileName = hStoreFile.getFileInfo().getHFileInfo().getHFileContext().getHFileName(); - long maxTimeStamp = dataTieringType.instance.getTimestamp(hStoreFile); + long maxTimeStamp = dataTieringType.getInstance().getTimestamp(hStoreFile); + LOG.debug("Max TS for file {} is {}", hFileName, new Date(maxTimeStamp)); long currentTimestamp = EnvironmentEdgeManager.getDelegate().currentTime(); long fileAge = currentTimestamp - maxTimeStamp; if (fileAge > hotDataAge) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java index 4e4b6edd25c3..83da5f54e43f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DataTieringType.java @@ -23,10 +23,15 @@ public enum DataTieringType { NONE(null), TIME_RANGE(new CellTSTiering()), - CUSTOM_CELL_VALUE(new CustomCellValueTiering()); + CUSTOM(new CustomTiering()); + + private final DataTiering instance; - final DataTiering instance; DataTieringType(DataTiering instance) { this.instance = instance; } + + public DataTiering getInstance() { + return instance; + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java index 5796d7c890e8..e01f062a0191 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DateTieredMultiFileWriter.java @@ -23,9 +23,8 @@ import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; -import org.apache.hadoop.hbase.ExtendedCell; import java.util.function.Function; -import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.ExtendedCell; import org.apache.yetus.audience.InterfaceAudience; /** 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 88eb59f69e80..dc13f190afaa 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 @@ -17,6 +17,8 @@ */ package org.apache.hadoop.hbase.regionserver; +import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY; + import java.io.IOException; import java.util.List; import org.apache.hadoop.conf.Configuration; @@ -31,7 +33,6 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.yetus.audience.InterfaceAudience; -import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY; /** * HBASE-15400 This store engine allows us to store data in date tiered layout with exponential @@ -47,8 +48,8 @@ public class DateTieredStoreEngine extends StoreEngine filesCompacting) { return compactionPolicy.needsCompaction(storeFileManager.getStoreFiles(), filesCompacting); 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 58d7fdf1778e..b5732d3b23ac 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 @@ -672,11 +672,11 @@ private void appendTrackedTimestampsToMetadata() throws IOException { } public void appendCustomCellTimestampsToMetadata(TimeRangeTracker timeRangeTracker) - throws IOException { + throws IOException { writer.appendCustomCellTimestampsToMetadata(timeRangeTracker); } - private void appendGeneralBloomfilter(final ExtendedCell cell) throws IOException { + private void appendGeneralBloomfilter(final ExtendedCell cell) throws IOException { if (this.generalBloomFilterWriter != null) { /* * http://2.bp.blogspot.com/_Cib_A77V54U/StZMrzaKufI/AAAAAAAAADo/ZhK7bGoJdMQ/s400/KeyValue. 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 2e679e1a84dc..069968294b84 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 @@ -404,7 +404,7 @@ protected abstract List commitWriter(T writer, FileDetails fd, protected abstract void abortWriter(T writer) throws IOException; protected List decorateCells(List cells) { - //no op + // no op return cells; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java index e6ee4757118b..538cbda2f3fe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java @@ -17,25 +17,27 @@ */ package org.apache.hadoop.hbase.regionserver.compactions; +import static org.apache.hadoop.hbase.regionserver.StoreEngine.STORE_ENGINE_CLASS_KEY; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieringValueProvider.TIERING_CELL_QUALIFIER; + +import java.io.IOException; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.yetus.audience.InterfaceAudience; -import java.io.IOException; -import static org.apache.hadoop.hbase.regionserver.StoreEngine.STORE_ENGINE_CLASS_KEY; -import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy.TIERING_CELL_QUALIFIER; @InterfaceAudience.Private public class CustomCellTieredUtils { public static void checkForModifyTable(TableDescriptor newTable) throws IOException { - for(ColumnFamilyDescriptor descriptor : newTable.getColumnFamilies()) { + for (ColumnFamilyDescriptor descriptor : newTable.getColumnFamilies()) { String storeEngineClassName = descriptor.getConfigurationValue(STORE_ENGINE_CLASS_KEY); - if (storeEngineClassName != null && storeEngineClassName. - contains("CustomCellTieredStoreEngine")) { - if( descriptor.getConfigurationValue(TIERING_CELL_QUALIFIER) == null ) { - throw new DoNotRetryIOException("StoreEngine " + storeEngineClassName + - " is missing required " + TIERING_CELL_QUALIFIER + " parameter."); + if ( + storeEngineClassName != null && storeEngineClassName.contains("CustomCellTieredStoreEngine") + ) { + if (descriptor.getConfigurationValue(TIERING_CELL_QUALIFIER) == null) { + throw new DoNotRetryIOException("StoreEngine " + storeEngineClassName + + " is missing required " + TIERING_CELL_QUALIFIER + " parameter."); } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java similarity index 60% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java index 76948691d89d..0c9d212a03a2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java @@ -17,6 +17,27 @@ */ package org.apache.hadoop.hbase.regionserver.compactions; +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; @@ -24,54 +45,47 @@ import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.Tag; import org.apache.hadoop.hbase.TagType; -import org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter; -import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter; -import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy.TIERING_CELL_QUALIFIER; /** * An extension of DateTieredCompactor, overriding the decorateCells method to allow for custom * values to be used for the different file tiers during compaction. */ @InterfaceAudience.Private -public class CustomCellTieredCompactor extends DateTieredCompactor { - +public class CustomCellTieringValueProvider implements CustomTieredCompactor.TieringValueProvider { + public static final String TIERING_CELL_QUALIFIER = "TIERING_CELL_QUALIFIER"; private byte[] tieringQualifier; - public CustomCellTieredCompactor(Configuration conf, HStore store) { - super(conf, store); + @Override + public void init(Configuration conf) throws Exception { tieringQualifier = Bytes.toBytes(conf.get(TIERING_CELL_QUALIFIER)); } @Override - protected List decorateCells(List cells) { - //if no tiering qualifier properly set, skips the whole flow - if(tieringQualifier!=null) { + public List decorateCells(List cells) { + // if no tiering qualifier properly set, skips the whole flow + if (tieringQualifier != null) { byte[] tieringValue = null; - //first iterates through the cells within a row, to find the tiering value for the row + // first iterates through the cells within a row, to find the tiering value for the row for (Cell cell : cells) { byte[] qualifier = new byte[cell.getQualifierLength()]; - System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), qualifier, 0, cell.getQualifierLength()); + System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), qualifier, 0, + cell.getQualifierLength()); if (Arrays.equals(qualifier, tieringQualifier)) { tieringValue = new byte[cell.getValueLength()]; - System.arraycopy(cell.getValueArray(), cell.getValueOffset(), tieringValue, 0, cell.getValueLength()); + System.arraycopy(cell.getValueArray(), cell.getValueOffset(), tieringValue, 0, + cell.getValueLength()); break; } } - if(tieringValue==null){ + if (tieringValue == null) { tieringValue = Bytes.toBytes(Long.MAX_VALUE); } - //now apply the tiering value as a tag to all cells within the row + // now apply the tiering value as a tag to all cells within the row Tag tieringValueTag = new ArrayBackedTag(TagType.CELL_VALUE_TIERING_TAG_TYPE, tieringValue); List newCells = new ArrayList<>(cells.size()); - for(ExtendedCell cell : cells) { + for (ExtendedCell cell : cells) { List tags = PrivateCellUtil.getTags(cell); tags.add(tieringValueTag); newCells.add(PrivateCellUtil.createCell(cell, tags)); @@ -82,20 +96,14 @@ protected List decorateCells(List cells) { } } - private long getTieringValue(ExtendedCell cell) { + @Override + public long getTieringValue(ExtendedCell cell) { Optional tagOptional = PrivateCellUtil.getTag(cell, TagType.CELL_VALUE_TIERING_TAG_TYPE); - if(tagOptional.isPresent()) { + if (tagOptional.isPresent()) { Tag tag = tagOptional.get(); - return Bytes.toLong(tag.getValueByteBuffer().array(), tag.getValueOffset(), tag.getValueLength()); + return Bytes.toLong(tag.getValueByteBuffer().array(), tag.getValueOffset(), + tag.getValueLength()); } return Long.MAX_VALUE; } - - @Override - protected DateTieredMultiFileWriter createMultiWriter(final CompactionRequestImpl request, - final List lowerBoundaries, final Map lowerBoundariesPolicies) { - return new CustomTieringMultiFileWriter(lowerBoundaries, lowerBoundariesPolicies, - needEmptyFile(request), CustomCellTieredCompactor.this::getTieringValue); - } - } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactionPolicy.java similarity index 70% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactionPolicy.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactionPolicy.java index e540bf62bf21..665f29fdc816 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactionPolicy.java @@ -17,6 +17,12 @@ */ package org.apache.hadoop.hbase.regionserver.compactions; +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.CUSTOM_TIERING_TIME_RANGE; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; import org.apache.commons.lang3.mutable.MutableLong; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HDFSBlocksDistribution; @@ -28,55 +34,47 @@ import org.apache.yetus.audience.InterfaceAudience; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; /** * Custom implementation of DateTieredCompactionPolicy that calculates compaction boundaries based * on the hbase.hstore.compaction.date.tiered.custom.age.limit.millis configuration property - * and the TIERING_CELL_MIN/TIERING_CELL_MAX stored on metadata of each store file. - * - * This policy would produce either one or two tiers: - * - One tier if either all files data age are older than the configured age limit or all files - * data age are younger than the configured age limit. - * - Two tiers if files have both younger and older data than the configured age limit. - * + * and the TIERING_CELL_MIN/TIERING_CELL_MAX stored on metadata of each store file. This policy + * would produce either one or two tiers: - One tier if either all files data age are older than the + * configured age limit or all files data age are younger than the configured age limit. - Two tiers + * if files have both younger and older data than the configured age limit. */ @InterfaceAudience.Private -public class CustomCellTieredCompactionPolicy extends DateTieredCompactionPolicy { +public class CustomTieredCompactionPolicy extends DateTieredCompactionPolicy { public static final String AGE_LIMIT_MILLIS = "hbase.hstore.compaction.date.tiered.custom.age.limit.millis"; - //Defaults to 10 years - public static final long DEFAULT_AGE_LIMIT_MILLIS = (long) (10L*365.25*24L*60L*60L*1000L); - - public static final String TIERING_CELL_QUALIFIER = "TIERING_CELL_QUALIFIER"; + // Defaults to 10 years + public static final long DEFAULT_AGE_LIMIT_MILLIS = + (long) (10L * 365.25 * 24L * 60L * 60L * 1000L); - private static final Logger LOG = LoggerFactory.getLogger(CustomCellTieredCompactionPolicy.class); + private static final Logger LOG = LoggerFactory.getLogger(CustomTieredCompactionPolicy.class); private long cutOffTimestamp; - public CustomCellTieredCompactionPolicy(Configuration conf, - StoreConfigInformation storeConfigInfo) throws IOException { + public CustomTieredCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) + throws IOException { super(conf, storeConfigInfo); - cutOffTimestamp = EnvironmentEdgeManager.currentTime() - - conf.getLong(AGE_LIMIT_MILLIS, DEFAULT_AGE_LIMIT_MILLIS); + cutOffTimestamp = EnvironmentEdgeManager.currentTime() + - conf.getLong(AGE_LIMIT_MILLIS, DEFAULT_AGE_LIMIT_MILLIS); } @Override - protected List getCompactBoundariesForMajor(Collection filesToCompact, long now) { + protected List getCompactBoundariesForMajor(Collection filesToCompact, + long now) { MutableLong min = new MutableLong(Long.MAX_VALUE); MutableLong max = new MutableLong(0); filesToCompact.forEach(f -> { - byte[] timeRangeBytes = f.getMetadataValue(TIERING_CELL_TIME_RANGE); + byte[] timeRangeBytes = f.getMetadataValue(CUSTOM_TIERING_TIME_RANGE); long minCurrent = Long.MAX_VALUE; long maxCurrent = 0; - if(timeRangeBytes!=null) { + if (timeRangeBytes != null) { try { TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(timeRangeBytes); timeRangeTracker.getMin(); @@ -86,10 +84,10 @@ protected List getCompactBoundariesForMajor(Collection filesTo LOG.warn("Got TIERING_CELL_TIME_RANGE info from file, but failed to parse it:", e); } } - if(minCurrent < min.getValue()) { + if (minCurrent < min.getValue()) { min.setValue(minCurrent); } - if(maxCurrent > max.getValue()) { + if (maxCurrent > max.getValue()) { max.setValue(maxCurrent); } }); @@ -108,33 +106,33 @@ protected List getCompactBoundariesForMajor(Collection filesTo @Override public CompactionRequestImpl selectMinorCompaction(ArrayList candidateSelection, boolean mayUseOffPeak, boolean mayBeStuck) throws IOException { - ArrayList filteredByPolicy = this.compactionPolicyPerWindow. - applyCompactionPolicy(candidateSelection, mayUseOffPeak, mayBeStuck); + ArrayList filteredByPolicy = this.compactionPolicyPerWindow + .applyCompactionPolicy(candidateSelection, mayUseOffPeak, mayBeStuck); return selectMajorCompaction(filteredByPolicy); } @Override public boolean shouldPerformMajorCompaction(Collection filesToCompact) - throws IOException{ + throws IOException { long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact); long now = EnvironmentEdgeManager.currentTime(); - if(isMajorCompactionTime(filesToCompact, now, lowTimestamp)) { + if (isMajorCompactionTime(filesToCompact, now, lowTimestamp)) { long cfTTL = this.storeConfigInfo.getStoreFileTtl(); int countLower = 0; int countHigher = 0; HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); - for(HStoreFile f : filesToCompact) { - if(checkForTtl(cfTTL, f)){ + for (HStoreFile f : filesToCompact) { + if (checkForTtl(cfTTL, f)) { return true; } - if(isMajorOrBulkloadResult(f, now - lowTimestamp)){ + if (isMajorOrBulkloadResult(f, now - lowTimestamp)) { return true; } - byte[] timeRangeBytes = f.getMetadataValue(TIERING_CELL_TIME_RANGE); + byte[] timeRangeBytes = f.getMetadataValue(CUSTOM_TIERING_TIME_RANGE); TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(timeRangeBytes); - if(timeRangeTracker.getMin() < cutOffTimestamp) { + if (timeRangeTracker.getMin() < cutOffTimestamp) { if (timeRangeTracker.getMax() > cutOffTimestamp) { - //Found at least one file crossing the cutOffTimestamp + // Found at least one file crossing the cutOffTimestamp return true; } else { countLower++; @@ -144,9 +142,9 @@ public boolean shouldPerformMajorCompaction(Collection filesToCompac } hdfsBlocksDistribution.add(f.getHDFSBlockDistribution()); } - //If we haven't found any files crossing the cutOffTimestamp, we have to check - //if there are at least more than one file on each tier and if so, perform compaction - if( countLower > 1 || countHigher > 1){ + // If we haven't found any files crossing the cutOffTimestamp, we have to check + // if there are at least more than one file on each tier and if so, perform compaction + if (countLower > 1 || countHigher > 1) { return true; } return checkBlockLocality(hdfsBlocksDistribution); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactor.java new file mode 100644 index 000000000000..47e4e142bdad --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactor.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.regionserver.compactions; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ExtendedCell; +import org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter; +import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class CustomTieredCompactor extends DateTieredCompactor { + + public static final String TIERING_VALUE_PROVIDER = + "hbase.hstore.custom-tiering-value.provider.class"; + private TieringValueProvider tieringValueProvider; + + public CustomTieredCompactor(Configuration conf, HStore store) throws IOException { + super(conf, store); + String className = + conf.get(TIERING_VALUE_PROVIDER, CustomCellTieringValueProvider.class.getName()); + try { + tieringValueProvider = + (TieringValueProvider) Class.forName(className).getConstructor().newInstance(); + tieringValueProvider.init(conf); + } catch (Exception e) { + throw new IOException("Unable to load configured tiering value provider '" + className + "'", + e); + } + } + + @Override + protected List decorateCells(List cells) { + return tieringValueProvider.decorateCells(cells); + } + + @Override + protected DateTieredMultiFileWriter createMultiWriter(final CompactionRequestImpl request, + final List lowerBoundaries, final Map lowerBoundariesPolicies) { + return new CustomTieringMultiFileWriter(lowerBoundaries, lowerBoundariesPolicies, + needEmptyFile(request), CustomTieredCompactor.this.tieringValueProvider::getTieringValue); + } + + public interface TieringValueProvider { + + void init(Configuration configuration) throws Exception; + + default List decorateCells(List cells) { + return cells; + } + + long getTieringValue(ExtendedCell cell); + } + +} 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 64c7678adbc6..2cce0d67d772 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 @@ -110,7 +110,7 @@ public boolean needsCompaction(Collection storeFiles, } protected boolean isMajorCompactionTime(Collection filesToCompact, long now, - long lowestModificationTime) throws IOException { + long lowestModificationTime) throws IOException { long mcTime = getNextMajorCompactTime(filesToCompact); if (filesToCompact == null || mcTime == 0) { if (LOG.isDebugEnabled()) { @@ -129,30 +129,34 @@ protected boolean isMajorCompactionTime(Collection filesToCompact, l return true; } - protected boolean checkForTtl(long ttl, HStoreFile file){ + protected boolean checkForTtl(long ttl, HStoreFile file) { OptionalLong minTimestamp = file.getMinimumTimestamp(); - long oldest = minTimestamp.isPresent() ? - EnvironmentEdgeManager.currentTime() - minTimestamp.getAsLong() : Long.MIN_VALUE; + long oldest = minTimestamp.isPresent() + ? EnvironmentEdgeManager.currentTime() - minTimestamp.getAsLong() + : Long.MIN_VALUE; if (ttl != Long.MAX_VALUE && oldest >= ttl) { LOG.debug("Major compaction triggered on store " + this + "; for TTL maintenance"); return true; } return false; } + protected boolean isMajorOrBulkloadResult(HStoreFile file, long timeDiff) { if (!file.isMajorCompactionResult() || file.isBulkLoadResult()) { - LOG.debug("Major compaction triggered on store " + this + ", because there are new files and time since last major compaction " - + timeDiff + "ms"); + LOG.debug("Major compaction triggered on store " + this + + ", because there are new files and time since last major compaction " + timeDiff + "ms"); return true; } return false; } protected boolean checkBlockLocality(HDFSBlocksDistribution hdfsBlocksDistribution) - throws UnknownHostException { - float blockLocalityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(DNS.getHostname(comConf.conf, DNS.ServerType.REGIONSERVER)); + throws UnknownHostException { + float blockLocalityIndex = hdfsBlocksDistribution + .getBlockLocalityIndex(DNS.getHostname(comConf.conf, DNS.ServerType.REGIONSERVER)); if (blockLocalityIndex < comConf.getMinLocalityToForceCompact()) { - LOG.debug("Major compaction triggered on store " + this + "; to make hdfs blocks local, current blockLocalityIndex is " + blockLocalityIndex + LOG.debug("Major compaction triggered on store " + this + + "; to make hdfs blocks local, current blockLocalityIndex is " + blockLocalityIndex + " (min " + comConf.getMinLocalityToForceCompact() + ")"); return true; } @@ -161,45 +165,49 @@ protected boolean checkBlockLocality(HDFSBlocksDistribution hdfsBlocksDistributi @Override public boolean shouldPerformMajorCompaction(Collection filesToCompact) - throws IOException { + throws IOException { long lowTimestamp = StoreUtils.getLowestTimestamp(filesToCompact); long now = EnvironmentEdgeManager.currentTime(); - if(isMajorCompactionTime(filesToCompact, now, lowTimestamp)) { + if (isMajorCompactionTime(filesToCompact, now, lowTimestamp)) { long cfTTL = this.storeConfigInfo.getStoreFileTtl(); HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); List boundaries = getCompactBoundariesForMajor(filesToCompact, now); boolean[] filesInWindow = new boolean[boundaries.size()]; for (HStoreFile file : filesToCompact) { OptionalLong minTimestamp = file.getMinimumTimestamp(); - if(checkForTtl(cfTTL, file)){ + if (checkForTtl(cfTTL, file)) { return true; } - if(isMajorOrBulkloadResult(file, now - lowTimestamp)){ + if (isMajorOrBulkloadResult(file, now - lowTimestamp)) { return true; } - int lowerWindowIndex = Collections.binarySearch(boundaries, minTimestamp.orElse(Long.MAX_VALUE)); - int upperWindowIndex = Collections.binarySearch(boundaries, file.getMaximumTimestamp().orElse(Long.MAX_VALUE)); + int lowerWindowIndex = + Collections.binarySearch(boundaries, minTimestamp.orElse(Long.MAX_VALUE)); + int upperWindowIndex = + Collections.binarySearch(boundaries, file.getMaximumTimestamp().orElse(Long.MAX_VALUE)); // Handle boundary conditions and negative values of binarySearch lowerWindowIndex = (lowerWindowIndex < 0) ? Math.abs(lowerWindowIndex + 2) : lowerWindowIndex; upperWindowIndex = (upperWindowIndex < 0) ? Math.abs(upperWindowIndex + 2) : upperWindowIndex; if (lowerWindowIndex != upperWindowIndex) { - LOG.debug( - "Major compaction triggered on store " + this + "; because file " + file.getPath() + " has data with timestamps cross window boundaries"); + LOG.debug("Major compaction triggered on store " + this + "; because file " + + file.getPath() + " has data with timestamps cross window boundaries"); return true; } else if (filesInWindow[upperWindowIndex]) { - LOG.debug("Major compaction triggered on store " + this + "; because there are more than one file in some windows"); + LOG.debug("Major compaction triggered on store " + this + + "; because there are more than one file in some windows"); return true; } else { filesInWindow[upperWindowIndex] = true; } hdfsBlocksDistribution.add(file.getHDFSBlockDistribution()); } - if(checkBlockLocality(hdfsBlocksDistribution)) { + if (checkBlockLocality(hdfsBlocksDistribution)) { return true; } - LOG.debug("Skipping major compaction of " + this + ", because the files are already major compacted"); + LOG.debug( + "Skipping major compaction of " + this + ", because the files are already major compacted"); } return false; } @@ -316,7 +324,8 @@ private DateTieredCompactionRequest generateCompactionRequest(ArrayList getCompactBoundariesForMajor(Collection filesToCompact, long now) { + protected 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/DateTieredCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DateTieredCompactor.java index 5dacf63ab6ce..9cef2ebc3144 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,7 +24,6 @@ import java.util.function.Consumer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.InternalScanner; @@ -69,7 +68,8 @@ public List compact(final CompactionRequestImpl request, final List @Override public DateTieredMultiFileWriter createWriter(InternalScanner scanner, FileDetails fd, - boolean shouldDropBehind, boolean major, Consumer writerCreationTracker) throws IOException { + boolean shouldDropBehind, boolean major, Consumer writerCreationTracker) + throws IOException { DateTieredMultiFileWriter writer = createMultiWriter(request, lowerBoundaries, lowerBoundariesPolicies); initMultiWriter(writer, scanner, fd, shouldDropBehind, major, writerCreationTracker); @@ -80,8 +80,8 @@ public DateTieredMultiFileWriter createWriter(InternalScanner scanner, FileDetai protected DateTieredMultiFileWriter createMultiWriter(final CompactionRequestImpl request, final List lowerBoundaries, final Map lowerBoundariesPolicies) { - return new DateTieredMultiFileWriter(lowerBoundaries, - lowerBoundariesPolicies, needEmptyFile(request), c -> c.getTimestamp()); + return new DateTieredMultiFileWriter(lowerBoundaries, lowerBoundariesPolicies, + needEmptyFile(request), c -> c.getTimestamp()); } @Override diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java index f031a96d15f4..153ad50419b4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileInlineToRootChunkConversion.java @@ -22,6 +22,7 @@ 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.CellBuilderType; import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; import org.apache.hadoop.hbase.HBaseClassTestRule; @@ -85,7 +86,7 @@ public void testWriteHFile() throws Exception { hfw.append(ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(k) .setFamily(HConstants.EMPTY_BYTE_ARRAY).setQualifier(HConstants.EMPTY_BYTE_ARRAY) .setTimestamp(HConstants.LATEST_TIMESTAMP).setType(KeyValue.Type.Maximum.getCode()) - .setValue(v).build()); + .setValue(v).setType(Cell.Type.Put).build()); } hfw.close(); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java index 9bf8138933a0..454c05287907 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java @@ -108,7 +108,7 @@ public class TestCustomCellDataTieringManager { private static final long DAY = 24 * 60 * 60 * 1000; private static Configuration defaultConf; private static FileSystem fs; - private static BlockCache blockCache; + private BlockCache blockCache; private static CacheConfig cacheConf; private static Path testDir; private static final Map testOnlineRegions = new HashMap<>(); @@ -128,7 +128,7 @@ public static void setupBeforeClass() throws Exception { testDir = TEST_UTIL.getDataTestDir(TestCustomCellDataTieringManager.class.getSimpleName()); defaultConf = TEST_UTIL.getConfiguration(); updateCommonConfigurations(); - assertTrue(DataTieringManager.instantiate(defaultConf, testOnlineRegions)); + DataTieringManager.instantiate(defaultConf, testOnlineRegions); dataTieringManager = DataTieringManager.getInstance(); rowKeyString = ""; } @@ -235,7 +235,7 @@ public void testHotDataWithPath() throws IOException { @Test public void testPrefetchWhenDataTieringEnabled() throws IOException { setPrefetchBlocksOnOpen(); - initializeTestEnvironment(); + this.blockCache = initializeTestEnvironment(); // Evict blocks from cache by closing the files and passing evict on close. // Then initialize the reader again. Since Prefetch on open is set to true, it should prefetch // those blocks. @@ -285,27 +285,26 @@ public void testColdDataFiles() throws IOException { @Test public void testCacheCompactedBlocksOnWriteDataTieringDisabled() throws IOException { setCacheCompactBlocksOnWrite(); - initializeTestEnvironment(); - - HRegion region = createHRegion("table3"); + this.blockCache = initializeTestEnvironment(); + HRegion region = createHRegion("table3", this.blockCache); testCacheCompactedBlocksOnWrite(region, true); } @Test public void testCacheCompactedBlocksOnWriteWithHotData() throws IOException { setCacheCompactBlocksOnWrite(); - initializeTestEnvironment(); - - HRegion region = createHRegion("table3", getConfWithCustomCellDataTieringEnabled(5 * DAY)); + this.blockCache = initializeTestEnvironment(); + HRegion region = + createHRegion("table3", getConfWithCustomCellDataTieringEnabled(5 * DAY), this.blockCache); testCacheCompactedBlocksOnWrite(region, true); } @Test public void testCacheCompactedBlocksOnWriteWithColdData() throws IOException { setCacheCompactBlocksOnWrite(); - initializeTestEnvironment(); - - HRegion region = createHRegion("table3", getConfWithCustomCellDataTieringEnabled(DAY)); + this.blockCache = initializeTestEnvironment(); + HRegion region = + createHRegion("table3", getConfWithCustomCellDataTieringEnabled(DAY), this.blockCache); testCacheCompactedBlocksOnWrite(region, false); } @@ -588,7 +587,7 @@ public void testCacheConfigShouldCacheFile() throws Exception { @Test public void testCacheOnReadColdFile() throws Exception { - initializeTestEnvironment(); + this.blockCache = initializeTestEnvironment(); // hStoreFiles[3] is a cold file. the blocks should not get loaded after a readBlock call. HStoreFile hStoreFile = hStoreFiles.get(3); BlockCacheKey cacheKey = new BlockCacheKey(hStoreFile.getPath(), 0, true, BlockType.DATA); @@ -597,7 +596,7 @@ public void testCacheOnReadColdFile() throws Exception { @Test public void testCacheOnReadHotFile() throws Exception { - initializeTestEnvironment(); + this.blockCache = initializeTestEnvironment(); // hStoreFiles[0] is a hot file. the blocks should get loaded after a readBlock call. HStoreFile hStoreFile = hStoreFiles.get(0); BlockCacheKey cacheKey = @@ -605,7 +604,7 @@ public void testCacheOnReadHotFile() throws Exception { testCacheOnRead(hStoreFile, cacheKey, -1, true); } - private void testCacheOnRead(HStoreFile hStoreFile, BlockCacheKey key, long onDiskBlockSize, + private void testCacheOnRead(HStoreFile hStoreFile, BlockCacheKey key, long onDiskBlockSize, boolean expectedCached) throws Exception { // Execute the read block API which will try to cache the block if the block is a hot block. hStoreFile.getReader().getHFileReader().readBlock(key.getOffset(), onDiskBlockSize, true, false, @@ -633,7 +632,7 @@ private void validateBlocks(Set keys, int expectedTotalKeys, int numColdBlocks++; } } catch (Exception e) { - LOG.debug("Error validating priority for key {}",key, e); + LOG.debug("Error validating priority for key {}", key, e); fail(e.getMessage()); } } @@ -693,24 +692,26 @@ private void testDataTieringMethodWithKeyNoException(DataTieringMethodCallerWith testDataTieringMethodWithKey(caller, key, expectedResult, null); } - private static void initializeTestEnvironment() throws IOException { - setupFileSystemAndCache(); - setupOnlineRegions(); + private static BlockCache initializeTestEnvironment() throws IOException { + BlockCache blockCache = setupFileSystemAndCache(); + setupOnlineRegions(blockCache); + return blockCache; } - private static void setupFileSystemAndCache() throws IOException { + private static BlockCache setupFileSystemAndCache() throws IOException { fs = HFileSystem.get(defaultConf); - blockCache = BlockCacheFactory.createBlockCache(defaultConf); + BlockCache blockCache = BlockCacheFactory.createBlockCache(defaultConf); cacheConf = new CacheConfig(defaultConf, blockCache); + return blockCache; } - private static void setupOnlineRegions() throws IOException { + private static void setupOnlineRegions(BlockCache blockCache) throws IOException { testOnlineRegions.clear(); hStoreFiles.clear(); long day = 24 * 60 * 60 * 1000; long currentTime = System.currentTimeMillis(); - HRegion region1 = createHRegion("table1"); + HRegion region1 = createHRegion("table1", blockCache); HStore hStore11 = createHStore(region1, "cf1", getConfWithCustomCellDataTieringEnabled(day)); hStoreFiles.add(createHStoreFile(hStore11.getStoreContext().getFamilyStoreDirectoryPath(), @@ -724,18 +725,16 @@ private static void setupOnlineRegions() throws IOException { region1.stores.put(Bytes.toBytes("cf1"), hStore11); region1.stores.put(Bytes.toBytes("cf2"), hStore12); - HRegion region2 = - createHRegion("table2", getConfWithCustomCellDataTieringEnabled((long) (2.5 * day))); + HRegion region2 = createHRegion("table2", + getConfWithCustomCellDataTieringEnabled((long) (2.5 * day)), blockCache); HStore hStore21 = createHStore(region2, "cf1"); hStoreFiles.add(createHStoreFile(hStore21.getStoreContext().getFamilyStoreDirectoryPath(), - hStore21.getReadOnlyConfiguration(), currentTime - 2 * day, - region2.getRegionFileSystem())); + hStore21.getReadOnlyConfiguration(), currentTime - 2 * day, region2.getRegionFileSystem())); hStore21.refreshStoreFiles(); HStore hStore22 = createHStore(region2, "cf2"); hStoreFiles.add(createHStoreFile(hStore22.getStoreContext().getFamilyStoreDirectoryPath(), - hStore22.getReadOnlyConfiguration(), currentTime - 3 * day, - region2.getRegionFileSystem())); + hStore22.getReadOnlyConfiguration(), currentTime - 3 * day, region2.getRegionFileSystem())); hStore22.refreshStoreFiles(); region2.stores.put(Bytes.toBytes("cf1"), hStore21); @@ -749,11 +748,12 @@ private static void setupOnlineRegions() throws IOException { testOnlineRegions.put(region2.getRegionInfo().getEncodedName(), region2); } - private static HRegion createHRegion(String table) throws IOException { - return createHRegion(table, defaultConf); + private static HRegion createHRegion(String table, BlockCache blockCache) throws IOException { + return createHRegion(table, defaultConf, blockCache); } - private static HRegion createHRegion(String table, Configuration conf) throws IOException { + private static HRegion createHRegion(String table, Configuration conf, BlockCache blockCache) + throws IOException { TableName tableName = TableName.valueOf(table); TableDescriptor htd = TableDescriptorBuilder.newBuilder(tableName) @@ -792,7 +792,6 @@ private static HStore createHStore(HRegion region, String columnFamily, Configur return new HStore(region, columnFamilyDescriptor, conf, false); } - private static HStoreFile createHStoreFile(Path storeDir, Configuration conf, long timestamp, HRegionFileSystem regionFs) throws IOException { String columnFamily = storeDir.getName(); @@ -802,15 +801,15 @@ private static HStoreFile createHStoreFile(Path storeDir, Configuration conf, lo writeStoreFileRandomData(storeFileWriter, Bytes.toBytes(columnFamily), timestamp); - StoreContext storeContext = - StoreContext.getBuilder().withRegionFileSystem(regionFs).build(); + StoreContext storeContext = StoreContext.getBuilder().withRegionFileSystem(regionFs).build(); StoreFileTracker sft = StoreFileTrackerFactory.create(conf, true, storeContext); - return new HStoreFile(fs, storeFileWriter.getPath(), conf, cacheConf, BloomType.NONE, true, sft); + return new HStoreFile(fs, storeFileWriter.getPath(), conf, cacheConf, BloomType.NONE, true, + sft); } private static Configuration getConfWithCustomCellDataTieringEnabled(long hotDataAge) { Configuration conf = new Configuration(defaultConf); - conf.set(DataTieringManager.DATATIERING_KEY, DataTieringType.CUSTOM_CELL_VALUE.name()); + conf.set(DataTieringManager.DATATIERING_KEY, DataTieringType.CUSTOM.name()); conf.set(DataTieringManager.DATATIERING_HOT_DATA_AGE_KEY, String.valueOf(hotDataAge)); return conf; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java index 18100aca0502..eedbad83f2c2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java @@ -17,12 +17,13 @@ */ package org.apache.hadoop.hbase.regionserver; -import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.CUSTOM_TIERING_TIME_RANGE; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; + import java.io.IOException; import java.util.ArrayList; import java.util.UUID; @@ -33,7 +34,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; -import org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.CustomTieredCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerForTest; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -56,40 +57,40 @@ public class TestCustomCellTieredCompactionPolicy { public static final byte[] FAMILY = Bytes.toBytes("cf"); - private HStoreFile createFile(Path file, long minValue, long maxValue, long size, int seqId) throws IOException { + private HStoreFile createFile(Path file, long minValue, long maxValue, long size, int seqId) + throws IOException { return createFile(mockRegionInfo(), file, minValue, maxValue, size, seqId, 0); } - private HStoreFile createFile(RegionInfo regionInfo, Path file, long minValue, long maxValue, long size, int seqId, - long ageInDisk) throws IOException { + private HStoreFile createFile(RegionInfo regionInfo, Path file, long minValue, long maxValue, + long size, int seqId, long ageInDisk) throws IOException { FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); - HRegionFileSystem regionFileSystem = new HRegionFileSystem(TEST_UTIL.getConfiguration(),fs, - file, regionInfo); + HRegionFileSystem regionFileSystem = + new HRegionFileSystem(TEST_UTIL.getConfiguration(), fs, file, regionInfo); StoreContext ctx = new StoreContext.Builder() .withColumnFamilyDescriptor(ColumnFamilyDescriptorBuilder.newBuilder(FAMILY).build()) .withRegionFileSystem(regionFileSystem).build(); StoreFileTrackerForTest sftForTest = new StoreFileTrackerForTest(TEST_UTIL.getConfiguration(), true, ctx); MockHStoreFile msf = - new MockHStoreFile(TEST_UTIL, file, size, ageInDisk, false, (long) seqId, - sftForTest); + new MockHStoreFile(TEST_UTIL, file, size, ageInDisk, false, (long) seqId, sftForTest); TimeRangeTracker timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); timeRangeTracker.setMin(minValue); timeRangeTracker.setMax(maxValue); - msf.setMetadataValue(TIERING_CELL_TIME_RANGE, TimeRangeTracker.toByteArray(timeRangeTracker)); + msf.setMetadataValue(CUSTOM_TIERING_TIME_RANGE, TimeRangeTracker.toByteArray(timeRangeTracker)); return msf; } - private CustomCellTieredCompactionPolicy mockAndCreatePolicy() throws Exception { + private CustomTieredCompactionPolicy mockAndCreatePolicy() throws Exception { RegionInfo mockedRegionInfo = mockRegionInfo(); return mockAndCreatePolicy(mockedRegionInfo); } - private CustomCellTieredCompactionPolicy mockAndCreatePolicy(RegionInfo regionInfo) throws Exception { + private CustomTieredCompactionPolicy mockAndCreatePolicy(RegionInfo regionInfo) throws Exception { StoreConfigInformation mockedStoreConfig = mock(StoreConfigInformation.class); when(mockedStoreConfig.getRegionInfo()).thenReturn(regionInfo); - CustomCellTieredCompactionPolicy policy = - new CustomCellTieredCompactionPolicy(TEST_UTIL.getConfiguration(), mockedStoreConfig); + CustomTieredCompactionPolicy policy = + new CustomTieredCompactionPolicy(TEST_UTIL.getConfiguration(), mockedStoreConfig); return policy; } @@ -101,55 +102,58 @@ private RegionInfo mockRegionInfo() { private Path preparePath() throws Exception { FileSystem fs = FileSystem.get(TEST_UTIL.getConfiguration()); - Path file = new Path(TEST_UTIL.getDataTestDir(), - UUID.randomUUID().toString().replaceAll("-", "")); + Path file = + new Path(TEST_UTIL.getDataTestDir(), UUID.randomUUID().toString().replaceAll("-", "")); fs.create(file); - return file; + return file; } + @Test public void testGetCompactBoundariesForMajorNoOld() throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 0)); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 1)); - assertEquals(1, - ((DateTieredCompactionRequest)policy.selectMajorCompaction(files)).getBoundaries().size()); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 1)); + assertEquals(1, + ((DateTieredCompactionRequest) policy.selectMajorCompaction(files)).getBoundaries().size()); } @Test public void testGetCompactBoundariesForMajorAllOld() throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); - //The default cut off age is 10 years, so any of the min/max value there should get in the old tier + // The default cut off age is 10 years, so any of the min/max value there should get in the old + // tier files.add(createFile(file, 0, 1, 1024, 0)); files.add(createFile(file, 2, 3, 1024, 1)); assertEquals(2, - ((DateTieredCompactionRequest)policy.selectMajorCompaction(files)).getBoundaries().size()); + ((DateTieredCompactionRequest) policy.selectMajorCompaction(files)).getBoundaries().size()); } @Test public void testGetCompactBoundariesForMajorOneOnEachSide() throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, 1, 1024, 0)); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), 1024, 1)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 1)); assertEquals(3, - ((DateTieredCompactionRequest)policy.selectMajorCompaction(files)).getBoundaries().size()); + ((DateTieredCompactionRequest) policy.selectMajorCompaction(files)).getBoundaries().size()); } @Test public void testGetCompactBoundariesForMajorOneCrossing() throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, EnvironmentEdgeManager.currentTime(), 1024, 0)); assertEquals(3, - ((DateTieredCompactionRequest)policy.selectMajorCompaction(files)).getBoundaries().size()); + ((DateTieredCompactionRequest) policy.selectMajorCompaction(files)).getBoundaries().size()); } @FunctionalInterface @@ -158,15 +162,17 @@ interface PolicyValidator { } private void testShouldPerformMajorCompaction(long min, long max, int numFiles, - PolicyValidator> validation) throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + PolicyValidator> validation) + throws Exception { + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); RegionInfo mockedRegionInfo = mockRegionInfo(); Path file = preparePath(); ArrayList files = new ArrayList<>(); ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge(); EnvironmentEdgeManager.injectEdge(timeMachine); - for(int i=0; i assertTrue(p.shouldPerformMajorCompaction(f))); + (p, f) -> assertTrue(p.shouldPerformMajorCompaction(f))); } @Test public void testShouldPerformMajorCompactionOneFileMinMaxLow() throws Exception { testShouldPerformMajorCompaction(0, 1, 1, - (p,f) -> assertFalse(p.shouldPerformMajorCompaction(f))); + (p, f) -> assertFalse(p.shouldPerformMajorCompaction(f))); } @Test public void testShouldPerformMajorCompactionOneFileMinMaxHigh() throws Exception { long currentTime = EnvironmentEdgeManager.currentTime(); testShouldPerformMajorCompaction(currentTime, currentTime, 1, - (p,f) -> assertFalse(p.shouldPerformMajorCompaction(f))); + (p, f) -> assertFalse(p.shouldPerformMajorCompaction(f))); } @Test public void testShouldPerformMajorCompactionTwoFilesMinMaxHigh() throws Exception { long currentTime = EnvironmentEdgeManager.currentTime(); testShouldPerformMajorCompaction(currentTime, currentTime, 2, - (p,f) -> assertTrue(p.shouldPerformMajorCompaction(f))); + (p, f) -> assertTrue(p.shouldPerformMajorCompaction(f))); } @Test public void testSelectMinorCompactionTwoFilesNoOld() throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 0)); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 1)); - //Shouldn't do minor compaction, as minimum number of files + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 1)); + // Shouldn't do minor compaction, as minimum number of files // for minor compactions is 3 - assertEquals(0, - policy.selectMinorCompaction(files, true, true).getFiles().size()); + assertEquals(0, policy.selectMinorCompaction(files, true, true).getFiles().size()); } @Test public void testSelectMinorCompactionThreeFilesNoOld() throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 0)); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 1)); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 2)); - assertEquals(3, - policy.selectMinorCompaction(files, true, true).getFiles().size()); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 1)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 2)); + assertEquals(3, policy.selectMinorCompaction(files, true, true).getFiles().size()); } @Test public void testSelectMinorCompactionThreeFilesAllOld() throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); - files.add(createFile(file, 0, 1, - 1024, 0)); - files.add(createFile(file, 1, 2, - 1024, 1)); - files.add(createFile(file, 3, 4, - 1024, 2)); - assertEquals(3, - policy.selectMinorCompaction(files, true, true).getFiles().size()); + files.add(createFile(file, 0, 1, 1024, 0)); + files.add(createFile(file, 1, 2, 1024, 1)); + files.add(createFile(file, 3, 4, 1024, 2)); + assertEquals(3, policy.selectMinorCompaction(files, true, true).getFiles().size()); } @Test public void testSelectMinorCompactionThreeFilesOneOldTwoNew() throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); - files.add(createFile(file, 0, 1, - 1024, 0)); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 1)); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 2)); - assertEquals(3, - policy.selectMinorCompaction(files, true, true).getFiles().size()); + files.add(createFile(file, 0, 1, 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 1)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 2)); + assertEquals(3, policy.selectMinorCompaction(files, true, true).getFiles().size()); } @Test public void testSelectMinorCompactionThreeFilesTwoOldOneNew() throws Exception { - CustomCellTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); - files.add(createFile(file, 0, 1, - 1024, 0)); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 1)); - files.add(createFile(file, EnvironmentEdgeManager.currentTime(), EnvironmentEdgeManager.currentTime(), - 1024, 2)); - assertEquals(3, - policy.selectMinorCompaction(files, true, true).getFiles().size()); + files.add(createFile(file, 0, 1, 1024, 0)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 1)); + files.add(createFile(file, EnvironmentEdgeManager.currentTime(), + EnvironmentEdgeManager.currentTime(), 1024, 2)); + assertEquals(3, policy.selectMinorCompaction(files, true, true).getFiles().size()); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java index ebe027591eca..bf82a531f199 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDataTieringManager.java @@ -61,7 +61,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheTestUtils; import org.apache.hadoop.hbase.io.hfile.HFileBlock; import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; -import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl; import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTracker; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory; @@ -811,7 +810,7 @@ private static Configuration getConfWithTimeRangeDataTieringEnabled(long hotData } static HStoreFile createHStoreFile(Path storeDir, Configuration conf, long timestamp, - HRegionFileSystem regionFs) throws IOException { + HRegionFileSystem regionFs) throws IOException { String columnFamily = storeDir.getName(); StoreFileWriter storeFileWriter = new StoreFileWriter.Builder(conf, cacheConf, fs) @@ -819,11 +818,11 @@ static HStoreFile createHStoreFile(Path storeDir, Configuration conf, long times writeStoreFileRandomData(storeFileWriter, Bytes.toBytes(columnFamily), timestamp); - StoreContext storeContext = - StoreContext.getBuilder().withRegionFileSystem(regionFs).build(); + StoreContext storeContext = StoreContext.getBuilder().withRegionFileSystem(regionFs).build(); StoreFileTracker sft = StoreFileTrackerFactory.create(conf, true, storeContext); - return new HStoreFile(fs, storeFileWriter.getPath(), conf, cacheConf, BloomType.NONE, true, sft); + return new HStoreFile(fs, storeFileWriter.getPath(), conf, cacheConf, BloomType.NONE, true, + sft); } /** diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java index e44e09758858..253bdb43567f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestCustomCellTieredCompactor.java @@ -17,8 +17,16 @@ */ package org.apache.hadoop.hbase.regionserver.compactions; -import org.apache.hadoop.hbase.HBaseClassTestRule; +import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.CUSTOM_TIERING_TIME_RANGE; +import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieringValueProvider.TIERING_CELL_QUALIFIER; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.hbase.HBaseClassTestRule; import org.apache.hadoop.hbase.HBaseTestingUtil; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.Waiter; @@ -28,7 +36,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.hadoop.hbase.regionserver.CustomCellTieredStoreEngine; +import org.apache.hadoop.hbase.regionserver.CustomTieredStoreEngine; import org.apache.hadoop.hbase.regionserver.TimeRangeTracker; import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.SmallTests; @@ -38,14 +46,6 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import static org.apache.hadoop.hbase.regionserver.CustomTieringMultiFileWriter.TIERING_CELL_TIME_RANGE; -import static org.apache.hadoop.hbase.regionserver.compactions.CustomCellTieredCompactionPolicy.TIERING_CELL_QUALIFIER; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; @Category({ RegionServerTests.class, SmallTests.class }) public class TestCustomCellTieredCompactor { @@ -75,7 +75,7 @@ public void tearDown() throws Exception { @Test public void testCustomCellTieredCompactor() throws Exception { ColumnFamilyDescriptorBuilder clmBuilder = ColumnFamilyDescriptorBuilder.newBuilder(FAMILY); - clmBuilder.setValue("hbase.hstore.engine.class", CustomCellTieredStoreEngine.class.getName()); + clmBuilder.setValue("hbase.hstore.engine.class", CustomTieredStoreEngine.class.getName()); clmBuilder.setValue(TIERING_CELL_QUALIFIER, "date"); TableName tableName = TableName.valueOf("testCustomCellTieredCompactor"); TableDescriptorBuilder tblBuilder = TableDescriptorBuilder.newBuilder(tableName); @@ -91,12 +91,11 @@ public void testCustomCellTieredCompactor() throws Exception { Put put = new Put(Bytes.toBytes(i)); put.addColumn(FAMILY, Bytes.toBytes("val"), Bytes.toBytes("v" + i)); put.addColumn(FAMILY, Bytes.toBytes("date"), - Bytes.toBytes(recordTime - (11L*366L*24L*60L*60L*1000L))); + Bytes.toBytes(recordTime - (11L * 366L * 24L * 60L * 60L * 1000L))); puts.add(put); - put = new Put(Bytes.toBytes(i+1000)); + put = new Put(Bytes.toBytes(i + 1000)); put.addColumn(FAMILY, Bytes.toBytes("val"), Bytes.toBytes("v" + (i + 1000))); - put.addColumn(FAMILY, Bytes.toBytes("date"), - Bytes.toBytes(recordTime)); + put.addColumn(FAMILY, Bytes.toBytes("date"), Bytes.toBytes(recordTime)); puts.add(put); table.put(puts); utility.flush(tableName); @@ -105,38 +104,38 @@ public void testCustomCellTieredCompactor() throws Exception { long firstCompactionTime = System.currentTimeMillis(); utility.getAdmin().majorCompact(tableName); Waiter.waitFor(utility.getConfiguration(), 5000, - () -> utility.getMiniHBaseCluster().getMaster() - .getLastMajorCompactionTimestamp(tableName) > firstCompactionTime); + () -> utility.getMiniHBaseCluster().getMaster().getLastMajorCompactionTimestamp(tableName) + > firstCompactionTime); long numHFiles = utility.getNumHFiles(tableName, FAMILY); - //The first major compaction would have no means to detect more than one tier, + // The first major compaction would have no means to detect more than one tier, // because without the min/max values available in the file info portion of the selected files // for compaction, CustomCellDateTieredCompactionPolicy has no means // to calculate the proper boundaries. assertEquals(1, numHFiles); - utility.getMiniHBaseCluster().getRegions(tableName).get(0).getStore(FAMILY).getStorefiles().forEach( - file -> { - byte[] rangeBytes = file.getMetadataValue(TIERING_CELL_TIME_RANGE); + utility.getMiniHBaseCluster().getRegions(tableName).get(0).getStore(FAMILY).getStorefiles() + .forEach(file -> { + byte[] rangeBytes = file.getMetadataValue(CUSTOM_TIERING_TIME_RANGE); assertNotNull(rangeBytes); try { TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(rangeBytes); - assertEquals((recordTime - (11L*366L*24L*60L*60L*1000L)), timeRangeTracker.getMin()); + assertEquals((recordTime - (11L * 366L * 24L * 60L * 60L * 1000L)), + timeRangeTracker.getMin()); assertEquals(recordTime, timeRangeTracker.getMax()); } catch (IOException e) { fail(e.getMessage()); } - } - ); - //now do major compaction again, to make sure we write two separate files + }); + // now do major compaction again, to make sure we write two separate files long secondCompactionTime = System.currentTimeMillis(); utility.getAdmin().majorCompact(tableName); Waiter.waitFor(utility.getConfiguration(), 5000, - () -> utility.getMiniHBaseCluster().getMaster() - .getLastMajorCompactionTimestamp(tableName) > secondCompactionTime); + () -> utility.getMiniHBaseCluster().getMaster().getLastMajorCompactionTimestamp(tableName) + > secondCompactionTime); numHFiles = utility.getNumHFiles(tableName, FAMILY); assertEquals(2, numHFiles); - utility.getMiniHBaseCluster().getRegions(tableName).get(0).getStore(FAMILY).getStorefiles().forEach( - file -> { - byte[] rangeBytes = file.getMetadataValue(TIERING_CELL_TIME_RANGE); + utility.getMiniHBaseCluster().getRegions(tableName).get(0).getStore(FAMILY).getStorefiles() + .forEach(file -> { + byte[] rangeBytes = file.getMetadataValue(CUSTOM_TIERING_TIME_RANGE); assertNotNull(rangeBytes); try { TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(rangeBytes); @@ -144,7 +143,6 @@ public void testCustomCellTieredCompactor() throws Exception { } catch (IOException e) { fail(e.getMessage()); } - } - ); + }); } } From adb40a492c9717c4b07705f5152c3bc30392c4ec Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Fri, 27 Jun 2025 23:04:05 +0100 Subject: [PATCH 08/11] HBASE-28463 Rebase time based priority branch (HBASE-28463) with latest master (and fix conflicts) Change-Id: I0704c8f060d06c9dc7b43d8d7c81a3274d8fd333 --- .../java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java | 3 ++- .../java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java index 3fcf75b39709..72ca37c0557c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java @@ -282,7 +282,8 @@ public boolean shouldCacheBlockOnRead(BlockCategory category) { public boolean shouldCacheBlockOnRead(BlockCategory category, HFileInfo hFileInfo, Configuration conf) { Optional cacheFileBlock = Optional.of(true); - if (getBlockCache().isPresent()) { + // For DATA blocks only, if BuckeCache is in use, we don't need to cache block again + if (getBlockCache().isPresent() && category.equals(BlockCategory.DATA)) { Optional result = getBlockCache().get().shouldCacheFile(hFileInfo, conf); if (result.isPresent()) { cacheFileBlock = result; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index cea5c0361f43..7e3a415d7d7f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -921,7 +921,7 @@ public void appendCustomCellTimestampsToMetadata(TimeRangeTracker timeRangeTrack * to include the timestamp of this key */ private void trackTimestamps(final ExtendedCell cell) { - if (Cell.Type.Put == cell.getType()) { + if (KeyValue.Type.Put == KeyValue.Type.codeToType(cell.getTypeByte())) { earliestPutTs = Math.min(earliestPutTs, cell.getTimestamp()); } timeRangeTracker.includeTimestamp(cell); From 283320568104c69bbc0647b4d4b33ff22245f0f0 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Tue, 8 Jul 2025 16:03:59 +0100 Subject: [PATCH 09/11] addressing review comments Change-Id: I434e61aafd31a0c47f656e23eee5239aef7910f1 --- .../hbase/io/hfile/HFileWriterImpl.java | 10 +++--- .../regionserver/CustomTieredStoreEngine.java | 4 +-- .../CustomTieringMultiFileWriter.java | 2 +- .../CustomCellTieringValueProvider.java | 17 ---------- ... => CustomDateTieredCompactionPolicy.java} | 6 ++-- .../TestCustomCellTieredCompactionPolicy.java | 32 +++++++++---------- 6 files changed, 27 insertions(+), 44 deletions(-) rename hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/{CustomTieredCompactionPolicy.java => CustomDateTieredCompactionPolicy.java} (96%) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 7e3a415d7d7f..04bf3ff7b323 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -129,11 +129,11 @@ public class HFileWriterImpl implements HFile.Writer { /** Cache configuration for caching data on write. */ protected final CacheConfig cacheConf; - public void setTimeRangeToTrack(Supplier timeRangeToTrack) { - this.timeRangeToTrack = timeRangeToTrack; + public void setTimeRangeTrackerForTiering(Supplier timeRangeTrackerForTiering) { + this.timeRangeTrackerForTiering = timeRangeTrackerForTiering; } - private Supplier timeRangeToTrack; + private Supplier timeRangeTrackerForTiering; /** * Name for this object used when logging or in toString. Is either the result of a toString on @@ -196,7 +196,7 @@ public HFileWriterImpl(final Configuration conf, CacheConfig cacheConf, Path pat this.hFileContext = fileContext; // TODO: Move this back to upper layer this.timeRangeTracker = TimeRangeTracker.create(TimeRangeTracker.Type.NON_SYNC); - this.timeRangeToTrack = () -> this.timeRangeTracker; + this.timeRangeTrackerForTiering = () -> this.timeRangeTracker; DataBlockEncoding encoding = hFileContext.getDataBlockEncoding(); if (encoding != DataBlockEncoding.NONE) { this.blockEncoder = new HFileDataBlockEncoderImpl(encoding); @@ -598,7 +598,7 @@ private BlockCacheKey buildCacheBlockKey(long offset, BlockType blockType) { } private boolean shouldCacheBlock(BlockCache cache, BlockCacheKey key) { - Optional result = cache.shouldCacheBlock(key, timeRangeToTrack.get().getMax(), conf); + Optional result = cache.shouldCacheBlock(key, timeRangeTrackerForTiering.get().getMax(), conf); return result.orElse(true); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java index 8ba5a0d5cf1b..354a14c3bed7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.CompoundConfiguration; -import org.apache.hadoop.hbase.regionserver.compactions.CustomTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.CustomDateTieredCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.CustomTieredCompactor; import org.apache.yetus.audience.InterfaceAudience; @@ -44,7 +44,7 @@ protected void createComponents(Configuration conf, HStore store, CellComparator CompoundConfiguration config = new CompoundConfiguration(); config.add(conf); config.add(store.conf); - config.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, CustomTieredCompactionPolicy.class.getName()); + config.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, CustomDateTieredCompactionPolicy.class.getName()); createCompactionPolicy(config, store); this.storeFileManager = new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, config, compactionPolicy.getConf()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java index 905b542a1826..d2b88a501ec5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieringMultiFileWriter.java @@ -57,7 +57,7 @@ public void append(ExtendedCell cell) throws IOException { timeRangeTracker.setMax(tieringValue); lowerBoundary2TimeRanger.put(entry.getKey(), timeRangeTracker); ((HFileWriterImpl) lowerBoundary2Writer.get(entry.getKey()).getLiveFileWriter()) - .setTimeRangeToTrack(() -> timeRangeTracker); + .setTimeRangeTrackerForTiering(() -> timeRangeTracker); } else { TimeRangeTracker timeRangeTracker = entry.getValue(); if (timeRangeTracker.getMin() > tieringValue) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java index 0c9d212a03a2..7de9ba1851e5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java @@ -17,23 +17,6 @@ */ package org.apache.hadoop.hbase.regionserver.compactions; -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ import java.util.ArrayList; import java.util.Arrays; import java.util.List; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomDateTieredCompactionPolicy.java similarity index 96% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactionPolicy.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomDateTieredCompactionPolicy.java index 665f29fdc816..63a6d1c5b593 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomDateTieredCompactionPolicy.java @@ -44,7 +44,7 @@ * if files have both younger and older data than the configured age limit. */ @InterfaceAudience.Private -public class CustomTieredCompactionPolicy extends DateTieredCompactionPolicy { +public class CustomDateTieredCompactionPolicy extends DateTieredCompactionPolicy { public static final String AGE_LIMIT_MILLIS = "hbase.hstore.compaction.date.tiered.custom.age.limit.millis"; @@ -53,11 +53,11 @@ public class CustomTieredCompactionPolicy extends DateTieredCompactionPolicy { public static final long DEFAULT_AGE_LIMIT_MILLIS = (long) (10L * 365.25 * 24L * 60L * 60L * 1000L); - private static final Logger LOG = LoggerFactory.getLogger(CustomTieredCompactionPolicy.class); + private static final Logger LOG = LoggerFactory.getLogger(CustomDateTieredCompactionPolicy.class); private long cutOffTimestamp; - public CustomTieredCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) + public CustomDateTieredCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) throws IOException { super(conf, storeConfigInfo); cutOffTimestamp = EnvironmentEdgeManager.currentTime() diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java index eedbad83f2c2..b0c6de9195d0 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java @@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.RegionInfo; -import org.apache.hadoop.hbase.regionserver.compactions.CustomTieredCompactionPolicy; +import org.apache.hadoop.hbase.regionserver.compactions.CustomDateTieredCompactionPolicy; import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest; import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerForTest; import org.apache.hadoop.hbase.testclassification.RegionServerTests; @@ -81,16 +81,16 @@ private HStoreFile createFile(RegionInfo regionInfo, Path file, long minValue, l return msf; } - private CustomTieredCompactionPolicy mockAndCreatePolicy() throws Exception { + private CustomDateTieredCompactionPolicy mockAndCreatePolicy() throws Exception { RegionInfo mockedRegionInfo = mockRegionInfo(); return mockAndCreatePolicy(mockedRegionInfo); } - private CustomTieredCompactionPolicy mockAndCreatePolicy(RegionInfo regionInfo) throws Exception { + private CustomDateTieredCompactionPolicy mockAndCreatePolicy(RegionInfo regionInfo) throws Exception { StoreConfigInformation mockedStoreConfig = mock(StoreConfigInformation.class); when(mockedStoreConfig.getRegionInfo()).thenReturn(regionInfo); - CustomTieredCompactionPolicy policy = - new CustomTieredCompactionPolicy(TEST_UTIL.getConfiguration(), mockedStoreConfig); + CustomDateTieredCompactionPolicy policy = + new CustomDateTieredCompactionPolicy(TEST_UTIL.getConfiguration(), mockedStoreConfig); return policy; } @@ -110,7 +110,7 @@ private Path preparePath() throws Exception { @Test public void testGetCompactBoundariesForMajorNoOld() throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, EnvironmentEdgeManager.currentTime(), @@ -123,7 +123,7 @@ public void testGetCompactBoundariesForMajorNoOld() throws Exception { @Test public void testGetCompactBoundariesForMajorAllOld() throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); // The default cut off age is 10 years, so any of the min/max value there should get in the old @@ -136,7 +136,7 @@ public void testGetCompactBoundariesForMajorAllOld() throws Exception { @Test public void testGetCompactBoundariesForMajorOneOnEachSide() throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, 1, 1024, 0)); @@ -148,7 +148,7 @@ public void testGetCompactBoundariesForMajorOneOnEachSide() throws Exception { @Test public void testGetCompactBoundariesForMajorOneCrossing() throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, EnvironmentEdgeManager.currentTime(), 1024, 0)); @@ -162,9 +162,9 @@ interface PolicyValidator { } private void testShouldPerformMajorCompaction(long min, long max, int numFiles, - PolicyValidator> validation) + PolicyValidator> validation) throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); RegionInfo mockedRegionInfo = mockRegionInfo(); Path file = preparePath(); ArrayList files = new ArrayList<>(); @@ -209,7 +209,7 @@ public void testShouldPerformMajorCompactionTwoFilesMinMaxHigh() throws Exceptio @Test public void testSelectMinorCompactionTwoFilesNoOld() throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, EnvironmentEdgeManager.currentTime(), @@ -223,7 +223,7 @@ public void testSelectMinorCompactionTwoFilesNoOld() throws Exception { @Test public void testSelectMinorCompactionThreeFilesNoOld() throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, EnvironmentEdgeManager.currentTime(), @@ -237,7 +237,7 @@ public void testSelectMinorCompactionThreeFilesNoOld() throws Exception { @Test public void testSelectMinorCompactionThreeFilesAllOld() throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, 1, 1024, 0)); @@ -248,7 +248,7 @@ public void testSelectMinorCompactionThreeFilesAllOld() throws Exception { @Test public void testSelectMinorCompactionThreeFilesOneOldTwoNew() throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, 1, 1024, 0)); @@ -261,7 +261,7 @@ public void testSelectMinorCompactionThreeFilesOneOldTwoNew() throws Exception { @Test public void testSelectMinorCompactionThreeFilesTwoOldOneNew() throws Exception { - CustomTieredCompactionPolicy policy = mockAndCreatePolicy(); + CustomDateTieredCompactionPolicy policy = mockAndCreatePolicy(); Path file = preparePath(); ArrayList files = new ArrayList<>(); files.add(createFile(file, 0, 1, 1024, 0)); From 63a290a2593d5942642b6146bcfe807c23c01a71 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Wed, 9 Jul 2025 11:48:16 +0100 Subject: [PATCH 10/11] addressing another round of reviews and spotless failures Change-Id: I992f3eebe00943c4f073d1c7ea96010db21d07e6 --- .../apache/hadoop/hbase/io/hfile/HFileWriterImpl.java | 3 ++- .../hbase/regionserver/CustomTieredStoreEngine.java | 3 ++- .../compactions/CustomCellTieringValueProvider.java | 11 +++-------- .../compactions/CustomDateTieredCompactionPolicy.java | 4 ++-- .../TestCustomCellTieredCompactionPolicy.java | 3 ++- 5 files changed, 11 insertions(+), 13 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java index 04bf3ff7b323..684aee3beaca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java @@ -598,7 +598,8 @@ private BlockCacheKey buildCacheBlockKey(long offset, BlockType blockType) { } private boolean shouldCacheBlock(BlockCache cache, BlockCacheKey key) { - Optional result = cache.shouldCacheBlock(key, timeRangeTrackerForTiering.get().getMax(), conf); + Optional result = + cache.shouldCacheBlock(key, timeRangeTrackerForTiering.get().getMax(), conf); return result.orElse(true); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java index 354a14c3bed7..518b31fb5be4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CustomTieredStoreEngine.java @@ -44,7 +44,8 @@ protected void createComponents(Configuration conf, HStore store, CellComparator CompoundConfiguration config = new CompoundConfiguration(); config.add(conf); config.add(store.conf); - config.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, CustomDateTieredCompactionPolicy.class.getName()); + config.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, + CustomDateTieredCompactionPolicy.class.getName()); createCompactionPolicy(config, store); this.storeFileManager = new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, config, compactionPolicy.getConf()); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java index 7de9ba1851e5..fca76bae8f83 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieringValueProvider.java @@ -18,12 +18,12 @@ package org.apache.hadoop.hbase.regionserver.compactions; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.ExtendedCell; import org.apache.hadoop.hbase.PrivateCellUtil; import org.apache.hadoop.hbase.Tag; @@ -52,13 +52,8 @@ public List decorateCells(List cells) { byte[] tieringValue = null; // first iterates through the cells within a row, to find the tiering value for the row for (Cell cell : cells) { - byte[] qualifier = new byte[cell.getQualifierLength()]; - System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(), qualifier, 0, - cell.getQualifierLength()); - if (Arrays.equals(qualifier, tieringQualifier)) { - tieringValue = new byte[cell.getValueLength()]; - System.arraycopy(cell.getValueArray(), cell.getValueOffset(), tieringValue, 0, - cell.getValueLength()); + if (CellUtil.matchingQualifier(cell, tieringQualifier)) { + tieringValue = CellUtil.cloneValue(cell); break; } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomDateTieredCompactionPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomDateTieredCompactionPolicy.java index 63a6d1c5b593..dcc97c63d024 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomDateTieredCompactionPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomDateTieredCompactionPolicy.java @@ -57,8 +57,8 @@ public class CustomDateTieredCompactionPolicy extends DateTieredCompactionPolicy private long cutOffTimestamp; - public CustomDateTieredCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo) - throws IOException { + public CustomDateTieredCompactionPolicy(Configuration conf, + StoreConfigInformation storeConfigInfo) throws IOException { super(conf, storeConfigInfo); cutOffTimestamp = EnvironmentEdgeManager.currentTime() - conf.getLong(AGE_LIMIT_MILLIS, DEFAULT_AGE_LIMIT_MILLIS); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java index b0c6de9195d0..c89e99197179 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellTieredCompactionPolicy.java @@ -86,7 +86,8 @@ private CustomDateTieredCompactionPolicy mockAndCreatePolicy() throws Exception return mockAndCreatePolicy(mockedRegionInfo); } - private CustomDateTieredCompactionPolicy mockAndCreatePolicy(RegionInfo regionInfo) throws Exception { + private CustomDateTieredCompactionPolicy mockAndCreatePolicy(RegionInfo regionInfo) + throws Exception { StoreConfigInformation mockedStoreConfig = mock(StoreConfigInformation.class); when(mockedStoreConfig.getRegionInfo()).thenReturn(regionInfo); CustomDateTieredCompactionPolicy policy = From 7496a26e22411fba888f9cbde5ae81466c2d4337 Mon Sep 17 00:00:00 2001 From: Wellington Ramos Chevreuil Date: Thu, 10 Jul 2025 11:02:54 +0100 Subject: [PATCH 11/11] addressing javadoc, checkstyle and spotless issues Change-Id: I49d287cddf81170514474b27a642c249a7bc79d7 --- .../org/apache/hadoop/hbase/io/hfile/BlockCache.java | 6 +++--- .../compactions/CustomCellTieredUtils.java | 3 +++ .../TestCustomCellDataTieringManager.java | 11 ++++++----- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java index be3f8beb4547..9297e7074a97 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BlockCache.java @@ -213,9 +213,9 @@ default Optional shouldCacheFile(HFileInfo hFileInfo, Configuration con * not be overridden by all implementing classes. In such cases, the returned Optional will be * empty. For subclasses implementing this logic, the returned Optional would contain the boolean * value reflecting if the passed block should indeed be cached. - * @param key The key representing the block to check if it should be cached. - * @param timeRangeTracker the time range tracker containing the timestamps - * @param conf The configuration object to use for determining caching behavior. + * @param key The key representing the block to check if it should be cached. + * @param maxTimeStamp The maximum timestamp for the block to check if it should be cached. + * @param conf The configuration object to use for determining caching behavior. * @return An empty Optional if this method is not supported; otherwise, the returned Optional * contains the boolean value indicating if the block should be cached. */ diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java index 538cbda2f3fe..f908b31e4ae5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CustomCellTieredUtils.java @@ -28,6 +28,9 @@ @InterfaceAudience.Private public class CustomCellTieredUtils { + private CustomCellTieredUtils() { + // Utility class, no instantiation + } public static void checkForModifyTable(TableDescriptor newTable) throws IOException { for (ColumnFamilyDescriptor descriptor : newTable.getColumnFamilies()) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java index 454c05287907..b01717dfa1f8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCustomCellDataTieringManager.java @@ -19,7 +19,6 @@ import static org.apache.hadoop.hbase.HConstants.BUCKET_CACHE_SIZE_KEY; import static org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.DEFAULT_ERROR_TOLERATION_DURATION; -import static org.apache.hadoop.hbase.regionserver.TestDataTieringManager.createHStoreFile; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -81,12 +80,14 @@ * This class is used to test the functionality of the DataTieringManager. * * The mock online regions are stored in {@link TestCustomCellDataTieringManager#testOnlineRegions}. - * For all tests, the setup of {@link TestCustomCellDataTieringManager#testOnlineRegions} occurs only once. + * For all tests, the setup of + * {@link TestCustomCellDataTieringManager#testOnlineRegions} occurs only once. * Please refer to {@link TestCustomCellDataTieringManager#setupOnlineRegions()} for the structure. - * Additionally, a list of all store files is maintained in {@link TestCustomCellDataTieringManager#hStoreFiles}. + * Additionally, a list of all store files is + * maintained in {@link TestCustomCellDataTieringManager#hStoreFiles}. * The characteristics of these store files are listed below: - * @formatter:off ## HStoreFile Information - * + * @formatter:off + * ## HStoreFile Information * | HStoreFile | Region | Store | DataTiering | isHot | * |------------------|--------------------|---------------------|-----------------------|-------| * | hStoreFile0 | region1 | hStore11 | CUSTOM_CELL_VALUE | true |