You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-user@hadoop.apache.org by Lucas Bernardi <lu...@gmail.com> on 2013/03/04 17:09:53 UTC

Re: map reduce and sync

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<FileStatus> listStatus(JobContext job) throws
IOException {
        List<FileStatus> listStatus = super.listStatus(job);
        List<FileStatus> 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 <lu...@gmail.com> 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 <lu...@gmail.com> wrote:
>
>> I didn't notice, thanks for the heads up.
>>
>>
>> On Mon, Feb 25, 2013 at 4:31 AM, Harsh J <ha...@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 <lu...@gmail.com>
>>> 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
>>> > <yh...@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 case
>>> you
>>> >> describe.
>>> >>
>>> >> Thanks
>>> >> Hemanth
>>> >>
>>> >>
>>> >> On Sun, Feb 24, 2013 at 3:33 AM, Lucas Bernardi <lu...@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 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
>>> >>> <yh...@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'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 <lu...@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 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
>>> >>>>> <yh...@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 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
>>>
>>
>>
>