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

fang fang chen commented on PIG-2753:
-------------------------------------

This is caused by the steps pig get configurations:
org.apache.pig.backend.hadoop.hbase.HBaseStorage:     
public void setLocation(String location, Job job) throws IOException {
        job.getConfiguration().setBoolean("pig.noSplitCombination", true);
        m_conf = job.getConfiguration();               //comments 1
        HBaseConfiguration.addHbaseResources(m_conf);              //comments 2
        // Make sure the HBase, ZooKeeper, and Guava jars get shipped.
        TableMapReduceUtil.addDependencyJars(job.getConfiguration(), 
            org.apache.hadoop.hbase.client.HTable.class,
            com.google.common.collect.Lists.class,
            org.apache.zookeeper.ZooKeeper.class);

        String tablename = location;
        if (location.startsWith("hbase://")){
           tablename = location.substring(8);
        }
        if (m_table == null) {

comments 1: First load configuration from job.xml(here the hbase configuration 
is right, i.e "hbase.zookeeper.quorum" is "node1")
comments 2: Then load from hbase configuration files(first hbase-defalt.xml, 
then hbase-site.xml). Then if there is no hbase configuration in tasktracker 
side, pig will load configuration from hbase-default.xml(which is include in 
hbase.jar) file and overwrite original configuration read from job.xml. i.e. 
"hbase.zookeeper.quorum" is "localhost" in hbase-default.xml).

                
> In distributed mapreduce mode, pig can not get correct hbase configuration
> --------------------------------------------------------------------------
>
>                 Key: PIG-2753
>                 URL: https://issues.apache.org/jira/browse/PIG-2753
>             Project: Pig
>          Issue Type: Bug
>          Components: piggybank, site
>    Affects Versions: 0.9.1
>         Environment: OS:Red Hat Enterprise Linux Server release 5.5 (Tikanga)
>  
>            Reporter: fang fang chen
>            Assignee: fang fang chen
>
> Hadoop/Hbase/Zookeeper/pig node distribution:
> hadoop nodes: {node1=[namenode, secondarynamenode, jobtracker], 
> node2=[datanode, tasktracker]}
> hbase nodes: {node1=[master, regionserver]}
> pig nodes: {node1, node2}
> zookeeper nodes: {node1}
> Operate hbase table in node1 pig shell like:
> test = LOAD 'hbase://table' USING 
> org.apache.pig.backend.hadoop.hbase.HBaseStorage( 'd:sWords','-loadKey true') 
> AS (ID: bytearray  , Words:chararray );
> result = FOREACH test GENERATE ID, com.pig.test(Words);
> --result = FOREACH AA GENERATE com.pig.test(Words), ID;
> --dump result;
> store result into 'table' using 
> org.apache.pig.backend.hadoop.hbase.HBaseStorage('d:drools_cat');
> --store result into 'AA_10_categs' using 
> org.apache.pig.backend.hadoop.hbase.HBaseStorage('d:cat');
> In tasktracker node, pig can not read hbase configuration in job.xml.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to