From c0717c7383d3e3772c43c5777526d8afc033b4fa Mon Sep 17 00:00:00 2001 From: Bryan Beaudreault Date: Tue, 9 Aug 2022 15:18:55 -0400 Subject: [PATCH] HBASE-27241 Add metrics for evaluating cost and effectiveness of bloom filters (#4669) Signed-off-by: Nick Dimiduk --- .../MetricsRegionServerSource.java | 12 +++ .../MetricsRegionServerWrapper.java | 15 ++++ .../MetricsTableWrapperAggregate.java | 18 +++++ .../MetricsRegionServerSourceImpl.java | 7 ++ .../regionserver/MetricsTableSourceImpl.java | 22 +++++ .../regionserver/MetricsTableWrapperStub.java | 25 ++++++ .../hbase/io/hfile/BloomFilterMetrics.java | 65 +++++++++++++++ .../hbase/io/hfile/CompoundBloomFilter.java | 25 +++++- .../io/hfile/CompoundBloomFilterWriter.java | 2 +- .../hadoop/hbase/regionserver/HStore.java | 15 ++++ .../hadoop/hbase/regionserver/HStoreFile.java | 24 +++++- .../MetricsRegionServerWrapperImpl.java | 27 +++++++ .../MetricsTableWrapperAggregateImpl.java | 66 +++++++++++++++ .../hadoop/hbase/regionserver/Store.java | 12 +++ .../hbase/regionserver/StoreEngine.java | 11 ++- .../hbase/regionserver/StoreFileReader.java | 21 +++-- .../hadoop/hbase/util/BloomFilterFactory.java | 8 +- .../MetricsRegionServerWrapperStub.java | 15 ++++ .../hadoop/hbase/regionserver/TestHStore.java | 81 +++++++++++++++++++ .../regionserver/TestMetricsRegionServer.java | 3 + .../TestMetricsTableAggregate.java | 7 ++ 21 files changed, 468 insertions(+), 13 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BloomFilterMetrics.java diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java index 20c70697ee5..d89f512858e 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java @@ -281,6 +281,18 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo String STATIC_INDEX_SIZE_DESC = "Uncompressed size of the static indexes."; String STATIC_BLOOM_SIZE = "staticBloomSize"; String STATIC_BLOOM_SIZE_DESC = "Uncompressed size of the static bloom filters."; + + String BLOOM_FILTER_REQUESTS_COUNT = "bloomFilterRequestsCount"; + String BLOOM_FILTER_REQUESTS_COUNT_DESC = "Count of requests to bloom filters."; + + String BLOOM_FILTER_NEGATIVE_RESULTS_COUNT = "bloomFilterNegativeResultsCount"; + String BLOOM_FILTER_NEGATIVE_RESULTS_COUNT_DESC = + "Count of bloom filter requests which returned a negative result."; + + String BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT = "bloomFilterEligibleRequestsCount"; + String BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT_DESC = + "Count of requests which could have used bloom filters but didn't because they weren't configured or loaded"; + String NUMBER_OF_MUTATIONS_WITHOUT_WAL = "mutationsWithoutWALCount"; String NUMBER_OF_MUTATIONS_WITHOUT_WAL_DESC = "Number of mutations that have been sent by clients with the write ahead logging turned off."; diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java index 46de6742db6..e2a50cec892 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java @@ -188,6 +188,21 @@ public interface MetricsRegionServerWrapper { */ long getTotalStaticBloomSize(); + /** + * Count of bloom filter requests. + */ + long getBloomFilterRequestsCount(); + + /** + * Count of bloom filter requests which return a negative result. + */ + long getBloomFilterNegativeResultsCount(); + + /** + * Count of requests which could have used bloom filters, but they weren't configured or loaded. + */ + long getBloomFilterEligibleRequestsCount(); + /** * Number of mutations received with WAL explicitly turned off. */ diff --git a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregate.java b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregate.java index e18ba51b3c2..b64c01aa799 100644 --- a/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregate.java +++ b/hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregate.java @@ -97,6 +97,24 @@ public interface MetricsTableWrapperAggregate { /** Returns Average age of store files for this table */ long getAvgStoreFileAge(String table); + /** Returns the size of the static indexes for this table */ + long getStaticIndexSize(String table); + + /** Returns the size of the static blooms for this table */ + long getStaticBloomSize(String table); + + /** Returns count of bloom filter requests for this table. */ + long getBloomFilterRequestsCount(String table); + + /** Returns count of bloom filter requests which return a negative result for this table. */ + long getBloomFilterNegativeResultsCount(String table); + + /** + * Returns count of requests which could have used bloom filters for this table, but they weren't + * configured or loaded. + */ + long getBloomFilterEligibleRequestsCount(String table); + /** Returns Number of reference files for this table */ long getNumReferenceFiles(String table); diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java index 996573b2da9..bfb4bce506a 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java @@ -502,6 +502,13 @@ public class MetricsRegionServerSourceImpl extends BaseSourceImpl rsWrap.getTotalStaticIndexSize()) .addGauge(Interns.info(STATIC_BLOOM_SIZE, STATIC_BLOOM_SIZE_DESC), rsWrap.getTotalStaticBloomSize()) + .addCounter(Interns.info(BLOOM_FILTER_REQUESTS_COUNT, BLOOM_FILTER_REQUESTS_COUNT_DESC), + rsWrap.getBloomFilterRequestsCount()) + .addCounter( + Interns.info(BLOOM_FILTER_NEGATIVE_RESULTS_COUNT, BLOOM_FILTER_NEGATIVE_RESULTS_COUNT_DESC), + rsWrap.getBloomFilterNegativeResultsCount()) + .addCounter(Interns.info(BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT, + BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT_DESC), rsWrap.getBloomFilterEligibleRequestsCount()) .addGauge(Interns.info(NUMBER_OF_MUTATIONS_WITHOUT_WAL, NUMBER_OF_MUTATIONS_WITHOUT_WAL_DESC), rsWrap.getNumMutationsWithoutWAL()) .addGauge(Interns.info(DATA_SIZE_WITHOUT_WAL, DATA_SIZE_WITHOUT_WAL_DESC), diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableSourceImpl.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableSourceImpl.java index f7d0068bd81..83577228f6d 100644 --- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableSourceImpl.java +++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableSourceImpl.java @@ -334,6 +334,28 @@ public class MetricsTableSourceImpl implements MetricsTableSource { Interns.info(tableNamePrefix + MetricsRegionServerSource.AVG_STORE_FILE_AGE, MetricsRegionServerSource.AVG_STORE_FILE_AGE_DESC), tableWrapperAgg.getAvgStoreFileAge(tableName.getNameAsString())); + mrb.addGauge( + Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_BLOOM_SIZE, + MetricsRegionServerSource.STATIC_BLOOM_SIZE_DESC), + tableWrapperAgg.getStaticBloomSize(tableName.getNameAsString())); + mrb.addGauge( + Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_INDEX_SIZE, + MetricsRegionServerSource.STATIC_INDEX_SIZE), + tableWrapperAgg.getStaticIndexSize(tableName.getNameAsString())); + mrb.addCounter( + Interns.info(tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT, + MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT_DESC), + tableWrapperAgg.getBloomFilterRequestsCount(tableName.getNameAsString())); + mrb.addCounter( + Interns.info( + tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_NEGATIVE_RESULTS_COUNT, + MetricsRegionServerSource.BLOOM_FILTER_NEGATIVE_RESULTS_COUNT_DESC), + tableWrapperAgg.getBloomFilterNegativeResultsCount(tableName.getNameAsString())); + mrb.addCounter( + Interns.info( + tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT, + MetricsRegionServerSource.BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT_DESC), + tableWrapperAgg.getBloomFilterEligibleRequestsCount(tableName.getNameAsString())); mrb.addGauge( Interns.info(tableNamePrefix + MetricsRegionServerSource.NUM_REFERENCE_FILES, MetricsRegionServerSource.NUM_REFERENCE_FILES_DESC), diff --git a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java index c1d42718ef6..591bc9e63b5 100644 --- a/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java +++ b/hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java @@ -112,6 +112,31 @@ public class MetricsTableWrapperStub implements MetricsTableWrapperAggregate { return 88; } + @Override + public long getStaticIndexSize(String table) { + return 101; + } + + @Override + public long getStaticBloomSize(String table) { + return 111; + } + + @Override + public long getBloomFilterRequestsCount(String table) { + return 222; + } + + @Override + public long getBloomFilterNegativeResultsCount(String table) { + return 333; + } + + @Override + public long getBloomFilterEligibleRequestsCount(String table) { + return 444; + } + @Override public Map getMemstoreOnlyRowReadsCount(String table) { Map map = new HashMap(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BloomFilterMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BloomFilterMetrics.java new file mode 100644 index 00000000000..313493bc7c5 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/BloomFilterMetrics.java @@ -0,0 +1,65 @@ +/* + * 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.io.hfile; + +import java.util.concurrent.atomic.LongAdder; +import org.apache.yetus.audience.InterfaceAudience; + +@InterfaceAudience.Private +public class BloomFilterMetrics { + + private final LongAdder eligibleRequests = new LongAdder(); + private final LongAdder requests = new LongAdder(); + private final LongAdder negativeResults = new LongAdder(); + + /** + * Increment bloom request count, and negative result count if !passed + */ + public void incrementRequests(boolean passed) { + requests.increment(); + if (!passed) { + negativeResults.increment(); + } + } + + /** + * Increment for cases where bloom filter could have been used but wasn't defined or loaded. + */ + public void incrementEligible() { + eligibleRequests.increment(); + } + + /** Returns Current value for bloom requests count */ + public long getRequestsCount() { + return requests.sum(); + } + + /** Returns Current value for bloom negative results count */ + public long getNegativeResultsCount() { + return negativeResults.sum(); + } + + /** + * Returns Current value for requests which could have used bloom filters but wasn't defined or + * loaded. + */ + public long getEligibleRequestsCount() { + return eligibleRequests.sum(); + } + +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java index 0d39d09c969..95bc1c7b83d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java @@ -40,6 +40,7 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase implements Bloo /** Used to load chunks on demand */ private HFile.Reader reader; + private final BloomFilterMetrics metrics; private HFileBlockIndex.BlockIndexReader index; @@ -52,10 +53,14 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase implements Bloo /** * De-serialization for compound Bloom filter metadata. Must be consistent with what * {@link CompoundBloomFilterWriter} does. - * @param meta serialized Bloom filter metadata without any magic blocks n + * @param meta serialized Bloom filter metadata without any magic blocks + * @param reader reader for hfile + * @param metrics for collecting bloom filter metrics. may be null */ - public CompoundBloomFilter(DataInput meta, HFile.Reader reader) throws IOException { + public CompoundBloomFilter(DataInput meta, HFile.Reader reader, BloomFilterMetrics metrics) + throws IOException { this.reader = reader; + this.metrics = metrics; totalByteSize = meta.readLong(); hashCount = meta.readInt(); @@ -86,6 +91,14 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase implements Bloo @Override public boolean contains(byte[] key, int keyOffset, int keyLength, ByteBuff bloom) { + boolean result = containsInternal(key, keyOffset, keyLength, bloom); + if (metrics != null) { + metrics.incrementRequests(result); + } + return result; + } + + private boolean containsInternal(byte[] key, int keyOffset, int keyLength, ByteBuff bloom) { int block = index.rootBlockContainingKey(key, keyOffset, keyLength); if (block < 0) { return false; // This key is not in the file. @@ -127,6 +140,14 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase implements Bloo @Override public boolean contains(Cell keyCell, ByteBuff bloom, BloomType type) { + boolean result = containsInternal(keyCell, bloom, type); + if (metrics != null) { + metrics.incrementRequests(result); + } + return result; + } + + private boolean containsInternal(Cell keyCell, ByteBuff bloom, BloomType type) { int block = index.rootBlockContainingKey(keyCell); if (block < 0) { return false; // This key is not in the file. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java index 84d120a5568..ab7ac822a98 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java @@ -234,7 +234,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase * This is modeled after {@link CompoundBloomFilterWriter.MetaWriter} for simplicity, although * the two metadata formats do not have to be consistent. This does have to be consistent with * how - * {@link CompoundBloomFilter#CompoundBloomFilter(DataInput, org.apache.hadoop.hbase.io.hfile.HFile.Reader)} + * {@link CompoundBloomFilter#CompoundBloomFilter(DataInput, org.apache.hadoop.hbase.io.hfile.HFile.Reader, BloomFilterMetrics)} * reads fields. */ @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index 9357d7c35f2..b0dfa92336e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -2425,4 +2425,19 @@ public class HStore return storeFileWriterCreationTrackers.stream().flatMap(t -> t.get().stream()) .collect(Collectors.toSet()); } + + @Override + public long getBloomFilterRequestsCount() { + return storeEngine.getBloomFilterMetrics().getRequestsCount(); + } + + @Override + public long getBloomFilterNegativeResultsCount() { + return storeEngine.getBloomFilterMetrics().getNegativeResultsCount(); + } + + @Override + public long getBloomFilterEligibleRequestsCount() { + return storeEngine.getBloomFilterMetrics().getEligibleRequestsCount(); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index 02254818188..11685268480 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.io.hfile.BlockType; +import org.apache.hadoop.hbase.io.hfile.BloomFilterMetrics; import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.ReaderContext; @@ -128,6 +129,7 @@ public class HStoreFile implements StoreFile { // Block cache configuration and reference. private final CacheConfig cacheConf; + private final BloomFilterMetrics metrics; // Indicates if the file got compacted private volatile boolean compactedAway = false; @@ -225,8 +227,26 @@ public class HStoreFile implements StoreFile { * @param cacheConf The cache configuration and block cache reference. */ public HStoreFile(StoreFileInfo fileInfo, BloomType cfBloomType, CacheConfig cacheConf) { + this(fileInfo, cfBloomType, cacheConf, null); + } + + /** + * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram + * depending on the underlying files (10-20MB?). + * @param fileInfo The store file information. + * @param cfBloomType The bloom type to use for this store file as specified by column family + * configuration. This may or may not be the same as the Bloom filter type + * actually present in the HFile, because column family configuration might + * change. If this is {@link BloomType#NONE}, the existing Bloom filter is + * ignored. + * @param cacheConf The cache configuration and block cache reference. + * @param metrics Tracks bloom filter requests and results. May be null. + */ + public HStoreFile(StoreFileInfo fileInfo, BloomType cfBloomType, CacheConfig cacheConf, + BloomFilterMetrics metrics) { this.fileInfo = fileInfo; this.cacheConf = cacheConf; + this.metrics = metrics; if (BloomFilterFactory.isGeneralBloomEnabled(fileInfo.getConf())) { this.cfBloomType = cfBloomType; } else { @@ -447,7 +467,7 @@ public class HStoreFile implements StoreFile { BloomType hfileBloomType = initialReader.getBloomFilterType(); if (cfBloomType != BloomType.NONE) { - initialReader.loadBloomfilter(BlockType.GENERAL_BLOOM_META); + initialReader.loadBloomfilter(BlockType.GENERAL_BLOOM_META, metrics); if (hfileBloomType != cfBloomType) { LOG.debug("HFile Bloom filter type for " + initialReader.getHFileReader().getName() + ": " + hfileBloomType + ", but " + cfBloomType + " specified in column family " @@ -459,7 +479,7 @@ public class HStoreFile implements StoreFile { } // load delete family bloom filter - initialReader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META); + initialReader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META, metrics); try { byte[] data = metadataMap.get(TIMERANGE_KEY); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index e45c48ca72b..79ed5204ab2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -97,6 +97,9 @@ class MetricsRegionServerWrapperImpl implements MetricsRegionServerWrapper { private volatile long storefileIndexSize = 0; private volatile long totalStaticIndexSize = 0; private volatile long totalStaticBloomSize = 0; + private volatile long bloomFilterRequestsCount = 0; + private volatile long bloomFilterNegativeResultsCount = 0; + private volatile long bloomFilterEligibleRequestsCount = 0; private volatile long numMutationsWithoutWAL = 0; private volatile long dataInMemoryWithoutWAL = 0; private volatile double percentFileLocal = 0; @@ -604,6 +607,21 @@ class MetricsRegionServerWrapperImpl implements MetricsRegionServerWrapper { return totalStaticBloomSize; } + @Override + public long getBloomFilterRequestsCount() { + return bloomFilterRequestsCount; + } + + @Override + public long getBloomFilterNegativeResultsCount() { + return bloomFilterNegativeResultsCount; + } + + @Override + public long getBloomFilterEligibleRequestsCount() { + return bloomFilterEligibleRequestsCount; + } + @Override public long getNumMutationsWithoutWAL() { return numMutationsWithoutWAL; @@ -766,6 +784,9 @@ class MetricsRegionServerWrapperImpl implements MetricsRegionServerWrapper { long tempStorefileIndexSize = 0; long tempTotalStaticIndexSize = 0; long tempTotalStaticBloomSize = 0; + long tempBloomFilterRequestsCount = 0; + long tempBloomFilterNegativeResultsCount = 0; + long tempBloomFilterEligibleRequestsCount = 0; long tempNumMutationsWithoutWAL = 0; long tempDataInMemoryWithoutWAL = 0; double tempPercentFileLocal = 0; @@ -869,6 +890,9 @@ class MetricsRegionServerWrapperImpl implements MetricsRegionServerWrapper { tempStorefileIndexSize += store.getStorefilesRootLevelIndexSize(); tempTotalStaticBloomSize += store.getTotalStaticBloomSize(); tempTotalStaticIndexSize += store.getTotalStaticIndexSize(); + tempBloomFilterRequestsCount += store.getBloomFilterRequestsCount(); + tempBloomFilterNegativeResultsCount += store.getBloomFilterNegativeResultsCount(); + tempBloomFilterEligibleRequestsCount += store.getBloomFilterEligibleRequestsCount(); tempFlushedCellsCount += store.getFlushedCellsCount(); tempCompactedCellsCount += store.getCompactedCellsCount(); tempMajorCompactedCellsCount += store.getMajorCompactedCellsCount(); @@ -961,6 +985,9 @@ class MetricsRegionServerWrapperImpl implements MetricsRegionServerWrapper { storefileIndexSize = tempStorefileIndexSize; totalStaticIndexSize = tempTotalStaticIndexSize; totalStaticBloomSize = tempTotalStaticBloomSize; + bloomFilterRequestsCount = tempBloomFilterRequestsCount; + bloomFilterNegativeResultsCount = tempBloomFilterNegativeResultsCount; + bloomFilterEligibleRequestsCount = tempBloomFilterEligibleRequestsCount; numMutationsWithoutWAL = tempNumMutationsWithoutWAL; dataInMemoryWithoutWAL = tempDataInMemoryWithoutWAL; percentFileLocal = tempPercentFileLocal; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java index c5ea89c4a9f..b6ba1471f77 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java @@ -93,6 +93,14 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr (long) store.getAvgStoreFileAge().getAsDouble() * store.getStorefilesCount(); } mt.storeCount += 1; + + mt.staticIndexSize += store.getTotalStaticIndexSize(); + mt.staticBloomSize += store.getTotalStaticBloomSize(); + + mt.bloomRequestsCount += store.getBloomFilterRequestsCount(); + mt.bloomNegativeResultsCount += store.getBloomFilterNegativeResultsCount(); + mt.bloomEligibleRequestsCount += store.getBloomFilterEligibleRequestsCount(); + tempKey = tbl.getNameAsString() + HASH + familyName; Long tempVal = mt.perStoreMemstoreOnlyReadCount.get(tempKey); if (tempVal == null) { @@ -302,6 +310,56 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr : (metricsTable.totalStoreFileAge / metricsTable.storeFileCount); } + @Override + public long getStaticIndexSize(String table) { + MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table)); + if (metricsTable == null) { + return 0; + } + + return metricsTable.staticIndexSize; + } + + @Override + public long getStaticBloomSize(String table) { + MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table)); + if (metricsTable == null) { + return 0; + } + + return metricsTable.staticBloomSize; + } + + @Override + public long getBloomFilterRequestsCount(String table) { + MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table)); + if (metricsTable == null) { + return 0; + } + + return metricsTable.bloomRequestsCount; + } + + @Override + public long getBloomFilterNegativeResultsCount(String table) { + MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table)); + if (metricsTable == null) { + return 0; + } + + return metricsTable.bloomNegativeResultsCount; + } + + @Override + public long getBloomFilterEligibleRequestsCount(String table) { + MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table)); + if (metricsTable == null) { + return 0; + } + + return metricsTable.bloomEligibleRequestsCount; + } + @Override public long getNumReferenceFiles(String table) { MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table)); @@ -348,7 +406,15 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr long maxStoreFileAge; long minStoreFileAge = Long.MAX_VALUE; long totalStoreFileAge; + + long staticIndexSize; + + long staticBloomSize; long referenceFileCount; + + long bloomRequestsCount; + long bloomNegativeResultsCount; + long bloomEligibleRequestsCount; long cpRequestCount; Map perStoreMemstoreOnlyReadCount = new HashMap<>(); Map perStoreMixedReadCount = new HashMap<>(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index 5f9cfc09432..cd6676b563c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -240,4 +240,16 @@ public interface Store { * if you try to set a configuration. */ Configuration getReadOnlyConfiguration(); + + /** Returns count of bloom filter results for this store. */ + long getBloomFilterRequestsCount(); + + /** Returns count of negative results for bloom filter requests for this store. */ + long getBloomFilterNegativeResultsCount(); + + /** + * Returns count of requests which could have used bloom filters, but they weren't configured or + * loaded. + */ + long getBloomFilterEligibleRequestsCount(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java index 3edbf21bb82..f6e3db0116b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreEngine.java @@ -37,6 +37,7 @@ import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.io.hfile.BloomFilterMetrics; import org.apache.hadoop.hbase.log.HBaseMarkers; import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy; @@ -98,6 +99,8 @@ public abstract class StoreEngine= numStoreFiles); + assertTrue(store.getBloomFilterNegativeResultsCount() > 0); + } + + @Test + public void testStoreBloomFilterMetricsWithBloomRow() throws IOException { + int numStoreFiles = 5; + writeAndRead(BloomType.ROWCOL, numStoreFiles); + + assertEquals(0, store.getBloomFilterEligibleRequestsCount()); + // hard to know exactly the numbers here, we are just trying to + // prove that they are incrementing + assertTrue(store.getBloomFilterRequestsCount() >= numStoreFiles); + assertTrue(store.getBloomFilterNegativeResultsCount() > 0); + } + + @Test + public void testStoreBloomFilterMetricsWithBloomRowPrefix() throws IOException { + int numStoreFiles = 5; + writeAndRead(BloomType.ROWPREFIX_FIXED_LENGTH, numStoreFiles); + + assertEquals(0, store.getBloomFilterEligibleRequestsCount()); + // hard to know exactly the numbers here, we are just trying to + // prove that they are incrementing + assertTrue(store.getBloomFilterRequestsCount() >= numStoreFiles); + } + + @Test + public void testStoreBloomFilterMetricsWithBloomNone() throws IOException { + int numStoreFiles = 5; + writeAndRead(BloomType.NONE, numStoreFiles); + + assertEquals(0, store.getBloomFilterRequestsCount()); + assertEquals(0, store.getBloomFilterNegativeResultsCount()); + + // hard to know exactly the numbers here, we are just trying to + // prove that they are incrementing + assertTrue(store.getBloomFilterEligibleRequestsCount() >= numStoreFiles); + } + + private void writeAndRead(BloomType bloomType, int numStoreFiles) throws IOException { + Configuration conf = HBaseConfiguration.create(); + FileSystem fs = FileSystem.get(conf); + + ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(family) + .setCompressionType(Compression.Algorithm.GZ).setBloomFilterType(bloomType) + .setConfiguration(BloomFilterUtil.PREFIX_LENGTH_KEY, "3").build(); + init(name.getMethodName(), conf, hcd); + + for (int i = 1; i <= numStoreFiles; i++) { + byte[] row = Bytes.toBytes("row" + i); + LOG.info("Adding some data for the store file #" + i); + long timeStamp = EnvironmentEdgeManager.currentTime(); + this.store.add(new KeyValue(row, family, qf1, timeStamp, (byte[]) null), null); + this.store.add(new KeyValue(row, family, qf2, timeStamp, (byte[]) null), null); + this.store.add(new KeyValue(row, family, qf3, timeStamp, (byte[]) null), null); + flush(i); + } + + // Verify the total number of store files + assertEquals(numStoreFiles, this.store.getStorefiles().size()); + + TreeSet columns = new TreeSet<>(Bytes.BYTES_COMPARATOR); + columns.add(qf1); + + for (int i = 1; i <= numStoreFiles; i++) { + KeyValueScanner scanner = + store.getScanner(new Scan(new Get(Bytes.toBytes("row" + i))), columns, 0); + scanner.peek(); + } + } + /** * Verify that compression and data block encoding are respected by the createWriter method, used * on store flush. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java index ba971946ced..82d50900376 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionServer.java @@ -99,6 +99,9 @@ public class TestMetricsRegionServer { HELPER.assertGauge("staticBloomSize", 408, serverSource); HELPER.assertGauge("mutationsWithoutWALCount", 409, serverSource); HELPER.assertGauge("mutationsWithoutWALSize", 410, serverSource); + HELPER.assertCounter("bloomFilterRequestsCount", 411, serverSource); + HELPER.assertCounter("bloomFilterNegativeResultsCount", 412, serverSource); + HELPER.assertCounter("bloomFilterEligibleRequestsCount", 413, serverSource); HELPER.assertGauge("percentFilesLocal", 99, serverSource); HELPER.assertGauge("percentFilesLocalSecondaryRegions", 99, serverSource); HELPER.assertGauge("compactionQueueLength", 411, serverSource); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableAggregate.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableAggregate.java index 169f7c54c99..4fc4e76d860 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableAggregate.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableAggregate.java @@ -97,6 +97,13 @@ public class TestMetricsTableAggregate { HELPER.assertGauge(pre + "avgStoreFileAge", 66, agg); HELPER.assertGauge(pre + "numReferenceFiles", 77, agg); HELPER.assertGauge(pre + "averageRegionSize", 88, agg); + + HELPER.assertGauge(pre + "staticIndexSize", 101, agg); + HELPER.assertGauge(pre + "staticBloomSize", 111, agg); + + HELPER.assertCounter(pre + "bloomFilterRequestsCount", 222, agg); + HELPER.assertCounter(pre + "bloomFilterNegativeResultsCount", 333, agg); + HELPER.assertCounter(pre + "bloomFilterEligibleRequestsCount", 444, agg); } @Test