Skip to content

Commit

Permalink
HBASE-27241 Add metrics for evaluating cost and effectiveness of bloo…
Browse files Browse the repository at this point in the history
…m filters (#4669)

Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
  • Loading branch information
bbeaudreault authored Aug 9, 2022
1 parent 5919b30 commit bfa53f5
Show file tree
Hide file tree
Showing 21 changed files with 467 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -288,6 +288,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.";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -506,6 +506,13 @@ private MetricsRecordBuilder addGaugesToMetricsRecordBuilder(MetricsRecordBuilde
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),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -198,6 +198,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.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -338,6 +338,28 @@ void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
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),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,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);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,30 @@ public long getCpRequestsCount(String table) {
return 99;
}

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<String, Long> getMemstoreOnlyRowReadsCount(String table) {
Map<String, Long> map = new HashMap<String, Long>();
Expand Down
Original file line number Diff line number Diff line change
@@ -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();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -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();
Expand Down Expand Up @@ -86,6 +91,14 @@ public CompoundBloomFilter(DataInput meta, HFile.Reader reader) throws IOExcepti

@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.
Expand Down Expand Up @@ -127,6 +140,14 @@ private HFileBlock getBloomBlock(int block) {

@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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,7 @@ public void readFields(DataInput in) throws IOException {
* 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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2425,4 +2425,19 @@ public Set<Path> getStoreFilesBeingWritten() {
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();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
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;
Expand Down Expand Up @@ -130,6 +131,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;
Expand Down Expand Up @@ -227,8 +229,26 @@ public HStoreFile(FileSystem fs, Path p, Configuration conf, CacheConfig cacheCo
* @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 {
Expand Down Expand Up @@ -443,7 +463,7 @@ private void open() throws IOException {

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 "
Expand All @@ -455,7 +475,7 @@ private void open() throws IOException {
}

// 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);
Expand Down
Loading

0 comments on commit bfa53f5

Please sign in to comment.