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

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

I see the exception. The new resource gets introduced in 
LogicalPlanBuilder.buildLoadOp when we process the load statement. When we 
create LOLoad, we will read schema from MyFileSystem, and MyFileSystem do 
expect the entry in myfs.xml there, here is the stack:
{code}
Caused by: java.lang.IllegalStateException: This is the error mentioned in 
PIG-3441
        at pig.pig3441.MyFileSystem.initialize(MyFileSystem.java:38)
        at 
org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2397)
        at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:89)
        at 
org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2431)
        at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2413)
        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:368)
        at 
org.apache.pig.backend.hadoop.datastorage.HDataStorage.init(HDataStorage.java:70)
        at 
org.apache.pig.backend.hadoop.datastorage.HDataStorage.<init>(HDataStorage.java:53)
        at 
org.apache.pig.builtin.JsonMetadata.findMetaFile(JsonMetadata.java:109)
        at org.apache.pig.builtin.JsonMetadata.getSchema(JsonMetadata.java:189)
        at org.apache.pig.builtin.PigStorage.getSchema(PigStorage.java:538)
        at 
org.apache.pig.newplan.logical.relational.LOLoad.getSchemaFromMetaData(LOLoad.java:175)
        at 
org.apache.pig.newplan.logical.relational.LOLoad.<init>(LOLoad.java:89)
        at 
org.apache.pig.parser.LogicalPlanBuilder.buildLoadOp(LogicalPlanBuilder.java:885)
{code}
LogicalPlanBuilder.buildLoadOp is after HExecutionEngine.init, where we inject 
all the configuration in. so the new entries in myfs-site.xml are not there. By 
the time we check missing resources again in MapReduceLauncher.launchPig, it is 
too late. The reason your fix works is because in HDataStorage.init:67, we 
create configuration with defaults and then pass to FileSystem.get().

Before we make potentially disrupting change to always create configuration 
with defaults, can we first introduce a config which you can pass additional 
resource file (myfs-site.xml)?

> 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