You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/04/20 13:31:33 UTC

[GitHub] [hbase] ramkrish86 opened a new pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

ramkrish86 opened a new pull request #1552:
URL: https://github.com/apache/hbase/pull/1552


   Initial PR to get the metrics that indicates the gets that happens on the memstore. We just try to get the count at the row level and not at the cell level.
   Only StoreScanner has details about how the cell or row was retrieved. Even if one of the cell is retrieved from the Memstore we will account the read to have used the memstore indicating it was trying to access latest data. We can add metric for scan too later. 
   Another thing to note is that the real time counter on the Store level will use MetricStore which will use the Counters with hbase-metric-api and not the hadoop's counters. 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r434376478



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSource.java
##########
@@ -0,0 +1,60 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface MetricsStoreAggregateSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Stores";
+
+  /**
+   * The name of the metrics context that metrics will be under.
+   */
+  String METRICS_CONTEXT = "regionserver";
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about Stores under a region";
+
+  /**
+   * The name of the metrics context that metrics will be under in jmx
+   */
+  String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;

Review comment:
       And seeing ur other comment -  this is where yo uwill get per region under each store I have how many requests. If I want to remove this then I may have to do something like how it has been done for Table vs Store - we will have region vs store (and it will be at the region level only). 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r439611922



##########
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:
       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




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r414469707



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -2898,4 +2915,40 @@ public int getMaxCompactedStoreFileRefCount() {
       ? maxCompactedStoreFileRefCount.getAsInt() : 0;
   }
 
