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

zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/master by this push:
     new 8785122141e HBASE-29107 shell: Improve 'count' performance (#6656)
8785122141e is described below

commit 8785122141ebf96d1e50530e60cdae0c83cff0b4
Author: Junegunn Choi <[email protected]>
AuthorDate: Sat Mar 1 01:05:53 2025 +0900

    HBASE-29107 shell: Improve 'count' performance (#6656)
    
    Signed-off-by: Duo Zhang <[email protected]>
    Signed-off-by: Nihal Jain <[email protected]>
    Signed-off-by: Viraj Jasani <[email protected]>
---
 hbase-shell/src/main/ruby/hbase/table.rb          | 29 ++++++++++-------------
 hbase-shell/src/main/ruby/shell/commands/count.rb | 11 ++-------
 2 files changed, 15 insertions(+), 25 deletions(-)

diff --git a/hbase-shell/src/main/ruby/hbase/table.rb 
b/hbase-shell/src/main/ruby/hbase/table.rb
index d265ca09556..aed5e1c26e5 100644
--- a/hbase-shell/src/main/ruby/hbase/table.rb
+++ b/hbase-shell/src/main/ruby/hbase/table.rb
@@ -321,24 +321,21 @@ EOF
     def _count_internal(interval = 1000, scan = nil, cacheBlocks=false)
       raise(ArgumentError, 'Scan argument should be 
org.apache.hadoop.hbase.client.Scan') \
         unless scan.nil? || scan.is_a?(org.apache.hadoop.hbase.client.Scan)
-      # We can safely set scanner caching with the first key only filter
 
-      if scan.nil?
-        scan = org.apache.hadoop.hbase.client.Scan.new
-        scan.setCacheBlocks(cacheBlocks)
-        scan.setCaching(10)
-        scan.setFilter(org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter.new)
-      else
-        scan.setCacheBlocks(cacheBlocks)
-        filter = scan.getFilter
-        firstKeyOnlyFilter = 
org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter.new
-        if filter.nil?
-          scan.setFilter(firstKeyOnlyFilter)
-        else
-          firstKeyOnlyFilter.setReversed(filter.isReversed)
-          scan.setFilter(org.apache.hadoop.hbase.filter.FilterList.new(filter, 
firstKeyOnlyFilter))
-        end
+      scan ||= org.apache.hadoop.hbase.client.Scan.new
+      scan.setCacheBlocks(cacheBlocks)
+
+      # Optimize counting by using FirstKeyOnlyFilter and KeyOnlyFilter
+      filters = [
+        org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter.new,
+        org.apache.hadoop.hbase.filter.KeyOnlyFilter.new
+      ]
+      filter = scan.getFilter
+      if filter
+        filters.each { |f| f.setReversed(filter.isReversed) }
+        filters = [filter, *filters]
       end
+      scan.setFilter(org.apache.hadoop.hbase.filter.FilterList.new(filters))
 
       # Run the scanner
       scanner = @table.getScanner(scan)
diff --git a/hbase-shell/src/main/ruby/shell/commands/count.rb 
b/hbase-shell/src/main/ruby/shell/commands/count.rb
index 70523588231..c850c712338 100644
--- a/hbase-shell/src/main/ruby/shell/commands/count.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/count.rb
@@ -25,16 +25,12 @@ module Shell
 Count the number of rows in a table.  Return value is the number of rows.
 This operation may take a LONG time (Run '$HADOOP_HOME/bin/hadoop jar
 hbase.jar rowcount' to run a counting mapreduce job). Current count is shown
-every 1000 rows by default. Count interval may be optionally specified. Scan
-caching is enabled on count scans by default. Default cache size is 10 rows.
-If your rows are small in size, you may want to increase this
-parameter. Examples:
+every 1000 rows by default. Count interval may be optionally specified.
+Examples:
 
  hbase> count 'ns1:t1'
  hbase> count 't1'
  hbase> count 't1', INTERVAL => 100000
- hbase> count 't1', CACHE => 1000
- hbase> count 't1', INTERVAL => 10, CACHE => 1000
  hbase> count 't1', FILTER => "
     (QualifierFilter (>=, 'binary:xyz')) AND (TimestampsFilter ( 123, 456))"
  hbase> count 't1', COLUMNS => ['c1', 'c2'], STARTROW => 'abc', STOPROW => 
'xyz'
@@ -44,8 +40,6 @@ t to table 't1', the corresponding commands would be:
 
  hbase> t.count
  hbase> t.count INTERVAL => 100000
- hbase> t.count CACHE => 1000
- hbase> t.count INTERVAL => 10, CACHE => 1000
  hbase> t.count FILTER => "
     (QualifierFilter (>=, 'binary:xyz')) AND (TimestampsFilter ( 123, 456))"
  hbase> t.count COLUMNS => ['c1', 'c2'], STARTROW => 'abc', STOPROW => 'xyz'
@@ -85,7 +79,6 @@ EOF
         # Merge params with defaults
         params = {
           'INTERVAL' => 1000,
-          'CACHE' => 10,
           'CACHE_BLOCKS' => false
         }.merge(params)
 

Reply via email to