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/13 01:36:23 UTC

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

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

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


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

commit 1c3eaedacb6c08cfc9360738a89c413740ff6c32
Author: Nick Dimiduk <nd...@apache.org>
AuthorDate: Wed Feb 12 17:36:12 2020 -0800

    HBASE-23802 Remove unnecessary Configuration instantiation in LossyAccounting (#1127) (#1166)
    
    Signed-off-by: stack <st...@apache.org>
---
 .../hadoop/hbase/coprocessor/MetaTableMetrics.java | 64 ++++++++++++----------
 .../apache/hadoop/hbase/util/LossyCounting.java    | 21 ++++---
 .../hadoop/hbase/util/TestLossyCounting.java       | 26 +++++----
 3 files changed, 63 insertions(+), 48 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 a5a880b..66aab20 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,24 +1,29 @@
-/**
- * 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.
+/*
+ * 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.coprocessor;
 
+import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-
-import com.google.common.collect.ImmutableMap;
-
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.TableName;
@@ -31,6 +36,7 @@ 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.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.LossyCounting;
 
 /**
@@ -48,15 +54,18 @@ public class MetaTableMetrics extends BaseRegionObserver {
   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().put(Put.class, MetaTableOps.PUT)
-          .put(Get.class, MetaTableOps.GET).put(Delete.class, MetaTableOps.DELETE).build();
+  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)
+              .build();
 
   @Override
   public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get, List<Cell> results)
@@ -93,13 +102,12 @@ public class MetaTableMetrics extends BaseRegionObserver {
    * @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);
+    final String tableRowKey = Bytes.toString(op.getRow());
     if (tableRowKey.isEmpty()) {
       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;
   }
 
   /**
@@ -108,13 +116,12 @@ public class MetaTableMetrics extends BaseRegionObserver {
    * @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);
+    final String tableRowKey = Bytes.toString(op.getRow());
     if (tableRowKey.isEmpty()) {
       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) {
@@ -257,8 +264,9 @@ public class MetaTableMetrics extends BaseRegionObserver {
           metrics.remove(key);
         }
       };
-      clientMetricsLossyCounting = new LossyCounting(listener);
-      regionMetricsLossyCounting = new LossyCounting(listener);
+      final Configuration conf = regionCoprocessorEnv.getConfiguration();
+      clientMetricsLossyCounting = new LossyCounting(conf, listener);
+      regionMetricsLossyCounting = new LossyCounting(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 255f720..ca00e80 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,8 +22,7 @@ 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.hadoop.hbase.classification.InterfaceAudience;
 
@@ -37,7 +36,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
  * Based on paper:
  * http://www.vldb.org/conf/2002/S10P03.pdf
  */
-
 @InterfaceAudience.Private
 public class LossyCounting {
   private long bucketSize;
@@ -50,6 +48,10 @@ public class LossyCounting {
     void sweep(String key);
   }
 
+  LossyCounting(double errorRate) {
+    this(errorRate, null);
+  }
+
   public LossyCounting(double errorRate, LossyCountingListener listener) {
     if (errorRate < 0.0 || errorRate > 1.0) {
       throw new IllegalArgumentException(" Lossy Counting error rate should be within range [0,1]");
@@ -62,9 +64,12 @@ public class LossyCounting {
     calculateCurrentTerm();
   }
 
-  public LossyCounting(LossyCountingListener listener) {
-    this(HBaseConfiguration.create().getDouble(HConstants.DEFAULT_LOSSY_COUNTING_ERROR_RATE, 0.02),
-        listener);
+  LossyCounting(Configuration conf) {
+    this(conf, null);
+  }
+
+  public LossyCounting(Configuration conf, LossyCountingListener listener) {
+    this(conf.getDouble(HConstants.DEFAULT_LOSSY_COUNTING_ERROR_RATE, 0.02), listener);
   }
 
   private void addByOne(String key) {
@@ -82,7 +87,7 @@ public class LossyCounting {
 
   public void add(String key) {
     addByOne(key);
-    if(totalDataCount % bucketSize == 0) {
+    if (totalDataCount % bucketSize == 0) {
       //sweep the entries at bucket boundaries
       sweep();
     }
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 548d31a..83016c0 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,7 +20,9 @@
 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.HBaseConfiguration;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 
@@ -30,31 +32,33 @@ import org.junit.experimental.categories.Category;
 @Category({MiscTests.class, SmallTests.class})
 public class TestLossyCounting {
 
+  private final Configuration conf = HBaseConfiguration.create();
+
   @Test
   public void testBucketSize() {
-    LossyCounting lossyCounting = new LossyCounting(0.01, null);
+    LossyCounting lossyCounting = new LossyCounting(0.01);
     assertEquals(100L, lossyCounting.getBucketSize());
-    LossyCounting lossyCounting2 = new LossyCounting(null);
+    LossyCounting lossyCounting2 = new LossyCounting(conf);
     assertEquals(50L, lossyCounting2.getBucketSize());
   }
 
   @Test
   public void testAddByOne() {
-    LossyCounting lossyCounting = new LossyCounting(0.01, null);
+    LossyCounting lossyCounting = new LossyCounting(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, null);
+    LossyCounting lossyCounting = new LossyCounting(0.01);
     for(int i = 0; i < 400; i++){
       String key = "" + i;
       lossyCounting.add(key);
@@ -66,7 +70,7 @@ public class TestLossyCounting {
 
   @Test
   public void testSweep2() {
-    LossyCounting lossyCounting = new LossyCounting(0.1, null);
+    LossyCounting lossyCounting = new LossyCounting(0.1);
     for(int i = 0; i < 10; i++){
       String key = "" + i;
       lossyCounting.add(key);
@@ -78,6 +82,4 @@ public class TestLossyCounting {
     }
     assertEquals(1L, lossyCounting.getDataSize());
   }
-
-
-}
\ No newline at end of file
+}