+  @Override
+  public long getReadRequestsFromStoreCount() {
+    return getRequestsFromStore.sum();
+  }
+
+  @Override
+  public long getGetRequestsCountFromMemstore() {
+    return getRequestsFromMemstore.sum();
+  }
+
+  @Override
+  public long getGetRequestsCountFromFile() {
+    return getRequestsFromFile.sum();
+  }
+
+  void incrGetRequestsFromStore() {
+    getRequestsFromStore.increment();
+    if (metricsStore != null) {
+      metricsStore.updateGet();
+    }
+  }
+
+  void updateMetricsStore(boolean memstoreRead) {
+    if (memstoreRead) {
+      getRequestsFromMemstore.increment();
+    } else {
+      getRequestsFromFile.increment();
+    }
+    if (metricsStore != null) {
+      if (memstoreRead) {
+        metricsStore.updateMemstoreGet();
+      }
+      metricsStore.updateFileGet();

Review comment:
       good catch. Thanks for the reviews. 
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-643267556


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 12s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 24s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 45s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in master failed.  |
   | -0 :warning: |  patch  |   7m  9s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 42s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 36s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 127m  5s |  hbase-server in the patch passed.  |
   |  |   | 155m 58s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux fe0828d027ab 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 6b7a93d10c |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/testReport/ |
   | Max. process+thread count | 4740 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-644739795


   Committing the patch today. Thanks for all the reviews @anoopsjohn, @saintstack  and @virajjasani .
   Got to learn about the metrics system here by doing this. 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r428596325



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreSourceImpl.java
##########
@@ -0,0 +1,211 @@
+/**
+ * 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.regionserver;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Interns;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreSourceImpl implements MetricsStoreSource {
+
+  private MetricsStoreWrapper storeWrapper;
+  private MetricsStoreAggregateSourceImpl aggreagate;
+  private AtomicBoolean closed = new AtomicBoolean(false);
+
+  private String storeNamePrefix;
+  private final MetricRegistry registry;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreSourceImpl.class);
+  String storeReadsKey;
+
+  String memstoreReadsKey;
+  String fileReadsKey;
+  private final Counter storeReads;
+  private final Counter memstoreReads;
+  private final Counter fileReads;
+
+  public MetricsStoreSourceImpl(MetricsStoreWrapper storeWrapper,
+      MetricsStoreAggregateSourceImpl aggreagate) {
+    this.storeWrapper = storeWrapper;
+    this.aggreagate = aggreagate;
+    aggreagate.register(this);
+
+    LOG.debug("Creating new MetricsRegionSourceImpl for table " + storeWrapper.getStoreName() + " "

Review comment:
       Pls correct log  
   for store storeWrapper.getRegionName() + " : " + storeWrapper.getStoreName() 

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreSourceImpl.java
##########
@@ -0,0 +1,211 @@
+/**
+ * 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.regionserver;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Interns;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreSourceImpl implements MetricsStoreSource {
+
+  private MetricsStoreWrapper storeWrapper;
+  private MetricsStoreAggregateSourceImpl aggreagate;
+  private AtomicBoolean closed = new AtomicBoolean(false);
+
+  private String storeNamePrefix;
+  private final MetricRegistry registry;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreSourceImpl.class);
+  String storeReadsKey;
+
+  String memstoreReadsKey;
+  String fileReadsKey;
+  private final Counter storeReads;
+  private final Counter memstoreReads;
+  private final Counter fileReads;

Review comment:
       Yes here.. No need to fileReads.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
##########
@@ -402,6 +402,8 @@
   String DELETE_BATCH_KEY = "deleteBatch";
   String GET_SIZE_KEY = "getSize";
   String GET_KEY = "get";
+  String MEMSTORE_GET_KEY = "getsOnMemstore";
+  String FILE_GET_KEY = "getsOnFile";

Review comment:
       #getsOnMemstore + #getsOnFile will be #get write?  Why we should keep both metric then?  We can keep only getsOnMemstore

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -2547,7 +2565,7 @@ public CacheConfig getCacheConfig() {
   }
 
   public static final long FIXED_OVERHEAD =
-      ClassSize.align(ClassSize.OBJECT + (27 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG)
+      ClassSize.align(ClassSize.OBJECT + (31 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG)

Review comment:
       Actually here added 5 more refs but considering 4 only. And infact 4 is enough when u keep getRequestsFromMemstore  and getRequestsFromStore .
   getRequestsFromStore => getRequests..  This is store anyways so 'FromStore' is implicit.

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -2884,4 +2901,41 @@ public int getMaxCompactedStoreFileRefCount() {
       ? maxCompactedStoreFileRefCount.getAsInt() : 0;
   }
 
+  @Override
+  public long getReadRequestsFromStoreCount() {
+    return getRequestsFromStore.sum();
+  }
+
+  @Override
+  public long getGetRequestsCountFromMemstore() {
+    return getRequestsFromMemstore.sum();
+  }
+
+  @Override
+  public long getGetRequestsCountFromFile() {
+    return getRequestsFromFile.sum();
+  }
+
+  void incrGetRequestsFromStore() {
+    getRequestsFromStore.increment();

Review comment:
       Actually we need to keep this counter here as well as update the counter on Metrics also?  The local Counter is used by?

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
##########
@@ -302,6 +302,14 @@ void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
               regionNamePrefix + MetricsRegionSource.MAX_FLUSH_QUEUE_SIZE,
               MetricsRegionSource.MAX_FLUSH_QUEUE_DESC),
           this.regionWrapper.getMaxFlushQueueSize());
+      mrb.addCounter(

Review comment:
       We are collecting the metric at every store level and then aggregating over region also?  Do we need that? A region might have say 3 CFs out of which only one is getting recent data gets. Aggregate at region level for these 3 CFs, how that can be used?  IMO we can avoid this.  And keep aggregate only at the RS level for table:cf

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregate.java
##########
@@ -107,6 +109,13 @@
    */
   long getNumReferenceFiles(String table);
 
+  /**
+   * @return number of get requests on memstore for this table
+   */
+  Map<String, Long> getMemstoreReadRequestCount(String table);
 
-
+  /**
+   * @return number of get requests from file for this table
+   */
+  Map<String, Long> getFileRequestCount(String table);

Review comment:
       Same here..  All places we can keep totalReads metric and memstoreReads. That should be enough to understand the pattern.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableSourceImpl.java
##########
@@ -311,6 +320,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.getMemstoreReadRequestCount(tableName.getNameAsString()),

Review comment:
       Here we are not adding aggregate of total gets on table:cf?

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreSourceImpl.java
##########
@@ -0,0 +1,211 @@
+/**
+ * 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.regionserver;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Interns;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreSourceImpl implements MetricsStoreSource {
+
+  private MetricsStoreWrapper storeWrapper;
+  private MetricsStoreAggregateSourceImpl aggreagate;
+  private AtomicBoolean closed = new AtomicBoolean(false);
+
+  private String storeNamePrefix;
+  private final MetricRegistry registry;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreSourceImpl.class);
+  String storeReadsKey;
+
+  String memstoreReadsKey;
+  String fileReadsKey;
+  private final Counter storeReads;
+  private final Counter memstoreReads;
+  private final Counter fileReads;
+
+  public MetricsStoreSourceImpl(MetricsStoreWrapper storeWrapper,
+      MetricsStoreAggregateSourceImpl aggreagate) {
+    this.storeWrapper = storeWrapper;
+    this.aggreagate = aggreagate;
+    aggreagate.register(this);
+
+    LOG.debug("Creating new MetricsRegionSourceImpl for table " + storeWrapper.getStoreName() + " "
+        + storeWrapper.getRegionName());
+
+    // we are using the hbase-metrics API
+    registry = aggreagate.getMetricRegistry();
+
+    storeNamePrefix = "Namespace_" + storeWrapper.getNamespace() + "_table_"
+        + storeWrapper.getTableName() + "_region_" + storeWrapper.getRegionName() + "_store_"
+        + storeWrapper.getStoreName() + "_metric_";
+
+    String suffix = "Count";
+
+    storeReadsKey = storeNamePrefix + MetricsRegionServerSource.GET_KEY + suffix;
+    // all the counters are hbase-metrics API
+    storeReads = registry.counter(storeReadsKey);
+
+    memstoreReadsKey = storeNamePrefix + MetricsRegionServerSource.MEMSTORE_GET_KEY + suffix;
+    memstoreReads = registry.counter(memstoreReadsKey);
+
+    fileReadsKey = storeNamePrefix + MetricsRegionServerSource.FILE_GET_KEY + suffix;
+    fileReads = registry.counter(fileReadsKey);
+
+  }
+
+  @Override
+  public void close() {
+    boolean wasClosed = closed.getAndSet(true);
+
+    // Has someone else already closed this for us?
+    if (wasClosed) {
+      return;
+    }
+
+    // Before removing the metrics remove this region from the aggregate region bean.
+    // This should mean that it's unlikely that snapshot and close happen at the same time.
+    aggreagate.deregister(this);
+
+    // While it's un-likely that snapshot and close happen at the same time it's still possible.
+    // So grab the lock to ensure that all calls to snapshot are done before we remove the metrics
+    synchronized (this) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Removing store Metrics: " + storeWrapper.getStoreName());
+      }
+
+      registry.remove(storeReadsKey);
+      registry.remove(memstoreReadsKey);
+      registry.remove(fileReadsKey);
+
+      storeWrapper = null;
+    }
+  }
+
+  @Override
+  public int compareTo(MetricsStoreSource source) {
+    if (!(source instanceof MetricsStoreSourceImpl)) {
+      return -1;
+    }
+
+    MetricsStoreSourceImpl impl = (MetricsStoreSourceImpl) source;
+    if (impl == null) {
+      return -1;
+    }
+
+    // TODO : make this better
+    return Long.compare(this.storeWrapper.getStoreName().hashCode(),
+      impl.storeWrapper.getStoreName().hashCode());
+  }
+
+  @Override
+  public void updateGet() {
+    storeReads.increment();
+  }
+
+  @Override
+  public void updateMemtoreGet() {
+    memstoreReads.increment();
+  }
+
+  @Override
+  public void updateFileGet() {
+    fileReads.increment();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return obj == this
+        || (obj instanceof MetricsStoreSourceImpl && compareTo((MetricsStoreSourceImpl) obj) == 0);
+  }
+
+  @Override
+  public int hashCode() {
+    return this.storeWrapper.getStoreName().hashCode();
+  }
+
+  void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
+
+    // If there is a close that started be double extra sure
+    // that we're not getting any locks and not putting data
+    // into the metrics that should be removed. So early out
+    // before even getting the lock.
+    if (closed.get()) {
+      return;
+    }
+
+    // Grab the read
+    // This ensures that removes of the metrics
+    // can't happen while we are putting them back in.
+    synchronized (this) {
+
+      // It's possible that a close happened between checking
+      // the closed variable and getting the lock.
+      if (closed.get()) {
+        return;
+      }
+      mrb.addGauge(Interns.info(storeNamePrefix + MetricsRegionServerSource.STOREFILE_COUNT,

Review comment:
       Why all these storeFile related metric comes here as part of this?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java
##########
@@ -70,25 +69,36 @@ public void run() {
           localMetricsTableMap.put(tbl, mt);
         }
         if (r.getStores() != null) {
+          long memstoreReadCount = 0l;
+          long fileReadCount = 0l;
+          String familyName = null;
           for (Store store : r.getStores()) {
+            familyName = store.getColumnFamilyName();
+
             mt.storeFileCount += store.getStorefilesCount();
-            mt.memstoreSize += (store.getMemStoreSize().getDataSize() +
-              store.getMemStoreSize().getHeapSize() + store.getMemStoreSize().getOffHeapSize());
+            mt.memstoreSize += (store.getMemStoreSize().getDataSize()
+                + store.getMemStoreSize().getHeapSize() + store.getMemStoreSize().getOffHeapSize());
             mt.storeFileSize += store.getStorefilesSize();
             mt.referenceFileCount += store.getNumReferenceFiles();
 
-            mt.maxStoreFileAge = Math.max(mt.maxStoreFileAge, store.getMaxStoreFileAge().getAsLong());
-            mt.minStoreFileAge = Math.min(mt.minStoreFileAge, store.getMinStoreFileAge().getAsLong());
-            mt.totalStoreFileAge = (long)store.getAvgStoreFileAge().getAsDouble() *
-                store.getStorefilesCount();
+            mt.maxStoreFileAge =
+                Math.max(mt.maxStoreFileAge, store.getMaxStoreFileAge().getAsLong());
+            mt.minStoreFileAge =
+                Math.min(mt.minStoreFileAge, store.getMinStoreFileAge().getAsLong());
+            mt.totalStoreFileAge =
+                (long) store.getAvgStoreFileAge().getAsDouble() * store.getStorefilesCount();
             mt.storeCount += 1;
+            memstoreReadCount += store.getGetRequestsCountFromMemstore();
+            fileReadCount += store.getGetRequestsCountFromFile();
+            mt.storeMemstoreGetCount.putIfAbsent(familyName, memstoreReadCount);
+            mt.storeFileGetCount.putIfAbsent(familyName, fileReadCount);
           }
+
           mt.regionCount += 1;
 
           mt.readRequestCount += r.getReadRequestsCount();
-          mt.filteredReadRequestCount += getFilteredReadRequestCount(tbl.getNameAsString());
+          mt.filteredReadRequestCount += r.getFilteredReadRequestsCount();

Review comment:
       Why this change? If any bug fix, pls keep it as another Jira

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -1002,6 +1012,14 @@ public Void call() throws IOException {
     } finally {
       this.lock.writeLock().unlock();
       this.archiveLock.unlock();
+      // moving it after the unlocking so
+      // that metrics closure does not affect them
+      if (this.metricsStore != null) {
+        metricsStore.close();

Review comment:
       This will do all the deregister stuff?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-638265477


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 18s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 25s |  master passed  |
   | -0 :warning: |  patch  |   2m  9s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 20s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 13s |  hbase-hadoop-compat: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m  5s |  hbase-server: The patch generated 3 new + 36 unchanged - 1 fixed = 39 total (was 37)  |
   | +1 :green_heart: |  whitespace  |   0m  1s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  11m  1s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 45s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 25s |  The patch does not generate ASF License warnings.  |
   |  |   |  34m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 50e6f90d250f 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8de8c44029 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop-compat.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r441335976



##########
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 don't have it already with the general read count?
   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 some of thees values per store should be equal to the total read count per region. 

##########
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 don't have it already with the general read count?
   
   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 some of thees values per store should be equal to the total read count per region. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r434369134



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSource.java
##########
@@ -0,0 +1,60 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface MetricsStoreAggregateSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Stores";
+
+  /**
+   * The name of the metrics context that metrics will be under.
+   */
+  String METRICS_CONTEXT = "regionserver";
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about Stores under a region";
+
+  /**
+   * The name of the metrics context that metrics will be under in jmx
+   */
+  String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;

Review comment:
       I was just trying to push all the store level metric per region under this. that is why collected it under RegionServer.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-617163971


   Some checkstyles seems to be coming from already existing code. Not sure. Fixed them. Lets see. Tests seems to be passing. The javadoc for hbase-hadoop2-compat seems to be failing due to
   
   > [ERROR] Exit code: 1 - javadoc: error - The code being documented uses modules but the packages defined in https://docs.oracle.com/javase/8/docs/api/ are in the unnamed module.
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-628487216


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 23s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 57s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 27s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   3m 20s |  master passed  |
   | -0 :warning: |  patch  |   3m  2s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 41s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 16s |  hbase-hadoop-compat: The patch generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m 29s |  hbase-server: The patch generated 2 new + 26 unchanged - 2 fixed = 28 total (was 28)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  12m 21s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 54s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 22s |  The patch does not generate ASF License warnings.  |
   |  |   |  40m 35s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux cff583fa3a90 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 5e32e08782 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop-compat.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | whitespace | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-636136273


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 29s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 50s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 39s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 33s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server in master failed.  |
   | -0 :warning: |  patch  |   7m 59s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 46s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 33s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 33s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 47s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 49s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 49s |  hbase-hadoop-compat in the patch passed.  |
   | -1 :x: |  unit  | 215m  9s |  hbase-server in the patch failed.  |
   |  |   | 248m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.10 Server=19.03.10 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux df9b3d9a0807 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / da2e03bb3b |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/testReport/ |
   | Max. process+thread count | 3001 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r428788807



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreSourceImpl.java
##########
@@ -0,0 +1,211 @@
+/**
+ * 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.regionserver;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.metrics.Interns;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreSourceImpl implements MetricsStoreSource {
+
+  private MetricsStoreWrapper storeWrapper;
+  private MetricsStoreAggregateSourceImpl aggreagate;
+  private AtomicBoolean closed = new AtomicBoolean(false);
+
+  private String storeNamePrefix;
+  private final MetricRegistry registry;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreSourceImpl.class);
+  String storeReadsKey;
+
+  String memstoreReadsKey;
+  String fileReadsKey;
+  private final Counter storeReads;
+  private final Counter memstoreReads;
+  private final Counter fileReads;
+
+  public MetricsStoreSourceImpl(MetricsStoreWrapper storeWrapper,
+      MetricsStoreAggregateSourceImpl aggreagate) {
+    this.storeWrapper = storeWrapper;
+    this.aggreagate = aggreagate;
+    aggreagate.register(this);
+
+    LOG.debug("Creating new MetricsRegionSourceImpl for table " + storeWrapper.getStoreName() + " "
+        + storeWrapper.getRegionName());
+
+    // we are using the hbase-metrics API
+    registry = aggreagate.getMetricRegistry();
+
+    storeNamePrefix = "Namespace_" + storeWrapper.getNamespace() + "_table_"
+        + storeWrapper.getTableName() + "_region_" + storeWrapper.getRegionName() + "_store_"
+        + storeWrapper.getStoreName() + "_metric_";
+
+    String suffix = "Count";
+
+    storeReadsKey = storeNamePrefix + MetricsRegionServerSource.GET_KEY + suffix;
+    // all the counters are hbase-metrics API
+    storeReads = registry.counter(storeReadsKey);
+
+    memstoreReadsKey = storeNamePrefix + MetricsRegionServerSource.MEMSTORE_GET_KEY + suffix;
+    memstoreReads = registry.counter(memstoreReadsKey);
+
+    fileReadsKey = storeNamePrefix + MetricsRegionServerSource.FILE_GET_KEY + suffix;
+    fileReads = registry.counter(fileReadsKey);
+
+  }
+
+  @Override
+  public void close() {
+    boolean wasClosed = closed.getAndSet(true);
+
+    // Has someone else already closed this for us?
+    if (wasClosed) {
+      return;
+    }
+
+    // Before removing the metrics remove this region from the aggregate region bean.
+    // This should mean that it's unlikely that snapshot and close happen at the same time.
+    aggreagate.deregister(this);
+
+    // While it's un-likely that snapshot and close happen at the same time it's still possible.
+    // So grab the lock to ensure that all calls to snapshot are done before we remove the metrics
+    synchronized (this) {
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Removing store Metrics: " + storeWrapper.getStoreName());
+      }
+
+      registry.remove(storeReadsKey);
+      registry.remove(memstoreReadsKey);
+      registry.remove(fileReadsKey);
+
+      storeWrapper = null;
+    }
+  }
+
+  @Override
+  public int compareTo(MetricsStoreSource source) {
+    if (!(source instanceof MetricsStoreSourceImpl)) {
+      return -1;
+    }
+
+    MetricsStoreSourceImpl impl = (MetricsStoreSourceImpl) source;
+    if (impl == null) {
+      return -1;
+    }
+
+    // TODO : make this better
+    return Long.compare(this.storeWrapper.getStoreName().hashCode(),
+      impl.storeWrapper.getStoreName().hashCode());
+  }
+
+  @Override
+  public void updateGet() {
+    storeReads.increment();
+  }
+
+  @Override
+  public void updateMemtoreGet() {
+    memstoreReads.increment();
+  }
+
+  @Override
+  public void updateFileGet() {
+    fileReads.increment();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    return obj == this
+        || (obj instanceof MetricsStoreSourceImpl && compareTo((MetricsStoreSourceImpl) obj) == 0);
+  }
+
+  @Override
+  public int hashCode() {
+    return this.storeWrapper.getStoreName().hashCode();
+  }
+
+  void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
+
+    // If there is a close that started be double extra sure
+    // that we're not getting any locks and not putting data
+    // into the metrics that should be removed. So early out
+    // before even getting the lock.
+    if (closed.get()) {
+      return;
+    }
+
+    // Grab the read
+    // This ensures that removes of the metrics
+    // can't happen while we are putting them back in.
+    synchronized (this) {
+
+      // It's possible that a close happened between checking
+      // the closed variable and getting the lock.
+      if (closed.get()) {
+        return;
+      }
+      mrb.addGauge(Interns.info(storeNamePrefix + MetricsRegionServerSource.STOREFILE_COUNT,

Review comment:
       At region level we had the aggregation across stores. Here it is per store. Just aggregation . We don do anything here to evaluate all this. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-637687015


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 36s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 16s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 23s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 44s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 22s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in master failed.  |
   | -0 :warning: |  patch  |   7m  7s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 45s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 34s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 125m 22s |  hbase-server in the patch passed.  |
   |  |   | 154m  8s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 70dbe1cd2058 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e5345b3a7c |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/testReport/ |
   | Max. process+thread count | 4076 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-636139889


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  2s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 57s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 21s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 16s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 54s |  master passed  |
   | -0 :warning: |  patch  |   7m 28s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  0s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 12s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 52s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 38s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 227m 49s |  hbase-server in the patch passed.  |
   |  |   | 257m 14s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.10 Server=19.03.10 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 56c4f7447e93 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / da2e03bb3b |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/testReport/ |
   | Max. process+thread count | 2520 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-619064466


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |  16m 26s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 44s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 49s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 11s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 11s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 27s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-hadoop2-compat in master failed.  |
   | -0 :warning: |  javadoc  |   1m  0s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 53s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m 13s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 57s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 23s |  hbase-hadoop2-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 51s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 31s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 36s |  hbase-hadoop2-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 185m 41s |  hbase-server in the patch passed.  |
   |  |   | 240m  7s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 13029288f7ff 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 023c6fe56b |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/testReport/ |
   | Max. process+thread count | 3428 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-628580345


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 24s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 49s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 46s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m 36s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server in master failed.  |
   | -0 :warning: |  patch  |   9m  2s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 35s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 27s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 27s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 19s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 41s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 192m 54s |  hbase-server in the patch passed.  |
   |  |   | 226m  3s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 78996e06e241 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 5e32e08782 |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/testReport/ |
   | Max. process+thread count | 3238 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r439611166



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java
##########
@@ -41,39 +40,42 @@
   private final HRegionServer regionServer;
   private ScheduledExecutorService executor;
   private Runnable runnable;
-  private long period;
+  private static final int PERIOD = 45;
   private ScheduledFuture<?> tableMetricsUpdateTask;
   private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap
     = new ConcurrentHashMap<>();
 
   public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
     this.regionServer = regionServer;
-    this.period = regionServer.getConfiguration().getLong(HConstants.REGIONSERVER_METRICS_PERIOD,
-      HConstants.DEFAULT_REGIONSERVER_METRICS_PERIOD) + 1000;
     this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
     this.runnable = new TableMetricsWrapperRunnable();
-    this.tableMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, period,
-      this.period, TimeUnit.MILLISECONDS);
+    this.tableMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, PERIOD,

Review comment:
       I would request to keep this out of this PR.  Raise another specific Jira to address this issue and may be below one also and get it committed as part of that. Later it will be easy for some one who is searching the change history/ bug fix history.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r414464537



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreWrapperImpl.java
##########
@@ -0,0 +1,194 @@
+/**
+ * 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.regionserver;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.OptionalDouble;
+import java.util.OptionalLong;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreWrapperImpl implements MetricsStoreWrapper, Closeable {
+
+  private final HStore store;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreWrapperImpl.class);
+
+  public static final int PERIOD = 45;
+  public static final String UNKNOWN = "unknown";
+  private ScheduledExecutorService executor;
+  private Runnable runnable;
+  // add others also. check if anything is redundant
+  private long numStoreFiles;
+  private long memstoreSize;
+  private long storeFileSize;
+  private long getsFromMemstore;
+  private long getsOnStore;
+  private long getsOnFile;
+  private long numReferenceFiles;
+  private long minStoreFileAge;
+  private long maxStoreFileAge;
+  private long avgStoreFileAge;
+  private long numHFiles;
+  private int storeRefCount;
+
+  private ScheduledFuture<?> storeMetricUpdateTask;
+
+  public MetricsStoreWrapperImpl(HStore store) {
+    this.store = store;
+    this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
+    this.runnable = new HStoreMetricsWrapperRunnable();
+    this.storeMetricUpdateTask =
+        this.executor.scheduleWithFixedDelay(this.runnable, PERIOD, PERIOD, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void close() throws IOException {
+    storeMetricUpdateTask.cancel(true);
+  }
+
+  @Override
+  public String getStoreName() {
+    return store.getColumnFamilyName();
+  }
+
+  @Override
+  public String getRegionName() {
+    return store.getRegionInfo().getRegionNameAsString();
+  }
+
+  @Override
+  public String getTableName() {
+    return store.getRegionInfo().getTable().getNameAsString();
+  }
+
+  @Override
+  public String getNamespace() {
+    return store.getTableName().getNamespaceAsString();
+  }
+
+  @Override
+  public long getNumStoreFiles() {
+    return numStoreFiles;
+  }
+
+  @Override
+  public long getMemStoreSize() {
+    // todo : change this - we need to expose data, heapsize and offheapdatasize

Review comment:
       For now lets be it this way. All other metrics has to be changed. We can do it that time.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r424941958



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSource.java
##########
@@ -0,0 +1,60 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface MetricsStoreAggregateSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Stores";
+
+  /**
+   * The name of the metrics context that metrics will be under.
+   */
+  String METRICS_CONTEXT = "regionserver";
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about Stores under a region";

Review comment:
       I have now tried to consolidate the metric at the table level also. Pls have a look. @anoopsjohn .




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r438696429



##########
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:
       Here the key will be &lt;regionName>_&lt;storeName>?
   At this level we should call CF not store?
   




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 merged pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 merged pull request #1552:
URL: https://github.com/apache/hbase/pull/1552


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-636016778


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 36s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 26s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 48s |  master passed  |
   | -0 :warning: |  patch  |   2m 27s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 50s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 13s |  hbase-hadoop-compat: The patch generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m 18s |  hbase-server: The patch generated 5 new + 25 unchanged - 1 fixed = 30 total (was 26)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  12m 41s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   3m  4s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 22s |  The patch does not generate ASF License warnings.  |
   |  |   |  41m  0s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.10 Server=19.03.10 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 6c7686a38062 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / da2e03bb3b |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop-compat.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | whitespace | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-643662066


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 35s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 30s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  3s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 22s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 16s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  master passed  |
   | -0 :warning: |  patch  |   7m 28s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 56s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m  7s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 37s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 198m 31s |  hbase-server in the patch passed.  |
   |  |   | 228m 22s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 0193253dc038 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fd5002d0da |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/testReport/ |
   | Max. process+thread count | 3819 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r434304384



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
##########
@@ -53,6 +53,10 @@
   String COPROCESSOR_EXECUTION_STATISTICS_DESC = "Statistics for coprocessor execution times";
   String REPLICA_ID = "replicaid";
   String REPLICA_ID_DESC = "The replica ID of a region. 0 is primary, otherwise is secondary";
+  String READ_REQUEST_ON_MEMSTORE = "readRequestCountOnMemstore";
+  String READ_REQUEST_ON_MEMSTORE_DESC = "Reads happening out of memstore";
+  String MIXED_READ_REQUEST_ON_STORE = "mixedReadRequestCountOnStore";
+  String MIXED_READ_REQUEST_ON_STORE_DESC = "Reads happening out of files and memstore on store";

Review comment:
       That constants I did not clean up. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-643080706


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  9s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  1s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 27s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 35s |  master passed  |
   | -0 :warning: |  patch  |   2m 19s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 12s |  hbase-hadoop-compat: The patch generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m 12s |  hbase-server: The patch generated 3 new + 37 unchanged - 1 fixed = 40 total (was 38)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 20s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 57s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 22s |  The patch does not generate ASF License warnings.  |
   |  |   |  38m 32s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux c518895e98e2 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 42a82df3af |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop-compat.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-638341617


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 48s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 15s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 39s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 54s |  master passed  |
   | -0 :warning: |  patch  |   6m 57s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 23s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 13s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 34s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 32s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 136m 19s |  hbase-server in the patch passed.  |
   |  |   | 162m 58s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 95dc53fa76c3 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8de8c44029 |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/testReport/ |
   | Max. process+thread count | 3987 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r428788326



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
##########
@@ -302,6 +302,14 @@ void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
               regionNamePrefix + MetricsRegionSource.MAX_FLUSH_QUEUE_SIZE,
               MetricsRegionSource.MAX_FLUSH_QUEUE_DESC),
           this.regionWrapper.getMaxFlushQueueSize());
