You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Robert Mahfoud <rm...@mediosystems.com> on 2011/04/20 21:18:21 UTC

LocalJobRunner and HBASE-2669 woes

I'm upgrading our code base now to HBase 0.90.1 (CDH3-U0) from 0.89.something.
I'm facing a very hard time tracking failures in our unit tests and working around them. Here's the situation:

1- Many of our unit tests run in-process M/R jobs using the LocalJobRunner.
2- We often do things before the M/R jobs and things afte the M/R to validate the outcome.
3- With HBASE-2669 adding the HCM.deleteAllConnections(true) to the TableOutputFormat.close() method, any connections that were open before the M/R job will be closed.
4- Furthermore, any Configuration instance that we used before to get an HTable or HBaseAdmin or anything else that uses HConnection's can no longer be used to initialize any new classes.

Here's an example that used to work before HBASE-2669 but now fails:
<code>

    public static void main(String[] argv) throws Exception {

        final String table1 = "atable" + System.currentTimeMillis();

        final String table2 = "btable" + System.currentTimeMillis();

        final String family = "afamily";


        Configuration config = HBaseConfiguration.create(); // I have a single-node HBase cluster started

                                                            // and no JobTracker configured.

        HBaseTestingUtility util = new HBaseTestingUtility(config);

        // create the table

        HTable t1 = util.createTable(Bytes.toBytes(table1), Bytes.toBytes(family));

        util.loadTable(t1, Bytes.toBytes(family));

        HBaseAdmin admin = new HBaseAdmin(new Configuration(config));



        HTable t2;

        if (admin.tableExists(table2)) {

            t2 = new HTable(new Configuration(config), Bytes.toBytes(table2));

            // make sure t2 is empty

            if (t2.getScanner(new Scan()).iterator().hasNext()) {

                throw new IllegalStateException("t2 is not empty");

            }

        }

        else {

            t2 = util.createTable(Bytes.toBytes(table2), Bytes.toBytes(family));

        }



        // export t1 to disk

        Path dir = new Path("/tmp/" + System.currentTimeMillis());

        Job ex = Export.createSubmittableJob(new Configuration(config), new String[]{table1, dir.toString(), });

        if (!ex.waitForCompletion(true)) {

            throw new RuntimeException("Export job failed.");

        }



        // import into t2

        Job im = Import.createSubmittableJob(new Configuration(config)

                    , new String[] {table2, dir.toString()});   // <-- this uses TableOutputFormat

        if (!im.waitForCompletion(true)) {  // <-- when done, all connections will be closed

            throw new RuntimeException("Import job failed.");

        }



        // do something that needs a connection

        HTableDescriptor[] tables = admin.listTables(); // <-- this will throw

                                                        //          IOException: HConnectionImplementation closed

        for(HTableDescriptor table : tables) {

            System.out.println(table.getNameAsString());

        }

        if (t2.getScanner(new Scan()).iterator().hasNext()) {

            throw new IllegalStateException("t2 is empty!");

        }

    }

</code>

Some issues I'm facing are much more subtle and harder to work around: e.g. it's impossible to run 2 M/R jobs in parallel (at least one of them using TableOutputFormat) because as soon as the first TableOutputFormat calls HCM.deleteAllConnections(...) the other's connections will be closed.

The questions that I have are the following:
1- Is there any simpler way to do the same thing?
2- Are there any best-practices that we are missing, and that could help with this issue?
3- Nothing turns up when we google "HBASE-2669 AND LocalJobRunner" and very few things come up when we google "HBASE-2669" alone. Do you know of anyone else facing a similar problem?
4- Should there be a configuration flag that turns off this behavior when using the LocalJobRunner?

--
Robert Mahfoud


Re: LocalJobRunner and HBASE-2669 woes

Posted by Ted Yu <yu...@gmail.com>.
For HBASE-3777, Karthick and I finally nailed down issues related to
finalizer that made TestTableMapReduce fail.
A final patch would be put up for review :-).

