You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2016/07/05 18:13:32 UTC

[1/2] hbase git commit: HBASE-16091 Canary takes lot more time when there are delete markers in the table (Vishal Khandelwal)

Repository: hbase
Updated Branches:
  refs/heads/0.98 47c19607c -> e3ef8b69b
  refs/heads/branch-1 84dd9cbcb -> 8efc6148b


HBASE-16091 Canary takes lot more time when there are delete markers in the table (Vishal Khandelwal)

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java

Amending-Author: Andrew Purtell <ap...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/8efc6148
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/8efc6148
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/8efc6148

Branch: refs/heads/branch-1
Commit: 8efc6148b9ccaa29d2608d1d7348d0d3c5d8158d
Parents: 84dd9cb
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Jul 5 10:11:08 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jul 5 10:34:27 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HConstants.java     |  2 +
 .../org/apache/hadoop/hbase/tool/Canary.java    | 62 +++++++++++++++-----
 .../hadoop/hbase/tool/TestCanaryTool.java       | 29 +++++++--
 3 files changed, 73 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8efc6148/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index e062989..769945b 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -1237,6 +1237,8 @@ public final class HConstants {
 
   public static final String HBASE_CANARY_WRITE_TABLE_CHECK_PERIOD_KEY =
       "hbase.canary.write.table.check.period";
+  
+  public static final String HBASE_CANARY_READ_RAW_SCAN_KEY = "hbase.canary.read.raw.enabled";
 
   /**
    * Configuration keys for programmatic JAAS configuration for secured ZK interaction

http://git-wip-us.apache.org/repos/asf/hbase/blob/8efc6148/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
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 7d37161..ca27e71 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
@@ -262,12 +262,15 @@ public final class Canary implements Tool {
     private HRegionInfo region;
     private Sink sink;
     private TaskType taskType;
+    private boolean rawScanEnabled;
 
-    RegionTask(Connection connection, HRegionInfo region, Sink sink, TaskType taskType) {
+    RegionTask(Connection connection, HRegionInfo region, Sink sink, TaskType taskType,
+        boolean rawScanEnabled) {
       this.connection = connection;
       this.region = region;
       this.sink = sink;
       this.taskType = taskType;
+      this.rawScanEnabled = rawScanEnabled;
     }
 
     @Override
@@ -321,6 +324,11 @@ public final class Canary implements Tool {
           get.addFamily(column.getName());
         } else {
           scan = new Scan();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("rawScan : %s for table: %s", rawScanEnabled,
+              tableDesc.getTableName()));
+          }
+          scan.setRaw(rawScanEnabled);
           scan.setCaching(1);
           scan.setCacheBlocks(false);
           scan.setFilter(new FirstKeyOnlyFilter());
@@ -746,6 +754,8 @@ public final class Canary implements Tool {
     System.err.println("   -treatFailureAsError treats read / write failure as error");
     System.err.println("   -writeTable    The table used for write sniffing."
         + " Default is hbase:canary");
+    System.err.println("   -Dhbase.canary.read.raw.enabled=<true/false> Use this flag to enable or disable raw scan during read canary test"
+        + " Default is false and raw is not enabled during scan");
     System.err
         .println("   -D<configProperty>=<value> assigning or override the configuration params");
     System.exit(USAGE_EXIT_CODE);
@@ -870,6 +880,7 @@ public final class Canary implements Tool {
     private float regionsLowerLimit;
     private float regionsUpperLimit;
     private int checkPeriod;
+    private boolean rawScanEnabled;
 
     public RegionMonitor(Connection connection, String[] monitorTargets, boolean useRegExp,
         Sink sink, ExecutorService executor, boolean writeSniffing, TableName writeTableName,
@@ -887,6 +898,7 @@ public final class Canary implements Tool {
       this.checkPeriod =
           conf.getInt(HConstants.HBASE_CANARY_WRITE_TABLE_CHECK_PERIOD_KEY,
             DEFAULT_WRITE_TABLE_CHECK_PERIOD);
+      this.rawScanEnabled = conf.getBoolean(HConstants.HBASE_CANARY_READ_RAW_SCAN_KEY, false);
     }
 
     @Override
@@ -898,7 +910,8 @@ public final class Canary implements Tool {
             String[] tables = generateMonitorTables(this.targets);
             this.initialized = true;
             for (String table : tables) {
-              taskFutures.addAll(Canary.sniff(admin, sink, table, executor, TaskType.READ));
+              taskFutures.addAll(Canary.sniff(admin, sink, table, executor, TaskType.READ,
+                this.rawScanEnabled));
             }
           } else {
             taskFutures.addAll(sniff(TaskType.READ));
@@ -914,8 +927,8 @@ public final class Canary implements Tool {
               lastCheckTime = EnvironmentEdgeManager.currentTime();
             }
             // sniff canary table with write operation
-            taskFutures.addAll(Canary.sniff(admin, sink,
-              admin.getTableDescriptor(writeTableName), executor, TaskType.WRITE));
+            taskFutures.addAll(Canary.sniff(admin, sink, admin.getTableDescriptor(writeTableName),
+              executor, TaskType.WRITE, this.rawScanEnabled));
           }
 
           for (Future<Void> future : taskFutures) {
@@ -987,7 +1000,7 @@ public final class Canary implements Tool {
       for (HTableDescriptor table : admin.listTables()) {
         if (admin.isTableEnabled(table.getTableName())
             && (!table.getTableName().equals(writeTableName))) {
-          taskFutures.addAll(Canary.sniff(admin, sink, table, executor, taskType));
+          taskFutures.addAll(Canary.sniff(admin, sink, table, executor, taskType, this.rawScanEnabled));
         }
       }
       return taskFutures;
@@ -1058,38 +1071,58 @@ public final class Canary implements Tool {
    * Canary entry point for specified table.
    * @throws Exception
    */
+  public static void sniff(final Admin admin, TableName tableName, boolean rawScanEnabled)
+      throws Exception {
+    sniff(admin, tableName, TaskType.READ, rawScanEnabled);
+  }
+  
+  /**
+   * Canary entry point for specified table.
+   * Keeping this method backward compatibility
+   * @throws Exception
+   */
   public static void sniff(final Admin admin, TableName tableName)
       throws Exception {
-    sniff(admin, tableName, TaskType.READ);
+    sniff(admin, tableName, TaskType.READ, false);
   }
 
   /**
    * Canary entry point for specified table with task type(read/write)
    * @throws Exception
    */
-  public static void sniff(final Admin admin, TableName tableName, TaskType taskType)
-      throws Exception {
+  public static void sniff(final Admin admin, TableName tableName, TaskType taskType,
+      boolean rawScanEnabled)   throws Exception {
     List<Future<Void>> taskFutures =
         Canary.sniff(admin, new StdOutSink(), tableName.getNameAsString(),
-          new ScheduledThreadPoolExecutor(1), taskType);
+          new ScheduledThreadPoolExecutor(1), taskType, rawScanEnabled);
     for (Future<Void> future : taskFutures) {
       future.get();
     }
   }
+  
+  /**
+   * Canary entry point for specified table with task type(read/write)
+   * Keeping this method backward compatible
+   * @throws Exception
+   */
+  public static void sniff(final Admin admin, TableName tableName, TaskType taskType)
+      throws Exception {
+    Canary.sniff(admin, tableName, taskType, false);
+  }
 
   /**
    * Canary entry point for specified table.
    * @throws Exception
    */
   private static List<Future<Void>> sniff(final Admin admin, final Sink sink, String tableName,
-      ExecutorService executor, TaskType taskType) throws Exception {
+      ExecutorService executor, TaskType taskType, boolean rawScanEnabled) throws Exception {
     if (LOG.isDebugEnabled()) {
       LOG.debug(String.format("checking table is enabled and getting table descriptor for table %s",
         tableName));
     }
     if (admin.isTableEnabled(TableName.valueOf(tableName))) {
       return Canary.sniff(admin, sink, admin.getTableDescriptor(TableName.valueOf(tableName)),
-        executor, taskType);
+        executor, taskType, rawScanEnabled);
     } else {
       LOG.warn(String.format("Table %s is not enabled", tableName));
     }
@@ -1100,7 +1133,8 @@ public final class Canary implements Tool {
    * Loops over regions that owns this table, and output some information abouts the state.
    */
   private static List<Future<Void>> sniff(final Admin admin, final Sink sink,
-      HTableDescriptor tableDesc, ExecutorService executor, TaskType taskType) throws Exception {
+      HTableDescriptor tableDesc, ExecutorService executor, TaskType taskType,
+      boolean rawScanEnabled) throws Exception {
 
     if (LOG.isDebugEnabled()) {
       LOG.debug(String.format("reading list of regions for table %s", tableDesc.getTableName()));
@@ -1114,8 +1148,8 @@ public final class Canary implements Tool {
     }
     List<RegionTask> tasks = new ArrayList<RegionTask>();
     try {
-      for (HRegionInfo region : admin.getTableRegions(tableDesc.getTableName())) {
-        tasks.add(new RegionTask(admin.getConnection(), region, sink, taskType));
+      for (HRegionInfo region : admin.getTableRegions(tableDesc.getTableName())) {        
+        tasks.add(new RegionTask(admin.getConnection(), region, sink, taskType, rawScanEnabled));
       }
     } finally {
       table.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/8efc6148/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
----------------------------------------------------------------------
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 05852b8..b5b0d37 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
@@ -19,26 +19,21 @@
 
 package org.apache.hadoop.hbase.tool;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil;
-import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Appender;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.spi.LoggingEvent;
 import com.google.common.collect.Iterables;
-import org.apache.hadoop.hbase.HConstants;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -148,6 +143,28 @@ public class TestCanaryTool {
     }));
   }
 
+  @Test
+  public void testRawScanConfig() throws Exception {
+    TableName tableName = TableName.valueOf("testTableRawScan");
+    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.RegionServerStdOutSink sink = spy(new Canary.RegionServerStdOutSink());
+    Canary canary = new Canary(executor, sink);
+    String[] args = { "-t", "10000", "testTableRawScan" };
+    org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(testingUtility.getConfiguration());
+    conf.setBoolean(HConstants.HBASE_CANARY_READ_RAW_SCAN_KEY, true);
+    ToolRunner.run(conf, canary, args);
+    verify(sink, atLeastOnce())
+        .publishReadTiming(isA(HRegionInfo.class), isA(HColumnDescriptor.class), anyLong());
+  }
+  
   private void runRegionserverCanary() throws Exception {
     ExecutorService executor = new ScheduledThreadPoolExecutor(1);
     Canary canary = new Canary(executor, new Canary.RegionServerStdOutSink());


[2/2] hbase git commit: HBASE-16091 Canary takes lot more time when there are delete markers in the table (Vishal Khandelwal)

Posted by ap...@apache.org.
HBASE-16091 Canary takes lot more time when there are delete markers in the table (Vishal Khandelwal)

Conflicts:
	hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
	hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
	hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java

Amending-Author: Andrew Purtell <ap...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/e3ef8b69
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/e3ef8b69
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/e3ef8b69

Branch: refs/heads/0.98
Commit: e3ef8b69bf6834b8a1b7e33aee53792e8ef1f7cb
Parents: 47c1960
Author: Andrew Purtell <ap...@apache.org>
Authored: Tue Jul 5 10:11:08 2016 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Tue Jul 5 11:10:26 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/HConstants.java     |  4 +-
 .../org/apache/hadoop/hbase/tool/Canary.java    | 51 +++++++++++++++-----
 .../hadoop/hbase/tool/TestCanaryTool.java       | 22 +++++++++
 3 files changed, 63 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/e3ef8b69/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
index f046784..466c26f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
@@ -1135,7 +1135,9 @@ public final class HConstants {
 
   public static final String HBASE_CANARY_WRITE_TABLE_CHECK_PERIOD_KEY =
       "hbase.canary.write.table.check.period";
-  
+
+  public static final String HBASE_CANARY_READ_RAW_SCAN_KEY = "hbase.canary.read.raw.enabled";
+
   /**
    * Config keys for programmatic JAAS config for secured ZK interaction
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/e3ef8b69/hbase-server/src/main/java/org/apache/hadoop/hbase/tool/Canary.java
----------------------------------------------------------------------
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 4f2e5fe..51f162f 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
@@ -264,12 +264,15 @@ public final class Canary implements Tool {
     private HRegionInfo region;
     private Sink sink;
     private TaskType taskType;
+    private boolean rawScanEnabled;
 
-    RegionTask(HConnection connection, HRegionInfo region, Sink sink, TaskType taskType) {
+    RegionTask(HConnection connection, HRegionInfo region, Sink sink, TaskType taskType,
+        boolean rawScanEnabled) {
       this.connection = connection;
       this.region = region;
       this.sink = sink;
       this.taskType = taskType;
+      this.rawScanEnabled = rawScanEnabled;
     }
 
     @Override
@@ -323,6 +326,11 @@ public final class Canary implements Tool {
           get.addFamily(column.getName());
         } else {
           scan = new Scan();
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(String.format("rawScan : %s for table: %s", rawScanEnabled,
+              tableDesc.getTableName()));
+          }
+          scan.setRaw(rawScanEnabled);
           scan.setCaching(1);
           scan.setCacheBlocks(false);
           scan.setFilter(new FirstKeyOnlyFilter());
@@ -733,6 +741,8 @@ public final class Canary implements Tool {
     System.err.println("   -treatFailureAsError treats read / write failure as error");
     System.err.println("   -writeTable    The table used for write sniffing."
         + " Default is hbase:canary");
+    System.err.println("   -Dhbase.canary.read.raw.enabled=<true/false> Use this flag to enable or disable raw scan during read canary test"
+        + " Default is false and raw is not enabled during scan");
     System.err
         .println("   -D<configProperty>=<value> assigning or override the configuration params");
     System.exit(USAGE_EXIT_CODE);
@@ -855,6 +865,7 @@ public final class Canary implements Tool {
     private float regionsLowerLimit;
     private float regionsUpperLimit;
     private int checkPeriod;
+    private boolean rawScanEnabled;
 
     public RegionMonitor(HConnection connection, String[] monitorTargets, boolean useRegExp,
         Sink sink, ExecutorService executor, boolean writeSniffing, TableName writeTableName,
@@ -872,6 +883,7 @@ public final class Canary implements Tool {
       this.checkPeriod =
           conf.getInt(HConstants.HBASE_CANARY_WRITE_TABLE_CHECK_PERIOD_KEY,
             DEFAULT_WRITE_TABLE_CHECK_PERIOD);
+      this.rawScanEnabled = conf.getBoolean(HConstants.HBASE_CANARY_READ_RAW_SCAN_KEY, false);
     }
 
     @Override
@@ -883,7 +895,8 @@ public final class Canary implements Tool {
             String[] tables = generateMonitorTables(this.targets);
             this.initialized = true;
             for (String table : tables) {
-              taskFutures.addAll(Canary.sniff(connection, sink, table, executor, TaskType.READ));
+              taskFutures.addAll(Canary.sniff(connection, sink, table, executor, TaskType.READ,
+                this.rawScanEnabled));
             }
           } else {
             taskFutures.addAll(sniff(TaskType.READ));
@@ -899,8 +912,8 @@ public final class Canary implements Tool {
               lastCheckTime = EnvironmentEdgeManager.currentTimeMillis();
             }
             // sniff canary table with write operation
-            taskFutures.addAll(Canary.sniff(connection, sink,
-              writeTableName.getNameAsString(), executor, TaskType.WRITE));
+            taskFutures.addAll(Canary.sniff(connection, sink, writeTableName.getNameAsString(),
+              executor, TaskType.WRITE, this.rawScanEnabled));
           }
 
           for (Future<Void> future : taskFutures) {
@@ -973,7 +986,7 @@ public final class Canary implements Tool {
         if (connection.isTableEnabled(table.getTableName())
             && (!table.getTableName().equals(writeTableName))) {
           taskFutures.addAll(Canary.sniff(connection, sink, table.getTableName(), executor,
-            taskType));
+            taskType, this.rawScanEnabled));
         }
       }
       return taskFutures;
@@ -1033,29 +1046,40 @@ public final class Canary implements Tool {
    * Canary entry point for specified table.
    * @throws Exception
    */
-  public static void sniff(final HConnection connection, TableName tableName, TaskType taskType)
-      throws Exception {
+  public static void sniff(final HConnection connection, TableName tableName, TaskType taskType,
+      boolean rawScanEnabled) throws Exception {
     List<Future<Void>> taskFutures =
         Canary.sniff(connection, new StdOutSink(), tableName.getNameAsString(),
-          new ScheduledThreadPoolExecutor(1), taskType);
+          new ScheduledThreadPoolExecutor(1), taskType, rawScanEnabled);
     for (Future<Void> future : taskFutures) {
       future.get();
     }
   }
+  
+  /**
+   * Canary entry point for specified table with task type(read/write)
+   * Keeping this method backward compatible
+   * @throws Exception
+   */
+  public static void sniff(final HConnection connection, TableName tableName, TaskType taskType)
+      throws Exception {
+    Canary.sniff(connection, tableName, taskType, false);
+  }
 
   /**
    * Canary entry point for specified table.
    * @throws Exception
    */
   private static List<Future<Void>> sniff(final HConnection connection, final Sink sink,
-    String tableName, ExecutorService executor, TaskType taskType) throws Exception {
+      String tableName, ExecutorService executor, TaskType taskType, boolean rawScanEnabled)
+      throws Exception {
     if (LOG.isDebugEnabled()) {
       LOG.debug(String.format("checking table is enabled and getting table descriptor for table %s",
         tableName));
     }
     if (connection.isTableEnabled(TableName.valueOf(tableName))) {
-      return Canary.sniff(connection, sink, TableName.valueOf(tableName), executor,
-        taskType);
+      return Canary.sniff(connection, sink, TableName.valueOf(tableName), executor, taskType,
+        rawScanEnabled);
     } else {
       LOG.warn(String.format("Table %s is not enabled", tableName));
     }
@@ -1066,7 +1090,8 @@ public final class Canary implements Tool {
    * Loops over regions that owns this table, and output some information abouts the state.
    */
   private static List<Future<Void>> sniff(final HConnection connection, final Sink sink,
-      TableName tableName, ExecutorService executor, TaskType taskType) throws Exception {
+      TableName tableName, ExecutorService executor, TaskType taskType, boolean rawScanEnabled)
+      throws Exception {
     if (LOG.isDebugEnabled()) {
       LOG.debug(String.format("reading list of regions for table %s", tableName));
     }
@@ -1079,7 +1104,7 @@ public final class Canary implements Tool {
     List<RegionTask> tasks = new ArrayList<RegionTask>();
     try {
       for (HRegionInfo region : ((HTable)table).getRegionLocations().keySet()) {
-        tasks.add(new RegionTask(connection, region, sink, taskType));
+        tasks.add(new RegionTask(connection, region, sink, taskType, rawScanEnabled));
       }
     } finally {
       table.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/e3ef8b69/hbase-server/src/test/java/org/apache/hadoop/hbase/tool/TestCanaryTool.java
----------------------------------------------------------------------
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 18c269a..9d1dd9e 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
@@ -140,6 +140,28 @@ public class TestCanaryTool {
     }));
   }
 
+  @Test
+  public void testRawScanConfig() throws Exception {
+    TableName tableName = TableName.valueOf("testTableRawScan");
+    HTable 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.add(FAMILY, COLUMN, iBytes);
+      table.put(p);
+    }
+    ExecutorService executor = new ScheduledThreadPoolExecutor(1);
+    Canary.RegionServerStdOutSink sink = spy(new Canary.RegionServerStdOutSink());
+    Canary canary = new Canary(executor, sink);
+    String[] args = { "-t", "10000", "testTableRawScan" };
+    org.apache.hadoop.conf.Configuration conf = new org.apache.hadoop.conf.Configuration(testingUtility.getConfiguration());
+    conf.setBoolean(HConstants.HBASE_CANARY_READ_RAW_SCAN_KEY, true);
+    ToolRunner.run(conf, canary, args);
+    verify(sink, atLeastOnce())
+        .publishReadTiming(isA(HRegionInfo.class), isA(HColumnDescriptor.class), anyLong());
+  }
+  
   private void runRegionserverCanary() throws Exception {
     ExecutorService executor = new ScheduledThreadPoolExecutor(1);
     Canary canary = new Canary(executor, new Canary.RegionServerStdOutSink());