You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2020/02/12 17:01:20 UTC

[hbase] branch branch-2.1 updated: HBASE-23802 Remove unnecessary Configuration instantiation in LossyAccounting (#1127)

This is an automated email from the ASF dual-hosted git repository.

ndimiduk pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.1 by this push:
     new c540d90  HBASE-23802 Remove unnecessary Configuration instantiation in LossyAccounting (#1127)
c540d90 is described below

commit c540d906df35cf43d324e5bcb0ca614e4f7878bd
Author: Nick Dimiduk <nd...@apache.org>
AuthorDate: Thu Feb 6 11:56:45 2020 -0800

    HBASE-23802 Remove unnecessary Configuration instantiation in LossyAccounting (#1127)
    
    Signed-off-by: stack <st...@apache.org>
---
 .../hadoop/hbase/coprocessor/MetaTableMetrics.java | 50 ++++++++++------------
 .../apache/hadoop/hbase/util/LossyCounting.java    | 25 +++++------
 .../hadoop/hbase/util/TestLossyCounting.java       | 35 +++++++--------
 3 files changed, 54 insertions(+), 56 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
index 70e8df1..0918e59 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MetaTableMetrics.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -20,12 +20,12 @@
 package org.apache.hadoop.hbase.coprocessor;
 
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.Set;
-
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.TableName;
@@ -36,13 +36,12 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Row;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.LossyCounting;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.yetus.audience.InterfaceAudience;
-
 import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
 
-
 /**
  * A coprocessor that collects metrics from meta table.
  * <p>
@@ -59,14 +58,14 @@ public class MetaTableMetrics implements RegionCoprocessor {
   private MetricRegistry registry;
   private LossyCounting clientMetricsLossyCounting, regionMetricsLossyCounting;
   private boolean active = false;
-  private Set<String> metrics = new HashSet<String>();
+  private Set<String> metrics = new HashSet<>();
 
   enum MetaTableOps {
-    GET, PUT, DELETE;
+    GET, PUT, DELETE,
   }
 
-  private ImmutableMap<Class, MetaTableOps> opsNameMap =
-      ImmutableMap.<Class, MetaTableOps>builder()
+  private ImmutableMap<Class<? extends Row>, MetaTableOps> opsNameMap =
+      ImmutableMap.<Class<? extends Row>, MetaTableOps>builder()
               .put(Put.class, MetaTableOps.PUT)
               .put(Get.class, MetaTableOps.GET)
               .put(Delete.class, MetaTableOps.DELETE)
@@ -93,7 +92,7 @@ public class MetaTableMetrics implements RegionCoprocessor {
 
     @Override
     public void preDelete(ObserverContext<RegionCoprocessorEnvironment> e, Delete delete,
-        WALEdit edit, Durability durability) throws IOException {
+        WALEdit edit, Durability durability) {
       registerAndMarkMetrics(e, delete);
     }
 
@@ -113,13 +112,12 @@ public class MetaTableMetrics implements RegionCoprocessor {
      * @param op such as get, put or delete.
      */
     private String getTableNameFromOp(Row op) {
-      String tableName = null;
-      String tableRowKey = new String(((Row) op).getRow(), StandardCharsets.UTF_8);
-      if (tableRowKey.isEmpty()) {
+      final String tableRowKey = Bytes.toString(op.getRow());
+      if (StringUtils.isEmpty(tableRowKey)) {
         return null;
       }
-      tableName = tableRowKey.split(",").length > 0 ? tableRowKey.split(",")[0] : null;
-      return tableName;
+      final String[] splits = tableRowKey.split(",");
+      return splits.length > 0 ? splits[0] : null;
     }
 
     /**
@@ -127,13 +125,12 @@ public class MetaTableMetrics implements RegionCoprocessor {
      * @param op  such as get, put or delete.
      */
     private String getRegionIdFromOp(Row op) {
-      String regionId = null;
-      String tableRowKey = new String(((Row) op).getRow(), StandardCharsets.UTF_8);
-      if (tableRowKey.isEmpty()) {
+      final String tableRowKey = Bytes.toString(op.getRow());
+      if (StringUtils.isEmpty(tableRowKey)) {
         return null;
       }
-      regionId = tableRowKey.split(",").length > 2 ? tableRowKey.split(",")[2] : null;
-      return regionId;
+      final String[] splits = tableRowKey.split(",");
+      return splits.length > 2 ? splits[2] : null;
     }
 
     private boolean isMetaTableOp(ObserverContext<RegionCoprocessorEnvironment> e) {
@@ -279,14 +276,13 @@ public class MetaTableMetrics implements RegionCoprocessor {
           .equals(TableName.META_TABLE_NAME)) {
       RegionCoprocessorEnvironment regionCoprocessorEnv = (RegionCoprocessorEnvironment) env;
       registry = regionCoprocessorEnv.getMetricRegistryForRegionServer();
-      LossyCounting.LossyCountingListener listener = new LossyCounting.LossyCountingListener(){
-        @Override public void sweep(String key) {
-          registry.remove(key);
-          metrics.remove(key);
-        }
+      LossyCounting.LossyCountingListener listener = key -> {
+        registry.remove(key);
+        metrics.remove(key);
       };
-      clientMetricsLossyCounting = new LossyCounting("clientMetaMetrics",listener);
-      regionMetricsLossyCounting = new LossyCounting("regionMetaMetrics",listener);
+      final Configuration conf = regionCoprocessorEnv.getConfiguration();
+      clientMetricsLossyCounting = new LossyCounting("clientMetaMetrics", conf, listener);
+      regionMetricsLossyCounting = new LossyCounting("regionMetaMetrics", conf, listener);
       // only be active mode when this region holds meta table.
       active = true;
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java
index ca1a014..34fed1b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/LossyCounting.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -22,15 +22,12 @@ package org.apache.hadoop.hbase.util;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
-
 /**
  * LossyCounting utility, bounded data structure that maintains approximate high frequency
  * elements in data stream.
@@ -41,13 +38,11 @@ import org.slf4j.LoggerFactory;
  * Based on paper:
  * http://www.vldb.org/conf/2002/S10P03.pdf
  */
-
 @InterfaceAudience.Private
 public class LossyCounting {
   private static final Logger LOG = LoggerFactory.getLogger(LossyCounting.class);
   private long bucketSize;
   private int currentTerm;
-  private double errorRate;
   private Map<String, Integer> data;
   private long totalDataCount;
   private String name;
@@ -57,8 +52,11 @@ public class LossyCounting {
     void sweep(String key);
   }
 
-  public LossyCounting(double errorRate, String name, LossyCountingListener listener) {
-    this.errorRate = errorRate;
+  LossyCounting(String name, double errorRate) {
+    this(name, errorRate, null);
+  }
+
+  public LossyCounting(String name, double errorRate, LossyCountingListener listener) {
     this.name = name;
     if (errorRate < 0.0 || errorRate > 1.0) {
       throw new IllegalArgumentException(" Lossy Counting error rate should be within range [0,1]");
@@ -71,9 +69,12 @@ public class LossyCounting {
     calculateCurrentTerm();
   }
 
-  public LossyCounting(String name, LossyCountingListener listener) {
-    this(HBaseConfiguration.create().getDouble(HConstants.DEFAULT_LOSSY_COUNTING_ERROR_RATE, 0.02),
-        name, listener);
+  LossyCounting(String name, Configuration conf) {
+    this(name, conf, null);
+  }
+
+  public LossyCounting(String name, Configuration conf, LossyCountingListener listener) {
+    this(name, conf.getDouble(HConstants.DEFAULT_LOSSY_COUNTING_ERROR_RATE, 0.02), listener);
   }
 
   private void addByOne(String key) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestLossyCounting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestLossyCounting.java
index 5240c40..f08a20c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestLossyCounting.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestLossyCounting.java
@@ -1,4 +1,4 @@
-/**
+/*
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -20,11 +20,12 @@
 package org.apache.hadoop.hbase.util;
 
 import static org.junit.Assert.assertEquals;
-
+import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -36,31 +37,33 @@ public class TestLossyCounting {
   public static final HBaseClassTestRule CLASS_RULE =
       HBaseClassTestRule.forClass(TestLossyCounting.class);
 
+  private final Configuration conf = HBaseConfiguration.create();
+
   @Test
   public void testBucketSize() {
-    LossyCounting lossyCounting = new LossyCounting(0.01, "testBucketSize", null);
+    LossyCounting lossyCounting = new LossyCounting("testBucketSize", 0.01);
     assertEquals(100L, lossyCounting.getBucketSize());
-    LossyCounting lossyCounting2 = new LossyCounting("testBucketSize2", null);
+    LossyCounting lossyCounting2 = new LossyCounting("testBucketSize2", conf);
     assertEquals(50L, lossyCounting2.getBucketSize());
   }
 
   @Test
   public void testAddByOne() {
-    LossyCounting lossyCounting = new LossyCounting(0.01, "testAddByOne", null);
-    for(int i = 0; i < 100; i++){
+    LossyCounting lossyCounting = new LossyCounting("testAddByOne", 0.01);
+    for (int i = 0; i < 100; i++) {
       String key = "" + i;
       lossyCounting.add(key);
     }
     assertEquals(100L, lossyCounting.getDataSize());
-    for(int i = 0; i < 100; i++){
+    for (int i = 0; i < 100; i++) {
       String key = "" + i;
-      assertEquals(true, lossyCounting.contains(key));
+      assertTrue(lossyCounting.contains(key));
     }
   }
 
   @Test
-  public void testSweep1() {
-    LossyCounting lossyCounting = new LossyCounting(0.01, "testSweep1", null);
+  public void testSweep1() throws Exception {
+    LossyCounting lossyCounting = new LossyCounting("testSweep1", 0.01);
     for(int i = 0; i < 400; i++){
       String key = "" + i;
       lossyCounting.add(key);
@@ -71,9 +74,9 @@ public class TestLossyCounting {
   }
 
   @Test
-  public void testSweep2() {
-    LossyCounting lossyCounting = new LossyCounting(0.1, "testSweep2", null);
-    for(int i = 0; i < 10; i++){
+  public void testSweep2() throws Exception {
+    LossyCounting lossyCounting = new LossyCounting("testSweep2", 0.1);
+    for (int i = 0; i < 10; i++) {
       String key = "" + i;
       lossyCounting.add(key);
     }
@@ -84,6 +87,4 @@ public class TestLossyCounting {
     }
     assertEquals(1L, lossyCounting.getDataSize());
   }
-
-
-}
\ No newline at end of file
+}