You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Dan <da...@gmail.com> on 2011/02/23 14:57:03 UTC

TableInputFormat configuration problems with 0.90

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,

Re: TableInputFormat configuration problems with 0.90

Posted by Dan <da...@danharvey.co.uk>.
Sure, I've put this up on Jira here :-

https://issues.apache.org/jira/browse/HBASE-3578

On Wed, Feb 23, 2011 at 11:36 PM, Jean-Daniel Cryans <jd...@apache.org>wrote:

> 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 <da...@gmail.com> 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 <jd...@apache.org>
> 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 <da...@gmail.com> 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 <da...@gmail.com> 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,
> >>>>
> >>>
> >
>

Re: TableInputFormat configuration problems with 0.90

Posted by Jean-Daniel Cryans <jd...@apache.org>.
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 <da...@gmail.com> 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 <jd...@apache.org> 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 <da...@gmail.com> 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 <da...@gmail.com> 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,
>>>>
>>>
>

Re: TableInputFormat configuration problems with 0.90

Posted by Dan Harvey <da...@gmail.com>.
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 <jd...@apache.org> 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 <da...@gmail.com> 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 <da...@gmail.com> 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,
>>> 
>> 

Re: TableInputFormat configuration problems with 0.90

Posted by Jean-Daniel Cryans <jd...@apache.org>.
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 <da...@gmail.com> 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 <da...@gmail.com> 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,
>>
>

Re: TableInputFormat configuration problems with 0.90

Posted by Dan <da...@gmail.com>.
Other than patching the TableInputFormat class, if you call

HBaseConfiguration.addHbaseResources(job.getConfiguration());

on the job whilst your setting up the map reduce job it will add the needed
configuration to Hadoop's configuration class.

On Wed, Feb 23, 2011 at 4:34 PM, Cavus,M.,Fa. Post Direkt <
M.Cavus@postdirekt.de> wrote:

> Hi Dan,
> I've the same problem. Have you sold the problem or have you got any
> workaround.
>
> Regards
> Musa
>
> -----Original Message-----
> From: Dan [mailto:danharvey42@gmail.com]
> Sent: Wednesday, February 23, 2011 4:19 PM
> To: user@hbase.apache.org
> Subject: Re: TableInputFormat configuration problems with 0.90
>
> 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 <da...@gmail.com> 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(TableIn
> putFormatBase.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,
> >
>

Re: TableInputFormat configuration problems with 0.90

Posted by Dan <da...@gmail.com>.
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 <da...@gmail.com> 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,
>