You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by xu...@apache.org on 2019/09/16 17:56:59 UTC

[hbase] branch branch-2 updated: HBASE-22804 Provide an API to get list of successful regions and total expected regions in Canary

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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new aec5638  HBASE-22804 Provide an API to get list of successful regions and total expected regions in Canary
aec5638 is described below

commit aec5638addffd56a7d142340f4f26fbdf145467b
Author: Caroline Zhou <ca...@salesforce.com>
AuthorDate: Wed Aug 7 13:42:18 2019 -0700

    HBASE-22804 Provide an API to get list of successful regions and total expected regions in Canary
    
    Signed-off-by: Xu Cang <xu...@apache.org>
---
 .../java/org/apache/hadoop/hbase/tool/Canary.java  | 139 ++++++++++++++++++++-
 .../apache/hadoop/hbase/tool/TestCanaryTool.java   |  53 ++++++++
 2 files changed, 191 insertions(+), 1 deletion(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
index ac03937..fae904a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
@@ -47,6 +47,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+
 import org.apache.commons.lang3.time.StopWatch;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
@@ -133,6 +134,10 @@ public final class Canary implements Tool {
     long incWriteFailureCount();
     Map<String,String> getWriteFailures();
     void updateWriteFailures(String regionName, String serverName);
+    long getReadSuccessCount();
+    long incReadSuccessCount();
+    long getWriteSuccessCount();
+    long incWriteSuccessCount();
   }
 
   /**
@@ -140,7 +145,9 @@ public final class Canary implements Tool {
    */
   public static class StdOutSink implements Sink {
     private AtomicLong readFailureCount = new AtomicLong(0),
-        writeFailureCount = new AtomicLong(0);
+        writeFailureCount = new AtomicLong(0),
+        readSuccessCount = new AtomicLong(0),
+        writeSuccessCount = new AtomicLong(0);
     private Map<String, String> readFailures = new ConcurrentHashMap<>();
     private Map<String, String> writeFailures = new ConcurrentHashMap<>();
 
@@ -183,6 +190,26 @@ public final class Canary implements Tool {
     public void updateWriteFailures(String regionName, String serverName) {
       writeFailures.put(regionName, serverName);
     }
+
+    @Override
+    public long getReadSuccessCount() {
+      return readSuccessCount.get();
+    }
+
+    @Override
+    public long incReadSuccessCount() {
+      return readSuccessCount.incrementAndGet();
+    }
+
+    @Override
+    public long getWriteSuccessCount() {
+      return writeSuccessCount.get();
+    }
+
+    @Override
+    public long incWriteSuccessCount() {
+      return writeSuccessCount.incrementAndGet();
+    }
   }
 
   /**
@@ -219,6 +246,7 @@ public final class Canary implements Tool {
   public static class RegionStdOutSink extends StdOutSink {
     private Map<String, LongAdder> perTableReadLatency = new HashMap<>();
     private LongAdder writeLatency = new LongAdder();
+    private final Map<String, RegionTaskResult> regionMap = new ConcurrentHashMap<>();
 
     public void publishReadFailure(ServerName serverName, RegionInfo region, Exception e) {
       incReadFailureCount();
@@ -234,6 +262,10 @@ public final class Canary implements Tool {
 
     public void publishReadTiming(ServerName serverName, RegionInfo region,
         ColumnFamilyDescriptor column, long msTime) {
+      incReadSuccessCount();
+      RegionTaskResult res = this.regionMap.get(region.getRegionNameAsString());
+      res.setReadSuccess();
+      res.setReadLatency(msTime);
       LOG.info("Read from {} on {} {} in {}ms", region.getRegionNameAsString(), serverName,
           column.getNameAsString(), msTime);
     }
@@ -252,6 +284,10 @@ public final class Canary implements Tool {
 
     public void publishWriteTiming(ServerName serverName, RegionInfo region,
         ColumnFamilyDescriptor column, long msTime) {
+      incWriteSuccessCount();
+      RegionTaskResult res = this.regionMap.get(region.getRegionNameAsString());
+      res.setWriteSuccess();
+      res.setWriteLatency(msTime);
       LOG.info("Write to {} on {} {} in {}ms",
         region.getRegionNameAsString(), serverName, column.getNameAsString(), msTime);
     }
@@ -273,6 +309,14 @@ public final class Canary implements Tool {
     public LongAdder getWriteLatency() {
       return this.writeLatency;
     }
+
+    public Map<String, RegionTaskResult> getRegionMap() {
+      return this.regionMap;
+    }
+
+    public int getTotalExpectedRegions() {
+      return this.regionMap.size();
+    }
   }
 
   /**
@@ -932,6 +976,96 @@ public final class Canary implements Tool {
   }
 
   /**
+   * Canary region mode-specific data structure which stores information about each region
+   * to be scanned
+   */
+  public static class RegionTaskResult {
+    private RegionInfo region;
+    private TableName tableName;
+    private ServerName serverName;
+    private AtomicLong readLatency = null;
+    private AtomicLong writeLatency = null;
+    private boolean readSuccess = false;
+    private boolean writeSuccess = false;
+
+    public RegionTaskResult(RegionInfo region, TableName tableName, ServerName serverName) {
+      this.region = region;
+      this.tableName = tableName;
+      this.serverName = serverName;
+    }
+
+    public RegionInfo getRegionInfo() {
+      return this.region;
+    }
+
+    public String getRegionNameAsString() {
+      return this.region.getRegionNameAsString();
+    }
+
+    public TableName getTableName() {
+      return this.tableName;
+    }
+
+    public String getTableNameAsString() {
+      return this.tableName.getNameAsString();
+    }
+
+    public ServerName getServerName() {
+      return this.serverName;
+    }
+
+    public String getServerNameAsString() {
+      return this.serverName.getServerName();
+    }
+
+    public long getReadLatency() {
+      if (this.readLatency == null) {
+        return -1;
+      }
+      return this.readLatency.get();
+    }
+
+    public void setReadLatency(long readLatency) {
+      if (this.readLatency != null) {
+        this.readLatency.set(readLatency);
+      } else {
+        this.readLatency = new AtomicLong(readLatency);
+      }
+    }
+
+    public long getWriteLatency() {
+      if (this.writeLatency == null) {
+        return -1;
+      }
+      return this.writeLatency.get();
+    }
+
+    public void setWriteLatency(long writeLatency) {
+      if (this.writeLatency != null) {
+        this.writeLatency.set(writeLatency);
+      } else {
+        this.writeLatency = new AtomicLong(writeLatency);
+      }
+    }
+
+    public boolean isReadSuccess() {
+      return this.readSuccess;
+    }
+
+    public void setReadSuccess() {
+      this.readSuccess = true;
+    }
+
+    public boolean isWriteSuccess() {
+      return this.writeSuccess;
+    }
+
+    public void setWriteSuccess() {
+      this.writeSuccess = true;
+    }
+  }
+
+  /**
    * A Factory method for {@link Monitor}.
    * Makes a RegionServerMonitor, or a ZooKeeperMonitor, or a RegionMonitor.
    * @param index a start index for monitor target
@@ -1346,6 +1480,9 @@ public final class Canary implements Tool {
           RegionInfo region = location.getRegion();
           tasks.add(new RegionTask(admin.getConnection(), region, rs, (RegionStdOutSink)sink,
               taskType, rawScanEnabled, rwLatency));
+          Map<String, RegionTaskResult> regionMap = ((RegionStdOutSink) sink).getRegionMap();
+          regionMap.put(region.getRegionNameAsString(), new RegionTaskResult(region,
+              region.getTable(), rs));
         }
         return executor.invokeAll(tasks);
       }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
index bc745f6c..d29f24a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
@@ -18,7 +18,10 @@
 package org.apache.hadoop.hbase.tool;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.ArgumentMatchers.argThat;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Matchers.eq;
@@ -29,6 +32,7 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
@@ -127,6 +131,55 @@ public class TestCanaryTool {
   }
 
   @Test
+  public void testCanaryRegionTaskResult() throws Exception {
+    TableName tableName = TableName.valueOf("testCanaryRegionTaskResult");
+    Table table = testingUtility.createTable(tableName, new byte[][] { FAMILY });
+    // insert some test rows
+    for (int i=0; i<1000; i++) {
+      byte[] iBytes = Bytes.toBytes(i);
+      Put p = new Put(iBytes);
+      p.addColumn(FAMILY, COLUMN, iBytes);
+      table.put(p);
+    }
+    ExecutorService executor = new ScheduledThreadPoolExecutor(1);
+    Canary.RegionStdOutSink sink = spy(new Canary.RegionStdOutSink());
+    Canary canary = new Canary(executor, sink);
+    String[] args = { "-writeSniffing", "-t", "10000", "testCanaryRegionTaskResult" };
+    assertEquals(0, ToolRunner.run(testingUtility.getConfiguration(), canary, args));
+
+    assertTrue("verify read success count > 0", sink.getReadSuccessCount() > 0);
+    assertTrue("verify write success count > 0", sink.getWriteSuccessCount() > 0);
+    verify(sink, atLeastOnce()).publishReadTiming(isA(ServerName.class), isA(RegionInfo.class),
+      isA(ColumnFamilyDescriptor.class), anyLong());
+    verify(sink, atLeastOnce()).publishWriteTiming(isA(ServerName.class), isA(RegionInfo.class),
+      isA(ColumnFamilyDescriptor.class), anyLong());
+
+    assertTrue("canary should expect to scan at least 1 region",
+      sink.getTotalExpectedRegions() > 0);
+    Map<String, Canary.RegionTaskResult> regionMap = sink.getRegionMap();
+    assertFalse("verify region map has size > 0", regionMap.isEmpty());
+
+    for (String regionName : regionMap.keySet()) {
+      Canary.RegionTaskResult res = regionMap.get(regionName);
+      assertNotNull("verify each expected region has a RegionTaskResult object in the map", res);
+      assertNotNull("verify getRegionNameAsString()", regionName);
+      assertNotNull("verify getRegionInfo()", res.getRegionInfo());
+      assertNotNull("verify getTableName()", res.getTableName());
+      assertNotNull("verify getTableNameAsString()", res.getTableNameAsString());
+      assertNotNull("verify getServerName()", res.getServerName());
+      assertNotNull("verify getServerNameAsString()", res.getServerNameAsString());
+
+      if (regionName.contains(Canary.DEFAULT_WRITE_TABLE_NAME.getNameAsString())) {
+        assertTrue("write to region " + regionName + " succeeded", res.isWriteSuccess());
+        assertTrue("write took some time", res.getWriteLatency() > -1);
+      } else {
+        assertTrue("read from region " + regionName + " succeeded", res.isReadSuccess());
+        assertTrue("read took some time", res.getReadLatency() > -1);
+      }
+    }
+  }
+
+  @Test
   @Ignore("Intermittent argument matching failures, see HBASE-18813")
   public void testReadTableTimeouts() throws Exception {
     final TableName [] tableNames = new TableName[2];