Return-Path: Delivered-To: apmail-hadoop-hdfs-user-archive@minotaur.apache.org Received: (qmail 85778 invoked from network); 9 Oct 2009 22:28:32 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 9 Oct 2009 22:28:32 -0000 Received: (qmail 59589 invoked by uid 500); 9 Oct 2009 22:28:32 -0000 Delivered-To: apmail-hadoop-hdfs-user-archive@hadoop.apache.org Received: (qmail 59523 invoked by uid 500); 9 Oct 2009 22:28:32 -0000 Mailing-List: contact hdfs-user-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hdfs-user@hadoop.apache.org Delivered-To: mailing list hdfs-user@hadoop.apache.org Received: (qmail 59514 invoked by uid 99); 9 Oct 2009 22:28:32 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 09 Oct 2009 22:28:32 +0000 X-ASF-Spam-Status: No, hits=2.2 required=10.0 tests=HTML_MESSAGE,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: local policy) Received: from [216.252.110.213] (HELO web56204.mail.re3.yahoo.com) (216.252.110.213) by apache.org (qpsmtpd/0.29) with SMTP; Fri, 09 Oct 2009 22:28:21 +0000 Received: (qmail 9215 invoked by uid 60001); 9 Oct 2009 22:27:59 -0000 DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=yahoo.com; s=s1024; t=1255127279; bh=105FJ6Zuvj8xeSxS8ZifdEF2FQ93BHtiCz446aK+KAM=; h=Message-ID:X-YMail-OSG:Received:X-Mailer:References:Date:From:Subject:To:In-Reply-To:MIME-Version:Content-Type; b=Cd4z6cbFPNumih2phtFOzIkJw5A9Bhmg5ufSAKUGy6dfssu0/YebwvF8Jf/LANM2k2sZiO7nk65z5oFo0G4HYrWpE4PLcknXUsfbj9Dai4/Y8lXaKusjiwa29djOmaA4nir2bUFmZIDfF2Fcn/P8K2N1633C2kIebm/q32RBXmo= DomainKey-Signature: a=rsa-sha1; q=dns; c=nofws; s=s1024; d=yahoo.com; h=Message-ID:X-YMail-OSG:Received:X-Mailer:References:Date:From:Subject:To:In-Reply-To:MIME-Version:Content-Type; b=EE5SLwu8uxmTw4HgkBynjKX4ps6JfN0P4IChvmFNuaiu23sR6y6V9+sU2kDM7FVAk2xjKaD50CELt/qxmkFjpwlhXeM2P+qS0K2caNb11an5TKSP4IMdOVfllQkUo2s9G/bK637FmFjFS2vKxhem+6guuDWdCEPyY03GbsrmNxU=; Message-ID: <234641.9030.qm@web56204.mail.re3.yahoo.com> X-YMail-OSG: ZDG8CuwVM1kgRh5XPdNZFPvCpu0wAPIBRXv5Z8xhAnSsMVNf4vCfJ_1vzgNOPkFcwL3om2D4U1NF8cEOmBY8GYaCy2SAUh90Z5s5tAgZLcH5..zCHef37GocWFj6KCf3DqdclbJfUUBTD7ipeQSXMnfRP6K4NpR0e1yY5U7Y6.J08FBu_vDfXvMXvTZwU02CDXDPqgpTCdKflml9lQcZod7hv92nV0l8cfKg6ePo4y40MTThl211c07dqVI2yJ_yFGRJMBdX3HB9ugUIYP32Ll1J7suJ8BahAVdSRavM06FwtrmgqURgzndC1_BR.WQRkMwQZaBKwkYw2k0nycsLE5pxmzdcuEaLpTNMnAarMCp7ymy85.tO87o5wXvp_e96NOPQiJobj_gjAnVyCBwbA1Ao Received: from [216.145.54.7] by web56204.mail.re3.yahoo.com via HTTP; Fri, 09 Oct 2009 15:27:59 PDT X-Mailer: YahooMailRC/182.10 YahooMailWebService/0.7.347.3 References: <7c962aed0910091313la775d35v520f6a40dddae2b5@mail.gmail.com> <786023.95783.qm@web56206.mail.re3.yahoo.com> <7c962aed0910091432g51af09dai3113b5218007594b@mail.gmail.com> <728045.8261.qm@web56207.mail.re3.yahoo.com> Date: Fri, 9 Oct 2009 15:27:59 -0700 (PDT) From: "Tsz Wo \(Nicholas\), Sze" Subject: Re: hflush not working for me? To: hdfs-user@hadoop.apache.org In-Reply-To: <728045.8261.qm@web56207.mail.re3.yahoo.com> MIME-Version: 1.0 Content-Type: multipart/alternative; boundary="0-807419493-1255127279=:9030" X-Virus-Checked: Checked by ClamAV on apache.org --0-807419493-1255127279=:9030 Content-Type: text/plain; charset=us-ascii 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" >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 >>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 >>>>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 >>>> >> --0-807419493-1255127279=:9030 Content-Type: text/html; charset=us-ascii
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

--0-807419493-1255127279=:9030--