[ 
https://issues.apache.org/jira/browse/HBASE-21800?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16754711#comment-16754711
 ] 

Sakthi edited comment on HBASE-21800 at 1/29/19 8:01 AM:
---------------------------------------------------------

The cause:

For a new meter, following is the code progress:
{code:java}
...
registerLossyCountingMeterIfNotPresent(e, clientRequestMeter, 
clientMetricsLossyCounting);
markMeterIfPresent(clientRequestMeter);
...
{code}
While registering the metric, in registerLossyCountingMeterIfNotPresent() , a 
newly added meter might not be added into the requestsMap if it becomes 
eligible to be swept off by the lossycounter:
{code:java}
    private void registerLossyCountingMeterIfNotPresent(
        ObserverContext<RegionCoprocessorEnvironment> e,
        String requestMeter, LossyCounting lossyCounting) {
      ...
      Set<String> metersToBeRemoved = lossyCounting.addByOne(requestMeter);
      if(!requestsMap.containsKey(requestMeter) && 
metersToBeRemoved.contains(requestMeter)){
        ...
        // newly added meter is swept by lossy counting cleanup. No need to put 
it into requestsMap.
        return;
      }
     ...
    }
{code}
But, after registering, it's assumed in markMeterIfPresent() that every attempt 
to register a meter would always be successful and hence it can be accessed 
through the requestsMap:
{code:java}
    private void markMeterIfPresent(String requestMeter) {
      ...
      Metric metric =
          requestsMap.get(requestMeter).isPresent() ? 
requestsMap.get(requestMeter).get() : null;
      ...
    }
{code}
Solution:
 A "requestsMap.containsKey(meterName)" check before accessing it.


was (Author: jatsakthi):
The cause:

For a new meter, following is the code progress:
{code:java}
...
registerLossyCountingMeterIfNotPresent(e, clientRequestMeter, 
clientMetricsLossyCounting);
markMeterIfPresent(clientRequestMeter);
...
{code}
While registering the metric, in registerLossyCountingMeterIfNotPresent() , a 
newly added meter might not be added into the requestsMap if it becomes 
eligible to be swept off by the lossycounter:
{code:java}
    private void registerLossyCountingMeterIfNotPresent(
        ObserverContext<RegionCoprocessorEnvironment> e,
        String requestMeter, LossyCounting lossyCounting) {
      ...
      Set<String> metersToBeRemoved = lossyCounting.addByOne(requestMeter);
      if(!requestsMap.containsKey(requestMeter) && 
metersToBeRemoved.contains(requestMeter)){
        ...
        // newly added meter is swept by lossy counting cleanup. No need to put 
it into requestsMap.
        return;
      }
     ...
    }
{code}
But, after registering, it's assume in markMeterIfPresent() that every attempt 
to register a meter would always be successful and hence it can be accessed 
through the requestsMap:
{code:java}
    private void markMeterIfPresent(String requestMeter) {
      ...
      Metric metric =
          requestsMap.get(requestMeter).isPresent() ? 
requestsMap.get(requestMeter).get() : null;
      ...
    }
{code}

Solution:
A "requestsMap.containsKey(meterName)" check before accessing it.

> RegionServer aborted due to NPE from MetaTableMetrics coprocessor
> -----------------------------------------------------------------
>
>                 Key: HBASE-21800
>                 URL: https://issues.apache.org/jira/browse/HBASE-21800
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Sakthi
>            Assignee: Sakthi
>            Priority: Major
>
> I was just playing around the code, trying to capture "Top k" table metrics 
> from MetaMetrics, when I bumped into this issue. Though currently we are not 
> capturing "Top K" table metrics, but we can encounter this issue because of 
> the "Top k Clients" that is implemented using the LossyAlgo.
>  
> RegionServer gets aborted due to a NPE from MetaTableMetrics coprocessor. The 
> log looks somewhat like this:
> {code:java}
> 2019-01-28 23:31:10,311 ERROR 
> [RpcServer.priority.FPBQ.Fifo.handler=19,queue=1,port=16020] 
> coprocessor.CoprocessorHost: The coprocessor 
> org.apache.hadoop.hbase.coprocessor.MetaTableMetrics threw 
> java.lang.NullPointerException
> java.lang.NullPointerException
>       at 
> org.apache.hadoop.hbase.coprocessor.MetaTableMetrics$ExampleRegionObserverMeta.markMeterIfPresent(MetaTableMetrics.java:123)
>       at 
> org.apache.hadoop.hbase.coprocessor.MetaTableMetrics$ExampleRegionObserverMeta.tableMetricRegisterAndMark2(MetaTableMetrics.java:233)
>       at 
> org.apache.hadoop.hbase.coprocessor.MetaTableMetrics$ExampleRegionObserverMeta.preGetOp(MetaTableMetrics.java:82)
>       at 
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$19.call(RegionCoprocessorHost.java:840)
>       at 
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$19.call(RegionCoprocessorHost.java:837)
>       at 
> org.apache.hadoop.hbase.coprocessor.CoprocessorHost$ObserverOperationWithoutResult.callObserver(CoprocessorHost.java:551)
>       at 
> org.apache.hadoop.hbase.coprocessor.CoprocessorHost.execOperation(CoprocessorHost.java:625)
>       at 
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.preGet(RegionCoprocessorHost.java:837)
>       at 
> org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2608)
>       at 
> org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2547)
>       at 
> org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:41998)
>       at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:413)
>       at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:130)
>       at 
> org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:324)
>       at 
> org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:304)
> 2019-01-28 23:31:10,314 ERROR 
> [RpcServer.priority.FPBQ.Fifo.handler=19,queue=1,port=16020] 
> regionserver.HRegionServer: ***** ABORTING region server 
> 10.0.0.24,16020,1548747043814: The coprocessor 
> org.apache.hadoop.hbase.coprocessor.MetaTableMetrics threw 
> java.lang.NullPointerException *****
> java.lang.NullPointerException
>       at 
> org.apache.hadoop.hbase.coprocessor.MetaTableMetrics$ExampleRegionObserverMeta.markMeterIfPresent(MetaTableMetrics.java:123)
>       at 
> org.apache.hadoop.hbase.coprocessor.MetaTableMetrics$ExampleRegionObserverMeta.tableMetricRegisterAndMark2(MetaTableMetrics.java:233)
>       at 
> org.apache.hadoop.hbase.coprocessor.MetaTableMetrics$ExampleRegionObserverMeta.preGetOp(MetaTableMetrics.java:82)
>       at 
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$19.call(RegionCoprocessorHost.java:840)
>       at 
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$19.call(RegionCoprocessorHost.java:837)
>       at 
> org.apache.hadoop.hbase.coprocessor.CoprocessorHost$ObserverOperationWithoutResult.callObserver(CoprocessorHost.java:551)
>       at 
> org.apache.hadoop.hbase.coprocessor.CoprocessorHost.execOperation(CoprocessorHost.java:625)
>       at 
> org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.preGet(RegionCoprocessorHost.java:837)
>       at 
> org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2608)
>       at 
> org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2547)
>       at 
> org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:41998)
>       at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:413)
>       at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:130)
>       at 
> org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:324)
>       at 
> org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:304) 
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to