Return-Path: X-Original-To: apmail-hadoop-user-archive@minotaur.apache.org Delivered-To: apmail-hadoop-user-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 43A26928C for ; Mon, 4 Mar 2013 16:10:28 +0000 (UTC) Received: (qmail 91009 invoked by uid 500); 4 Mar 2013 16:10:22 -0000 Delivered-To: apmail-hadoop-user-archive@hadoop.apache.org Received: (qmail 90884 invoked by uid 500); 4 Mar 2013 16:10:22 -0000 Mailing-List: contact user-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@hadoop.apache.org Delivered-To: mailing list user@hadoop.apache.org Received: (qmail 90877 invoked by uid 99); 4 Mar 2013 16:10:22 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 04 Mar 2013 16:10:22 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of lucejb@gmail.com designates 209.85.219.54 as permitted sender) Received: from [209.85.219.54] (HELO mail-oa0-f54.google.com) (209.85.219.54) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 04 Mar 2013 16:10:14 +0000 Received: by mail-oa0-f54.google.com with SMTP id n12so9215697oag.41 for ; Mon, 04 Mar 2013 08:09:53 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:x-received:in-reply-to:references:date:message-id :subject:from:to:content-type; bh=gg4RPdIC8E2b1G/i9UoRjTNWJzpl+8rT11X5he8MUNM=; b=XWHxFwr0eR6VKN/B11ndmVuXKETmReCF8hSQIabDDlBPKFiVrXYU7ue+fYFdMIr7Zw FWhAQwHoofmooNyq1LW2wPsSSUXFSsfyOA0dD6GbNAfpzcjTf8WPcsnAwPuMPNbMaAyG vnkMAb7G8DA2hYR731UHb+TGlIlSwiq9L9/gQ6+Jn5HsymfMQpcpDMByDqVZnarEH8/1 lZmGL1Qu+JpYSdnO22qkYAgQEvVanLyHlzgeej9tuwLozUnWw+74UBn9xdnzIVMnKF2z BL6Wm8b8P5k4r+xQTHj9YTNpG6vRko17nl8zZkbihbHTZknEm2vx/FiCYSlIskkTWPXI 8yWQ== MIME-Version: 1.0 X-Received: by 10.60.172.84 with SMTP id ba20mr15749485oec.10.1362413393773; Mon, 04 Mar 2013 08:09:53 -0800 (PST) Received: by 10.76.27.73 with HTTP; Mon, 4 Mar 2013 08:09:53 -0800 (PST) In-Reply-To: References: Date: Mon, 4 Mar 2013 13:09:53 -0300 Message-ID: Subject: Re: map reduce and sync From: Lucas Bernardi To: user@hadoop.apache.org Content-Type: multipart/alternative; boundary=bcaec55408ac5f8f8f04d71b960a X-Virus-Checked: Checked by ClamAV on apache.org --bcaec55408ac5f8f8f04d71b960a Content-Type: text/plain; charset=ISO-8859-1 Ok, so I found a workaround for this issue, I share it here for others: So the key problem is that hadoop won't update the file size until the file is closed, then the FileInputFormat will see never-closed-files as empty files and generate no splits for the map reduce process. To fix this problem I changed the way the file length is calculated, overriding the listStatus mehtod in a new InputFormat implementation, which inherits from FileInputFormat: @Override protected List listStatus(JobContext job) throws IOException { List listStatus = super.listStatus(job); List result = Lists.newArrayList(); DFSClient dfsClient = null; try { dfsClient = new DFSClient(job.getConfiguration()); for (FileStatus fileStatus : listStatus) { long len = fileStatus.getLen(); if (len == 0) { DFSInputStream open = dfsClient.open(fileStatus.getPath().toUri().getPath()); long fileLength = open.getFileLength(); open.close(); FileStatus fileStatus2 = new FileStatus(fileLength, fileStatus.isDir(), fileStatus.getReplication(), fileStatus.getBlockSize(), fileStatus.getModificationTime(), fileStatus.getAccessTime(), fileStatus.getPermission(), fileStatus.getOwner(), fileStatus.getGroup(), fileStatus.getPath()); result.add(fileStatus2); } else { result.add(fileStatus); } } } finally { if (dfsClient != null) { dfsClient.close(); } } return result; } this worked just fine for me. What do you think? Thanks! Lucas On Mon, Feb 25, 2013 at 7:03 PM, Lucas Bernardi wrote: > It looks like getSplits in FileInputFormat is ignoring 0 lenght files.... > That also would explain the weird behavior of tail, which seems to always > jump to the start since file length is 0. > > So, basically, sync doesn't update file length, any code based on file > size, is unreliable. > > Am I right? > > How can I get around this? > > Lucas > > > On Mon, Feb 25, 2013 at 12:38 PM, Lucas Bernardi wrote: > >> I didn't notice, thanks for the heads up. >> >> >> On Mon, Feb 25, 2013 at 4:31 AM, Harsh J wrote: >> >>> Just an aside (I've not tried to look at the original issue yet), but >>> Scribe has not been maintained (nor has seen a release) in over a year >>> now -- looking at the commit history. Same case with both Facebook and >>> Twitter's fork. >>> >>> On Mon, Feb 25, 2013 at 7:16 AM, Lucas Bernardi >>> wrote: >>> > Yeah I looked at scribe, looks good but sounds like too much for my >>> problem. >>> > I'd rather make it work the simple way. Could you pleas post your >>> code, may >>> > be I'm doing something wrong on the sync side. Maybe a buffer size, >>> block >>> > size or some other parameter is different... >>> > >>> > Thanks! >>> > Lucas >>> > >>> > >>> > On Sun, Feb 24, 2013 at 10:31 PM, Hemanth Yamijala >>> > wrote: >>> >> >>> >> I am using the same version of Hadoop as you. >>> >> >>> >> Can you look at something like Scribe, which AFAIK fits the use case >>> you >>> >> describe. >>> >> >>> >> Thanks >>> >> Hemanth >>> >> >>> >> >>> >> On Sun, Feb 24, 2013 at 3:33 AM, Lucas Bernardi >>> wrote: >>> >>> >>> >>> That is exactly what I did, but in my case, it is like if the file >>> were >>> >>> empty, the job counters say no bytes read. >>> >>> I'm using hadoop 1.0.3 which version did you try? >>> >>> >>> >>> What I'm trying to do is just some basic analyitics on a product >>> search >>> >>> system. There is a search service, every time a user performs a >>> search, the >>> >>> search string, and the results are stored in this file, and the file >>> is >>> >>> sync'ed. I'm actually using pig to do some basic counts, it doesn't >>> work, >>> >>> like I described, because the file looks empty for the map reduce >>> >>> components. I thought it was about pig, but I wasn't sure, so I >>> tried a >>> >>> simple mr job, and used the word count to test the map reduce >>> compoinents >>> >>> actually see the sync'ed bytes. >>> >>> >>> >>> Of course if I close the file, everything works perfectly, but I >>> don't >>> >>> want to close the file every while, since that means I should create >>> another >>> >>> one (since no append support), and that would end up with too many >>> tiny >>> >>> files, something we know is bad for mr performance, and I don't want >>> to add >>> >>> more parts to this (like a file merging tool). I think unign sync is >>> a clean >>> >>> solution, since we don't care about writing performance, so I'd >>> rather keep >>> >>> it like this if I can make it work. >>> >>> >>> >>> Any idea besides hadoop version? >>> >>> >>> >>> Thanks! >>> >>> >>> >>> Lucas >>> >>> >>> >>> >>> >>> >>> >>> On Sat, Feb 23, 2013 at 11:54 AM, Hemanth Yamijala >>> >>> wrote: >>> >>>> >>> >>>> Hi Lucas, >>> >>>> >>> >>>> I tried something like this but got different results. >>> >>>> >>> >>>> I wrote code that opened a file on HDFS, wrote a line and called >>> sync. >>> >>>> Without closing the file, I ran a wordcount with that file as >>> input. It did >>> >>>> work fine and was able to count the words that were sync'ed (even >>> though the >>> >>>> file length seems to come as 0 like you noted in fs -ls) >>> >>>> >>> >>>> So, not sure what's happening in your case. In the MR job, do the >>> job >>> >>>> counters indicate no bytes were read ? >>> >>>> >>> >>>> On a different note though, if you can describe a little more what >>> you >>> >>>> are trying to accomplish, we could probably work a better solution. >>> >>>> >>> >>>> Thanks >>> >>>> hemanth >>> >>>> >>> >>>> >>> >>>> On Sat, Feb 23, 2013 at 7:15 PM, Lucas Bernardi >>> >>>> wrote: >>> >>>>> >>> >>>>> Helo Hemanth, thanks for answering. >>> >>>>> The file is open by a separate process not map reduce related at >>> all. >>> >>>>> You can think of it as a servlet, receiving requests, and writing >>> them to >>> >>>>> this file, every time a request is received it is written and >>> >>>>> org.apache.hadoop.fs.FSDataOutputStream.sync() is invoked. >>> >>>>> >>> >>>>> At the same time, I want to run a map reduce job over this file. >>> Simply >>> >>>>> runing the word count example doesn't seem to work, it is like if >>> the file >>> >>>>> were empty. >>> >>>>> >>> >>>>> hadoop -fs -tail works just fine, and reading the file using >>> >>>>> org.apache.hadoop.fs.FSDataInputStream also works ok. >>> >>>>> >>> >>>>> Last thing, the web interface doesn't see the contents, and command >>> >>>>> hadoop -fs -ls says the file is empty. >>> >>>>> >>> >>>>> What am I doing wrong? >>> >>>>> >>> >>>>> Thanks! >>> >>>>> >>> >>>>> Lucas >>> >>>>> >>> >>>>> >>> >>>>> >>> >>>>> On Sat, Feb 23, 2013 at 4:37 AM, Hemanth Yamijala >>> >>>>> wrote: >>> >>>>>> >>> >>>>>> Could you please clarify, are you opening the file in your mapper >>> code >>> >>>>>> and reading from there ? >>> >>>>>> >>> >>>>>> Thanks >>> >>>>>> Hemanth >>> >>>>>> >>> >>>>>> On Friday, February 22, 2013, Lucas Bernardi wrote: >>> >>>>>>> >>> >>>>>>> Hello there, I'm trying to use hadoop map reduce to process an >>> open >>> >>>>>>> file. The writing process, writes a line to the file and syncs >>> the file to >>> >>>>>>> readers. >>> >>>>>>> (org.apache.hadoop.fs.FSDataOutputStream.sync()). >>> >>>>>>> >>> >>>>>>> If I try to read the file from another process, it works fine, at >>> >>>>>>> least using >>> >>>>>>> org.apache.hadoop.fs.FSDataInputStream. >>> >>>>>>> >>> >>>>>>> hadoop -fs -tail also works just fine >>> >>>>>>> >>> >>>>>>> But it looks like map reduce doesn't read any data. I tried >>> using the >>> >>>>>>> word count example, same thing, it is like if the file were >>> empty for the >>> >>>>>>> map reduce framework. >>> >>>>>>> >>> >>>>>>> I'm using hadoop 1.0.3. and pig 0.10.0 >>> >>>>>>> >>> >>>>>>> I need some help around this. >>> >>>>>>> >>> >>>>>>> Thanks! >>> >>>>>>> >>> >>>>>>> Lucas >>> >>>>> >>> >>>>> >>> >>>> >>> >>> >>> >> >>> > >>> >>> >>> >>> -- >>> Harsh J >>> >> >> > --bcaec55408ac5f8f8f04d71b960a Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable Ok, so I found a workaround for this issue, I share it here for others:So the key problem is that hadoop won't update the file size until the= file is closed, then the FileInputFormat will see never-closed-files as em= pty files and generate no splits for the map reduce process.

