FAQ
I'm putting together some unit tests up in our application that exercise
hflush. I'm using minidfscluster and a jar made by building head of the
0.21 branch of hdfs (from about a minute ago).

Code opens a file, writes a bunch of edits, invokes hflush (by calling sync
on DFSDataOutputStream instance) and then, without closing the Writer, opens
a Reader on same file. This Reader does not see any edits not to mind edits
up to the sync invocation.

I can trace the code and see how on hflush it sends the queued packets of
edits.

I studied TestReadWhileWriting. I've set setBoolean("dfs.support.append",
true) before minidfscluster spins up. I can't set soft lease to 1 second
because not in same package so I just wait out the default minute. It
doesn't seem to make a difference.

Do I have to do open as another user?

Thanks for any pointers,
St.Ack

Search Discussions

  • Stack at Oct 9, 2009 at 8:25 pm
    I just tried opening Reader as another user but that made no difference.
    St.Ack
    On Fri, Oct 9, 2009 at 1:13 PM, stack wrote:

    I'm putting together some unit tests up in our application that exercise
    hflush. I'm using minidfscluster and a jar made by building head of the
    0.21 branch of hdfs (from about a minute ago).

    Code opens a file, writes a bunch of edits, invokes hflush (by calling sync
    on DFSDataOutputStream instance) and then, without closing the Writer, opens
    a Reader on same file. This Reader does not see any edits not to mind edits
    up to the sync invocation.

    I can trace the code and see how on hflush it sends the queued packets of
    edits.

    I studied TestReadWhileWriting. I've set setBoolean("dfs.support.append",
    true) before minidfscluster spins up. I can't set soft lease to 1 second
    because not in same package so I just wait out the default minute. It
    doesn't seem to make a difference.

    Do I have to do open as another user?

    Thanks for any pointers,
    St.Ack
  • Tsz Wo \(Nicholas\), Sze at Oct 9, 2009 at 8:28 pm
    Hi St.Ack,
    ... soft lease to 1 second ...
    You are right that you don't have to change soft lease. It is for append but not related to hflash.
    Do I have to do open as another user?
    This should not be necessary.

    Could you send me/post your test?

    Nicholas Sze



    From: stack <stack@duboce.net>
    To: hdfs-user@hadoop.apache.org
    Sent: Fri, October 9, 2009 1:13:37 PM
    Subject: hflush not working for me?
    I'm putting together some unit tests up in our application that exercise hflush. I'm using minidfscluster and a jar made by building head of the 0.21 branch of hdfs (from about a minute ago).
    Code opens a file, writes a bunch of edits, invokes hflush (by calling sync on DFSDataOutputStream instance) and then, without closing the Writer, opens a Reader on same file. This Reader does not see any edits not to mind edits up to the sync invocation.

    I can trace the code and see how on hflush it sends the queued packets of edits.

    I studied TestReadWhileWriting. I've set setBoolean("dfs.support.append", true) before minidfscluster spins up. I can't set soft lease to 1 second because not in same package so I just wait out the default minute. It doesn't seem to make a difference.

    Do I have to do open as another user?

    Thanks for any pointers,
    St.Ack
  • Stack at Oct 9, 2009 at 9:33 pm

    On Fri, Oct 9, 2009 at 1:27 PM, Tsz Wo (Nicholas), Sze wrote:

    Hi St.Ack,
    ... soft lease to 1 second ...
    You are right that you don't have to change soft lease. It is for append
    but not related to hflash.
    I should not have to set it then? I can remove this 70 second pause in
    middle of my test?


    Do I have to do open as another user?
    This should not be necessary.

    Could you send me/post your test?
    Sure, as long as you don't hold this ugly code against me ever after.

    I checked in the code so you could try it:
    http://svn.apache.org/repos/asf/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java

    Its the first test, testSync.

    It starts out by copying whats down in the hdfs testReadWhileWriting. That
    bit works fine.

    Then comes the ugly stuff.

    HLog is our write-ahead log wrapper. Internally it writes out to a
    SequenceFile.Writer. The SequenceFile.Writer has been doctored using
    reflection so the out datamember is non-private. A call to HLog.sync runs
    the SequenceFile.Writer.sync -- which DOES NOT call sync on the backing
    output stream -- and then it calls sync on the now accessible out stream
    (Sorry its so ugly -- I'm trying to hack stuff up fast so all of hbase gets
    access to this new facility). If I trace in the debugger, I can see that
    the sync on the out data member goes down into hflush. Queued up edits are
    flushed. It seems like it should be working.

    Do I have to do some doctoring of the reader? (It doesn't seem so given that
    the code at the head of this test works).

    Thanks for taking a look Nicholas.

    To run the test, you can do "ant clean jar test -Dtestcase=TestHLog".

    (Let me know if you want an eclipse .project + .classpath so you can get it
    up in an ide to run debugger).

    St.Ack




    Nicholas Sze


    *From:* stack <stack@duboce.net>
    *To:* hdfs-user@hadoop.apache.org
    *Sent:* Fri, October 9, 2009 1:13:37 PM
    *Subject:* hflush not working for me?

    I'm putting together some unit tests up in our application that exercise
    hflush. I'm using minidfscluster and a jar made by building head of the
    0.21 branch of hdfs (from about a minute ago).

    Code opens a file, writes a bunch of edits, invokes hflush (by calling sync
    on DFSDataOutputStream instance) and then, without closing the Writer, opens
    a Reader on same file. This Reader does not see any edits not to mind edits
    up to the sync invocation.

    I can trace the code and see how on hflush it sends the queued packets of
    edits.

    I studied TestReadWhileWriting. I've set setBoolean("dfs.support.append",
    true) before minidfscluster spins up. I can't set soft lease to 1 second
    because not in same package so I just wait out the default minute. It
    doesn't seem to make a difference.

    Do I have to do open as another user?

    Thanks for any pointers,
    St.Ack
  • Tsz Wo \(Nicholas\), Sze at Oct 9, 2009 at 9:42 pm
    Soft lease is for another writer to obtain the file lease if the original writer appears to abandon the file. In the current TestReadWhileWriting (not counting part (c) and (d)), there is only one writer. So soft lease is not related.

    Will check your test.

    Nicholas



    From: stack <stack@duboce.net>
    To: hdfs-user@hadoop.apache.org
    Sent: Fri, October 9, 2009 2:32:02 PM
    Subject: Re: hflush not working for me?
    On Fri, Oct 9, 2009 at 1:27 PM, Tsz Wo (Nicholas), Sze wrote:
    Hi St.Ack,
    ... soft lease to 1 second ...
    You are right that you don't have to change soft lease. It is for append but not related to hflash.
    I should not have to set it then? I can remove this 70 second pause in middle of my test?


    Do I have to do open as another user?
    This should not be necessary.

    Could you send me/post your test?
    Sure, as long as you don't hold this ugly code against me ever after.

    I checked in the code so you could try it: http://svn.apache.org/repos/asf/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java

    Its the first test, testSync.

    It starts out by copying whats down in the hdfs testReadWhileWriting. That bit works fine.

    Then comes the ugly stuff.

    HLog is our write-ahead log wrapper. Internally it writes out to a SequenceFile.Writer. The SequenceFile.Writer has been doctored using reflection so the out datamember is non-private. A call to HLog.sync runs the SequenceFile.Writer.sync -- which DOES NOT call sync on the backing output stream -- and then it calls sync on the now accessible out stream (Sorry its so ugly -- I'm trying to hack stuff up fast so all of hbase gets access to this new facility). If I trace in the debugger, I can see that the sync on the out data member goes down into hflush. Queued up edits are flushed. It seems like it should be working.

    Do I have to do some doctoring of the reader? (It doesn't seem so given that the code at the head of this test works).

    Thanks for taking a look Nicholas.

    To run the test, you can do "ant clean jar test -Dtestcase=TestHLog".

    (Let me know if you want an eclipse .project + .classpath so you can get it up in an ide to run debugger).

    St.Ack




    Nicholas Sze


    From: stack <stack@duboce.net>
    To: hdfs-user@hadoop.apache.org
    Sent: Fri, October 9, 2009 1:13:37 PM
    Subject: hflush not working for me?

    I'm putting together some unit tests up in our application that exercise hflush. I'm using minidfscluster and a jar made by building head of the 0.21 branch of hdfs (from about a minute ago).
    Code opens a file, writes a bunch of edits, invokes hflush (by calling sync on DFSDataOutputStream instance) and then, without closing the Writer, opens a Reader on same file. This Reader does not see any edits not to mind edits up to the sync invocation.

    I can trace the code and see how on hflush it sends the queued packets of edits.

    I studied TestReadWhileWriting. I've set setBoolean("dfs.support.append", true) before minidfscluster spins up. I can't set soft lease to 1 second because not in same package so I just wait out the default minute. It doesn't seem to make a difference.

    Do I have to do open as another user?

    Thanks for any pointers,
    St.Ack
  • Tsz Wo \(Nicholas\), Sze at Oct 9, 2009 at 10:28 pm
    Hi St.Ack,

    HLog uses SequenceFile.Reader. When a SequenceFile.Reader is constructed, it calls fs.getFileStatus(file).getLen(). However, fs.getFileStatus(file).getLen() does not return the hflushed length for un-closed file since the Namenode does not know the hflushed length. DFSClient have to ask a datanode for the length last block which is being written; see also HDFS-570. That's why "hflush not working for you". Sorry ...

    We have to change SequenceFile.Reader in order to support reading on unclosed SequenceFile.

    Nicholas



    From: "Tsz Wo (Nicholas), Sze" <s29752-hadoopuser@yahoo.com>
    To: hdfs-user@hadoop.apache.org
    Sent: Fri, October 9, 2009 2:40:34 PM
    Subject: Re: hflush not working for me?


    Soft lease is for another writer to obtain the file lease if the original writer appears to abandon the file. In the current TestReadWhileWriting (not counting part (c) and (d)), there is only one writer. So soft lease is not related.

    Will check your test.

    Nicholas



    From: stack <stack@duboce.net>
    To: hdfs-user@hadoop.apache.org
    Sent: Fri, October 9, 2009 2:32:02 PM
    Subject: Re: hflush not working for me?
    On Fri, Oct 9, 2009 at 1:27 PM, Tsz Wo (Nicholas), Sze wrote:
    Hi St.Ack,
    ... soft lease to 1 second ...
    You are right that you don't have to change soft lease. It is for append but not related to hflash.
    I should not have to set it then? I can remove this 70 second pause in middle of my test?


    Do I have to do open as another user?
    This should not be necessary.

    Could you send me/post your test?
    Sure, as long as you don't hold this ugly code against me ever after.

    I checked in the code so you could try it: http://svn.apache.org/repos/asf/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java

    Its the first test, testSync.

    It starts out by copying whats down in the hdfs testReadWhileWriting. That bit works fine.

    Then comes the ugly stuff.

    HLog is our write-ahead log wrapper. Internally it writes out to a SequenceFile.Writer. The SequenceFile.Writer has been doctored using reflection so the out datamember is non-private. A call to HLog.sync runs the SequenceFile.Writer.sync -- which DOES NOT call sync on the backing output stream -- and then it calls sync on the now accessible out stream (Sorry its so ugly -- I'm trying to hack stuff up fast so all of hbase gets access to this new facility). If I trace in the debugger, I can see that the sync on the out data member goes down into hflush. Queued up edits are flushed. It seems like it should be working.

    Do I have to do some doctoring of the reader? (It doesn't seem so given that the code at the head of this test works).

    Thanks for taking a look Nicholas.

    To run the test, you can do "ant clean jar test -Dtestcase=TestHLog".

    (Let me know if you want an eclipse .project + .classpath so you can get it up in an ide to run debugger).

    St.Ack




    Nicholas Sze


    From: stack <stack@duboce.net>
    To: hdfs-user@hadoop.apache.org
    Sent: Fri, October 9, 2009 1:13:37 PM
    Subject: hflush not working for me?

    I'm putting together some unit tests up in our application that exercise hflush. I'm using minidfscluster and a jar made by building head of the 0.21 branch of hdfs (from about a minute ago).
    Code opens a file, writes a bunch of edits, invokes hflush (by calling sync on DFSDataOutputStream instance) and then, without closing the Writer, opens a Reader on same file. This Reader does not see any edits not to mind edits up to the sync invocation.

    I can trace the code and see how on hflush it sends the queued packets of edits.

    I studied TestReadWhileWriting. I've set setBoolean("dfs.support.append", true) before minidfscluster spins up. I can't set soft lease to 1 second because not in same package so I just wait out the default minute. It doesn't seem to make a difference.

    Do I have to do open as another user?

    Thanks for any pointers,
    St.Ack
  • Stack at Oct 9, 2009 at 10:40 pm
    Thank you for figuring it Nicholas.
    St.Ack
    On Fri, Oct 9, 2009 at 3:27 PM, Tsz Wo (Nicholas), Sze wrote:

    Hi St.Ack,

    HLog uses SequenceFile.Reader. When a SequenceFile.Reader is constructed,
    it calls fs.getFileStatus(file).getLen(). However,
    fs.getFileStatus(file).getLen() does not return the hflushed length for
    un-closed file since the Namenode does not know the hflushed length.
    DFSClient have to ask a datanode for the length last block which is being
    written; see also HDFS-570. That's why "hflush not working for you". Sorry
    ...

    We have to change SequenceFile.Reader in order to support reading on
    unclosed SequenceFile.

    Nicholas


    *From:* "Tsz Wo (Nicholas), Sze" <s29752-hadoopuser@yahoo.com>
    *To:* hdfs-user@hadoop.apache.org
    *Sent:* Fri, October 9, 2009 2:40:34 PM

    *Subject:* Re: hflush not working for me?

    Soft lease is for another writer to obtain the file lease if the original
    writer appears to abandon the file. In the current TestReadWhileWriting
    (not counting part (c) and (d)), there is only one writer. So soft lease is
    not related.

    Will check your test.

    Nicholas


    *From:* stack <stack@duboce.net>
    *To:* hdfs-user@hadoop.apache.org
    *Sent:* Fri, October 9, 2009 2:32:02 PM
    *Subject:* Re: hflush not working for me?

    On Fri, Oct 9, 2009 at 1:27 PM, Tsz Wo (Nicholas), Sze <
    s29752-hadoopuser@yahoo.com> wrote:
    Hi St.Ack,
    ... soft lease to 1 second ...
    You are right that you don't have to change soft lease. It is for append
    but not related to hflash.
    I should not have to set it then? I can remove this 70 second pause in
    middle of my test?


    Do I have to do open as another user?
    This should not be necessary.

    Could you send me/post your test?
    Sure, as long as you don't hold this ugly code against me ever after.

    I checked in the code so you could try it:
    http://svn.apache.org/repos/asf/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/regionserver/TestHLog.java

    Its the first test, testSync.

    It starts out by copying whats down in the hdfs testReadWhileWriting. That
    bit works fine.

    Then comes the ugly stuff.

    HLog is our write-ahead log wrapper. Internally it writes out to a
    SequenceFile.Writer. The SequenceFile.Writer has been doctored using
    reflection so the out datamember is non-private. A call to HLog.sync runs
    the SequenceFile.Writer.sync -- which DOES NOT call sync on the backing
    output stream -- and then it calls sync on the now accessible out stream
    (Sorry its so ugly -- I'm trying to hack stuff up fast so all of hbase gets
    access to this new facility). If I trace in the debugger, I can see that
    the sync on the out data member goes down into hflush. Queued up edits are
    flushed. It seems like it should be working.

    Do I have to do some doctoring of the reader? (It doesn't seem so given
    that the code at the head of this test works).

    Thanks for taking a look Nicholas.

    To run the test, you can do "ant clean jar test -Dtestcase=TestHLog".

    (Let me know if you want an eclipse .project + .classpath so you can get it
    up in an ide to run debugger).

    St.Ack




    Nicholas Sze


    *From:* stack <stack@duboce.net>
    *To:* hdfs-user@hadoop.apache.org
    *Sent:* Fri, October 9, 2009 1:13:37 PM
    *Subject:* hflush not working for me?

    I'm putting together some unit tests up in our application that exercise
    hflush. I'm using minidfscluster and a jar made by building head of the
    0.21 branch of hdfs (from about a minute ago).

    Code opens a file, writes a bunch of edits, invokes hflush (by calling
    sync on DFSDataOutputStream instance) and then, without closing the Writer,
    opens a Reader on same file. This Reader does not see any edits not to mind
    edits up to the sync invocation.

    I can trace the code and see how on hflush it sends the queued packets of
    edits.

    I studied TestReadWhileWriting. I've set setBoolean("dfs.support.append",
    true) before minidfscluster spins up. I can't set soft lease to 1 second
    because not in same package so I just wait out the default minute. It
    doesn't seem to make a difference.

    Do I have to do open as another user?

    Thanks for any pointers,
    St.Ack

Related Discussions

Discussion Navigation
viewthread | post
Discussion Overview
grouphdfs-user @
categorieshadoop
postedOct 9, '09 at 8:15p
activeOct 9, '09 at 10:40p
posts7
users2
websitehadoop.apache.org...
irc#hadoop

2 users in discussion

Stack: 4 posts Tsz Wo \(Nicholas\), Sze: 3 posts

People

Translate

site design / logo © 2022 Grokbase