In the end, we expect user to use (better tuned) API wisely.
We will add more javadoc for HTable and the new HConnectionKey class.

Take a look at the new TestTableMapReduce code for recommended usage of
HTable.

Cheers

On Wed, Apr 20, 2011 at 7:20 PM, Robert Mahfoud <ro...@gmail.com>wrote:

> On Wed, Apr 20, 2011 at 12:38 PM, Stack <st...@duboce.net> wrote:
>
> > On Wed, Apr 20, 2011 at 12:18 PM, Robert Mahfoud
> > <rm...@mediosystems.com> wrote:
> > > I'm upgrading our code base now to HBase 0.90.1 (CDH3-U0) from
> > 0.89.something.
> > > I'm facing a very hard time tracking failures in our unit tests and
> > working around them. Here's the situation:
> > >
> > > 1- Many of our unit tests run in-process M/R jobs using the
> > LocalJobRunner.
> > > 2- We often do things before the M/R jobs and things afte the M/R to
> > validate the outcome.
> > > 3- With HBASE-2669 adding the HCM.deleteAllConnections(true) to the
> > TableOutputFormat.close() method, any connections that were open before
> the
> > M/R job will be closed.
> >
> >
> > This came up last week.  There the user made a copy of TOF because
> > this.table is private preventing changing how TOF#close works.  If we
> > added an accessor for the table to TOF so you could override its close
> > behavior, would that work for you?
> >
> >
> Thanks for the suggestion. I'm going to try to do that and add a
> configuration flag to the copied class. I will look for the update that
> makes this.table protected.
> I think that this wasn't a wise design choice since one wouldn't expect
> using an "incidental" class (TOF) to have such a pervasive side effect.
>
> Adding a configuration flag could also be a quick and easy fix until a
> global solution emerges (maybe as part of HBASE-3766 or HBASE-37777).
>
>
> >
> > > Some issues I'm facing are much more subtle and harder to work around:
> > e.g. it's impossible to run 2 M/R jobs in parallel (at least one of them
> > using TableOutputFormat) because as soon as the first TableOutputFormat
> > calls HCM.deleteAllConnections(...) the other's connections will be
> closed.
> > >
> >
> > Yes.
> >
> > > The questions that I have are the following:
> > > 1- Is there any simpler way to do the same thing?
> > > 2- Are there any best-practices that we are missing, and that could
> help
> > with this issue?
> >
> > We changed TOF.  It looks like messed up the case where folks are
> > running multiple MR jobs all up in the one jvm context.
> >
> >
> > > 3- Nothing turns up when we google "HBASE-2669 AND LocalJobRunner" and
> > very few things come up when we google "HBASE-2669" alone. Do you know of
> > anyone else facing a similar problem?
> > > 4- Should there be a configuration flag that turns off this behavior
> when
> > using the LocalJobRunner?
> > >
> >
> > See above.
> >
> > Sorry for the inconvenience.
> >
> > St.Ack
> >
>
>
>
> --
> Robert Mahfoud
> http://www.1apple1day.com/
>

Re: LocalJobRunner and HBASE-2669 woes

Posted by Stack <st...@duboce.net>.
On Wed, Apr 20, 2011 at 7:20 PM, Robert Mahfoud
<ro...@gmail.com> wrote:
> I think that this wasn't a wise design choice since one wouldn't expect
> using an "incidental" class (TOF) to have such a pervasive side effect.
>

Agreed.

Better testing -- coverage and exercise of candidate release versions
-- should have caught this one.

Sorry for the inconvenience,
St.Ack

Re: LocalJobRunner and HBASE-2669 woes

Posted by Robert Mahfoud <ro...@gmail.com>.
On Wed, Apr 20, 2011 at 12:38 PM, Stack <st...@duboce.net> wrote:

