Yeah it should, also I'm pretty sure you're right to say that this
regression comes from HBASE-2036... would you mind opening a jira?

Thanks for the report and the digging Dan!

J-D

On Wed, Feb 23, 2011 at 3:30 PM, Dan Harvey <[email protected]> wrote:
> Ah ok, most of the time we were using the default Hadoop configuration object 
> and not the HBase one.
>
> I guess that's a change between 0.20 and 0.90? Would it not make sense for 
> the TableMapReduceUtil class to do that for you? As you'll need it in every 
> HBase map reduce job.
>
> Anyway, I guess we should just change our job configs to include the HBase 
> configuration too then it would work fine.
>
> Thanks,
>
> On 23 Feb 2011, at 19:44, Jean-Daniel Cryans <[email protected]> wrote:
>
>> How do you create the configuration object Dan? Are you doing:
>>
>> Configuration conf = HBaseConfiguration.create();
>> Job job = new Job(conf, "somename");
>>
>> or are you just creating a normal Configuration?
>>
>> BTW the code I wrote is what I expect people do and what I'm doing myself.
>>
>> J-D
>>
>> On Wed, Feb 23, 2011 at 7:19 AM, Dan <[email protected]> wrote:
>>> Or the other way would be adding the HBase configs to the Hadoop config,
>>> which I think maybe what is intended.
>>>
>>> If I do it whilst I'm setting up the job
>>> with HBaseConfiguration.addHbaseResources() it works fine, should
>>> the TableMapReduceUtil.initTableMapperJob do this for you?
>>>
>>> I think this was the change that remove the old HBaseConfiguration from
>>> being used https://issues.apache.org/jira/browse/HBASE-2036, but no other
>>> way to add the HBase configs for a TableInputFormat seems to have been
>>> added?
>>>
>>> Sorry if I'm completely missing something here!
>>>
>>> On Wed, Feb 23, 2011 at 1:57 PM, Dan <[email protected]> wrote:
>>>
>>>> Hey,
>>>>
>>>> I'm just testing our code to move over to 0.90 and I'm finding some issues
>>>> with the map/reduce jobs we've written using TableInputFormat.
>>>>
>>>> We setup the jobs using TableMapReduceUtil.initTableMapperJob(..); which
>>>> worked fine in 0.20.6 but now throws the following errors when I try to run
>>>> them :-
>>>>
>>>> 11/02/23 12:38:44 ERROR zookeeper.ZKConfig: no clientPort found in zoo.cfg
>>>> 11/02/23 12:38:44 ERROR mapreduce.TableInputFormat:
>>>> org.apache.hadoop.hbase.ZooKeeperConnectionException: java.io.IOException:
>>>> Unable to determine ZooKeeper ensemble
>>>> ...
>>>> Exception in thread "main" java.io.IOException: No table was provided.
>>>> at
>>>> org.apache.hadoop.hbase.mapreduce.TableInputFormatBase.getSplits(TableInputFormatBase.java:130)
>>>> ...
>>>>
>>>> Which I think basically means it can't find the quorum/port for zookeeper
>>>> in the hbase configuration object from the hbase-*.xml files.
>>>> Looking into this a bit further it seems the TableInputFormat's setConf()
>>>> method creates the HTable using the Hadoop configuration object which
>>>> doesn't contain any of the HBase configuration...
>>>>
>>>> So am I setting the job up incorrectly? If not it doesn't seem to make
>>>> sense to me that the HTable should be constructed with the hadoop
>>>> configuration so I guess there might be a bug in the TableInputFormat.
>>>> Modifying it to not take the hadoop configuration object allows the job to
>>>> work correctly and picks up the hbase configuration I've put on the class
>>>> path, https://gist.github.com/840403. If that's the case I'll file a bug
>>>> report and add the patch.
>>>>
>>>> Thanks,
>>>>
>>>
>

Reply via email to