HBASE-27241 Add metrics for evaluating cost and effectiveness of bloom filters (#4669)

Signed-off-by: Nick Dimiduk <ndimiduk@apache.org>
This commit is contained in:
Bryan Beaudreault 2022-08-09 15:18:55 -04:00 committed by GitHub
parent 5919b30b6d
commit bfa53f5609
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
21 changed files with 467 additions and 13 deletions

View File

@ -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.";

View File

@ -506,6 +506,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),

View File

@ -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.
*/

View File

@ -338,6 +338,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),

View File

@ -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);

View File

@ -117,6 +117,30 @@ public class MetricsTableWrapperStub implements MetricsTableWrapperAggregate {
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>();

View File

@ -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();
}
}

View File

@ -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.

View File

@ -235,7 +235,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

View File

@ -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();
}
}

View File

@ -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;
@ -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;
@ -227,8 +229,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 {
@ -443,7 +463,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 "
@ -455,7 +475,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);

View File

@ -99,6 +99,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;
@ -616,6 +619,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;
@ -778,6 +796,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;
@ -884,6 +905,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();
@ -984,6 +1008,9 @@ class MetricsRegionServerWrapperImpl implements MetricsRegionServerWrapper {
storefileIndexSize = tempStorefileIndexSize;
totalStaticIndexSize = tempTotalStaticIndexSize;
totalStaticBloomSize = tempTotalStaticBloomSize;
bloomFilterRequestsCount = tempBloomFilterRequestsCount;
bloomFilterNegativeResultsCount = tempBloomFilterNegativeResultsCount;
bloomFilterEligibleRequestsCount = tempBloomFilterEligibleRequestsCount;
numMutationsWithoutWAL = tempNumMutationsWithoutWAL;
dataInMemoryWithoutWAL = tempDataInMemoryWithoutWAL;
percentFileLocal = tempPercentFileLocal;

View File

@ -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) {
@ -303,6 +311,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));
@ -349,7 +407,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<String, Long> perStoreMemstoreOnlyReadCount = new HashMap<>();
Map<String, Long> perStoreMixedReadCount = new HashMap<>();

View File

@ -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();
}

View File

@ -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<SF extends StoreFlusher, CP extends Compaction
protected CP compactionPolicy;
protected C compactor;
protected SFM storeFileManager;
private final BloomFilterMetrics bloomFilterMetrics = new BloomFilterMetrics();
private Configuration conf;
private StoreContext ctx;
private RegionCoprocessorHost coprocessorHost;
@ -217,8 +220,8 @@ public abstract class StoreEngine<SF extends StoreFlusher, CP extends Compaction
public HStoreFile createStoreFileAndReader(StoreFileInfo info) throws IOException {
info.setRegionCoprocessorHost(coprocessorHost);
HStoreFile storeFile =
new HStoreFile(info, ctx.getFamily().getBloomFilterType(), ctx.getCacheConf());
HStoreFile storeFile = new HStoreFile(info, ctx.getFamily().getBloomFilterType(),
ctx.getCacheConf(), bloomFilterMetrics);
storeFile.initReader();
return storeFile;
}
@ -541,4 +544,8 @@ public abstract class StoreEngine<SF extends StoreFlusher, CP extends Compaction
ReadWriteLock getLock() {
return storeLock;
}
public BloomFilterMetrics getBloomFilterMetrics() {
return bloomFilterMetrics;
}
}

View File