> On Wed, Apr 20, 2011 at 12:18 PM, Robert Mahfoud
> <rm...@mediosystems.com> wrote:
> > I'm upgrading our code base now to HBase 0.90.1 (CDH3-U0) from
> 0.89.something.
> > I'm facing a very hard time tracking failures in our unit tests and
> working around them. Here's the situation:
> >
> > 1- Many of our unit tests run in-process M/R jobs using the
> LocalJobRunner.
> > 2- We often do things before the M/R jobs and things afte the M/R to
> validate the outcome.
> > 3- With HBASE-2669 adding the HCM.deleteAllConnections(true) to the
> TableOutputFormat.close() method, any connections that were open before the
> M/R job will be closed.
>
>
> This came up last week.  There the user made a copy of TOF because
> this.table is private preventing changing how TOF#close works.  If we
> added an accessor for the table to TOF so you could override its close
> behavior, would that work for you?
>
>
Thanks for the suggestion. I'm going to try to do that and add a
configuration flag to the copied class. I will look for the update that
makes this.table protected.
I think that this wasn't a wise design choice since one wouldn't expect
using an "incidental" class (TOF) to have such a pervasive side effect.

Adding a configuration flag could also be a quick and easy fix until a
global solution emerges (maybe as part of HBASE-3766 or HBASE-37777).


>
> > Some issues I'm facing are much more subtle and harder to work around:
> e.g. it's impossible to run 2 M/R jobs in parallel (at least one of them
> using TableOutputFormat) because as soon as the first TableOutputFormat
> calls HCM.deleteAllConnections(...) the other's connections will be closed.
> >
>
> Yes.
>
> > The questions that I have are the following:
> > 1- Is there any simpler way to do the same thing?
> > 2- Are there any best-practices that we are missing, and that could help
> with this issue?
>
> We changed TOF.  It looks like messed up the case where folks are
> running multiple MR jobs all up in the one jvm context.
>
>
> > 3- Nothing turns up when we google "HBASE-2669 AND LocalJobRunner" and
> very few things come up when we google "HBASE-2669" alone. Do you know of
> anyone else facing a similar problem?
> > 4- Should there be a configuration flag that turns off this behavior when
> using the LocalJobRunner?
> >
>
> See above.
>
> Sorry for the inconvenience.
>
> St.Ack
>



-- 
Robert Mahfoud
http://www.1apple1day.com/

Re: LocalJobRunner and HBASE-2669 woes

Posted by Stack <st...@duboce.net>.
On Wed, Apr 20, 2011 at 12:18 PM, Robert Mahfoud
<rm...@mediosystems.com> wrote:
> I'm upgrading our code base now to HBase 0.90.1 (CDH3-U0) from 0.89.something.
> I'm facing a very hard time tracking failures in our unit tests and working around them. Here's the situation:
>
> 1- Many of our unit tests run in-process M/R jobs using the LocalJobRunner.
> 2- We often do things before the M/R jobs and things afte the M/R to validate the outcome.
> 3- With HBASE-2669 adding the HCM.deleteAllConnections(true) to the TableOutputFormat.close() method, any connections that were open before the M/R job will be closed.


This came up last week.  There the user made a copy of TOF because
this.table is private preventing changing how TOF#close works.  If we
added an accessor for the table to TOF so you could override its close
behavior, would that work for you?



> Some issues I'm facing are much more subtle and harder to work around: e.g. it's impossible to run 2 M/R jobs in parallel (at least one of them using TableOutputFormat) because as soon as the first TableOutputFormat calls HCM.deleteAllConnections(...) the other's connections will be closed.
>

Yes.

> The questions that I have are the following:
> 1- Is there any simpler way to do the same thing?
> 2- Are there any best-practices that we are missing, and that could help with this issue?

We changed TOF.  It looks like messed up the case where folks are
running multiple MR jobs all up in the one jvm context.


> 3- Nothing turns up when we google "HBASE-2669 AND LocalJobRunner" and very few things come up when we google "HBASE-2669" alone. Do you know of anyone else facing a similar problem?
> 4- Should there be a configuration flag that turns off this behavior when using the LocalJobRunner?
>

See above.

Sorry for the inconvenience.

St.Ack