hadoop-pig-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Alan Gates (JIRA)" <j...@apache.org>
Subject [jira] Commented: (PIG-1174) Creation of output path should be done by storage function
Date Tue, 05 Jan 2010 00:24:54 GMT

    [ https://issues.apache.org/jira/browse/PIG-1174?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12796449#action_12796449
] 

Alan Gates commented on PIG-1174:
---------------------------------

Delegating creation of the output path to the storage function is not trivial.  The storage
function is invoked on every reducer (or every mapper for map only jobs).  So delaying creation
until the storage function will create a race condition that the storage functions will handle.
 And if the solution is just to let the first one win and all the rest error out and ignore
the error, for a large job this will still bombard the namenode with hundreds or thousands
of bogus mkdir requests.  It also has the problem that all the storage functions that get
an error can't tell if it's really an error (there's old data there they are overwriting)
versus they just lost the race and another function has already created it.

We are reworking the way load and store function interact with InputFormats and OutputFormats
(see PIG-966 for full details).  This will push the responsibility of file creation onto the
OutputFormat.  This may partially address your concerns.

> Creation of output path should be done by storage function
> ----------------------------------------------------------
>
>                 Key: PIG-1174
>                 URL: https://issues.apache.org/jira/browse/PIG-1174
>             Project: Pig
>          Issue Type: Bug
>            Reporter: Bill Graham
>
> When executing a STORE command, Pig creates the output location before the storage function
gets called. This causes problems with storage functions that have logic to determine the
output location. See this thread:
> http://www.mail-archive.com/pig-user%40hadoop.apache.org/msg01538.html
> For example, when making a request like this:
> STORE A INTO '/my/home/output' USING MultiStorage('/my/home/output','0', 'none', '\t');
> Pig creates a file '/my/home/output' and then an exception is thrown when MultiStorage
tries to make a directory under '/my/home/output'. The workaround is to instead specify a
dummy location as the first path like so:
> STORE A INTO '/my/home/output/temp' USING MultiStorage('/my/home/output','0', 'none',
'\t');
> Two changes should be made:
> 1. The path specified in the INTO clause should be available to the storage function
so it doesn't need to be duplicated.
> 2. The creation of the output paths should be delegated to the storage function.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message