@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.client.Scan;
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.HFileBlock;
@ -64,6 +65,7 @@ public class StoreFileReader {
protected BloomFilter generalBloomFilter = null;
protected BloomFilter deleteFamilyBloomFilter = null;
private BloomFilterMetrics bloomFilterMetrics = null;
protected BloomType bloomFilterType;
private final HFile.Reader reader;
protected long sequenceID = -1;
@ -261,6 +263,9 @@ public class StoreFileReader {
case ROWPREFIX_FIXED_LENGTH:
return passesGeneralRowPrefixBloomFilter(scan);
default:
if (scan.isGetScan()) {
bloomFilterMetrics.incrementEligible();
}
return true;
}
}
@ -300,6 +305,7 @@ public class StoreFileReader {
private boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) {
BloomFilter bloomFilter = this.generalBloomFilter;
if (bloomFilter == null) {
bloomFilterMetrics.incrementEligible();
return true;
}
@ -320,6 +326,7 @@ public class StoreFileReader {
public boolean passesGeneralRowColBloomFilter(Cell cell) {
BloomFilter bloomFilter = this.generalBloomFilter;
if (bloomFilter == null) {
bloomFilterMetrics.incrementEligible();
return true;
}
// Used in ROW_COL bloom
@ -341,6 +348,7 @@ public class StoreFileReader {
private boolean passesGeneralRowPrefixBloomFilter(Scan scan) {
BloomFilter bloomFilter = this.generalBloomFilter;
if (bloomFilter == null) {
bloomFilterMetrics.incrementEligible();
return true;
}
@ -491,12 +499,13 @@ public class StoreFileReader {
}
public void loadBloomfilter() {
this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
this.loadBloomfilter(BlockType.GENERAL_BLOOM_META, null);
this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META, null);
}
public void loadBloomfilter(BlockType blockType) {
public void loadBloomfilter(BlockType blockType, BloomFilterMetrics metrics) {
try {
this.bloomFilterMetrics = metrics;
if (blockType == BlockType.GENERAL_BLOOM_META) {
if (this.generalBloomFilter != null) return; // Bloom has been loaded
@ -506,7 +515,7 @@ public class StoreFileReader {
if (bloomFilterType == BloomType.NONE) {
throw new IOException("valid bloom filter type not found in FileInfo");
} else {
generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader, metrics);
if (LOG.isTraceEnabled()) {
LOG.trace("Loaded " + bloomFilterType.toString() + " "
+ generalBloomFilter.getClass().getSimpleName() + " metadata for "
@ -519,7 +528,9 @@ public class StoreFileReader {
DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
if (bloomMeta != null) {
deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
// don't pass in metrics for the delete family bloom for now since the
// goal is to give users insight into blooms _they_ configured.
deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader, null);
LOG.info(
"Loaded Delete Family Bloom (" + deleteFamilyBloomFilter.getClass().getSimpleName()
+ ") metadata for " + reader.getName());

View File

@ -21,6 +21,7 @@ import java.io.DataInput;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.io.hfile.BloomFilterMetrics;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterBase;
@ -85,10 +86,15 @@ public final class BloomFilterFactory {
*/
public static BloomFilter createFromMeta(DataInput meta, HFile.Reader reader)
throws IllegalArgumentException, IOException {
return createFromMeta(meta, reader, null);
}
public static BloomFilter createFromMeta(DataInput meta, HFile.Reader reader,
BloomFilterMetrics metrics) throws IllegalArgumentException, IOException {
int version = meta.readInt();
switch (version) {
case CompoundBloomFilterBase.VERSION:
return new CompoundBloomFilter(meta, reader);
return new CompoundBloomFilter(meta, reader, metrics);
default:
throw new IllegalArgumentException("Bad bloom filter format version " + version);

View File

@ -242,6 +242,21 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe
return 410;
}
@Override
public long getBloomFilterRequestsCount() {
return 411;
}
@Override
public long getBloomFilterNegativeResultsCount() {
return 412;
}
@Override
public long getBloomFilterEligibleRequestsCount() {
return 413;
}
@Override
public double getPercentFileLocal() {
return 99;

View File

@ -108,6 +108,7 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.BloomFilterUtil;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@ -311,6 +312,86 @@ public class TestHStore {
});
}
@Test
public void testStoreBloomFilterMetricsWithBloomRowCol() 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 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<byte[]> 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.

View File

@ -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);

View File

@ -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