hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HBASE-2481) Client is not getting UnknownScannerExceptions; they are being eaten
Date Fri, 23 Apr 2010 21:37:51 GMT

    [ https://issues.apache.org/jira/browse/HBASE-2481?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12860385#action_12860385
] 

stack commented on HBASE-2481:
------------------------------

Here is script to demo problem.  It scans 99 rows in a PE table then pauses for 60 seconds.
 When it resumes, there is nothing else to fetch (though table has millions of rows) and over
on serverside is a USE:

{code}
$ more bin/slow_scan.rb 
# To run: ./bin/hbase org.jruby.Main bin/slow_scan.rb
include Java
import org.apache.hadoop.hbase.util.Bytes
import org.apache.hadoop.hbase.HConstants
import org.apache.hadoop.hbase.client.HTable
import org.apache.hadoop.hbase.client.Scan
import org.apache.hadoop.hbase.HBaseConfiguration
import org.apache.hadoop.hbase.util.FSUtils
import org.apache.hadoop.fs.FileSystem
import java.lang.Thread

# Get configuration to use.
c = HBaseConfiguration.new()

# Set hadoop filesystem configuration using the hbase.rootdir.
# Otherwise, we'll always use localhost though the hbase.rootdir
# might be pointing at hdfs location.
c.set("fs.default.name", c.get(HConstants::HBASE_DIR))
fs = FileSystem.get(c)

# Clean mentions of table from .META.
# Scan the .META. and remove all lines that begin with tablename
t = HTable.new(c, "TestTable")
scan = Scan.new()
scanner = t.getScanner(scan)
count = 0
while (result = scanner.next())
  rowid = Bytes.toString(result.getRow())
  puts rowid
  count = count + 1 
  # If count == 100, pause for 60 seconds.
  if count == 100 
    Thread.sleep(1000 * 60)
  end
end
scanner.close()
{code}

> Client is not getting UnknownScannerExceptions; they are being eaten
> --------------------------------------------------------------------
>
>                 Key: HBASE-2481
>                 URL: https://issues.apache.org/jira/browse/HBASE-2481
>             Project: Hadoop HBase
>          Issue Type: Bug
>    Affects Versions: 0.20.4
>            Reporter: stack
>            Priority: Blocker
>
> This was reported by mudphone on IRC and confirmed by myself in quick test.  If the client
takes too long going back to the RS, the RS will throw an UnknownScannerException but it doesn't
get back to the client.  Instead, the client scan silently ends.  Marking this blocker.  Its
actually in 0.20.4.  Thats what I was testing.  Mayhaps an RC sinker?

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message