Skip to content

Commit c0717c7

Browse files
committed
HBASE-27241 Add metrics for evaluating cost and effectiveness of bloom filters (#4669)
Signed-off-by: Nick Dimiduk <[email protected]>
1 parent 187c084 commit c0717c7

File tree

21 files changed

+468
-13
lines changed

21 files changed

+468
-13
lines changed

hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -281,6 +281,18 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo
281281
String STATIC_INDEX_SIZE_DESC = "Uncompressed size of the static indexes.";
282282
String STATIC_BLOOM_SIZE = "staticBloomSize";
283283
String STATIC_BLOOM_SIZE_DESC = "Uncompressed size of the static bloom filters.";
284+
285+
String BLOOM_FILTER_REQUESTS_COUNT = "bloomFilterRequestsCount";
286+
String BLOOM_FILTER_REQUESTS_COUNT_DESC = "Count of requests to bloom filters.";
287+
288+
String BLOOM_FILTER_NEGATIVE_RESULTS_COUNT = "bloomFilterNegativeResultsCount";
289+
String BLOOM_FILTER_NEGATIVE_RESULTS_COUNT_DESC =
290+
"Count of bloom filter requests which returned a negative result.";
291+
292+
String BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT = "bloomFilterEligibleRequestsCount";
293+
String BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT_DESC =
294+
"Count of requests which could have used bloom filters but didn't because they weren't configured or loaded";
295+
284296
String NUMBER_OF_MUTATIONS_WITHOUT_WAL = "mutationsWithoutWALCount";
285297
String NUMBER_OF_MUTATIONS_WITHOUT_WAL_DESC =
286298
"Number of mutations that have been sent by clients with the write ahead logging turned off.";

hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapper.java

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -188,6 +188,21 @@ public interface MetricsRegionServerWrapper {
188188
*/
189189
long getTotalStaticBloomSize();
190190

191+
/**
192+
* Count of bloom filter requests.
193+
*/
194+
long getBloomFilterRequestsCount();
195+
196+
/**
197+
* Count of bloom filter requests which return a negative result.
198+
*/
199+
long getBloomFilterNegativeResultsCount();
200+
201+
/**
202+
* Count of requests which could have used bloom filters, but they weren't configured or loaded.
203+
*/
204+
long getBloomFilterEligibleRequestsCount();
205+
191206
/**
192207
* Number of mutations received with WAL explicitly turned off.
193208
*/

hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregate.java

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -97,6 +97,24 @@ public interface MetricsTableWrapperAggregate {
9797
/** Returns Average age of store files for this table */
9898
long getAvgStoreFileAge(String table);
9999

100+
/** Returns the size of the static indexes for this table */
101+
long getStaticIndexSize(String table);
102+
103+
/** Returns the size of the static blooms for this table */
104+
long getStaticBloomSize(String table);
105+
106+
/** Returns count of bloom filter requests for this table. */
107+
long getBloomFilterRequestsCount(String table);
108+
109+
/** Returns count of bloom filter requests which return a negative result for this table. */
110+
long getBloomFilterNegativeResultsCount(String table);
111+
112+
/**
113+
* Returns count of requests which could have used bloom filters for this table, but they weren't
114+
* configured or loaded.
115+
*/
116+
long getBloomFilterEligibleRequestsCount(String table);
117+
100118
/** Returns Number of reference files for this table */
101119
long getNumReferenceFiles(String table);
102120

hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceImpl.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -502,6 +502,13 @@ private MetricsRecordBuilder addGaugesToMetricsRecordBuilder(MetricsRecordBuilde
502502
rsWrap.getTotalStaticIndexSize())
503503
.addGauge(Interns.info(STATIC_BLOOM_SIZE, STATIC_BLOOM_SIZE_DESC),
504504
rsWrap.getTotalStaticBloomSize())
505+
.addCounter(Interns.info(BLOOM_FILTER_REQUESTS_COUNT, BLOOM_FILTER_REQUESTS_COUNT_DESC),
506+
rsWrap.getBloomFilterRequestsCount())
507+
.addCounter(
508+
Interns.info(BLOOM_FILTER_NEGATIVE_RESULTS_COUNT, BLOOM_FILTER_NEGATIVE_RESULTS_COUNT_DESC),
509+
rsWrap.getBloomFilterNegativeResultsCount())
510+
.addCounter(Interns.info(BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT,
511+
BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT_DESC), rsWrap.getBloomFilterEligibleRequestsCount())
505512
.addGauge(Interns.info(NUMBER_OF_MUTATIONS_WITHOUT_WAL, NUMBER_OF_MUTATIONS_WITHOUT_WAL_DESC),
506513
rsWrap.getNumMutationsWithoutWAL())
507514
.addGauge(Interns.info(DATA_SIZE_WITHOUT_WAL, DATA_SIZE_WITHOUT_WAL_DESC),

hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableSourceImpl.java

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -334,6 +334,28 @@ void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
334334
Interns.info(tableNamePrefix + MetricsRegionServerSource.AVG_STORE_FILE_AGE,
335335
MetricsRegionServerSource.AVG_STORE_FILE_AGE_DESC),
336336
tableWrapperAgg.getAvgStoreFileAge(tableName.getNameAsString()));
337+
mrb.addGauge(
338+
Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_BLOOM_SIZE,
339+
MetricsRegionServerSource.STATIC_BLOOM_SIZE_DESC),
340+
tableWrapperAgg.getStaticBloomSize(tableName.getNameAsString()));
341+
mrb.addGauge(
342+
Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_INDEX_SIZE,
343+
MetricsRegionServerSource.STATIC_INDEX_SIZE),
344+
tableWrapperAgg.getStaticIndexSize(tableName.getNameAsString()));
345+
mrb.addCounter(
346+
Interns.info(tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT,
347+
MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT_DESC),
348+
tableWrapperAgg.getBloomFilterRequestsCount(tableName.getNameAsString()));
349+
mrb.addCounter(
350+
Interns.info(
351+
tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_NEGATIVE_RESULTS_COUNT,
352+
MetricsRegionServerSource.BLOOM_FILTER_NEGATIVE_RESULTS_COUNT_DESC),
353+
tableWrapperAgg.getBloomFilterNegativeResultsCount(tableName.getNameAsString()));
354+
mrb.addCounter(
355+
Interns.info(
356+
tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT,
357+
MetricsRegionServerSource.BLOOM_FILTER_ELIGIBLE_REQUESTS_COUNT_DESC),
358+
tableWrapperAgg.getBloomFilterEligibleRequestsCount(tableName.getNameAsString()));
337359
mrb.addGauge(
338360
Interns.info(tableNamePrefix + MetricsRegionServerSource.NUM_REFERENCE_FILES,
339361
MetricsRegionServerSource.NUM_REFERENCE_FILES_DESC),

hbase-hadoop2-compat/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java

Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -112,6 +112,31 @@ public long getAvgRegionSize(String table) {
112112
return 88;
113113
}
114114

115+
@Override
116+
public long getStaticIndexSize(String table) {
117+
return 101;
118+
}
119+
120+
@Override
121+
public long getStaticBloomSize(String table) {
122+
return 111;
123+
}
124+
125+
@Override
126+
public long getBloomFilterRequestsCount(String table) {
127+
return 222;
128+
}
129+
130+
@Override
131+
public long getBloomFilterNegativeResultsCount(String table) {
132+
return 333;
133+
}
134+
135+
@Override
136+
public long getBloomFilterEligibleRequestsCount(String table) {
137+
return 444;
138+
}
139+
115140
@Override
116141
public Map<String, Long> getMemstoreOnlyRowReadsCount(String table) {
117142
Map<String, Long> map = new HashMap<String, Long>();
Lines changed: 65 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,65 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hbase.io.hfile;
19+
20+
import java.util.concurrent.atomic.LongAdder;
21+
import org.apache.yetus.audience.InterfaceAudience;
22+
23+
@InterfaceAudience.Private
24+
public class BloomFilterMetrics {
25+
26+
private final LongAdder eligibleRequests = new LongAdder();
27+
private final LongAdder requests = new LongAdder();
28+
private final LongAdder negativeResults = new LongAdder();
29+
30+
/**
31+
* Increment bloom request count, and negative result count if !passed
32+
*/
33+
public void incrementRequests(boolean passed) {
34+
requests.increment();
35+
if (!passed) {
36+
negativeResults.increment();
37+
}
38+
}
39+
40+
/**
41+
* Increment for cases where bloom filter could have been used but wasn't defined or loaded.
42+
*/
43+
public void incrementEligible() {
44+
eligibleRequests.increment();
45+
}
46+
47+
/** Returns Current value for bloom requests count */
48+
public long getRequestsCount() {
49+
return requests.sum();
50+
}
51+
52+
/** Returns Current value for bloom negative results count */
53+
public long getNegativeResultsCount() {
54+
return negativeResults.sum();
55+
}
56+
57+
/**
58+
* Returns Current value for requests which could have used bloom filters but wasn't defined or
59+
* loaded.
60+
*/
61+
public long getEligibleRequestsCount() {
62+
return eligibleRequests.sum();
63+
}
64+
65+
}

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilter.java

Lines changed: 23 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,7 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase implements Bloo
4040

4141
/** Used to load chunks on demand */
4242
private HFile.Reader reader;
43+
private final BloomFilterMetrics metrics;
4344

4445
private HFileBlockIndex.BlockIndexReader index;
4546

@@ -52,10 +53,14 @@ public class CompoundBloomFilter extends CompoundBloomFilterBase implements Bloo
5253
/**
5354
* De-serialization for compound Bloom filter metadata. Must be consistent with what
5455
* {@link CompoundBloomFilterWriter} does.
55-
* @param meta serialized Bloom filter metadata without any magic blocks n
56+
* @param meta serialized Bloom filter metadata without any magic blocks
57+
* @param reader reader for hfile
58+
* @param metrics for collecting bloom filter metrics. may be null
5659
*/
57-
public CompoundBloomFilter(DataInput meta, HFile.Reader reader) throws IOException {
60+
public CompoundBloomFilter(DataInput meta, HFile.Reader reader, BloomFilterMetrics metrics)
61+
throws IOException {
5862
this.reader = reader;
63+
this.metrics = metrics;
5964

6065
totalByteSize = meta.readLong();
6166
hashCount = meta.readInt();
@@ -86,6 +91,14 @@ public CompoundBloomFilter(DataInput meta, HFile.Reader reader) throws IOExcepti
8691

8792
@Override
8893
public boolean contains(byte[] key, int keyOffset, int keyLength, ByteBuff bloom) {
94+
boolean result = containsInternal(key, keyOffset, keyLength, bloom);
95+
if (metrics != null) {
96+
metrics.incrementRequests(result);
97+
}
98+
return result;
99+
}
100+
101+
private boolean containsInternal(byte[] key, int keyOffset, int keyLength, ByteBuff bloom) {
89102
int block = index.rootBlockContainingKey(key, keyOffset, keyLength);
90103
if (block < 0) {
91104
return false; // This key is not in the file.
@@ -127,6 +140,14 @@ private HFileBlock getBloomBlock(int block) {
127140

128141
@Override
129142
public boolean contains(Cell keyCell, ByteBuff bloom, BloomType type) {
143+
boolean result = containsInternal(keyCell, bloom, type);
144+
if (metrics != null) {
145+
metrics.incrementRequests(result);
146+
}
147+
return result;
148+
}
149+
150+
private boolean containsInternal(Cell keyCell, ByteBuff bloom, BloomType type) {
130151
int block = index.rootBlockContainingKey(keyCell);
131152
if (block < 0) {
132153
return false; // This key is not in the file.

hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CompoundBloomFilterWriter.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -234,7 +234,7 @@ public void readFields(DataInput in) throws IOException {
234234
* This is modeled after {@link CompoundBloomFilterWriter.MetaWriter} for simplicity, although
235235
* the two metadata formats do not have to be consistent. This does have to be consistent with
236236
* how
237-
* {@link CompoundBloomFilter#CompoundBloomFilter(DataInput, org.apache.hadoop.hbase.io.hfile.HFile.Reader)}
237+
* {@link CompoundBloomFilter#CompoundBloomFilter(DataInput, org.apache.hadoop.hbase.io.hfile.HFile.Reader, BloomFilterMetrics)}
238238
* reads fields.
239239
*/
240240
@Override

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2425,4 +2425,19 @@ Set<Path> getStoreFilesBeingWritten() {
24252425
return storeFileWriterCreationTrackers.stream().flatMap(t -> t.get().stream())
24262426
.collect(Collectors.toSet());
24272427
}
2428+
2429+
@Override
2430+
public long getBloomFilterRequestsCount() {
2431+
return storeEngine.getBloomFilterMetrics().getRequestsCount();
2432+
}
2433+
2434+
@Override
2435+
public long getBloomFilterNegativeResultsCount() {
2436+
return storeEngine.getBloomFilterMetrics().getNegativeResultsCount();
2437+
}
2438+
2439+
@Override
2440+
public long getBloomFilterEligibleRequestsCount() {
2441+
return storeEngine.getBloomFilterMetrics().getEligibleRequestsCount();
2442+
}
24282443
}

0 commit comments

Comments
 (0)