To fix this problem I changed the way the file length i= s calculated, overriding the listStatus mehtod in a new InputFormat impleme= ntation, which inherits from FileInputFormat:

=A0 =A0 @Override
=A0 =A0 protected List<FileStatus> l= istStatus(JobContext job) throws IOException {
=A0 =A0 =A0 =A0 Li= st<FileStatus> listStatus =3D super.listStatus(job);
=A0 = =A0 =A0 =A0 List<FileStatus> result =3D Lists.newArrayList();
=A0 =A0 =A0 =A0 DFSClient dfsClient =3D null;
=A0 =A0 =A0 = =A0 try {
=A0 =A0 =A0 =A0 =A0 =A0 dfsClient =3D new DFSClient(job= .getConfiguration());
=A0 =A0 =A0 =A0 =A0 =A0 for (FileStatus fil= eStatus : listStatus) {
=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 long len = =3D fileStatus.getLen();
=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 if (len =3D=3D 0) {
=A0 =A0 = =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 DFSInputStream open =3D dfsClient.open(file= Status.getPath().toUri().getPath());
=A0 =A0 =A0 =A0 =A0 =A0 =A0 = =A0 =A0 =A0 long fileLength =3D open.getFileLength();
=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 open.close();
=A0 =A0 =A0= =A0 =A0 =A0 =A0 =A0 =A0 =A0 FileStatus fileStatus2 =3D new FileStatus(file= Length, fileStatus.isDir(), fileStatus.getReplication(),
=A0 =A0 = =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 fileStatus.getBlockSize(), fileStat= us.getModificationTime(), fileStatus.getAccessTime(),
=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 fileStatus.getPermissi= on(), fileStatus.getOwner(), fileStatus.getGroup(), fileStatus.getPath());<= /div>
=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 result.add(fileStatus2);<= /div>
=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 } else {
=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 result.add(fileStatus);
<= div>=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 }
=A0 =A0 =A0 =A0 =A0 =A0 }
=A0 =A0 =A0 =A0 } finally {
=A0 =A0 =A0 =A0 =A0 =A0 if (= dfsClient !=3D null) {
=A0 =A0 =A0 =A0 =A0 =A0 =A0 =A0 dfsClient.= close();
=A0 =A0 =A0 =A0 =A0 =A0 }
=A0 =A0 =A0 =A0 }
=A0 = =A0 =A0 =A0 return result;
=A0 =A0 }

