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

Daniel Dai commented on PIG-3441:
---------------------------------

Let's first figure out when does myfs-site.xml get injected. In 
HExecutionEngine.init, Pig will create a Configuration with defaults 
(getExecConf), and put all entries in PigContext.properties. And everytime Pig 
need a Configuration, create Configuration without defaults and inject all 
PigContext.properties. What I find is when Pig invokes HExecutionEngine.init, 
some default resources is not yet get added, and what I do in the patch is to 
create a Configuration object in later stage and inject missing entries 
(MapReduceLauncher.launchPig). So let's check:
1. First tell me is myfs-default.xml/myfs-site.xml default resource (using 
Configuration.addDefaultResource) or regular resource (using 
Configuration.addResource)
2. After HExecutionEngine.init, does PigContext.properties contains your entry?
3. In MapReduceLauncher.launchPig line 
"ConfigurationUtil.updateProperties(HExecutionEngine.getExecConf(pc.getProperties()),
 pc.getProperties());", does Configuration object created in getExecConf has 
your resource? And after updateProperties, does PigContext.properties contains 
your entry?

> Allow Pig to use default resources from Configuration objects
> -------------------------------------------------------------
>
>                 Key: PIG-3441
>                 URL: https://issues.apache.org/jira/browse/PIG-3441
>             Project: Pig
>          Issue Type: Bug
>          Components: impl
>    Affects Versions: 0.11.1
>            Reporter: Bhooshan Mogal
>            Assignee: Daniel Dai
>         Attachments: PIG-3441-2.patch, PIG-3441-3.patch, PIG-3441.patch, 
> PIG-3441_1.patch
>
>
> Pig currently ignores parameters from configuration files added statically to 
> Configuration objects as Configuration.addDefaultResource(filename.xml).
> Consider the following scenario -
> In a hadoop FileSystem driver for a non-HDFS filesystem you load properties 
> specific to that FileSystem in a static initializer block in the class that 
> extends org.apache.hadoop.fs.Filesystem for your FileSystem like below - 
> {code}
> class MyFileSystem extends FileSystem {
>         static {
>               Configuration.addDefaultResource("myfs-default.xml");
>               Configuration.addDefaultResource("myfs-site.xml");
>       }
> }
> {code}
> Interfaces like the Hadoop CLI, Hive, Hadoop M/R can find configuration 
> parameters defined in these configuration files as long as they are on the 
> classpath.
> However, Pig cannot find parameters from these files, because it ignores 
> configuration files added statically.
> Pig should allow users to specify if they would like pig to read parameters 
> from resources loaded statically.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to