+      mrb.addCounter(

Review comment:
       At the store level only it comes as per region per store. This is something we already have. Jut adding those two new metric here. So should be ok.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r412880431



##########
File path: hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java
##########
@@ -45,6 +46,16 @@ private synchronized MetricsRegionAggregateSourceImpl getRegionAggregate() {
     }
   }
 
+  private synchronized MetricsStoreAggregateSourceImpl
+      getStoreAggregate(MetricsStoreWrapper wrapper) {

Review comment:
       wrapper argument can be removed?

##########
File path: hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSourceImpl.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.regionserver;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
+import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreAggregateSourceImpl extends BaseSourceImpl
+    implements MetricsStoreAggregateSource {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreAggregateSourceImpl.class);
+
+  private final MetricsExecutorImpl executor = new MetricsExecutorImpl();
+
+  private final Set<MetricsStoreSource> storeSources =
+      Collections.newSetFromMap(new ConcurrentHashMap<MetricsStoreSource, Boolean>());

Review comment:
       nit: `Collections.newSetFromMap(new ConcurrentHashMap<>())`

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStore.java
##########
@@ -0,0 +1,59 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class MetricsStore {
+  private final MetricsStoreSource source;
+  private MetricsStoreWrapper storeWrapper;
+
+  public MetricsStore(final MetricsStoreWrapper wrapper, Configuration conf) {

Review comment:
       We can remove conf

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreWrapperImpl.java
##########
@@ -0,0 +1,194 @@
+/**
+ * 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.regionserver;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.OptionalDouble;
+import java.util.OptionalLong;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreWrapperImpl implements MetricsStoreWrapper, Closeable {
+
+  private final HStore store;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreWrapperImpl.class);

Review comment:
       Would you like to use LOG somewhere? Or maybe dropped the idea and hence it's not being used?

##########
File path: hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSourceImpl.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.regionserver;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
+import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreAggregateSourceImpl extends BaseSourceImpl
+    implements MetricsStoreAggregateSource {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreAggregateSourceImpl.class);
+
+  private final MetricsExecutorImpl executor = new MetricsExecutorImpl();
+
+  private final Set<MetricsStoreSource> storeSources =
+      Collections.newSetFromMap(new ConcurrentHashMap<MetricsStoreSource, Boolean>());
+
+  public MetricsStoreAggregateSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+
+  public MetricsStoreAggregateSourceImpl(String metricsName,
+                                          String metricsDescription,
+                                          String metricsContext,
+                                          String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+
+    // Every few mins clean the JMX cache.
+    executor.getExecutor().scheduleWithFixedDelay(new Runnable() {
+      public void run() {
+        JmxCacheBuster.clearJmxCache();
+      }
+    }, 5, 5, TimeUnit.MINUTES);
+  }
+
+  public MetricRegistry getMetricRegistry() {
+    return registry;
+  }
+
+  @Override
+  public void register(MetricsStoreSource source) {
+    storeSources.add(source);
+    clearCache();
+  }
+
+  @Override
+  public void deregister(MetricsStoreSource toRemove) {
+    try {
+      storeSources.remove(toRemove);
+    } catch (Exception e) {
+      // Ignored. If this errors out it means that someone is double
+      // closing the region source and the region is already nulled out.
+      LOG.info(

Review comment:
       Good to have LOG.error here?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreWrapperImpl.java
##########
@@ -0,0 +1,194 @@
+/**
+ * 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.regionserver;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.OptionalDouble;
+import java.util.OptionalLong;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreWrapperImpl implements MetricsStoreWrapper, Closeable {
+
+  private final HStore store;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreWrapperImpl.class);
+
+  public static final int PERIOD = 45;
+  public static final String UNKNOWN = "unknown";
+  private ScheduledExecutorService executor;
+  private Runnable runnable;
+  // add others also. check if anything is redundant
+  private long numStoreFiles;
+  private long memstoreSize;
+  private long storeFileSize;
+  private long getsFromMemstore;
+  private long getsOnStore;
+  private long getsOnFile;
+  private long numReferenceFiles;
+  private long minStoreFileAge;
+  private long maxStoreFileAge;
+  private long avgStoreFileAge;
+  private long numHFiles;
+  private int storeRefCount;
+
+  private ScheduledFuture<?> storeMetricUpdateTask;
+
+  public MetricsStoreWrapperImpl(HStore store) {
+    this.store = store;
+    this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
+    this.runnable = new HStoreMetricsWrapperRunnable();
+    this.storeMetricUpdateTask =
+        this.executor.scheduleWithFixedDelay(this.runnable, PERIOD, PERIOD, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void close() throws IOException {
+    storeMetricUpdateTask.cancel(true);
+  }
+
+  @Override
+  public String getStoreName() {
+    return store.getColumnFamilyName();
+  }
+
+  @Override
+  public String getRegionName() {
+    return store.getRegionInfo().getRegionNameAsString();
+  }
+
+  @Override
+  public String getTableName() {
+    return store.getRegionInfo().getTable().getNameAsString();
+  }
+
+  @Override
+  public String getNamespace() {
+    return store.getTableName().getNamespaceAsString();
+  }
+
+  @Override
+  public long getNumStoreFiles() {
+    return numStoreFiles;
+  }
+
+  @Override
+  public long getMemStoreSize() {
+    // todo : change this - we need to expose data, heapsize and offheapdatasize

Review comment:
       Noted. For this change, is it good to give name `memstoreDataSize` ?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -2547,7 +2565,7 @@ public CacheConfig getCacheConfig() {
   }
 
   public static final long FIXED_OVERHEAD =
-      ClassSize.align(ClassSize.OBJECT + (27 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG)
+      ClassSize.align(ClassSize.OBJECT + (31 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG)

Review comment:
       This change is a must or it is an improvement? Just trying to understand it better :)

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -2898,4 +2915,40 @@ public int getMaxCompactedStoreFileRefCount() {
       ? maxCompactedStoreFileRefCount.getAsInt() : 0;
   }
 
+  @Override
+  public long getReadRequestsFromStoreCount() {
+    return getRequestsFromStore.sum();
+  }
+
+  @Override
+  public long getGetRequestsCountFromMemstore() {
+    return getRequestsFromMemstore.sum();
+  }
+
+  @Override
+  public long getGetRequestsCountFromFile() {
+    return getRequestsFromFile.sum();
+  }
+
+  void incrGetRequestsFromStore() {
+    getRequestsFromStore.increment();
+    if (metricsStore != null) {
+      metricsStore.updateGet();
+    }
+  }
+
+  void updateMetricsStore(boolean memstoreRead) {
+    if (memstoreRead) {
+      getRequestsFromMemstore.increment();
+    } else {
+      getRequestsFromFile.increment();
+    }
+    if (metricsStore != null) {
+      if (memstoreRead) {
+        metricsStore.updateMemstoreGet();
+      }
+      metricsStore.updateFileGet();

Review comment:
       This should be an else statement?

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreWrapperImpl.java
##########
@@ -0,0 +1,194 @@
+/**
+ * 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.regionserver;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.OptionalDouble;
+import java.util.OptionalLong;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreWrapperImpl implements MetricsStoreWrapper, Closeable {
+
+  private final HStore store;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreWrapperImpl.class);
+
+  public static final int PERIOD = 45;
+  public static final String UNKNOWN = "unknown";
+  private ScheduledExecutorService executor;
+  private Runnable runnable;

Review comment:
       executor and runnable both could be local variable

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java
##########
@@ -608,7 +613,17 @@ public boolean next(List<Cell> outResult, ScannerContext scannerContext) throws
           if (f != null) {
             cell = f.transformCell(cell);
           }
-
+          // tracking gets only and currently per row
+          // and not per cell. Even scans metrics at the region level are
+          // being tracked row wise.
+          if (get && !trackGets) {
+            if (!heap.current.isFileScanner()) {
+              updateMetricsStore(true);
+            } else {
+              updateMetricsStore(false);

Review comment:
       Can be simplified to `updateMetricsStore(!heap.current.isFileScanner())`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-636015297


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 36s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  2s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 19s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 17s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 54s |  master passed  |
   | -0 :warning: |  patch  |   7m 30s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m  0s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 17s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m  4s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 52s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 37s |  hbase-hadoop-compat in the patch passed.  |
   | -1 :x: |  unit  |   9m 12s |  hbase-server in the patch failed.  |
   |  |   |  38m 15s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.10 Server=19.03.10 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 24e1245c431f 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / da2e03bb3b |
   | Default Java | 1.8.0_232 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/testReport/ |
   | Max. process+thread count | 436 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r438696429



##########
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:
       Here the key will be &lt;regionName>_<storeName>?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-643219434


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 25s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 58s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 24s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 35s |  master passed  |
   | -0 :warning: |  patch  |   2m 19s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 11s |  hbase-hadoop-compat: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m 13s |  hbase-server: The patch generated 3 new + 37 unchanged - 1 fixed = 40 total (was 38)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m  4s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 56s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 21s |  The patch does not generate ASF License warnings.  |
   |  |   |  37m 15s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 033ff9757222 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 6b7a93d10c |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop-compat.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-636036414


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 32s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 34s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 18s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 28s |  master passed  |
   | -0 :warning: |  patch  |   2m 12s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 23s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 13s |  hbase-hadoop-compat: The patch generated 2 new + 0 unchanged - 0 fixed = 2 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m  4s |  hbase-server: The patch generated 5 new + 25 unchanged - 1 fixed = 30 total (was 26)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  11m  8s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 46s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 26s |  The patch does not generate ASF License warnings.  |
   |  |   |  35m  0s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.10 Server=19.03.10 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 3d965e3ffeb2 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / da2e03bb3b |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop-compat.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | whitespace | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/9/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-620008887


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 33s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 29s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  2s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  9s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 20s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 30s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 30s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  2s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  6s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 27s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 28s |  hbase-hadoop2-compat in the patch passed.  |
   | -1 :x: |  unit  | 136m 18s |  hbase-server in the patch failed.  |
   |  |   | 163m 25s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 45b34e410ef3 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 84f2e95dd5 |
   | Default Java | 1.8.0_232 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/testReport/ |
   | Max. process+thread count | 4658 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r434304818



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
##########
@@ -170,4 +170,15 @@
    *   all compacted store files that belong to this region
    */
   long getMaxCompactedStoreFileRefCount();
+
+  /**
+   * @return the number of reads on memstore
+   */
+  long getMemstoreReadRequestsCount();

Review comment:
       I just followed a model where currently we have accouting at table level and region level. So following the same path - We added per region per store metric and this one is nothing but a region level aggregation. This will tell on the region how many reads are from memstore. (across all stores). 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-616642238


   Will check the test case failure. Seems metrics needs to be explicitly handled. And other Javadoc issues needs to be fixed
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-617048227


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   4m 24s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 25s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 19s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 32s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 54s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 22s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m  6s |  hbase-server: The patch generated 2 new + 28 unchanged - 0 fixed = 30 total (was 28)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |   5m 32s |  Patch does not cause any errors with Hadoop 3.1.2.  |
   | +1 :green_heart: |  spotbugs  |   3m 24s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 37s |  The patch does not generate ASF License warnings.  |
   |  |   |  34m 45s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 040c9b4d4bff 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / f48b50964c |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | whitespace | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-624097657


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 17s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 13s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 29s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 51s |  master passed  |
   | -0 :warning: |  patch  |   6m 39s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 40s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 12s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 12s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 25s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 51s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 36s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 195m 35s |  hbase-server in the patch passed.  |
   |  |   | 222m 49s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b3aa42e3cb2c 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fdbf45886d |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/testReport/ |
   | Max. process+thread count | 3213 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-617104797


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 33s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 19s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 51s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 41s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop2-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 58s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 44s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 44s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 18s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 16s |  hbase-hadoop2-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 42s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 30s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 30s |  hbase-hadoop2-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 121m 46s |  hbase-server in the patch passed.  |
   |  |   | 151m 43s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d341cbb03205 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / f48b50964c |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/testReport/ |
   | Max. process+thread count | 4293 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r434598232



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSource.java
##########
@@ -0,0 +1,60 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface MetricsStoreAggregateSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Stores";
+
+  /**
+   * The name of the metrics context that metrics will be under.
+   */
+  String METRICS_CONTEXT = "regionserver";
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about Stores under a region";
+
+  /**
+   * The name of the metrics context that metrics will be under in jmx
+   */
+  String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;

Review comment:
       I think to avoid this problem of having another metric exposed at Store level which will list all the region names and its corresponding read metrics and also we have a region level metric where we expose across all stores in that region how many reads happened.  
   Both come under different Mbean - one is sub=Regions and another is sub=Stores. (this is added newly in this patch)
   
   The patch already has an aggregated metric at the Table level where we report a metric 
   
   > NameSpace_<Namespacename>table<tablename>_store<storename>_metric<metricName>
   We will just use this and add a similar metric at the region level which aggregates it per store
   as we have for table level. Will that be ok ? 
   @saintstack - what you think? 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] virajjasani commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
virajjasani commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r413246058



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreWrapperImpl.java
##########
@@ -0,0 +1,194 @@
+/**
+ * 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.regionserver;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.OptionalDouble;
+import java.util.OptionalLong;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreWrapperImpl implements MetricsStoreWrapper, Closeable {
+
+  private final HStore store;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreWrapperImpl.class);
+
+  public static final int PERIOD = 45;
+  public static final String UNKNOWN = "unknown";
+  private ScheduledExecutorService executor;
+  private Runnable runnable;
+  // add others also. check if anything is redundant
+  private long numStoreFiles;
+  private long memstoreSize;
+  private long storeFileSize;
+  private long getsFromMemstore;
+  private long getsOnStore;
+  private long getsOnFile;
+  private long numReferenceFiles;
+  private long minStoreFileAge;
+  private long maxStoreFileAge;
+  private long avgStoreFileAge;
+  private long numHFiles;
+  private int storeRefCount;
+
+  private ScheduledFuture<?> storeMetricUpdateTask;
+
+  public MetricsStoreWrapperImpl(HStore store) {
+    this.store = store;
+    this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
+    this.runnable = new HStoreMetricsWrapperRunnable();
+    this.storeMetricUpdateTask =
+        this.executor.scheduleWithFixedDelay(this.runnable, PERIOD, PERIOD, TimeUnit.SECONDS);
+  }
+
+  @Override
+  public void close() throws IOException {
+    storeMetricUpdateTask.cancel(true);
+  }
+
+  @Override
+  public String getStoreName() {
+    return store.getColumnFamilyName();
+  }
+
+  @Override
+  public String getRegionName() {
+    return store.getRegionInfo().getRegionNameAsString();
+  }
+
+  @Override
+  public String getTableName() {
+    return store.getRegionInfo().getTable().getNameAsString();
+  }
+
+  @Override
+  public String getNamespace() {
+    return store.getTableName().getNamespaceAsString();
+  }
+
+  @Override
+  public long getNumStoreFiles() {
+    return numStoreFiles;
+  }
+
+  @Override
+  public long getMemStoreSize() {
+    // todo : change this - we need to expose data, heapsize and offheapdatasize

Review comment:
       So that next time when we add heapsize and offheapdatasize, we can give name: `memstoreHeapSize`, `memstoreOffHeapSize` etc.
   Your call.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-643639680


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 35s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 10s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 29s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 46s |  master passed  |
   | -0 :warning: |  patch  |   2m 26s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 53s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 19s |  hbase-server: The patch generated 2 new + 37 unchanged - 1 fixed = 39 total (was 38)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 39s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   3m 29s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 23s |  The patch does not generate ASF License warnings.  |
   |  |   |  41m 40s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 72f8728e7952 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fd5002d0da |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-643164646


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 11s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 56s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 14s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  4s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 52s |  master passed  |
   | -0 :warning: |  patch  |   7m 15s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 42s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 13s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m  6s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 51s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 35s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 203m 19s |  hbase-server in the patch passed.  |
   |  |   | 231m 36s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 8966aca66120 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 42a82df3af |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/testReport/ |
   | Max. process+thread count | 3320 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-637724253


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 18s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 20s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 16s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m  5s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 55s |  master passed  |
   | -0 :warning: |  patch  |   7m 18s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 47s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 13s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 13s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m  2s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 52s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 39s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 200m 15s |  hbase-server in the patch passed.  |
   |  |   | 228m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.9 Server=19.03.9 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux bc38ae8e6c19 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e5345b3a7c |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/testReport/ |
   | Max. process+thread count | 3526 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r434304341



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
##########
@@ -53,6 +53,10 @@
   String COPROCESSOR_EXECUTION_STATISTICS_DESC = "Statistics for coprocessor execution times";
   String REPLICA_ID = "replicaid";
   String REPLICA_ID_DESC = "The replica ID of a region. 0 is primary, otherwise is secondary";
+  String READ_REQUEST_ON_MEMSTORE = "readRequestCountOnMemstore";
+  String READ_REQUEST_ON_MEMSTORE_DESC = "Reads happening out of memstore";
+  String MIXED_READ_REQUEST_ON_STORE = "mixedReadRequestCountOnStore";
+  String MIXED_READ_REQUEST_ON_STORE_DESC = "Reads happening out of files and memstore on store";

Review comment:
       The one above is not needed - MEMSTORE_GET_KEY  and FILE_GET_KEY  in MetricsREgionServerSource because I had removed all those additional metric. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-617181237


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 19s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 33s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 48s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 20s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 29s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |   5m 23s |  Patch does not cause any errors with Hadoop 3.1.2.  |
   | +1 :green_heart: |  spotbugs  |   3m 19s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 36s |  The patch does not generate ASF License warnings.  |
   |  |   |  30m 11s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux f8793569c32b 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / f48b50964c |
   | Max. process+thread count | 94 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r428787492



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
##########
@@ -402,6 +402,8 @@
   String DELETE_BATCH_KEY = "deleteBatch";
   String GET_SIZE_KEY = "getSize";
   String GET_KEY = "get";
+  String MEMSTORE_GET_KEY = "getsOnMemstore";
+  String FILE_GET_KEY = "getsOnFile";

Review comment:
       Some of the gets that lands on the StoreScanners does not actually get accounted into actual get. Probably that row does not exist. So I thought it is better to track both. Also the overhead is very minimal. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r430396089



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -2884,4 +2901,41 @@ public int getMaxCompactedStoreFileRefCount() {
       ? maxCompactedStoreFileRefCount.getAsInt() : 0;
   }
 
+  @Override
+  public long getReadRequestsFromStoreCount() {
+    return getRequestsFromStore.sum();
+  }
+
+  @Override
+  public long getGetRequestsCountFromMemstore() {
+    return getRequestsFromMemstore.sum();
+  }
+
+  @Override
+  public long getGetRequestsCountFromFile() {
+    return getRequestsFromFile.sum();
+  }
+
+  void incrGetRequestsFromStore() {
+    getRequestsFromStore.increment();

Review comment:
       The one direclty inder HStore is used by the Region level and Table level aggregators which deals with HStore. This gets printed periodically. The other one is at the MetricsStore level which is the real time one. For every request it will be displayed at JMX MBean level. 

##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java
##########
@@ -70,25 +69,36 @@ public void run() {
           localMetricsTableMap.put(tbl, mt);
         }
         if (r.getStores() != null) {
+          long memstoreReadCount = 0l;
+          long fileReadCount = 0l;
+          String familyName = null;
           for (Store store : r.getStores()) {
+            familyName = store.getColumnFamilyName();
+
             mt.storeFileCount += store.getStorefilesCount();
-            mt.memstoreSize += (store.getMemStoreSize().getDataSize() +
-              store.getMemStoreSize().getHeapSize() + store.getMemStoreSize().getOffHeapSize());
+            mt.memstoreSize += (store.getMemStoreSize().getDataSize()
+                + store.getMemStoreSize().getHeapSize() + store.getMemStoreSize().getOffHeapSize());
             mt.storeFileSize += store.getStorefilesSize();
             mt.referenceFileCount += store.getNumReferenceFiles();
 
-            mt.maxStoreFileAge = Math.max(mt.maxStoreFileAge, store.getMaxStoreFileAge().getAsLong());
-            mt.minStoreFileAge = Math.min(mt.minStoreFileAge, store.getMinStoreFileAge().getAsLong());
-            mt.totalStoreFileAge = (long)store.getAvgStoreFileAge().getAsDouble() *
-                store.getStorefilesCount();
+            mt.maxStoreFileAge =
+                Math.max(mt.maxStoreFileAge, store.getMaxStoreFileAge().getAsLong());
+            mt.minStoreFileAge =
+                Math.min(mt.minStoreFileAge, store.getMinStoreFileAge().getAsLong());
+            mt.totalStoreFileAge =
+                (long) store.getAvgStoreFileAge().getAsDouble() * store.getStorefilesCount();
             mt.storeCount += 1;
+            memstoreReadCount += store.getGetRequestsCountFromMemstore();
+            fileReadCount += store.getGetRequestsCountFromFile();
+            mt.storeMemstoreGetCount.putIfAbsent(familyName, memstoreReadCount);
+            mt.storeFileGetCount.putIfAbsent(familyName, fileReadCount);
           }
+
           mt.regionCount += 1;
 
           mt.readRequestCount += r.getReadRequestsCount();
-          mt.filteredReadRequestCount += getFilteredReadRequestCount(tbl.getNameAsString());
+          mt.filteredReadRequestCount += r.getFilteredReadRequestsCount();

Review comment:
       This was wrong. It is a simple change. So I thought it is better to make this change hhere. If you are particular i can make the change in separete JIRA.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-619940983


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 44s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 11s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 36s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   3m 12s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 52s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 20s |  hbase-server: The patch generated 1 new + 28 unchanged - 0 fixed = 29 total (was 28)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 47s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   3m 40s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 31s |  The patch does not generate ASF License warnings.  |
   |  |   |  42m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 7665fba5568d 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 84f2e95dd5 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r414464389



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreWrapperImpl.java
##########
@@ -0,0 +1,194 @@
+/**
+ * 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.regionserver;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.OptionalDouble;
+import java.util.OptionalLong;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreWrapperImpl implements MetricsStoreWrapper, Closeable {
+
+  private final HStore store;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreWrapperImpl.class);

Review comment:
       Removed it.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-628549596


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 31s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 30s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 15s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 32s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  master passed  |
   | -0 :warning: |  patch  |   6m 47s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 23s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 12s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 12s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 32s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 53s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 33s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 133m 37s |  hbase-server in the patch passed.  |
   |  |   | 159m 33s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux d762199036e5 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 5e32e08782 |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/testReport/ |
   | Max. process+thread count | 4392 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/7/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r414466487



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreWrapperImpl.java
##########
@@ -0,0 +1,194 @@
+/**
+ * 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.regionserver;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.OptionalDouble;
+import java.util.OptionalLong;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.metrics2.MetricsExecutor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreWrapperImpl implements MetricsStoreWrapper, Closeable {
+
+  private final HStore store;
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreWrapperImpl.class);
+
+  public static final int PERIOD = 45;
+  public static final String UNKNOWN = "unknown";
+  private ScheduledExecutorService executor;
+  private Runnable runnable;

Review comment:
       Sure. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-636017424


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   2m 56s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 19s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 44s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 29s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 22s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in master failed.  |
   | -0 :warning: |  patch  |   7m 42s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 27s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 27s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 19s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 42s |  hbase-hadoop-compat in the patch passed.  |
   | -1 :x: |  unit  |   9m 33s |  hbase-server in the patch failed.  |
   |  |   |  42m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.9 Server=19.03.9 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b9e0be44ede2 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / da2e03bb3b |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/testReport/ |
   | Max. process+thread count | 640 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/8/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r439553774



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java
##########
@@ -41,39 +40,42 @@
   private final HRegionServer regionServer;
   private ScheduledExecutorService executor;
   private Runnable runnable;
-  private long period;
+  private static final int PERIOD = 45;
   private ScheduledFuture<?> tableMetricsUpdateTask;
   private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap
     = new ConcurrentHashMap<>();
 
   public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
     this.regionServer = regionServer;
-    this.period = regionServer.getConfiguration().getLong(HConstants.REGIONSERVER_METRICS_PERIOD,
-      HConstants.DEFAULT_REGIONSERVER_METRICS_PERIOD) + 1000;
     this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
     this.runnable = new TableMetricsWrapperRunnable();
-    this.tableMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, period,
-      this.period, TimeUnit.MILLISECONDS);
+    this.tableMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, PERIOD,

Review comment:
       Is this a bug fix? (Like the one below) .. Can u explain .. Previously it was a config based thing and now hard coded.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r428789194



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
##########
@@ -1002,6 +1012,14 @@ public Void call() throws IOException {
     } finally {
       this.lock.writeLock().unlock();
       this.archiveLock.unlock();
+      // moving it after the unlocking so
+      // that metrics closure does not affect them
+      if (this.metricsStore != null) {
+        metricsStore.close();

Review comment:
       Yes. It will do the deregister.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-616581627


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 51s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 35s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   3m  2s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 12s |  hbase-hadoop2-compat: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m 11s |  hbase-server: The patch generated 5 new + 28 unchanged - 0 fixed = 33 total (was 28)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 9 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |   5m 59s |  Patch does not cause any errors with Hadoop 3.1.2.  |
   | -1 :x: |  spotbugs  |   0m 40s |  hbase-hadoop2-compat generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 32s |  The patch does not generate ASF License warnings.  |
   |  |   |  33m 41s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hbase-hadoop2-compat |
   |  |  org.apache.hadoop.hbase.regionserver.MetricsStoreSourceImpl defines equals and uses Object.hashCode()  At MetricsStoreSourceImpl.java:Object.hashCode()  At MetricsStoreSourceImpl.java:[line 139] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 8501d6c232a9 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / bcacc4ce93 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop2-compat.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | whitespace | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-general-check/output/whitespace-eol.txt |
   | spotbugs | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-general-check/output/new-spotbugs-hbase-hadoop2-compat.html |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-643653541


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 30s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 55s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 23s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 44s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in master failed.  |
   | -0 :warning: |  patch  |   7m  6s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 59s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 23s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 23s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 41s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 38s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 34s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 126m 35s |  hbase-server in the patch passed.  |
   |  |   | 154m 55s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 1d8748f47e1f 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fd5002d0da |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/testReport/ |
   | Max. process+thread count | 4393 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/14/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r439581163



##########
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:
       You mean the key should be Columnfamily? So _STORE - that should be _ColumnFamily?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r434207957



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
##########
@@ -402,6 +402,8 @@
   String DELETE_BATCH_KEY = "deleteBatch";
   String GET_SIZE_KEY = "getSize";
   String GET_KEY = "get";
+  String MEMSTORE_GET_KEY = "getsOnMemstore";
+  String FILE_GET_KEY = "getsOnFile";

Review comment:
       In other words, it'd be better if the accounting aligned.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java
##########
@@ -53,6 +53,10 @@
   String COPROCESSOR_EXECUTION_STATISTICS_DESC = "Statistics for coprocessor execution times";
   String REPLICA_ID = "replicaid";
   String REPLICA_ID_DESC = "The replica ID of a region. 0 is primary, otherwise is secondary";
+  String READ_REQUEST_ON_MEMSTORE = "readRequestCountOnMemstore";
+  String READ_REQUEST_ON_MEMSTORE_DESC = "Reads happening out of memstore";
+  String MIXED_READ_REQUEST_ON_STORE = "mixedReadRequestCountOnStore";
+  String MIXED_READ_REQUEST_ON_STORE_DESC = "Reads happening out of files and memstore on store";

Review comment:
       Yeah, don't we have this accounted already? Seems redundant.

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSource.java
##########
@@ -0,0 +1,60 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface MetricsStoreAggregateSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Stores";
+
+  /**
+   * The name of the metrics context that metrics will be under.
+   */
+  String METRICS_CONTEXT = "regionserver";
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about Stores under a region";
+
+  /**
+   * The name of the metrics context that metrics will be under in jmx
+   */
+  String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME;

Review comment:
       This model doesn't seem right. There is no such thing as a Store on a RegionServer. The RegionServer hosts Regions. A Region hosts Stores.  This is an aggregate of all Stores on the RegionServer?
   
   I could imagine RegionServer,sub=Region,sub=Store... with a Bean per Store but then we'd probably have too many MBeans. 
   
   Do we have a Region at this level in MBean hierarchy?

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java
##########
@@ -170,4 +170,15 @@
    *   all compacted store files that belong to this region
    */
   long getMaxCompactedStoreFileRefCount();
+
+  /**
+   * @return the number of reads on memstore
+   */
+  long getMemstoreReadRequestsCount();

Review comment:
       ... this if for the Region?  Can I ask for the number of reads on memstore on a Store?

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java
##########
@@ -302,6 +302,14 @@ void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
               regionNamePrefix + MetricsRegionSource.MAX_FLUSH_QUEUE_SIZE,
               MetricsRegionSource.MAX_FLUSH_QUEUE_DESC),
           this.regionWrapper.getMaxFlushQueueSize());