this worked just fine for me.

What do you think= ?

Thanks!
Lucas

On Mon, Feb 25, 2013 at= 7:03 PM, Lucas Bernardi <lucejb@gmail.com> wrote:
It looks like getSplits in=A0FileInputFormat is ignoring 0 lenght files....=
That also would explain the weird behavior of tail, which seems to alw= ays jump to the start since file length is 0.

So, = basically, sync doesn't update file length, any code based on file size= , is unreliable.

Am I right?

How can I get arou= nd this?

Lucas


On Mon, Feb 25, 2013 at 12:38 PM, L= ucas Bernardi <lucejb@gmail.com> wrote:
I didn't notice, thanks for the heads up= .


On Mon, Feb 25, 2013 at 4:31 = AM, Harsh J <harsh@cloudera.com> wrote:
Just an aside (I've not tried to look at= the original issue yet), but
Scribe has not been maintained (nor has seen a release) in over a year
now -- looking at the commit history. Same case with both Facebook and
Twitter's fork.

On Mon, Feb 25, 2013 at 7:16 AM, Lucas Bernardi <lucejb@gmail.com> wrote:
> Yeah I looked at scribe, looks good but sounds like too much for my pr= oblem.
> I'd rather make it work the simple way. Could you pleas post your = code, may
> be I'm doing something wrong on the sync side. Maybe a buffer size= , block
> size or some other =A0parameter is different...
>
> Thanks!
> Lucas
>
>
> On Sun, Feb 24, 2013 at 10:31 PM, Hemanth Yamijala
> <yhe= manth@thoughtworks.com> wrote:
>>
>> I am using the same version of Hadoop as you.
>>
>> Can you look at something like Scribe, which AFAIK fits the use ca= se you
>> describe.
>>
>> Thanks
>> Hemanth
>>
>>
>> On Sun, Feb 24, 2013 at 3:33 AM, Lucas Bernardi <lucejb@gmail.com> wrote:
>>>
>>> That is exactly what I did, but in my case, it is like if the = file were
>>> empty, the job counters say no bytes read.
>>> I'm using hadoop 1.0.3 which version did you try?
>>>
>>> What I'm trying to do is just some basic analyitics on a p= roduct search
>>> system. There is a search service, every time a user performs = a search, the
>>> search string, and the results are stored in this file, and th= e file is
>>> sync'ed. I'm actually using pig to do some basic count= s, it doesn't work,
>>> like I described, because the file looks empty for the map red= uce
>>> components. I thought it was about pig, but I wasn't sure,= so I tried a
>>> simple mr job, and used the word count to test the map reduce = compoinents
>>> actually see the sync'ed bytes.
>>>
>>> Of course if I close the file, everything works perfectly, but= I don't
>>> want to close the file every while, since that means I should = create another
>>> one (since no append support), and that would end up with too = many tiny
>>> files, something we know is bad for mr performance, and I don&= #39;t want to add
>>> more parts to this (like a file merging tool). I think unign s= ync is a clean
>>> solution, since we don't care about writing performance, s= o I'd rather keep
>>> it like this if I can make it work.
>>>
>>> Any idea besides hadoop version?
>>>
>>> Thanks!
>>>
>>> Lucas
>>>
>>>
>>>
>>> On Sat, Feb 23, 2013 at 11:54 AM, Hemanth Yamijala
>>> <yhemanth@thoughtworks.com> wrote:
>>>>
>>>> Hi Lucas,
>>>>
>>>> I tried something like this but got different results.
>>>>
>>>> I wrote code that opened a file on HDFS, wrote a line and = called sync.
>>>> Without closing the file, I ran a wordcount with that file= as input. It did
>>>> work fine and was able to count the words that were sync&#= 39;ed (even though the
>>>> file length seems to come as 0 like you noted in fs -ls) >>>>
>>>> So, not sure what's happening in your case. In the MR = job, do the job
>>>> counters indicate no bytes were read ?
>>>>
>>>> On a different note though, if you can describe a little m= ore what you
>>>> are trying to accomplish, we could probably work a better = solution.
>>>>
>>>> Thanks
>>>> hemanth
>>>>
>>>>
>>>> On Sat, Feb 23, 2013 at 7:15 PM, Lucas Bernardi <lucejb@gmail.com> >>>> wrote:
>>>>>
>>>>> Helo Hemanth, thanks for answering.
>>>>> The file is open by a separate process not map reduce = related at all.
>>>>> You can think of it as a servlet, receiving requests, = and writing them to
>>>>> this file, every time a request is received it is writ= ten and
>>>>> org.apache.hadoop.fs.FSDataOutputStream.sync() is invo= ked.
>>>>>
>>>>> At the same time, I want to run a map reduce job over = this file. Simply
>>>>> runing the word count example doesn't seem to work= , it is like if the file
>>>>> were empty.
>>>>>
>>>>> hadoop -fs -tail works just fine, and reading the file= using
>>>>> org.apache.hadoop.fs.FSDataInputStream also works ok.<= br> >>>>>
>>>>> Last thing, the web interface doesn't see the cont= ents, and command
>>>>> hadoop -fs -ls says the file is empty.
>>>>>
>>>>> What am I doing wrong?
>>>>>
>>>>> Thanks!
>>>>>
>>>>> Lucas
>>>>>
>>>>>
>>>>>
>>>>> On Sat, Feb 23, 2013 at 4:37 AM, Hemanth Yamijala
>>>>> <yhemanth@thoughtworks.com> wrote:
>>>>>>
>>>>>> Could you please clarify, are you opening the file= in your mapper code
>>>>>> and reading from there ?
>>>>>>
>>>>>> Thanks
>>>>>> Hemanth
>>>>>>
>>>>>> On Friday, February 22, 2013, Lucas Bernardi wrote= :
>>>>>>>
>>>>>>> Hello there, I'm trying to use hadoop map = reduce to process an open
>>>>>>> file. The writing process, writes a line to th= e file and syncs the file to
>>>>>>> readers.
>>>>>>> (org.apache.hadoop.fs.FSDataOutputStream.sync(= )).
>>>>>>>
>>>>>>> If I try to read the file from another process= , it works fine, at
>>>>>>> least using
>>>>>>> org.apache.hadoop.fs.FSDataInputStream.
>>>>>>>
>>>>>>> hadoop -fs -tail also works just fine
>>>>>>>
>>>>>>> But it looks like map reduce doesn't read = any data. I tried using the
>>>>>>> word count example, same thing, it is like if = the file were empty for the
>>>>>>> map reduce framework.
>>>>>>>
>>>>>>> I'm using hadoop 1.0.3. and pig 0.10.0
>>>>>>>
>>>>>>> I need some help around this.
>>>>>>>
>>>>>>> Thanks!
>>>>>>>
>>>>>>> Lucas
>>>>>
>>>>>
>>>>
>>>
>>
>



--
Harsh J



--bcaec55408ac5f8f8f04d71b960a--