saintstack commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r440935577



##########
File path: 
hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
##########
@@ -302,6 +308,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);

Review comment:
       We still need this count? We don't have it already with the general read 
count?

##########
File path: 
hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableSourceImpl.java
##########
@@ -311,6 +322,27 @@ 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.getMemstoreReadRequestsCount(tableName.getNameAsString()),
+          MetricsRegionSource.READ_REQUEST_ON_MEMSTORE,
+          MetricsRegionSource.READ_REQUEST_ON_MEMSTORE_DESC);
+        addGauge(mrb, 
tableWrapperAgg.getMixedRequestsCount(tableName.getNameAsString()),
+          MetricsRegionSource.MIXED_READ_REQUEST_ON_STORE,
+          MetricsRegionSource.MIXED_READ_REQUEST_ON_STORE_DESC);
+      }
+    }
+  }
+
+  private void addGauge(MetricsRecordBuilder mrb, Map<String, Long> metricMap, 
String metricName,
+      String metricDesc) {
+    if (metricMap != null) {
+      Iterator<Entry<String, Long>> iterator = metricMap.entrySet().iterator();
+      while (iterator.hasNext()) {
+        Entry<String, Long> entry = iterator.next();
+        // append 'store' and its name to the metric
+        mrb.addGauge(Interns.info(this.tableNamePrefixPart1 + _STORE
+            + entry.getKey().split(MetricsTableWrapperAggregate.UNDERSCORE)[1]

Review comment:
       This looks like it was addressed.

##########
File path: 
hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
##########
@@ -77,10 +83,10 @@ 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_";

Review comment:
       This '_metric_' addition to name was of no value?

##########
File path: 
hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
##########
@@ -170,4 +172,15 @@
    *   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();

Review comment:
       Yeah, do we need to keep this count? It doesn't overlap w/ another?

##########
File path: 
hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
##########
@@ -302,6 +308,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);

Review comment:
       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.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to