+      mrb.addCounter(

Review comment:
       I don't follow the above. We already have account at the store level per region? Why then would we add these counters?

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSource.java
##########
@@ -0,0 +1,60 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface MetricsStoreAggregateSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Stores";

Review comment:
       If a single Store, should be called 'Store' not 'Stores'?

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSource.java
##########
@@ -0,0 +1,60 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface MetricsStoreAggregateSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Stores";

Review comment:
       Metrics for a Store for more than one Store? If for more than one Store, why not in Region?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r434206117



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSource.java
##########
@@ -402,6 +402,8 @@
   String DELETE_BATCH_KEY = "deleteBatch";
   String GET_SIZE_KEY = "getSize";
   String GET_KEY = "get";
+  String MEMSTORE_GET_KEY = "getsOnMemstore";
+  String FILE_GET_KEY = "getsOnFile";

Review comment:
       When discrepancy, how will operators figure out why the discrepancy?
   
   Or, why don't we count gets for which there is no row?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r434304891



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSource.java
##########
@@ -0,0 +1,60 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface MetricsStoreAggregateSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Stores";

Review comment:
       Ok .will make it Store. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-638345369


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 53s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 37s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 38s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   8m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 26s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 56s |  hbase-server in master failed.  |
   | -0 :warning: |  patch  |   9m 55s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 17s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 30s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 31s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 31s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 58s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 36s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 125m 40s |  hbase-server in the patch passed.  |
   |  |   | 160m 52s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux cd13240fb373 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 8de8c44029 |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/testReport/ |
   | Max. process+thread count | 4463 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/11/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r414463943



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStore.java
##########
@@ -0,0 +1,59 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class MetricsStore {
+  private final MetricsStoreSource source;
+  private MetricsStoreWrapper storeWrapper;
+
+  public MetricsStore(final MetricsStoreWrapper wrapper, Configuration conf) {

Review comment:
       Done.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-643308040


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 28s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 46s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 20s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 22s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 28s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  0s |  master passed  |
   | -0 :warning: |  patch  |   7m 48s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 36s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 25s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 15s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 52s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 36s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 204m  8s |  hbase-server in the patch passed.  |
   |  |   | 235m 39s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 5e059f646d85 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 1dac9f69c4 |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/testReport/ |
   | Max. process+thread count | 3118 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/13/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-624003442


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  7s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 46s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 24s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 35s |  master passed  |
   | -0 :warning: |  patch  |   2m 20s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 39s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 12s |  hbase-server: The patch generated 1 new + 28 unchanged - 0 fixed = 29 total (was 28)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 18s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 53s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 21s |  The patch does not generate ASF License warnings.  |
   |  |   |  37m 58s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 9cc89820a1bc 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fdbf45886d |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-619049841


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  5s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 43s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 26s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 30s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  5s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 41s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 28s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 28s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 41s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  4s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 23s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 30s |  hbase-hadoop2-compat in the patch passed.  |
   | -1 :x: |  unit  | 188m  4s |  hbase-server in the patch failed.  |
   |  |   | 216m 23s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3c114740535b 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 023c6fe56b |
   | Default Java | 1.8.0_232 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/testReport/ |
   | Max. process+thread count | 3467 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-617289974


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 36s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 34s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  4s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 32s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 32s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  6s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 56s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 32s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 32s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 34s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  5s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 25s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 29s |  hbase-hadoop2-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 215m 35s |  hbase-server in the patch passed.  |
   |  |   | 245m 10s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux b69feac99e58 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / f48b50964c |
   | Default Java | 1.8.0_232 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/testReport/ |
   | Max. process+thread count | 2938 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r414459588



##########
File path: hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerSourceFactoryImpl.java
##########
@@ -45,6 +46,16 @@ private synchronized MetricsRegionAggregateSourceImpl getRegionAggregate() {
     }
   }
 
+  private synchronized MetricsStoreAggregateSourceImpl
+      getStoreAggregate(MetricsStoreWrapper wrapper) {

Review comment:
       Ya not needed. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r423007901



##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSource.java
##########
@@ -0,0 +1,60 @@
+/**
+ * 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.regionserver;
+
+import org.apache.hadoop.hbase.metrics.BaseSource;
+import org.apache.yetus.audience.InterfaceAudience;
+
+
+@InterfaceAudience.Private
+public interface MetricsStoreAggregateSource extends BaseSource {
+  /**
+   * The name of the metrics
+   */
+  String METRICS_NAME = "Stores";
+
+  /**
+   * The name of the metrics context that metrics will be under.
+   */
+  String METRICS_CONTEXT = "regionserver";
+
+  /**
+   * Description
+   */
+  String METRICS_DESCRIPTION = "Metrics about Stores under a region";

Review comment:
       So this will treat only one store under a region for this aggregation. 
   When we want to see an aggregated metric across all the store instances (under diff regions) for a table, this wont help right? (I mean metric for table CF) Do you think that will be more useful?  The stuff of memstore hit rate and all at a CF level would be useful IMO.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r441335976



##########
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 don't have it already with the general read count?
   
   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. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-620020029


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 34s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m  3s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 11s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   7m  7s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop2-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 48s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 18s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 24s |  the patch passed  |
   | +1 :green_heart: |  compile  |   2m  8s |  the patch passed  |
   | +1 :green_heart: |  javac  |   2m  8s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   7m 11s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-hadoop2-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 55s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 33s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 37s |  hbase-hadoop2-compat in the patch passed.  |
   | -1 :x: |  unit  | 136m  8s |  hbase-server in the patch failed.  |
   |  |   | 173m 28s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 3dcf923a9a15 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 84f2e95dd5 |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/testReport/ |
   | Max. process+thread count | 4030 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/5/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-637607597


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  9s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 27s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 53s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 23s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 34s |  master passed  |
   | -0 :warning: |  patch  |   2m 19s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 12s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 44s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 12s |  hbase-hadoop-compat: The patch generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  checkstyle  |   1m 11s |  hbase-server: The patch generated 1 new + 36 unchanged - 1 fixed = 37 total (was 37)  |
   | -0 :warning: |  whitespace  |   0m  0s |  The patch has 1 line(s) that end in whitespace. Use git apply --whitespace=fix <<patch_file>>. Refer https://git-scm.com/docs/git-apply  |
   | +1 :green_heart: |  hadoopcheck  |  12m 15s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   2m 55s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 21s |  The patch does not generate ASF License warnings.  |
   |  |   |  38m 19s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.11 Server=19.03.11 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux d82c4007eef6 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / e5345b3a7c |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-general-check/output/diff-checkstyle-hbase-hadoop-compat.txt |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | whitespace | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/artifact/yetus-general-check/output/whitespace-eol.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/10/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r441333721



##########
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:
       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. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] saintstack commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r414463271



##########
File path: hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSourceImpl.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.regionserver;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
+import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreAggregateSourceImpl extends BaseSourceImpl
+    implements MetricsStoreAggregateSource {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreAggregateSourceImpl.class);
+
+  private final MetricsExecutorImpl executor = new MetricsExecutorImpl();
+
+  private final Set<MetricsStoreSource> storeSources =
+      Collections.newSetFromMap(new ConcurrentHashMap<MetricsStoreSource, Boolean>());
+
+  public MetricsStoreAggregateSourceImpl() {
+    this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
+  }
+
+
+  public MetricsStoreAggregateSourceImpl(String metricsName,
+                                          String metricsDescription,
+                                          String metricsContext,
+                                          String metricsJmxContext) {
+    super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
+
+    // Every few mins clean the JMX cache.
+    executor.getExecutor().scheduleWithFixedDelay(new Runnable() {
+      public void run() {
+        JmxCacheBuster.clearJmxCache();
+      }
+    }, 5, 5, TimeUnit.MINUTES);
+  }
+
+  public MetricRegistry getMetricRegistry() {
+    return registry;
+  }
+
+  @Override
+  public void register(MetricsStoreSource source) {
+    storeSources.add(source);
+    clearCache();
+  }
+
+  @Override
+  public void deregister(MetricsStoreSource toRemove) {
+    try {
+      storeSources.remove(toRemove);
+    } catch (Exception e) {
+      // Ignored. If this errors out it means that someone is double
+      // closing the region source and the region is already nulled out.
+      LOG.info(

Review comment:
       Actually the removal is not needed to handle throw catch. I just used what was being done in MetricsREgionAggregateSource. Seems that is not needed.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-624064859


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 52s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 39s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 18s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 38s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 33s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 43s |  hbase-server in master failed.  |
   | -0 :warning: |  patch  |   7m 58s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m  9s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 42s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 42s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 43s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 44s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 36s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 130m 47s |  hbase-server in the patch passed.  |
   |  |   | 164m 44s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2f5c95b89a0a 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / fdbf45886d |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/testReport/ |
   | Max. process+thread count | 4103 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/6/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-617285363


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 37s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 55s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 50s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop2-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 45s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 14s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 37s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 51s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 51s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 56s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 21s |  hbase-hadoop2-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 49s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 30s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 38s |  hbase-hadoop2-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 203m 41s |  hbase-server in the patch passed.  |
   |  |   | 237m  9s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 2723897fdae4 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / f48b50964c |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/testReport/ |
   | Max. process+thread count | 2968 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/3/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-616583060


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  9s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 53s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 27s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 26s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m  6s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 40s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 28s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 28s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m 19s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 14s |  hbase-hadoop-compat generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -0 :warning: |  javadoc  |   0m 36s |  hbase-server generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 25s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 27s |  hbase-hadoop2-compat in the patch passed.  |
   | -1 :x: |  unit  |   7m 55s |  hbase-server in the patch failed.  |
   |  |   |  35m 40s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 8399350b51c6 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / bcacc4ce93 |
   | Default Java | 1.8.0_232 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk8-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/testReport/ |
   | Max. process+thread count | 760 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] anoopsjohn commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
anoopsjohn commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r438696429



##########
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:
       Here the key will be <regionName>_<storeName>?

##########
File path: hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregate.java
##########
@@ -107,6 +109,13 @@
    */
   long getNumReferenceFiles(String table);
 
+  /**
+   * @return number of get requests from memstore per store for this table
+   */
+  Map<String, Long> getMemstoreReadRequestsCount(String table);
 
-
+  /**
+   * @return number of get requests from file per store for this table
+   */
+  Map<String, Long> getMixedRequestsCount(String table);

Review comment:
       We dont say whether it is read reqs in this method name.  This is the total  #rows reads from this Store right?  Can we name in that way and avoid the term mixed?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-643171427


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   3m 10s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m 42s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 27s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   6m 20s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 20s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 40s |  hbase-server in master failed.  |
   | -0 :warning: |  patch  |   7m 40s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 34s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 29s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 29s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   6m 23s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 41s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 41s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 212m  5s |  hbase-server in the patch passed.  |
   |  |   | 245m 49s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.9 Server=19.03.9 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 76d9aa57d59e 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 42a82df3af |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/testReport/ |
   | Max. process+thread count | 3075 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/12/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-616582904


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   0m 40s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  3s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 32s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   4m  9s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 43s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  8s |  branch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 19s |  hbase-hadoop-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop2-compat in master failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in master failed.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 15s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 58s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 41s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 41s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   5m  9s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 17s |  hbase-hadoop-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 18s |  hbase-hadoop2-compat in the patch failed.  |
   | -0 :warning: |  javadoc  |   0m 39s |  hbase-server in the patch failed.  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 27s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 30s |  hbase-hadoop2-compat in the patch passed.  |
   | -1 :x: |  unit  |   6m 53s |  hbase-server in the patch failed.  |
   |  |   |  35m 25s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk11-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 48ac3509d3e2 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / bcacc4ce93 |
   | Default Java | 2020-01-14 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk11-hadoop3-check/output/branch-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-hadoop2-compat.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk11-hadoop3-check/output/patch-javadoc-hbase-server.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/artifact/yetus-jdk11-hadoop3-check/output/patch-unit-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/testReport/ |
   | Max. process+thread count | 695 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/1/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-618960621


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  5s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 31s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 49s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 33s |  master passed  |
   | +1 :green_heart: |  spotbugs  |   2m 54s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 11s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 38s |  the patch passed  |
   | -0 :warning: |  checkstyle  |   1m 12s |  hbase-server: The patch generated 3 new + 28 unchanged - 0 fixed = 31 total (was 28)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  hadoopcheck  |  12m 15s |  Patch does not cause any errors with Hadoop 3.1.2 3.2.1.  |
   | +1 :green_heart: |  spotbugs  |   3m 23s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  asflicense  |   0m 31s |  The patch does not generate ASF License warnings.  |
   |  |   |  39m 44s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-general-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | dupname asflicense spotbugs hadoopcheck hbaseanti checkstyle |
   | uname | Linux 6a74884112e6 4.15.0-91-generic #92-Ubuntu SMP Fri Feb 28 11:09:48 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / 023c6fe56b |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/artifact/yetus-general-check/output/diff-checkstyle-hbase-server.txt |
   | Max. process+thread count | 84 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/4/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) spotbugs=3.1.12 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r439580608



##########
File path: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java
##########
@@ -41,39 +40,42 @@
   private final HRegionServer regionServer;
   private ScheduledExecutorService executor;
   private Runnable runnable;
-  private long period;
+  private static final int PERIOD = 45;
   private ScheduledFuture<?> tableMetricsUpdateTask;
   private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap
     = new ConcurrentHashMap<>();
 
   public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
     this.regionServer = regionServer;
-    this.period = regionServer.getConfiguration().getLong(HConstants.REGIONSERVER_METRICS_PERIOD,
-      HConstants.DEFAULT_REGIONSERVER_METRICS_PERIOD) + 1000;
     this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
     this.runnable = new TableMetricsWrapperRunnable();
-    this.tableMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, period,
-      this.period, TimeUnit.MILLISECONDS);
+    this.tableMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, PERIOD,

Review comment:
       This is not a bug fix and also the config based update I believe it was just added because the MetricsREgionServer based aggregate therad was having that config. But if you see the MetricRegionSource it is same 45 sec. The period of updation was rather too frequent. I just changed it to be in sync with MetricREgionSource.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] Apache-HBase commented on issue #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-617108940


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   3m 56s |  Docker mode activated.  |
   | -0 :warning: |  yetus  |   0m  4s |  Unprocessed flag(s): --brief-report-file --spotbugs-strict-precheck --whitespace-eol-ignore-list --whitespace-tabs-ignore-list --quick-hadoopcheck  |
   ||| _ Prechecks _ |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 33s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   3m 37s |  master passed  |
   | +1 :green_heart: |  compile  |   1m 30s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 59s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   1m 11s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 16s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   3m 22s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 30s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 30s |  the patch passed  |
   | +1 :green_heart: |  shadedjars  |   4m 57s |  patch has no errors when building our shaded downstream artifacts.  |
   | -0 :warning: |  javadoc  |   0m 36s |  hbase-server generated 3 new + 0 unchanged - 0 fixed = 3 total (was 0)  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 27s |  hbase-hadoop-compat in the patch passed.  |
   | +1 :green_heart: |  unit  |   0m 28s |  hbase-hadoop2-compat in the patch passed.  |
   | +1 :green_heart: |  unit  | 132m  1s |  hbase-server in the patch passed.  |
   |  |   | 161m 59s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.8 Server=19.03.8 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-jdk8-hadoop3-check/output/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1552 |
   | Optional Tests | javac javadoc unit shadedjars compile |
   | uname | Linux 64438ef070dd 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/hbase-personality.sh |
   | git revision | master / f48b50964c |
   | Default Java | 1.8.0_232 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/artifact/yetus-jdk8-hadoop3-check/output/diff-javadoc-javadoc-hbase-server.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/testReport/ |
   | Max. process+thread count | 4872 (vs. ulimit of 12500) |
   | modules | C: hbase-hadoop-compat hbase-hadoop2-compat hbase-server U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1552/2/console |
   | versions | git=2.17.1 maven=(cecedd343002696d0abb50b32b541b8a6ba2883f) |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#issuecomment-635993029


   A new push which has the following changes
   -> Both gets and scans are covered in the same patch (we don do anything special for these just treat as same).
   -> Whether it is a scan or get - when we are at the StoreScanner loop where we check cell by cell till a row is done - we check if all the cells in that current loop till we return are from memstore or from file. Only then the corresponding metric is incremented. We no longer track the total gets that has landed on the StoreScanner. Say if out of 10 cells even if one cell is from memstore and remaining from file we don't account it with this metric. 
   -> The table level metric will now have a new aggregated metric
   
   >     "Namespace_default_table_TestTable_store_info0_metric_readRequestCountOnMemstore":1766,
   
   >    "Namespace_default_table_TestTable_store_info0_metric_readRequestCountOnFiles": 37828,
   
   This will be mapped per store for each table. 
   -> the patch also handles the cases where same family names can be across tables. Earlier that was not getting handled and now at the MetricStoreSource layer we add the region name and store name for the uniqueness.
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [hbase] ramkrish86 commented on a change in pull request #1552: HBASE-24205 Create metric to know the number of reads that happens fr…

Posted by GitBox <gi...@apache.org>.
ramkrish86 commented on a change in pull request #1552:
URL: https://github.com/apache/hbase/pull/1552#discussion_r414461189



##########
File path: hbase-hadoop2-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsStoreAggregateSourceImpl.java
##########
@@ -0,0 +1,116 @@
+/**
+ * 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.regionserver;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
+import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class MetricsStoreAggregateSourceImpl extends BaseSourceImpl
+    implements MetricsStoreAggregateSource {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricsStoreAggregateSourceImpl.class);
+
+  private final MetricsExecutorImpl executor = new MetricsExecutorImpl();
+
+  private final Set<MetricsStoreSource> storeSources =
+      Collections.newSetFromMap(new ConcurrentHashMap<MetricsStoreSource, Boolean>());

Review comment:
       the compiler was saying some warnings. Hence left it as is. 




----------------------------------------------------------------
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:
users@infra.apache.org