You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "stack (JIRA)" <ji...@apache.org> on 2011/01/07 19:35:45 UTC
[jira] Commented: (HBASE-3038) WALReaderFSDataInputStream.getPos()
fails if Filesize > MAX_INT
[ https://issues.apache.org/jira/browse/HBASE-3038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12978906#action_12978906 ]
stack commented on HBASE-3038:
------------------------------
Nicolas:
Could we not do this reflection once only in constructor rather than per getPos lookup?
{code}
try {
Field fIn = FilterInputStream.class.getDeclaredField("in");
fIn.setAccessible(true);
Object realIn = fIn.get(this.in);
Method getFileLength = realIn.getClass().
getMethod("getFileLength", new Class<?> []{});
getFileLength.setAccessible(true);
long realLength = ((Long)getFileLength.
invoke(realIn, new Object []{})).longValue();
assert(realLength >= this.length);
adjust = realLength - this.length;
} catch(Exception e) {
{code}
> WALReaderFSDataInputStream.getPos() fails if Filesize > MAX_INT
> ---------------------------------------------------------------
>
> Key: HBASE-3038
> URL: https://issues.apache.org/jira/browse/HBASE-3038
> Project: HBase
> Issue Type: Bug
> Components: regionserver
> Affects Versions: 0.89.20100621, 0.90.0
> Reporter: Nicolas Spiegelberg
> Assignee: Nicolas Spiegelberg
> Priority: Critical
> Fix For: 0.90.0
>
> Attachments: 3038-addendum.txt, HBASE-3038.patch
>
>
> WALReaderFSDataInputStream.getPos() uses this.in.available() to determine the actual length of the file. Except that available() returns an int instead of a long. Therefore, our current logic is broke when trying to read a split log > 2GB.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.