-
Notifications
You must be signed in to change notification settings - Fork 3.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
HBASE-24205 Create metric to know the number of reads that happens fr… #1552
Changes from 14 commits
dbfc3d0
e43ed9c
002e7ec
9e44c4a
5844bdb
b9a04b0
41670b3
5c254a3
32c1a82
923f356
e481f99
d0f89d5
66d72cb
a46d74f
4252420
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,8 @@ | |
|
||
package org.apache.hadoop.hbase.regionserver; | ||
|
||
import java.util.Map; | ||
import java.util.Map.Entry; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
|
||
import org.apache.hadoop.hbase.metrics.Interns; | ||
|
@@ -33,6 +35,8 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource { | |
|
||
private static final Logger LOG = LoggerFactory.getLogger(MetricsRegionSourceImpl.class); | ||
|
||
private static final String _STORE = "_store_"; | ||
|
||
private AtomicBoolean closed = new AtomicBoolean(false); | ||
|
||
// Non-final so that we can null out the wrapper | ||
|
@@ -45,6 +49,8 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource { | |
private final DynamicMetricsRegistry registry; | ||
|
||
private final String regionNamePrefix; | ||
private final String regionNamePrefix1; | ||
private final String regionNamePrefix2; | ||
private final String regionPutKey; | ||
private final String regionDeleteKey; | ||
private final String regionGetKey; | ||
|
@@ -77,10 +83,11 @@ public MetricsRegionSourceImpl(MetricsRegionWrapper regionWrapper, | |
|
||
registry = agg.getMetricsRegistry(); | ||
|
||
regionNamePrefix = "Namespace_" + regionWrapper.getNamespace() + | ||
"_table_" + regionWrapper.getTableName() + | ||
"_region_" + regionWrapper.getRegionName() + | ||
"_metric_"; | ||
regionNamePrefix1 = "Namespace_" + regionWrapper.getNamespace() + | ||
"_table_" + regionWrapper.getTableName() + | ||
"_region_" + regionWrapper.getRegionName(); | ||
regionNamePrefix2 = "_metric_"; | ||
regionNamePrefix = regionNamePrefix1 + regionNamePrefix2; | ||
|
||
String suffix = "Count"; | ||
|
||
|
@@ -302,6 +309,24 @@ void snapshot(MetricsRecordBuilder mrb, boolean ignored) { | |
regionNamePrefix + MetricsRegionSource.MAX_FLUSH_QUEUE_SIZE, | ||
MetricsRegionSource.MAX_FLUSH_QUEUE_DESC), | ||
this.regionWrapper.getMaxFlushQueueSize()); | ||
addCounter(mrb, this.regionWrapper.getMemstoreOnlyRowReadsCount(), | ||
MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE, | ||
MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE_DESC); | ||
addCounter(mrb, this.regionWrapper.getMixedRowReadsCount(), | ||
MetricsRegionSource.MIXED_ROW_READS, | ||
MetricsRegionSource.MIXED_ROW_READS_ON_STORE_DESC); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We still need this count? We don't have it already with the general read count? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. My aversion to the extra counting is that we already do so much; it costs us loads in cpu. Was trying to do less if we can. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are asking in terms of CPU that we add on while collecting the metric? Ifyou see we do collect the metric at the HStore level per row when the StoreScanner completes a row process. That is now a longadder. Seems it is more performant than AtomicLong. Also the above change that we have done at the region level is nothing but just get that metric when that runnable thread keeps running. We don do any metric collection at this level. Are you still thinking it may be a problem. @saintstack ? BTW thanks for your review here. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
This is a read count across all stores. But now what we get additionally is per store how much is the read count that hit both memstore and files - also one more where we say how many rows per store came out of memstore only. Ideally the sum of these values per store should be equal to the total read count per region. |
||
} | ||
} | ||
|
||
private void addCounter(MetricsRecordBuilder mrb, Map<String, Long> metricMap, String metricName, | ||
String metricDesc) { | ||
if (metricMap != null) { | ||
for (Entry<String, Long> entry : metricMap.entrySet()) { | ||
// append 'store' and its name to the metric | ||
mrb.addCounter(Interns.info( | ||
this.regionNamePrefix1 + _STORE + entry.getKey() + this.regionNamePrefix2 + metricName, | ||
metricDesc), entry.getValue()); | ||
} | ||
} | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,8 @@ | |
|
||
package org.apache.hadoop.hbase.regionserver; | ||
|
||
import java.util.Map; | ||
|
||
import org.apache.yetus.audience.InterfaceAudience; | ||
|
||
/** | ||
|
@@ -170,4 +172,15 @@ public interface MetricsRegionWrapper { | |
* all compacted store files that belong to this region | ||
*/ | ||
long getMaxCompactedStoreFileRefCount(); | ||
|
||
/** | ||
* @return the number of row reads completely on memstore per store | ||
*/ | ||
Map<String, Long> getMemstoreOnlyRowReadsCount(); | ||
|
||
/** | ||
* @return the number of row reads on memstore and file per store | ||
*/ | ||
Map<String, Long> getMixedRowReadsCount(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, do we need to keep this count? It doesn't overlap w/ another? |
||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -61,7 +61,10 @@ | |
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_SUCCESS_DESC; | ||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_SUCCESS_KEY; | ||
|
||
import java.util.Map; | ||
import java.util.Map.Entry; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
|
||
import org.apache.hadoop.hbase.TableName; | ||
import org.apache.hadoop.hbase.metrics.Interns; | ||
import org.apache.hadoop.metrics2.MetricHistogram; | ||
|
@@ -75,6 +78,8 @@ | |
@InterfaceAudience.Private | ||
public class MetricsTableSourceImpl implements MetricsTableSource { | ||
|
||
private static final String _STORE = "_store_"; | ||
|
||
private static final Logger LOG = LoggerFactory.getLogger(MetricsTableSourceImpl.class); | ||
|
||
private AtomicBoolean closed = new AtomicBoolean(false); | ||
|
@@ -87,6 +92,8 @@ public class MetricsTableSourceImpl implements MetricsTableSource { | |
private final MetricsTableAggregateSourceImpl agg; | ||
private final DynamicMetricsRegistry registry; | ||
private final String tableNamePrefix; | ||
private final String tableNamePrefixPart1; | ||
private final String tableNamePrefixPart2; | ||
private final TableName tableName; | ||
private final int hashCode; | ||
|
||
|
@@ -127,8 +134,11 @@ public MetricsTableSourceImpl(String tblName, | |
|
||
this.tableWrapperAgg = tblWrapperAgg; | ||
this.registry = agg.getMetricsRegistry(); | ||
this.tableNamePrefix = "Namespace_" + this.tableName.getNamespaceAsString() + | ||
"_table_" + this.tableName.getQualifierAsString() + "_metric_"; | ||
this.tableNamePrefixPart1 = "Namespace_" + this.tableName.getNamespaceAsString() + | ||
"_table_" + this.tableName.getQualifierAsString(); | ||
this.tableNamePrefixPart2 = "_metric_"; | ||
this.tableNamePrefix = tableNamePrefixPart1 + | ||
tableNamePrefixPart2; | ||
this.hashCode = this.tableName.hashCode(); | ||
} | ||
|
||
|
@@ -311,6 +321,25 @@ void snapshot(MetricsRecordBuilder mrb, boolean ignored) { | |
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.NUM_REFERENCE_FILES, | ||
MetricsRegionServerSource.NUM_REFERENCE_FILES_DESC), | ||
tableWrapperAgg.getNumReferenceFiles(tableName.getNameAsString())); | ||
addGauge(mrb, tableWrapperAgg.getMemstoreOnlyRowReadsCount(tableName.getNameAsString()), | ||
MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE, | ||
MetricsRegionSource.ROW_READS_ONLY_ON_MEMSTORE_DESC); | ||
addGauge(mrb, tableWrapperAgg.getMixedRowReadsCount(tableName.getNameAsString()), | ||
MetricsRegionSource.MIXED_ROW_READS, | ||
MetricsRegionSource.MIXED_ROW_READS_ON_STORE_DESC); | ||
} | ||
} | ||
} | ||
|
||
private void addGauge(MetricsRecordBuilder mrb, Map<String, Long> metricMap, String metricName, | ||
String metricDesc) { | ||
if (metricMap != null) { | ||
for (Entry<String, Long> entry : metricMap.entrySet()) { | ||
// append 'store' and its name to the metric | ||
mrb.addGauge(Interns.info(this.tableNamePrefixPart1 + _STORE | ||
+ entry.getKey().split(MetricsTableWrapperAggregate.UNDERSCORE)[1] | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here the key will be <regionName>_<storeName>? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You mean the key should be Columnfamily? So _STORE - that should be _ColumnFamily? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Store is an instance of CF. So here what we have is an aggregated value across all instances (Stores) of a given CF in a table. So the name CF make sense than STORE? WDYT There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This looks like it was addressed. |
||
+ this.tableNamePrefixPart2 + metricName, | ||
metricDesc), entry.getValue()); | ||
} | ||
} | ||
} | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This 'metric' addition to name was of no value?