You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Dru Jensen <dr...@gmail.com> on 2008/07/30 22:36:11 UTC

newbie - map reduce not distributing

Hello,

I created a map/reduce process by extending the TableMap and  
TableReduce API but for some reason
when I run multiple mappers, in the logs its showing that the same  
rows are being processed by each Mapper.

When I say logs, I mean in the hadoop task tracker (localhost:50030)  
and drilling down into the logs.

Do I need to manually perform a TableSplit or is this supposed to be  
done automatically?

If its something I need to do manually, can someone point me to some  
sample code?

If its supposed to be automatic and each mapper was supposed to get  
its own set of rows,
should I write up a bug for this?  I using trunk 0.2.0 on hadoop trunk  
0.17.2.

thanks,
Dru

Re: newbie - map reduce not distributing

Posted by Dru Jensen <dr...@gmail.com>.
J-D, Andy, St.Ack,

The patch fixed my problem.   Thanks again for your help.

Dru

On Aug 4, 2008, at 9:32 AM, Dru Jensen wrote:

> Thanks Andrew.  I will test the patch and verify everything is  
> working.
>
> On Aug 3, 2008, at 11:57 PM, Andrew Purtell wrote:
>
>> Opened HBASE-793.
>> https://issues.apache.org/jira/browse/HBASE-793
>>
>>  - Andy
>>
>>> From: stack <st...@duboce.net>
>>> Subject: Re: newbie - map reduce not distributing
>>> To: hbase-user@hadoop.apache.org
>>> Date: Saturday, August 2, 2008, 1:08 PM
>>>
>>> Thank you for persevering Dru.
>>>
>>> Indeed, a bug in getStartKeys will make us process all
>>> tables that have a column family name in common.
>> [...]
>>> Dru Jensen wrote:
>>>> I found what is causing the same rows being sent to
>>>> multiple map tasks.
>>>>
>>>> If you have the same column family name in other
>>>> tables, the Test will send the same rows to multiple map
>>>> reducers.
>> [...]
>>
>>
>>
>


Re: newbie - map reduce not distributing

Posted by Andrew Purtell <ap...@yahoo.com>.
Thanks Dru.

  - Andy


> From: Dru Jensen <dr...@gmail.com>
> Subject: Re: newbie - map reduce not distributing
> To: hbase-user@hadoop.apache.org
> Date: Monday, August 4, 2008, 9:32 AM
>
> Thanks Andrew.  I will test the patch and verify everything
> is working.
> 
> On Aug 3, 2008, at 11:57 PM, Andrew Purtell wrote:
> > Opened HBASE-793.
> > https://issues.apache.org/jira/browse/HBASE-793
> >
> >   - Andy



      

Re: newbie - map reduce not distributing

Posted by Dru Jensen <dr...@gmail.com>.
Thanks Andrew.  I will test the patch and verify everything is working.

On Aug 3, 2008, at 11:57 PM, Andrew Purtell wrote:

> Opened HBASE-793.
> https://issues.apache.org/jira/browse/HBASE-793
>
>   - Andy
>
>> From: stack <st...@duboce.net>
>> Subject: Re: newbie - map reduce not distributing
>> To: hbase-user@hadoop.apache.org
>> Date: Saturday, August 2, 2008, 1:08 PM
>>
>> Thank you for persevering Dru.
>>
>> Indeed, a bug in getStartKeys will make us process all
>> tables that have a column family name in common.
> [...]
>> Dru Jensen wrote:
>>> I found what is causing the same rows being sent to
>>> multiple map tasks.
>>>
>>> If you have the same column family name in other
>>> tables, the Test will send the same rows to multiple map
>>> reducers.
> [...]
>
>
>


Re: newbie - map reduce not distributing

Posted by Andrew Purtell <ap...@yahoo.com>.
Opened HBASE-793.
https://issues.apache.org/jira/browse/HBASE-793

   - Andy

> From: stack <st...@duboce.net>
> Subject: Re: newbie - map reduce not distributing
> To: hbase-user@hadoop.apache.org
> Date: Saturday, August 2, 2008, 1:08 PM
>
> Thank you for persevering Dru.
> 
> Indeed, a bug in getStartKeys will make us process all
> tables that have a column family name in common.
[...]
> Dru Jensen wrote:
> > I found what is causing the same rows being sent to
> > multiple map tasks. 
> >
> > If you have the same column family name in other
> > tables, the Test will send the same rows to multiple map
> > reducers.
[...]


      

Re: newbie - map reduce not distributing

Posted by Dru Jensen <dr...@gmail.com>.
Thanks St.Ack and J-D for your help on this.

On Aug 2, 2008, at 1:08 PM, stack wrote:

> Thank you for persevering Dru.
>
> Indeed, a bug in getStartKeys will make us process all tables that  
> have a column family name in common.
>
> Here's the code:
>
> public byte[][] getStartKeys() throws IOException {
>   final List<byte[]> keyList = new ArrayList<byte[]>();
>
>   MetaScannerVisitor visitor = new MetaScannerVisitor() {
>     public boolean processRow(RowResult rowResult) throws  
> IOException {
>       HRegionInfo info = Writables.getHRegionInfo(
>           rowResult.get(HConstants.COL_REGIONINFO));
>
>       if (!(info.isOffline() || info.isSplit())) {
>         keyList.add(info.getStartKey());
>       }
>       return true;
>     }
>
>   };
>   MetaScanner.metaScan(configuration, visitor, this.tableName);
>   return keyList.toArray(new byte[keyList.size()][]);
> }
>
> The above Visitor is visiting the meta table.  Its checking column  
> family name.  Any region that is not offlined or split gets added to  
> the list of regions.  Its not checking that the region belongs to  
> the wanted table.
>
> Would suggest adding something like a:
>
>       if (!Bytes.equals(rowResult.getRow(), getTableName())) {
>         return true;
>       }
>
> before the column check.
>
> For bonus points, you might want to make it so we stop scanning  
> after we've passed out all regions for the wanted table.
>
> If the above change fixes things, mind making an issue and attaching  
> patch?  Would be nice if we could get it into 0.2.0.
>
> Thanks for being persistent.
>
> St.Ack
>
> Dru Jensen wrote:
>> J-D,
>>
>> I found what is causing the same rows being sent to multiple map  
>> tasks.
>> If you have the same column family name in other tables, the Test  
>> will send the same rows to multiple map reducers.
>>
>> I'm attaching the DEBUG logs and the test class.
>>
>> thanks,
>> Dru
>>
>>
>> ------------------------------------------------------------------------
>>
>>
>> ------------------------------------------------------------------------
>>
>>
>> ------------------------------------------------------------------------
>>
>>
>> Steps to duplicate:
>>
>> 1. "hadoop dfs -rmr /hbase" and started hbase with no tables.
>>
>> 2. launch hbase shell and create table:
>> create 'test', 'content'
>> put 'test','test','content:test','testing'
>> put 'test','test2','content:test','testing2'
>>
>> 3. Run the Test MapReduce, only one map task is launched.  This is  
>> correct.
>> Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
>> 08/08/01 10:23:07 INFO mapred.JobClient: Running job:  
>> job_200808010944_0006
>> 08/08/01 10:23:08 INFO mapred.JobClient:  map 0% reduce 0%
>> 08/08/01 10:23:12 INFO mapred.JobClient:  map 100% reduce 0%
>> 08/08/01 10:23:14 INFO mapred.JobClient:  map 100% reduce 100%
>> 08/08/01 10:23:15 INFO mapred.JobClient: Job complete:  
>> job_200808010944_0006
>> 08/08/01 10:23:15 INFO mapred.JobClient: Counters: 13
>> 08/08/01 10:23:15 INFO mapred.JobClient:   Job Counters 08/08/01  
>> 10:23:15 INFO mapred.JobClient:     Launched map tasks=1
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Launched reduce tasks=1
>> 08/08/01 10:23:15 INFO mapred.JobClient:   Map-Reduce Framework
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Map input records=2
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Map output records=0
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Map input bytes=0
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Map output bytes=0
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Combine input records=0
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Combine output records=0
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Reduce input groups=0
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Reduce input records=0
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Reduce output records=0
>> 08/08/01 10:23:15 INFO mapred.JobClient:   File Systems
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Local bytes read=136
>> 08/08/01 10:23:15 INFO mapred.JobClient:     Local bytes written=280
>>
>> 4. create another table with 'content' column family
>> create 'weird', 'content'
>>
>> 5. Run the same test.  Two map tasks are launched even though  
>> column is in a different table.
>> Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
>> 08/08/01 10:24:15 INFO mapred.JobClient: Running job:  
>> job_200808010944_0007
>> 08/08/01 10:24:16 INFO mapred.JobClient:  map 0% reduce 0%
>> 08/08/01 10:24:19 INFO mapred.JobClient:  map 50% reduce 0%
>> 08/08/01 10:24:21 INFO mapred.JobClient:  map 100% reduce 0%
>> 08/08/01 10:24:26 INFO mapred.JobClient:  map 100% reduce 16%
>> 08/08/01 10:24:27 INFO mapred.JobClient:  map 100% reduce 100%
>> 08/08/01 10:24:28 INFO mapred.JobClient: Job complete:  
>> job_200808010944_0007
>> 08/08/01 10:24:28 INFO mapred.JobClient: Counters: 13
>> 08/08/01 10:24:28 INFO mapred.JobClient:   Job Counters 08/08/01  
>> 10:24:28 INFO mapred.JobClient:     Launched map tasks=2
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Launched reduce tasks=1
>> 08/08/01 10:24:28 INFO mapred.JobClient:   Map-Reduce Framework
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Map input records=4
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Map output records=0
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Map input bytes=0
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Map output bytes=0
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Combine input records=0
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Combine output records=0
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Reduce input groups=0
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Reduce input records=0
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Reduce output records=0
>> 08/08/01 10:24:28 INFO mapred.JobClient:   File Systems
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Local bytes read=136
>> 08/08/01 10:24:28 INFO mapred.JobClient:     Local bytes written=424
>>
>> 6. create another table that doesn't have 'content' column family
>> create 'notweird', 'notcontent'
>>
>> 7. run the test.  Still only 2 map tasks.
>> Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
>> 08/08/01 10:24:57 INFO mapred.JobClient: Running job:  
>> job_200808010944_0008
>> 08/08/01 10:24:58 INFO mapred.JobClient:  map 0% reduce 0%
>> 08/08/01 10:25:02 INFO mapred.JobClient:  map 50% reduce 0%
>> 08/08/01 10:25:03 INFO mapred.JobClient:  map 100% reduce 0%
>> 08/08/01 10:25:09 INFO mapred.JobClient:  map 100% reduce 100%
>> 08/08/01 10:25:10 INFO mapred.JobClient: Job complete:  
>> job_200808010944_0008
>> 08/08/01 10:25:10 INFO mapred.JobClient: Counters: 13
>> 08/08/01 10:25:10 INFO mapred.JobClient:   Job Counters 08/08/01  
>> 10:25:10 INFO mapred.JobClient:     Launched map tasks=2
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Launched reduce tasks=1
>> 08/08/01 10:25:10 INFO mapred.JobClient:   Map-Reduce Framework
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Map input records=4
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Map output records=0
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Map input bytes=0
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Map output bytes=0
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Combine input records=0
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Combine output records=0
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Reduce input groups=0
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Reduce input records=0
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Reduce output records=0
>> 08/08/01 10:25:10 INFO mapred.JobClient:   File Systems
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Local bytes read=136
>> 08/08/01 10:25:10 INFO mapred.JobClient:     Local bytes written=424
>>
>> 8. create another table with two families.  One called 'content'
>> create 'weirdest', 'content','notcontent'
>>
>> 9. run test.  3 map tasks are launched.
>> Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
>> 08/08/01 10:25:41 INFO mapred.JobClient: Running job:  
>> job_200808010944_0009
>> 08/08/01 10:25:42 INFO mapred.JobClient:  map 0% reduce 0%
>> 08/08/01 10:25:45 INFO mapred.JobClient:  map 33% reduce 0%
>> 08/08/01 10:25:46 INFO mapred.JobClient:  map 100% reduce 0%
>> 08/08/01 10:25:50 INFO mapred.JobClient:  map 100% reduce 100%
>> 08/08/01 10:25:51 INFO mapred.JobClient: Job complete:  
>> job_200808010944_0009
>> 08/08/01 10:25:51 INFO mapred.JobClient: Counters: 13
>> 08/08/01 10:25:51 INFO mapred.JobClient:   Job Counters 08/08/01  
>> 10:25:51 INFO mapred.JobClient:     Launched map tasks=3
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Launched reduce tasks=1
>> 08/08/01 10:25:51 INFO mapred.JobClient:   Map-Reduce Framework
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Map input records=6
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Map output records=0
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Map input bytes=0
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Map output bytes=0
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Combine input records=0
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Combine output records=0
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Reduce input groups=0
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Reduce input records=0
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Reduce output records=0
>> 08/08/01 10:25:51 INFO mapred.JobClient:   File Systems
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Local bytes read=136
>> 08/08/01 10:25:51 INFO mapred.JobClient:     Local bytes written=568
>>
>>
>> On Jul 31, 2008, at 5:42 PM, Jean-Daniel Cryans wrote:
>>
>>> Dru,
>>>
>>> There is something truly weird with your setup. I would advise  
>>> running your
>>> code (the simple one that only logs the rows) with DEBUG on. See the
>>> faq<http://wiki.apache.org/hadoop/Hbase/FAQ#5>on how to do it. Then
>>> get back with syslog and stdout. This way we will have
>>> more informations on how scanners are handling this.
>>>
>>> Also FYI, I ran the same code as yours with 0.2.0 on my setup and  
>>> had no
>>> problems.
>>>
>>> J-D
>>>
>>> On Thu, Jul 31, 2008 at 7:06 PM, Dru Jensen <drujensen@gmail.com <mailto:drujensen@gmail.com 
>>> >> wrote:
>>>
>>>> UPDATE:  I modified the RowCounter example and verified that it  
>>>> is sending
>>>> the same row to multiple map tasks also. Is this a known bug or  
>>>> am I doing
>>>> something truly as(s)inine?  Any help is appreciated.
>>>>
>>>>
>>>> On Jul 30, 2008, at 3:02 PM, Dru Jensen wrote:
>>>>
>>>> J-D,
>>>>>
>>>>> Again, thank you for your help on this.
>>>>>
>>>>> hitting the HBASE Master port 60010:
>>>>> System 1 - 2 regions
>>>>> System 2 - 1 region
>>>>> System 3 - 3 regions
>>>>>
>>>>> In order to demonstrate the behavior I'm seeing, I wrote a test  
>>>>> class.
>>>>>
>>>>> public class Test extends Configured implements Tool {
>>>>>
>>>>>  public static class Map extends TableMap {
>>>>>
>>>>>      @Override
>>>>>      public void map(ImmutableBytesWritable key, RowResult row,
>>>>> OutputCollector output, Reporter r) throws IOException {
>>>>>
>>>>>          String key_str = new String(key.get());
>>>>>          System.out.println("map: key = " + key_str);
>>>>>      }
>>>>>
>>>>>  }
>>>>>
>>>>>  public class Reduce extends TableReduce {
>>>>>
>>>>>      @Override
>>>>>      public void reduce(WritableComparable key, Iterator values,
>>>>> OutputCollector output, Reporter r) throws IOException {
>>>>>
>>>>>      }
>>>>>
>>>>>  }
>>>>>
>>>>>  public int run(String[] args) throws Exception {
>>>>>      JobConf job = new JobConf(getConf(), Test.class);
>>>>>      job.setJobName("Test");
>>>>>
>>>>>      job.setNumMapTasks(4);
>>>>>      job.setNumReduceTasks(1);
>>>>>
>>>>>      Map.initJob("test", "content:", Map.class, HStoreKey.class,
>>>>> HbaseMapWritable.class, job);
>>>>>      Reduce.initJob("test", Reduce.class, job);
>>>>>
>>>>>      JobClient.runJob(job);
>>>>>      return 0;
>>>>>  }
>>>>>
>>>>>  public static void main(String[] args) throws Exception {
>>>>>      int res = ToolRunner.run(new Configuration(), new Test(),  
>>>>> args);
>>>>>      System.exit(res);
>>>>>  }
>>>>> }
>>>>>
>>>>> In hbase shell:
>>>>> create 'test','content'
>>>>> put 'test','test','content:test','testing'
>>>>> put 'test','test2','content:test','testing2'
>>>>>
>>>>>
>>>>> The Hadoop log results:
>>>>> Task Logs: 'task_200807301447_0001_m_000000_0'
>>>>>
>>>>>
>>>>>
>>>>> stdout logs
>>>>> map: key = test
>>>>> map: key = test2
>>>>>
>>>>>
>>>>> stderr logs
>>>>>
>>>>>
>>>>> syslog logs
>>>>> 2008-07-30 14:51:16,410 INFO  
>>>>> org.apache.hadoop.metrics.jvm.JvmMetrics:
>>>>> Initializing JVM Metrics with processName=MAP, sessionId=
>>>>> 2008-07-30 14:51:16,507 INFO org.apache.hadoop.mapred.MapTask:
>>>>> numReduceTasks: 1
>>>>> 2008-07-30 14:51:17,120 INFO  
>>>>> org.apache.hadoop.mapred.TaskRunner: Task
>>>>> 'task_200807301447_0001_m_000000_0' done.
>>>>>
>>>>> Task Logs: 'task_200807301447_0001_m_000001_0'
>>>>>
>>>>>
>>>>>
>>>>> stdout logs
>>>>> map: key = test
>>>>> map: key = test2
>>>>>
>>>>>
>>>>> stderr logs
>>>>>
>>>>>
>>>>> syslog logs
>>>>> 2008-07-30 14:51:16,410 INFO  
>>>>> org.apache.hadoop.metrics.jvm.JvmMetrics:
>>>>> Initializing JVM Metrics with processName=MAP, sessionId=
>>>>> 2008-07-30 14:51:16,509 INFO org.apache.hadoop.mapred.MapTask:
>>>>> numReduceTasks: 1
>>>>> 2008-07-30 14:51:17,118 INFO  
>>>>> org.apache.hadoop.mapred.TaskRunner: Task
>>>>> 'task_200807301447_0001_m_000001_0' done.
>>>>>
>>>>> Tasks 3 and 4 are the same.
>>>>>
>>>>> Each map task is seeing the same rows.  Any help to prevent this  
>>>>> is
>>>>> appreciated.
>>>>>
>>>>> Thanks,
>>>>> Dru
>>>>>
>>>>>
>>>>> On Jul 30, 2008, at 2:22 PM, Jean-Daniel Cryans wrote:
>>>>>
>>>>> Dru,
>>>>>>
>>>>>> It is not supposed to process many times the same rows. Can I  
>>>>>> see the log
>>>>>> you're talking about? Also, how many regions do you have in  
>>>>>> your table?
>>>>>> (info available in the web UI).
>>>>>>
>>>>>> thx
>>>>>>
>>>>>> J-D
>>>>>>
>>>>>> On Wed, Jul 30, 2008 at 5:04 PM, Dru Jensen  
>>>>>> <drujensen@gmail.com <ma...@gmail.com>> wrote:
>>>>>>
>>>>>> J-D,
>>>>>>>
>>>>>>> thanks for your quick response.   I have 4 mapping processes  
>>>>>>> running on
>>>>>>> 3
>>>>>>> systems.
>>>>>>>
>>>>>>> Are the same rows being processed 4 times by each mapping  
>>>>>>> processor?
>>>>>>> According to the logs they are.
>>>>>>>
>>>>>>> When I run a map/reduce against a file, only one row gets  
>>>>>>> logged per
>>>>>>> mapper.  Why would this be different for hbase tables?
>>>>>>>
>>>>>>> I would think only one mapping process would process that one  
>>>>>>> row and it
>>>>>>> would only show up once in only one log.
>>>>>>> preferable it would be the same system that has the region.
>>>>>>>
>>>>>>> I only want one row to be processed once.  Is there anyway to  
>>>>>>> change
>>>>>>> this
>>>>>>> behavior without running only 1 mapper?
>>>>>>>
>>>>>>> thanks,
>>>>>>> Dru
>>>>>>>
>>>>>>>
>>>>>>> On Jul 30, 2008, at 1:44 PM, Jean-Daniel Cryans wrote:
>>>>>>>
>>>>>>> Dru,
>>>>>>>
>>>>>>>>
>>>>>>>> The regions will split when achieving a certain threshold so  
>>>>>>>> if you
>>>>>>>> want
>>>>>>>> your computing to be distributed, you will have to have more  
>>>>>>>> data.
>>>>>>>>
>>>>>>>> Regards,
>>>>>>>>
>>>>>>>> J-D
>>>>>>>>
>>>>>>>> On Wed, Jul 30, 2008 at 4:36 PM, Dru Jensen <drujensen@gmail.com 
>>>>>>>>  <ma...@gmail.com>>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>> Hello,
>>>>>>>>
>>>>>>>>>
>>>>>>>>> I created a map/reduce process by extending the TableMap and
>>>>>>>>> TableReduce
>>>>>>>>> API but for some reason
>>>>>>>>> when I run multiple mappers, in the logs its showing that  
>>>>>>>>> the same
>>>>>>>>> rows
>>>>>>>>> are
>>>>>>>>> being processed by each Mapper.
>>>>>>>>>
>>>>>>>>> When I say logs, I mean in the hadoop task tracker  
>>>>>>>>> (localhost:50030)
>>>>>>>>> and
>>>>>>>>> drilling down into the logs.
>>>>>>>>>
>>>>>>>>> Do I need to manually perform a TableSplit or is this  
>>>>>>>>> supposed to be
>>>>>>>>> done
>>>>>>>>> automatically?
>>>>>>>>>
>>>>>>>>> If its something I need to do manually, can someone point me  
>>>>>>>>> to some
>>>>>>>>> sample
>>>>>>>>> code?
>>>>>>>>>
>>>>>>>>> If its supposed to be automatic and each mapper was supposed  
>>>>>>>>> to get
>>>>>>>>> its
>>>>>>>>> own
>>>>>>>>> set of rows,
>>>>>>>>> should I write up a bug for this?  I using trunk 0.2.0 on  
>>>>>>>>> hadoop trunk
>>>>>>>>> 0.17.2.
>>>>>>>>>
>>>>>>>>> thanks,
>>>>>>>>> Dru
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>
>>>>
>>
>> =
>


Re: newbie - map reduce not distributing

Posted by stack <st...@duboce.net>.
Thank you for persevering Dru.

Indeed, a bug in getStartKeys will make us process all tables that have 
a column family name in common.

Here's the code:

  public byte[][] getStartKeys() throws IOException {
    final List<byte[]> keyList = new ArrayList<byte[]>();

    MetaScannerVisitor visitor = new MetaScannerVisitor() {
      public boolean processRow(RowResult rowResult) throws IOException {
        HRegionInfo info = Writables.getHRegionInfo(
            rowResult.get(HConstants.COL_REGIONINFO));

        if (!(info.isOffline() || info.isSplit())) {
          keyList.add(info.getStartKey());
        }
        return true;
      }

    };
    MetaScanner.metaScan(configuration, visitor, this.tableName);
    return keyList.toArray(new byte[keyList.size()][]);
  }

The above Visitor is visiting the meta table.  Its checking column 
family name.  Any region that is not offlined or split gets added to the 
list of regions.  Its not checking that the region belongs to the wanted 
table.

Would suggest adding something like a:

        if (!Bytes.equals(rowResult.getRow(), getTableName())) {
          return true;
        }

before the column check.

For bonus points, you might want to make it so we stop scanning after 
we've passed out all regions for the wanted table.

If the above change fixes things, mind making an issue and attaching 
patch?  Would be nice if we could get it into 0.2.0.

Thanks for being persistent.

St.Ack

Dru Jensen wrote:
> J-D,
>
> I found what is causing the same rows being sent to multiple map tasks. 
>
> If you have the same column family name in other tables, the Test will 
> send the same rows to multiple map reducers.
>
> I'm attaching the DEBUG logs and the test class.
>
> thanks,
> Dru
>
>
> ------------------------------------------------------------------------
>
>
> ------------------------------------------------------------------------
>
>
> ------------------------------------------------------------------------
>
>
> Steps to duplicate:
>
> 1. "hadoop dfs -rmr /hbase" and started hbase with no tables.
>
> 2. launch hbase shell and create table:
> create 'test', 'content'
> put 'test','test','content:test','testing'
> put 'test','test2','content:test','testing2'
>
> 3. Run the Test MapReduce, only one map task is launched.  This is 
> correct.
> Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
> 08/08/01 10:23:07 INFO mapred.JobClient: Running job: 
> job_200808010944_0006
> 08/08/01 10:23:08 INFO mapred.JobClient:  map 0% reduce 0%
> 08/08/01 10:23:12 INFO mapred.JobClient:  map 100% reduce 0%
> 08/08/01 10:23:14 INFO mapred.JobClient:  map 100% reduce 100%
> 08/08/01 10:23:15 INFO mapred.JobClient: Job complete: 
> job_200808010944_0006
> 08/08/01 10:23:15 INFO mapred.JobClient: Counters: 13
> 08/08/01 10:23:15 INFO mapred.JobClient:   Job Counters 
> 08/08/01 10:23:15 INFO mapred.JobClient:     Launched map tasks=1
> 08/08/01 10:23:15 INFO mapred.JobClient:     Launched reduce tasks=1
> 08/08/01 10:23:15 INFO mapred.JobClient:   Map-Reduce Framework
> 08/08/01 10:23:15 INFO mapred.JobClient:     Map input records=2
> 08/08/01 10:23:15 INFO mapred.JobClient:     Map output records=0
> 08/08/01 10:23:15 INFO mapred.JobClient:     Map input bytes=0
> 08/08/01 10:23:15 INFO mapred.JobClient:     Map output bytes=0
> 08/08/01 10:23:15 INFO mapred.JobClient:     Combine input records=0
> 08/08/01 10:23:15 INFO mapred.JobClient:     Combine output records=0
> 08/08/01 10:23:15 INFO mapred.JobClient:     Reduce input groups=0
> 08/08/01 10:23:15 INFO mapred.JobClient:     Reduce input records=0
> 08/08/01 10:23:15 INFO mapred.JobClient:     Reduce output records=0
> 08/08/01 10:23:15 INFO mapred.JobClient:   File Systems
> 08/08/01 10:23:15 INFO mapred.JobClient:     Local bytes read=136
> 08/08/01 10:23:15 INFO mapred.JobClient:     Local bytes written=280
>
> 4. create another table with 'content' column family
> create 'weird', 'content'
>
> 5. Run the same test.  Two map tasks are launched even though column 
> is in a different table.
> Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
> 08/08/01 10:24:15 INFO mapred.JobClient: Running job: 
> job_200808010944_0007
> 08/08/01 10:24:16 INFO mapred.JobClient:  map 0% reduce 0%
> 08/08/01 10:24:19 INFO mapred.JobClient:  map 50% reduce 0%
> 08/08/01 10:24:21 INFO mapred.JobClient:  map 100% reduce 0%
> 08/08/01 10:24:26 INFO mapred.JobClient:  map 100% reduce 16%
> 08/08/01 10:24:27 INFO mapred.JobClient:  map 100% reduce 100%
> 08/08/01 10:24:28 INFO mapred.JobClient: Job complete: 
> job_200808010944_0007
> 08/08/01 10:24:28 INFO mapred.JobClient: Counters: 13
> 08/08/01 10:24:28 INFO mapred.JobClient:   Job Counters 
> 08/08/01 10:24:28 INFO mapred.JobClient:     Launched map tasks=2
> 08/08/01 10:24:28 INFO mapred.JobClient:     Launched reduce tasks=1
> 08/08/01 10:24:28 INFO mapred.JobClient:   Map-Reduce Framework
> 08/08/01 10:24:28 INFO mapred.JobClient:     Map input records=4
> 08/08/01 10:24:28 INFO mapred.JobClient:     Map output records=0
> 08/08/01 10:24:28 INFO mapred.JobClient:     Map input bytes=0
> 08/08/01 10:24:28 INFO mapred.JobClient:     Map output bytes=0
> 08/08/01 10:24:28 INFO mapred.JobClient:     Combine input records=0
> 08/08/01 10:24:28 INFO mapred.JobClient:     Combine output records=0
> 08/08/01 10:24:28 INFO mapred.JobClient:     Reduce input groups=0
> 08/08/01 10:24:28 INFO mapred.JobClient:     Reduce input records=0
> 08/08/01 10:24:28 INFO mapred.JobClient:     Reduce output records=0
> 08/08/01 10:24:28 INFO mapred.JobClient:   File Systems
> 08/08/01 10:24:28 INFO mapred.JobClient:     Local bytes read=136
> 08/08/01 10:24:28 INFO mapred.JobClient:     Local bytes written=424
>
> 6. create another table that doesn't have 'content' column family
> create 'notweird', 'notcontent'
>
> 7. run the test.  Still only 2 map tasks.
> Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
> 08/08/01 10:24:57 INFO mapred.JobClient: Running job: 
> job_200808010944_0008
> 08/08/01 10:24:58 INFO mapred.JobClient:  map 0% reduce 0%
> 08/08/01 10:25:02 INFO mapred.JobClient:  map 50% reduce 0%
> 08/08/01 10:25:03 INFO mapred.JobClient:  map 100% reduce 0%
> 08/08/01 10:25:09 INFO mapred.JobClient:  map 100% reduce 100%
> 08/08/01 10:25:10 INFO mapred.JobClient: Job complete: 
> job_200808010944_0008
> 08/08/01 10:25:10 INFO mapred.JobClient: Counters: 13
> 08/08/01 10:25:10 INFO mapred.JobClient:   Job Counters 
> 08/08/01 10:25:10 INFO mapred.JobClient:     Launched map tasks=2
> 08/08/01 10:25:10 INFO mapred.JobClient:     Launched reduce tasks=1
> 08/08/01 10:25:10 INFO mapred.JobClient:   Map-Reduce Framework
> 08/08/01 10:25:10 INFO mapred.JobClient:     Map input records=4
> 08/08/01 10:25:10 INFO mapred.JobClient:     Map output records=0
> 08/08/01 10:25:10 INFO mapred.JobClient:     Map input bytes=0
> 08/08/01 10:25:10 INFO mapred.JobClient:     Map output bytes=0
> 08/08/01 10:25:10 INFO mapred.JobClient:     Combine input records=0
> 08/08/01 10:25:10 INFO mapred.JobClient:     Combine output records=0
> 08/08/01 10:25:10 INFO mapred.JobClient:     Reduce input groups=0
> 08/08/01 10:25:10 INFO mapred.JobClient:     Reduce input records=0
> 08/08/01 10:25:10 INFO mapred.JobClient:     Reduce output records=0
> 08/08/01 10:25:10 INFO mapred.JobClient:   File Systems
> 08/08/01 10:25:10 INFO mapred.JobClient:     Local bytes read=136
> 08/08/01 10:25:10 INFO mapred.JobClient:     Local bytes written=424
>
> 8. create another table with two families.  One called 'content'
> create 'weirdest', 'content','notcontent'
>
> 9. run test.  3 map tasks are launched.
> Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
> 08/08/01 10:25:41 INFO mapred.JobClient: Running job: 
> job_200808010944_0009
> 08/08/01 10:25:42 INFO mapred.JobClient:  map 0% reduce 0%
> 08/08/01 10:25:45 INFO mapred.JobClient:  map 33% reduce 0%
> 08/08/01 10:25:46 INFO mapred.JobClient:  map 100% reduce 0%
> 08/08/01 10:25:50 INFO mapred.JobClient:  map 100% reduce 100%
> 08/08/01 10:25:51 INFO mapred.JobClient: Job complete: 
> job_200808010944_0009
> 08/08/01 10:25:51 INFO mapred.JobClient: Counters: 13
> 08/08/01 10:25:51 INFO mapred.JobClient:   Job Counters 
> 08/08/01 10:25:51 INFO mapred.JobClient:     Launched map tasks=3
> 08/08/01 10:25:51 INFO mapred.JobClient:     Launched reduce tasks=1
> 08/08/01 10:25:51 INFO mapred.JobClient:   Map-Reduce Framework
> 08/08/01 10:25:51 INFO mapred.JobClient:     Map input records=6
> 08/08/01 10:25:51 INFO mapred.JobClient:     Map output records=0
> 08/08/01 10:25:51 INFO mapred.JobClient:     Map input bytes=0
> 08/08/01 10:25:51 INFO mapred.JobClient:     Map output bytes=0
> 08/08/01 10:25:51 INFO mapred.JobClient:     Combine input records=0
> 08/08/01 10:25:51 INFO mapred.JobClient:     Combine output records=0
> 08/08/01 10:25:51 INFO mapred.JobClient:     Reduce input groups=0
> 08/08/01 10:25:51 INFO mapred.JobClient:     Reduce input records=0
> 08/08/01 10:25:51 INFO mapred.JobClient:     Reduce output records=0
> 08/08/01 10:25:51 INFO mapred.JobClient:   File Systems
> 08/08/01 10:25:51 INFO mapred.JobClient:     Local bytes read=136
> 08/08/01 10:25:51 INFO mapred.JobClient:     Local bytes written=568
>
>
> On Jul 31, 2008, at 5:42 PM, Jean-Daniel Cryans wrote:
>
>> Dru,
>>
>> There is something truly weird with your setup. I would advise 
>> running your
>> code (the simple one that only logs the rows) with DEBUG on. See the
>> faq<http://wiki.apache.org/hadoop/Hbase/FAQ#5>on how to do it. Then
>> get back with syslog and stdout. This way we will have
>> more informations on how scanners are handling this.
>>
>> Also FYI, I ran the same code as yours with 0.2.0 on my setup and had no
>> problems.
>>
>> J-D
>>
>> On Thu, Jul 31, 2008 at 7:06 PM, Dru Jensen <drujensen@gmail.com 
>> <ma...@gmail.com>> wrote:
>>
>>> UPDATE:  I modified the RowCounter example and verified that it is 
>>> sending
>>> the same row to multiple map tasks also. Is this a known bug or am I 
>>> doing
>>> something truly as(s)inine?  Any help is appreciated.
>>>
>>>
>>> On Jul 30, 2008, at 3:02 PM, Dru Jensen wrote:
>>>
>>> J-D,
>>>>
>>>> Again, thank you for your help on this.
>>>>
>>>> hitting the HBASE Master port 60010:
>>>> System 1 - 2 regions
>>>> System 2 - 1 region
>>>> System 3 - 3 regions
>>>>
>>>> In order to demonstrate the behavior I'm seeing, I wrote a test class.
>>>>
>>>> public class Test extends Configured implements Tool {
>>>>
>>>>   public static class Map extends TableMap {
>>>>
>>>>       @Override
>>>>       public void map(ImmutableBytesWritable key, RowResult row,
>>>> OutputCollector output, Reporter r) throws IOException {
>>>>
>>>>           String key_str = new String(key.get());
>>>>           System.out.println("map: key = " + key_str);
>>>>       }
>>>>
>>>>   }
>>>>
>>>>   public class Reduce extends TableReduce {
>>>>
>>>>       @Override
>>>>       public void reduce(WritableComparable key, Iterator values,
>>>> OutputCollector output, Reporter r) throws IOException {
>>>>
>>>>       }
>>>>
>>>>   }
>>>>
>>>>   public int run(String[] args) throws Exception {
>>>>       JobConf job = new JobConf(getConf(), Test.class);
>>>>       job.setJobName("Test");
>>>>
>>>>       job.setNumMapTasks(4);
>>>>       job.setNumReduceTasks(1);
>>>>
>>>>       Map.initJob("test", "content:", Map.class, HStoreKey.class,
>>>> HbaseMapWritable.class, job);
>>>>       Reduce.initJob("test", Reduce.class, job);
>>>>
>>>>       JobClient.runJob(job);
>>>>       return 0;
>>>>   }
>>>>
>>>>   public static void main(String[] args) throws Exception {
>>>>       int res = ToolRunner.run(new Configuration(), new Test(), args);
>>>>       System.exit(res);
>>>>   }
>>>> }
>>>>
>>>> In hbase shell:
>>>> create 'test','content'
>>>> put 'test','test','content:test','testing'
>>>> put 'test','test2','content:test','testing2'
>>>>
>>>>
>>>> The Hadoop log results:
>>>> Task Logs: 'task_200807301447_0001_m_000000_0'
>>>>
>>>>
>>>>
>>>> stdout logs
>>>> map: key = test
>>>> map: key = test2
>>>>
>>>>
>>>> stderr logs
>>>>
>>>>
>>>> syslog logs
>>>> 2008-07-30 14:51:16,410 INFO org.apache.hadoop.metrics.jvm.JvmMetrics:
>>>> Initializing JVM Metrics with processName=MAP, sessionId=
>>>> 2008-07-30 14:51:16,507 INFO org.apache.hadoop.mapred.MapTask:
>>>> numReduceTasks: 1
>>>> 2008-07-30 14:51:17,120 INFO org.apache.hadoop.mapred.TaskRunner: Task
>>>> 'task_200807301447_0001_m_000000_0' done.
>>>>
>>>> Task Logs: 'task_200807301447_0001_m_000001_0'
>>>>
>>>>
>>>>
>>>> stdout logs
>>>> map: key = test
>>>> map: key = test2
>>>>
>>>>
>>>> stderr logs
>>>>
>>>>
>>>> syslog logs
>>>> 2008-07-30 14:51:16,410 INFO org.apache.hadoop.metrics.jvm.JvmMetrics:
>>>> Initializing JVM Metrics with processName=MAP, sessionId=
>>>> 2008-07-30 14:51:16,509 INFO org.apache.hadoop.mapred.MapTask:
>>>> numReduceTasks: 1
>>>> 2008-07-30 14:51:17,118 INFO org.apache.hadoop.mapred.TaskRunner: Task
>>>> 'task_200807301447_0001_m_000001_0' done.
>>>>
>>>> Tasks 3 and 4 are the same.
>>>>
>>>> Each map task is seeing the same rows.  Any help to prevent this is
>>>> appreciated.
>>>>
>>>> Thanks,
>>>> Dru
>>>>
>>>>
>>>> On Jul 30, 2008, at 2:22 PM, Jean-Daniel Cryans wrote:
>>>>
>>>> Dru,
>>>>>
>>>>> It is not supposed to process many times the same rows. Can I see 
>>>>> the log
>>>>> you're talking about? Also, how many regions do you have in your 
>>>>> table?
>>>>> (info available in the web UI).
>>>>>
>>>>> thx
>>>>>
>>>>> J-D
>>>>>
>>>>> On Wed, Jul 30, 2008 at 5:04 PM, Dru Jensen <drujensen@gmail.com 
>>>>> <ma...@gmail.com>> wrote:
>>>>>
>>>>> J-D,
>>>>>>
>>>>>> thanks for your quick response.   I have 4 mapping processes 
>>>>>> running on
>>>>>> 3
>>>>>> systems.
>>>>>>
>>>>>> Are the same rows being processed 4 times by each mapping processor?
>>>>>> According to the logs they are.
>>>>>>
>>>>>> When I run a map/reduce against a file, only one row gets logged per
>>>>>> mapper.  Why would this be different for hbase tables?
>>>>>>
>>>>>> I would think only one mapping process would process that one row 
>>>>>> and it
>>>>>> would only show up once in only one log.
>>>>>> preferable it would be the same system that has the region.
>>>>>>
>>>>>> I only want one row to be processed once.  Is there anyway to change
>>>>>> this
>>>>>> behavior without running only 1 mapper?
>>>>>>
>>>>>> thanks,
>>>>>> Dru
>>>>>>
>>>>>>
>>>>>> On Jul 30, 2008, at 1:44 PM, Jean-Daniel Cryans wrote:
>>>>>>
>>>>>> Dru,
>>>>>>
>>>>>>>
>>>>>>> The regions will split when achieving a certain threshold so if you
>>>>>>> want
>>>>>>> your computing to be distributed, you will have to have more data.
>>>>>>>
>>>>>>> Regards,
>>>>>>>
>>>>>>> J-D
>>>>>>>
>>>>>>> On Wed, Jul 30, 2008 at 4:36 PM, Dru Jensen <drujensen@gmail.com 
>>>>>>> <ma...@gmail.com>>
>>>>>>> wrote:
>>>>>>>
>>>>>>> Hello,
>>>>>>>
>>>>>>>>
>>>>>>>> I created a map/reduce process by extending the TableMap and
>>>>>>>> TableReduce
>>>>>>>> API but for some reason
>>>>>>>> when I run multiple mappers, in the logs its showing that the same
>>>>>>>> rows
>>>>>>>> are
>>>>>>>> being processed by each Mapper.
>>>>>>>>
>>>>>>>> When I say logs, I mean in the hadoop task tracker 
>>>>>>>> (localhost:50030)
>>>>>>>> and
>>>>>>>> drilling down into the logs.
>>>>>>>>
>>>>>>>> Do I need to manually perform a TableSplit or is this supposed 
>>>>>>>> to be
>>>>>>>> done
>>>>>>>> automatically?
>>>>>>>>
>>>>>>>> If its something I need to do manually, can someone point me to 
>>>>>>>> some
>>>>>>>> sample
>>>>>>>> code?
>>>>>>>>
>>>>>>>> If its supposed to be automatic and each mapper was supposed to get
>>>>>>>> its
>>>>>>>> own
>>>>>>>> set of rows,
>>>>>>>> should I write up a bug for this?  I using trunk 0.2.0 on 
>>>>>>>> hadoop trunk
>>>>>>>> 0.17.2.
>>>>>>>>
>>>>>>>> thanks,
>>>>>>>> Dru
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>
>>>
>
> =


Re: newbie - map reduce not distributing

Posted by Dru Jensen <dr...@gmail.com>.
J-D,

I found what is causing the same rows being sent to multiple map tasks.

If you have the same column family name in other tables, the Test will  
send the same rows to multiple map reducers.

I'm attaching the DEBUG logs and the test class.

thanks,
Dru



Steps to duplicate:

1. "hadoop dfs -rmr /hbase" and started hbase with no tables.

2. launch hbase shell and create table:
create 'test', 'content'
put 'test','test','content:test','testing'
put 'test','test2','content:test','testing2'

3. Run the Test MapReduce, only one map task is launched.  This is  
correct.
Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
08/08/01 10:23:07 INFO mapred.JobClient: Running job:  
job_200808010944_0006
08/08/01 10:23:08 INFO mapred.JobClient:  map 0% reduce 0%
08/08/01 10:23:12 INFO mapred.JobClient:  map 100% reduce 0%
08/08/01 10:23:14 INFO mapred.JobClient:  map 100% reduce 100%
08/08/01 10:23:15 INFO mapred.JobClient: Job complete:  
job_200808010944_0006
08/08/01 10:23:15 INFO mapred.JobClient: Counters: 13
08/08/01 10:23:15 INFO mapred.JobClient:   Job Counters
08/08/01 10:23:15 INFO mapred.JobClient:     Launched map tasks=1
08/08/01 10:23:15 INFO mapred.JobClient:     Launched reduce tasks=1
08/08/01 10:23:15 INFO mapred.JobClient:   Map-Reduce Framework
08/08/01 10:23:15 INFO mapred.JobClient:     Map input records=2
08/08/01 10:23:15 INFO mapred.JobClient:     Map output records=0
08/08/01 10:23:15 INFO mapred.JobClient:     Map input bytes=0
08/08/01 10:23:15 INFO mapred.JobClient:     Map output bytes=0
08/08/01 10:23:15 INFO mapred.JobClient:     Combine input records=0
08/08/01 10:23:15 INFO mapred.JobClient:     Combine output records=0
08/08/01 10:23:15 INFO mapred.JobClient:     Reduce input groups=0
08/08/01 10:23:15 INFO mapred.JobClient:     Reduce input records=0
08/08/01 10:23:15 INFO mapred.JobClient:     Reduce output records=0
08/08/01 10:23:15 INFO mapred.JobClient:   File Systems
08/08/01 10:23:15 INFO mapred.JobClient:     Local bytes read=136
08/08/01 10:23:15 INFO mapred.JobClient:     Local bytes written=280

4. create another table with 'content' column family
create 'weird', 'content'

5. Run the same test.  Two map tasks are launched even though column  
is in a different table.
Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
08/08/01 10:24:15 INFO mapred.JobClient: Running job:  
job_200808010944_0007
08/08/01 10:24:16 INFO mapred.JobClient:  map 0% reduce 0%
08/08/01 10:24:19 INFO mapred.JobClient:  map 50% reduce 0%
08/08/01 10:24:21 INFO mapred.JobClient:  map 100% reduce 0%
08/08/01 10:24:26 INFO mapred.JobClient:  map 100% reduce 16%
08/08/01 10:24:27 INFO mapred.JobClient:  map 100% reduce 100%
08/08/01 10:24:28 INFO mapred.JobClient: Job complete:  
job_200808010944_0007
08/08/01 10:24:28 INFO mapred.JobClient: Counters: 13
08/08/01 10:24:28 INFO mapred.JobClient:   Job Counters
08/08/01 10:24:28 INFO mapred.JobClient:     Launched map tasks=2
08/08/01 10:24:28 INFO mapred.JobClient:     Launched reduce tasks=1
08/08/01 10:24:28 INFO mapred.JobClient:   Map-Reduce Framework
08/08/01 10:24:28 INFO mapred.JobClient:     Map input records=4
08/08/01 10:24:28 INFO mapred.JobClient:     Map output records=0
08/08/01 10:24:28 INFO mapred.JobClient:     Map input bytes=0
08/08/01 10:24:28 INFO mapred.JobClient:     Map output bytes=0
08/08/01 10:24:28 INFO mapred.JobClient:     Combine input records=0
08/08/01 10:24:28 INFO mapred.JobClient:     Combine output records=0
08/08/01 10:24:28 INFO mapred.JobClient:     Reduce input groups=0
08/08/01 10:24:28 INFO mapred.JobClient:     Reduce input records=0
08/08/01 10:24:28 INFO mapred.JobClient:     Reduce output records=0
08/08/01 10:24:28 INFO mapred.JobClient:   File Systems
08/08/01 10:24:28 INFO mapred.JobClient:     Local bytes read=136
08/08/01 10:24:28 INFO mapred.JobClient:     Local bytes written=424

6. create another table that doesn't have 'content' column family
create 'notweird', 'notcontent'

7. run the test.  Still only 2 map tasks.
Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
08/08/01 10:24:57 INFO mapred.JobClient: Running job:  
job_200808010944_0008
08/08/01 10:24:58 INFO mapred.JobClient:  map 0% reduce 0%
08/08/01 10:25:02 INFO mapred.JobClient:  map 50% reduce 0%
08/08/01 10:25:03 INFO mapred.JobClient:  map 100% reduce 0%
08/08/01 10:25:09 INFO mapred.JobClient:  map 100% reduce 100%
08/08/01 10:25:10 INFO mapred.JobClient: Job complete:  
job_200808010944_0008
08/08/01 10:25:10 INFO mapred.JobClient: Counters: 13
08/08/01 10:25:10 INFO mapred.JobClient:   Job Counters
08/08/01 10:25:10 INFO mapred.JobClient:     Launched map tasks=2
08/08/01 10:25:10 INFO mapred.JobClient:     Launched reduce tasks=1
08/08/01 10:25:10 INFO mapred.JobClient:   Map-Reduce Framework
08/08/01 10:25:10 INFO mapred.JobClient:     Map input records=4
08/08/01 10:25:10 INFO mapred.JobClient:     Map output records=0
08/08/01 10:25:10 INFO mapred.JobClient:     Map input bytes=0
08/08/01 10:25:10 INFO mapred.JobClient:     Map output bytes=0
08/08/01 10:25:10 INFO mapred.JobClient:     Combine input records=0
08/08/01 10:25:10 INFO mapred.JobClient:     Combine output records=0
08/08/01 10:25:10 INFO mapred.JobClient:     Reduce input groups=0
08/08/01 10:25:10 INFO mapred.JobClient:     Reduce input records=0
08/08/01 10:25:10 INFO mapred.JobClient:     Reduce output records=0
08/08/01 10:25:10 INFO mapred.JobClient:   File Systems
08/08/01 10:25:10 INFO mapred.JobClient:     Local bytes read=136
08/08/01 10:25:10 INFO mapred.JobClient:     Local bytes written=424

8. create another table with two families.  One called 'content'
create 'weirdest', 'content','notcontent'

9. run test.  3 map tasks are launched.
Jensens-MacBook-2:hadoop drujensen$ ./test.sh Test
08/08/01 10:25:41 INFO mapred.JobClient: Running job:  
job_200808010944_0009
08/08/01 10:25:42 INFO mapred.JobClient:  map 0% reduce 0%
08/08/01 10:25:45 INFO mapred.JobClient:  map 33% reduce 0%
08/08/01 10:25:46 INFO mapred.JobClient:  map 100% reduce 0%
08/08/01 10:25:50 INFO mapred.JobClient:  map 100% reduce 100%
08/08/01 10:25:51 INFO mapred.JobClient: Job complete:  
job_200808010944_0009
08/08/01 10:25:51 INFO mapred.JobClient: Counters: 13
08/08/01 10:25:51 INFO mapred.JobClient:   Job Counters
08/08/01 10:25:51 INFO mapred.JobClient:     Launched map tasks=3
08/08/01 10:25:51 INFO mapred.JobClient:     Launched reduce tasks=1
08/08/01 10:25:51 INFO mapred.JobClient:   Map-Reduce Framework
08/08/01 10:25:51 INFO mapred.JobClient:     Map input records=6
08/08/01 10:25:51 INFO mapred.JobClient:     Map output records=0
08/08/01 10:25:51 INFO mapred.JobClient:     Map input bytes=0
08/08/01 10:25:51 INFO mapred.JobClient:     Map output bytes=0
08/08/01 10:25:51 INFO mapred.JobClient:     Combine input records=0
08/08/01 10:25:51 INFO mapred.JobClient:     Combine output records=0
08/08/01 10:25:51 INFO mapred.JobClient:     Reduce input groups=0
08/08/01 10:25:51 INFO mapred.JobClient:     Reduce input records=0
08/08/01 10:25:51 INFO mapred.JobClient:     Reduce output records=0
08/08/01 10:25:51 INFO mapred.JobClient:   File Systems
08/08/01 10:25:51 INFO mapred.JobClient:     Local bytes read=136
08/08/01 10:25:51 INFO mapred.JobClient:     Local bytes written=568


On Jul 31, 2008, at 5:42 PM, Jean-Daniel Cryans wrote:

> Dru,
>
> There is something truly weird with your setup. I would advise  
> running your
> code (the simple one that only logs the rows) with DEBUG on. See the
> faq<http://wiki.apache.org/hadoop/Hbase/FAQ#5>on how to do it. Then
> get back with syslog and stdout. This way we will have
> more informations on how scanners are handling this.
>
> Also FYI, I ran the same code as yours with 0.2.0 on my setup and  
> had no
> problems.
>
> J-D
>
> On Thu, Jul 31, 2008 at 7:06 PM, Dru Jensen <dr...@gmail.com>  
> wrote:
>
>> UPDATE:  I modified the RowCounter example and verified that it is  
>> sending
>> the same row to multiple map tasks also. Is this a known bug or am  
>> I doing
>> something truly as(s)inine?  Any help is appreciated.
>>
>>
>> On Jul 30, 2008, at 3:02 PM, Dru Jensen wrote:
>>
>> J-D,
>>>
>>> Again, thank you for your help on this.
>>>
>>> hitting the HBASE Master port 60010:
>>> System 1 - 2 regions
>>> System 2 - 1 region
>>> System 3 - 3 regions
>>>
>>> In order to demonstrate the behavior I'm seeing, I wrote a test  
>>> class.
>>>
>>> public class Test extends Configured implements Tool {
>>>
>>>   public static class Map extends TableMap {
>>>
>>>       @Override
>>>       public void map(ImmutableBytesWritable key, RowResult row,
>>> OutputCollector output, Reporter r) throws IOException {
>>>
>>>           String key_str = new String(key.get());
>>>           System.out.println("map: key = " + key_str);
>>>       }
>>>
>>>   }
>>>
>>>   public class Reduce extends TableReduce {
>>>
>>>       @Override
>>>       public void reduce(WritableComparable key, Iterator values,
>>> OutputCollector output, Reporter r) throws IOException {
>>>
>>>       }
>>>
>>>   }
>>>
>>>   public int run(String[] args) throws Exception {
>>>       JobConf job = new JobConf(getConf(), Test.class);
>>>       job.setJobName("Test");
>>>
>>>       job.setNumMapTasks(4);
>>>       job.setNumReduceTasks(1);
>>>
>>>       Map.initJob("test", "content:", Map.class, HStoreKey.class,
>>> HbaseMapWritable.class, job);
>>>       Reduce.initJob("test", Reduce.class, job);
>>>
>>>       JobClient.runJob(job);
>>>       return 0;
>>>   }
>>>
>>>   public static void main(String[] args) throws Exception {
>>>       int res = ToolRunner.run(new Configuration(), new Test(),  
>>> args);
>>>       System.exit(res);
>>>   }
>>> }
>>>
>>> In hbase shell:
>>> create 'test','content'
>>> put 'test','test','content:test','testing'
>>> put 'test','test2','content:test','testing2'
>>>
>>>
>>> The Hadoop log results:
>>> Task Logs: 'task_200807301447_0001_m_000000_0'
>>>
>>>
>>>
>>> stdout logs
>>> map: key = test
>>> map: key = test2
>>>
>>>
>>> stderr logs
>>>
>>>
>>> syslog logs
>>> 2008-07-30 14:51:16,410 INFO  
>>> org.apache.hadoop.metrics.jvm.JvmMetrics:
>>> Initializing JVM Metrics with processName=MAP, sessionId=
>>> 2008-07-30 14:51:16,507 INFO org.apache.hadoop.mapred.MapTask:
>>> numReduceTasks: 1
>>> 2008-07-30 14:51:17,120 INFO org.apache.hadoop.mapred.TaskRunner:  
>>> Task
>>> 'task_200807301447_0001_m_000000_0' done.
>>>
>>> Task Logs: 'task_200807301447_0001_m_000001_0'
>>>
>>>
>>>
>>> stdout logs
>>> map: key = test
>>> map: key = test2
>>>
>>>
>>> stderr logs
>>>
>>>
>>> syslog logs
>>> 2008-07-30 14:51:16,410 INFO  
>>> org.apache.hadoop.metrics.jvm.JvmMetrics:
>>> Initializing JVM Metrics with processName=MAP, sessionId=
>>> 2008-07-30 14:51:16,509 INFO org.apache.hadoop.mapred.MapTask:
>>> numReduceTasks: 1
>>> 2008-07-30 14:51:17,118 INFO org.apache.hadoop.mapred.TaskRunner:  
>>> Task
>>> 'task_200807301447_0001_m_000001_0' done.
>>>
>>> Tasks 3 and 4 are the same.
>>>
>>> Each map task is seeing the same rows.  Any help to prevent this is
>>> appreciated.
>>>
>>> Thanks,
>>> Dru
>>>
>>>
>>> On Jul 30, 2008, at 2:22 PM, Jean-Daniel Cryans wrote:
>>>
>>> Dru,
>>>>
>>>> It is not supposed to process many times the same rows. Can I see  
>>>> the log
>>>> you're talking about? Also, how many regions do you have in your  
>>>> table?
>>>> (info available in the web UI).
>>>>
>>>> thx
>>>>
>>>> J-D
>>>>
>>>> On Wed, Jul 30, 2008 at 5:04 PM, Dru Jensen <dr...@gmail.com>  
>>>> wrote:
>>>>
>>>> J-D,
>>>>>
>>>>> thanks for your quick response.   I have 4 mapping processes  
>>>>> running on
>>>>> 3
>>>>> systems.
>>>>>
>>>>> Are the same rows being processed 4 times by each mapping  
>>>>> processor?
>>>>> According to the logs they are.
>>>>>
>>>>> When I run a map/reduce against a file, only one row gets logged  
>>>>> per
>>>>> mapper.  Why would this be different for hbase tables?
>>>>>
>>>>> I would think only one mapping process would process that one  
>>>>> row and it
>>>>> would only show up once in only one log.
>>>>> preferable it would be the same system that has the region.
>>>>>
>>>>> I only want one row to be processed once.  Is there anyway to  
>>>>> change
>>>>> this
>>>>> behavior without running only 1 mapper?
>>>>>
>>>>> thanks,
>>>>> Dru
>>>>>
>>>>>
>>>>> On Jul 30, 2008, at 1:44 PM, Jean-Daniel Cryans wrote:
>>>>>
>>>>> Dru,
>>>>>
>>>>>>
>>>>>> The regions will split when achieving a certain threshold so if  
>>>>>> you
>>>>>> want
>>>>>> your computing to be distributed, you will have to have more  
>>>>>> data.
>>>>>>
>>>>>> Regards,
>>>>>>
>>>>>> J-D
>>>>>>
>>>>>> On Wed, Jul 30, 2008 at 4:36 PM, Dru Jensen <dr...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>> Hello,
>>>>>>
>>>>>>>
>>>>>>> I created a map/reduce process by extending the TableMap and
>>>>>>> TableReduce
>>>>>>> API but for some reason
>>>>>>> when I run multiple mappers, in the logs its showing that the  
>>>>>>> same
>>>>>>> rows
>>>>>>> are
>>>>>>> being processed by each Mapper.
>>>>>>>
>>>>>>> When I say logs, I mean in the hadoop task tracker (localhost: 
>>>>>>> 50030)
>>>>>>> and
>>>>>>> drilling down into the logs.
>>>>>>>
>>>>>>> Do I need to manually perform a TableSplit or is this supposed  
>>>>>>> to be
>>>>>>> done
>>>>>>> automatically?
>>>>>>>
>>>>>>> If its something I need to do manually, can someone point me  
>>>>>>> to some
>>>>>>> sample
>>>>>>> code?
>>>>>>>
>>>>>>> If its supposed to be automatic and each mapper was supposed  
>>>>>>> to get
>>>>>>> its
>>>>>>> own
>>>>>>> set of rows,
>>>>>>> should I write up a bug for this?  I using trunk 0.2.0 on  
>>>>>>> hadoop trunk
>>>>>>> 0.17.2.
>>>>>>>
>>>>>>> thanks,
>>>>>>> Dru
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>
>>


Re: newbie - map reduce not distributing

Posted by Jean-Daniel Cryans <jd...@gmail.com>.
Dru,

There is something truly weird with your setup. I would advise running your
code (the simple one that only logs the rows) with DEBUG on. See the
faq<http://wiki.apache.org/hadoop/Hbase/FAQ#5>on how to do it. Then
get back with syslog and stdout. This way we will have
more informations on how scanners are handling this.

Also FYI, I ran the same code as yours with 0.2.0 on my setup and had no
problems.

J-D

On Thu, Jul 31, 2008 at 7:06 PM, Dru Jensen <dr...@gmail.com> wrote:

> UPDATE:  I modified the RowCounter example and verified that it is sending
> the same row to multiple map tasks also. Is this a known bug or am I doing
> something truly as(s)inine?  Any help is appreciated.
>
>
> On Jul 30, 2008, at 3:02 PM, Dru Jensen wrote:
>
>  J-D,
>>
>> Again, thank you for your help on this.
>>
>> hitting the HBASE Master port 60010:
>> System 1 - 2 regions
>> System 2 - 1 region
>> System 3 - 3 regions
>>
>> In order to demonstrate the behavior I'm seeing, I wrote a test class.
>>
>> public class Test extends Configured implements Tool {
>>
>>    public static class Map extends TableMap {
>>
>>        @Override
>>        public void map(ImmutableBytesWritable key, RowResult row,
>> OutputCollector output, Reporter r) throws IOException {
>>
>>            String key_str = new String(key.get());
>>            System.out.println("map: key = " + key_str);
>>        }
>>
>>    }
>>
>>    public class Reduce extends TableReduce {
>>
>>        @Override
>>        public void reduce(WritableComparable key, Iterator values,
>> OutputCollector output, Reporter r) throws IOException {
>>
>>        }
>>
>>    }
>>
>>    public int run(String[] args) throws Exception {
>>        JobConf job = new JobConf(getConf(), Test.class);
>>        job.setJobName("Test");
>>
>>        job.setNumMapTasks(4);
>>        job.setNumReduceTasks(1);
>>
>>        Map.initJob("test", "content:", Map.class, HStoreKey.class,
>> HbaseMapWritable.class, job);
>>        Reduce.initJob("test", Reduce.class, job);
>>
>>        JobClient.runJob(job);
>>        return 0;
>>    }
>>
>>    public static void main(String[] args) throws Exception {
>>        int res = ToolRunner.run(new Configuration(), new Test(), args);
>>        System.exit(res);
>>    }
>> }
>>
>> In hbase shell:
>> create 'test','content'
>> put 'test','test','content:test','testing'
>> put 'test','test2','content:test','testing2'
>>
>>
>> The Hadoop log results:
>> Task Logs: 'task_200807301447_0001_m_000000_0'
>>
>>
>>
>> stdout logs
>> map: key = test
>> map: key = test2
>>
>>
>> stderr logs
>>
>>
>> syslog logs
>> 2008-07-30 14:51:16,410 INFO org.apache.hadoop.metrics.jvm.JvmMetrics:
>> Initializing JVM Metrics with processName=MAP, sessionId=
>> 2008-07-30 14:51:16,507 INFO org.apache.hadoop.mapred.MapTask:
>> numReduceTasks: 1
>> 2008-07-30 14:51:17,120 INFO org.apache.hadoop.mapred.TaskRunner: Task
>> 'task_200807301447_0001_m_000000_0' done.
>>
>> Task Logs: 'task_200807301447_0001_m_000001_0'
>>
>>
>>
>> stdout logs
>> map: key = test
>> map: key = test2
>>
>>
>> stderr logs
>>
>>
>> syslog logs
>> 2008-07-30 14:51:16,410 INFO org.apache.hadoop.metrics.jvm.JvmMetrics:
>> Initializing JVM Metrics with processName=MAP, sessionId=
>> 2008-07-30 14:51:16,509 INFO org.apache.hadoop.mapred.MapTask:
>> numReduceTasks: 1
>> 2008-07-30 14:51:17,118 INFO org.apache.hadoop.mapred.TaskRunner: Task
>> 'task_200807301447_0001_m_000001_0' done.
>>
>> Tasks 3 and 4 are the same.
>>
>> Each map task is seeing the same rows.  Any help to prevent this is
>> appreciated.
>>
>> Thanks,
>> Dru
>>
>>
>> On Jul 30, 2008, at 2:22 PM, Jean-Daniel Cryans wrote:
>>
>>  Dru,
>>>
>>> It is not supposed to process many times the same rows. Can I see the log
>>> you're talking about? Also, how many regions do you have in your table?
>>> (info available in the web UI).
>>>
>>> thx
>>>
>>> J-D
>>>
>>> On Wed, Jul 30, 2008 at 5:04 PM, Dru Jensen <dr...@gmail.com> wrote:
>>>
>>>  J-D,
>>>>
>>>> thanks for your quick response.   I have 4 mapping processes running on
>>>> 3
>>>> systems.
>>>>
>>>> Are the same rows being processed 4 times by each mapping processor?
>>>> According to the logs they are.
>>>>
>>>> When I run a map/reduce against a file, only one row gets logged per
>>>> mapper.  Why would this be different for hbase tables?
>>>>
>>>> I would think only one mapping process would process that one row and it
>>>> would only show up once in only one log.
>>>> preferable it would be the same system that has the region.
>>>>
>>>> I only want one row to be processed once.  Is there anyway to change
>>>> this
>>>> behavior without running only 1 mapper?
>>>>
>>>> thanks,
>>>> Dru
>>>>
>>>>
>>>> On Jul 30, 2008, at 1:44 PM, Jean-Daniel Cryans wrote:
>>>>
>>>> Dru,
>>>>
>>>>>
>>>>> The regions will split when achieving a certain threshold so if you
>>>>> want
>>>>> your computing to be distributed, you will have to have more data.
>>>>>
>>>>> Regards,
>>>>>
>>>>> J-D
>>>>>
>>>>> On Wed, Jul 30, 2008 at 4:36 PM, Dru Jensen <dr...@gmail.com>
>>>>> wrote:
>>>>>
>>>>> Hello,
>>>>>
>>>>>>
>>>>>> I created a map/reduce process by extending the TableMap and
>>>>>> TableReduce
>>>>>> API but for some reason
>>>>>> when I run multiple mappers, in the logs its showing that the same
>>>>>> rows
>>>>>> are
>>>>>> being processed by each Mapper.
>>>>>>
>>>>>> When I say logs, I mean in the hadoop task tracker (localhost:50030)
>>>>>> and
>>>>>> drilling down into the logs.
>>>>>>
>>>>>> Do I need to manually perform a TableSplit or is this supposed to be
>>>>>> done
>>>>>> automatically?
>>>>>>
>>>>>> If its something I need to do manually, can someone point me to some
>>>>>> sample
>>>>>> code?
>>>>>>
>>>>>> If its supposed to be automatic and each mapper was supposed to get
>>>>>> its
>>>>>> own
>>>>>> set of rows,
>>>>>> should I write up a bug for this?  I using trunk 0.2.0 on hadoop trunk
>>>>>> 0.17.2.
>>>>>>
>>>>>> thanks,
>>>>>> Dru
>>>>>>
>>>>>>
>>>>>>
>>>>
>>
>

Re: newbie - map reduce not distributing

Posted by Dru Jensen <dr...@gmail.com>.
UPDATE:  I modified the RowCounter example and verified that it is  
sending the same row to multiple map tasks also. Is this a known bug  
or am I doing something truly as(s)inine?  Any help is appreciated.

On Jul 30, 2008, at 3:02 PM, Dru Jensen wrote:

> J-D,
>
> Again, thank you for your help on this.
>
> hitting the HBASE Master port 60010:
> System 1 - 2 regions
> System 2 - 1 region
> System 3 - 3 regions
>
> In order to demonstrate the behavior I'm seeing, I wrote a test class.
>
> public class Test extends Configured implements Tool {
>
>     public static class Map extends TableMap {
>
>         @Override
>         public void map(ImmutableBytesWritable key, RowResult row,  
> OutputCollector output, Reporter r) throws IOException {
>
>             String key_str = new String(key.get());
>             System.out.println("map: key = " + key_str);
>         }
>
>     }
>
>     public class Reduce extends TableReduce {
>
>         @Override
>         public void reduce(WritableComparable key, Iterator values,  
> OutputCollector output, Reporter r) throws IOException {
>
>         }
>
>     }
>
>     public int run(String[] args) throws Exception {
>         JobConf job = new JobConf(getConf(), Test.class);
>         job.setJobName("Test");
>
>         job.setNumMapTasks(4);
>         job.setNumReduceTasks(1);
>
>         Map.initJob("test", "content:", Map.class, HStoreKey.class,  
> HbaseMapWritable.class, job);
>         Reduce.initJob("test", Reduce.class, job);
>
>         JobClient.runJob(job);
>         return 0;
>     }
>
>     public static void main(String[] args) throws Exception {
>         int res = ToolRunner.run(new Configuration(), new Test(),  
> args);
>         System.exit(res);
>     }
> }
>
> In hbase shell:
> create 'test','content'
> put 'test','test','content:test','testing'
> put 'test','test2','content:test','testing2'
>
>
> The Hadoop log results:
> Task Logs: 'task_200807301447_0001_m_000000_0'
>
>
>
> stdout logs
> map: key = test
> map: key = test2
>
>
> stderr logs
>
>
> syslog logs
> 2008-07-30 14:51:16,410 INFO  
> org.apache.hadoop.metrics.jvm.JvmMetrics: Initializing JVM Metrics  
> with processName=MAP, sessionId=
> 2008-07-30 14:51:16,507 INFO org.apache.hadoop.mapred.MapTask:  
> numReduceTasks: 1
> 2008-07-30 14:51:17,120 INFO org.apache.hadoop.mapred.TaskRunner:  
> Task 'task_200807301447_0001_m_000000_0' done.
>
> Task Logs: 'task_200807301447_0001_m_000001_0'
>
>
>
> stdout logs
> map: key = test
> map: key = test2
>
>
> stderr logs
>
>
> syslog logs
> 2008-07-30 14:51:16,410 INFO  
> org.apache.hadoop.metrics.jvm.JvmMetrics: Initializing JVM Metrics  
> with processName=MAP, sessionId=
> 2008-07-30 14:51:16,509 INFO org.apache.hadoop.mapred.MapTask:  
> numReduceTasks: 1
> 2008-07-30 14:51:17,118 INFO org.apache.hadoop.mapred.TaskRunner:  
> Task 'task_200807301447_0001_m_000001_0' done.
>
> Tasks 3 and 4 are the same.
>
> Each map task is seeing the same rows.  Any help to prevent this is  
> appreciated.
>
> Thanks,
> Dru
>
>
> On Jul 30, 2008, at 2:22 PM, Jean-Daniel Cryans wrote:
>
>> Dru,
>>
>> It is not supposed to process many times the same rows. Can I see  
>> the log
>> you're talking about? Also, how many regions do you have in your  
>> table?
>> (info available in the web UI).
>>
>> thx
>>
>> J-D
>>
>> On Wed, Jul 30, 2008 at 5:04 PM, Dru Jensen <dr...@gmail.com>  
>> wrote:
>>
>>> J-D,
>>>
>>> thanks for your quick response.   I have 4 mapping processes  
>>> running on 3
>>> systems.
>>>
>>> Are the same rows being processed 4 times by each mapping processor?
>>> According to the logs they are.
>>>
>>> When I run a map/reduce against a file, only one row gets logged per
>>> mapper.  Why would this be different for hbase tables?
>>>
>>> I would think only one mapping process would process that one row  
>>> and it
>>> would only show up once in only one log.
>>> preferable it would be the same system that has the region.
>>>
>>> I only want one row to be processed once.  Is there anyway to  
>>> change this
>>> behavior without running only 1 mapper?
>>>
>>> thanks,
>>> Dru
>>>
>>>
>>> On Jul 30, 2008, at 1:44 PM, Jean-Daniel Cryans wrote:
>>>
>>> Dru,
>>>>
>>>> The regions will split when achieving a certain threshold so if  
>>>> you want
>>>> your computing to be distributed, you will have to have more data.
>>>>
>>>> Regards,
>>>>
>>>> J-D
>>>>
>>>> On Wed, Jul 30, 2008 at 4:36 PM, Dru Jensen <dr...@gmail.com>  
>>>> wrote:
>>>>
>>>> Hello,
>>>>>
>>>>> I created a map/reduce process by extending the TableMap and  
>>>>> TableReduce
>>>>> API but for some reason
>>>>> when I run multiple mappers, in the logs its showing that the  
>>>>> same rows
>>>>> are
>>>>> being processed by each Mapper.
>>>>>
>>>>> When I say logs, I mean in the hadoop task tracker (localhost: 
>>>>> 50030) and
>>>>> drilling down into the logs.
>>>>>
>>>>> Do I need to manually perform a TableSplit or is this supposed  
>>>>> to be done
>>>>> automatically?
>>>>>
>>>>> If its something I need to do manually, can someone point me to  
>>>>> some
>>>>> sample
>>>>> code?
>>>>>
>>>>> If its supposed to be automatic and each mapper was supposed to  
>>>>> get its
>>>>> own
>>>>> set of rows,
>>>>> should I write up a bug for this?  I using trunk 0.2.0 on hadoop  
>>>>> trunk
>>>>> 0.17.2.
>>>>>
>>>>> thanks,
>>>>> Dru
>>>>>
>>>>>
>>>
>


Re: newbie - map reduce not distributing

Posted by Dru Jensen <dr...@gmail.com>.
J-D,

Again, thank you for your help on this.

hitting the HBASE Master port 60010:
System 1 - 2 regions
System 2 - 1 region
System 3 - 3 regions

In order to demonstrate the behavior I'm seeing, I wrote a test class.

public class Test extends Configured implements Tool {

     public static class Map extends TableMap {

         @Override
         public void map(ImmutableBytesWritable key, RowResult row,  
OutputCollector output, Reporter r) throws IOException {

             String key_str = new String(key.get());
             System.out.println("map: key = " + key_str);
         }

     }

     public class Reduce extends TableReduce {

         @Override
         public void reduce(WritableComparable key, Iterator values,  
OutputCollector output, Reporter r) throws IOException {

         }

     }

     public int run(String[] args) throws Exception {
         JobConf job = new JobConf(getConf(), Test.class);
         job.setJobName("Test");

         job.setNumMapTasks(4);
         job.setNumReduceTasks(1);

         Map.initJob("test", "content:", Map.class, HStoreKey.class,  
HbaseMapWritable.class, job);
         Reduce.initJob("test", Reduce.class, job);

         JobClient.runJob(job);
         return 0;
     }

     public static void main(String[] args) throws Exception {
         int res = ToolRunner.run(new Configuration(), new Test(),  
args);
         System.exit(res);
     }
}

In hbase shell:
create 'test','content'
put 'test','test','content:test','testing'
put 'test','test2','content:test','testing2'


The Hadoop log results:
Task Logs: 'task_200807301447_0001_m_000000_0'



stdout logs
map: key = test
map: key = test2


stderr logs


syslog logs
2008-07-30 14:51:16,410 INFO org.apache.hadoop.metrics.jvm.JvmMetrics:  
Initializing JVM Metrics with processName=MAP, sessionId=
2008-07-30 14:51:16,507 INFO org.apache.hadoop.mapred.MapTask:  
numReduceTasks: 1
2008-07-30 14:51:17,120 INFO org.apache.hadoop.mapred.TaskRunner: Task  
'task_200807301447_0001_m_000000_0' done.

Task Logs: 'task_200807301447_0001_m_000001_0'



stdout logs
map: key = test
map: key = test2


stderr logs


syslog logs
2008-07-30 14:51:16,410 INFO org.apache.hadoop.metrics.jvm.JvmMetrics:  
Initializing JVM Metrics with processName=MAP, sessionId=
2008-07-30 14:51:16,509 INFO org.apache.hadoop.mapred.MapTask:  
numReduceTasks: 1
2008-07-30 14:51:17,118 INFO org.apache.hadoop.mapred.TaskRunner: Task  
'task_200807301447_0001_m_000001_0' done.

Tasks 3 and 4 are the same.

Each map task is seeing the same rows.  Any help to prevent this is  
appreciated.

Thanks,
Dru


On Jul 30, 2008, at 2:22 PM, Jean-Daniel Cryans wrote:

> Dru,
>
> It is not supposed to process many times the same rows. Can I see  
> the log
> you're talking about? Also, how many regions do you have in your  
> table?
> (info available in the web UI).
>
> thx
>
> J-D
>
> On Wed, Jul 30, 2008 at 5:04 PM, Dru Jensen <dr...@gmail.com>  
> wrote:
>
>> J-D,
>>
>> thanks for your quick response.   I have 4 mapping processes  
>> running on 3
>> systems.
>>
>> Are the same rows being processed 4 times by each mapping processor?
>> According to the logs they are.
>>
>> When I run a map/reduce against a file, only one row gets logged per
>> mapper.  Why would this be different for hbase tables?
>>
>> I would think only one mapping process would process that one row  
>> and it
>> would only show up once in only one log.
>> preferable it would be the same system that has the region.
>>
>> I only want one row to be processed once.  Is there anyway to  
>> change this
>> behavior without running only 1 mapper?
>>
>> thanks,
>> Dru
>>
>>
>> On Jul 30, 2008, at 1:44 PM, Jean-Daniel Cryans wrote:
>>
>> Dru,
>>>
>>> The regions will split when achieving a certain threshold so if  
>>> you want
>>> your computing to be distributed, you will have to have more data.
>>>
>>> Regards,
>>>
>>> J-D
>>>
>>> On Wed, Jul 30, 2008 at 4:36 PM, Dru Jensen <dr...@gmail.com>  
>>> wrote:
>>>
>>> Hello,
>>>>
>>>> I created a map/reduce process by extending the TableMap and  
>>>> TableReduce
>>>> API but for some reason
>>>> when I run multiple mappers, in the logs its showing that the  
>>>> same rows
>>>> are
>>>> being processed by each Mapper.
>>>>
>>>> When I say logs, I mean in the hadoop task tracker (localhost: 
>>>> 50030) and
>>>> drilling down into the logs.
>>>>
>>>> Do I need to manually perform a TableSplit or is this supposed to  
>>>> be done
>>>> automatically?
>>>>
>>>> If its something I need to do manually, can someone point me to  
>>>> some
>>>> sample
>>>> code?
>>>>
>>>> If its supposed to be automatic and each mapper was supposed to  
>>>> get its
>>>> own
>>>> set of rows,
>>>> should I write up a bug for this?  I using trunk 0.2.0 on hadoop  
>>>> trunk
>>>> 0.17.2.
>>>>
>>>> thanks,
>>>> Dru
>>>>
>>>>
>>


Re: newbie - map reduce not distributing

Posted by Jean-Daniel Cryans <jd...@gmail.com>.
Dru,

It is not supposed to process many times the same rows. Can I see the log
you're talking about? Also, how many regions do you have in your table?
(info available in the web UI).

thx

J-D

On Wed, Jul 30, 2008 at 5:04 PM, Dru Jensen <dr...@gmail.com> wrote:

> J-D,
>
> thanks for your quick response.   I have 4 mapping processes running on 3
> systems.
>
> Are the same rows being processed 4 times by each mapping processor?
>  According to the logs they are.
>
> When I run a map/reduce against a file, only one row gets logged per
> mapper.  Why would this be different for hbase tables?
>
> I would think only one mapping process would process that one row and it
> would only show up once in only one log.
> preferable it would be the same system that has the region.
>
> I only want one row to be processed once.  Is there anyway to change this
> behavior without running only 1 mapper?
>
> thanks,
> Dru
>
>
> On Jul 30, 2008, at 1:44 PM, Jean-Daniel Cryans wrote:
>
>  Dru,
>>
>> The regions will split when achieving a certain threshold so if you want
>> your computing to be distributed, you will have to have more data.
>>
>> Regards,
>>
>> J-D
>>
>> On Wed, Jul 30, 2008 at 4:36 PM, Dru Jensen <dr...@gmail.com> wrote:
>>
>>  Hello,
>>>
>>> I created a map/reduce process by extending the TableMap and TableReduce
>>> API but for some reason
>>> when I run multiple mappers, in the logs its showing that the same rows
>>> are
>>> being processed by each Mapper.
>>>
>>> When I say logs, I mean in the hadoop task tracker (localhost:50030) and
>>> drilling down into the logs.
>>>
>>> Do I need to manually perform a TableSplit or is this supposed to be done
>>> automatically?
>>>
>>> If its something I need to do manually, can someone point me to some
>>> sample
>>> code?
>>>
>>> If its supposed to be automatic and each mapper was supposed to get its
>>> own
>>> set of rows,
>>> should I write up a bug for this?  I using trunk 0.2.0 on hadoop trunk
>>> 0.17.2.
>>>
>>> thanks,
>>> Dru
>>>
>>>
>

Re: newbie - map reduce not distributing

Posted by Dru Jensen <dr...@gmail.com>.
J-D,

thanks for your quick response.   I have 4 mapping processes running  
on 3 systems.

Are the same rows being processed 4 times by each mapping processor?   
According to the logs they are.

When I run a map/reduce against a file, only one row gets logged per  
mapper.  Why would this be different for hbase tables?

I would think only one mapping process would process that one row and  
it would only show up once in only one log.
preferable it would be the same system that has the region.

I only want one row to be processed once.  Is there anyway to change  
this behavior without running only 1 mapper?

thanks,
Dru

On Jul 30, 2008, at 1:44 PM, Jean-Daniel Cryans wrote:

> Dru,
>
> The regions will split when achieving a certain threshold so if you  
> want
> your computing to be distributed, you will have to have more data.
>
> Regards,
>
> J-D
>
> On Wed, Jul 30, 2008 at 4:36 PM, Dru Jensen <dr...@gmail.com>  
> wrote:
>
>> Hello,
>>
>> I created a map/reduce process by extending the TableMap and  
>> TableReduce
>> API but for some reason
>> when I run multiple mappers, in the logs its showing that the same  
>> rows are
>> being processed by each Mapper.
>>
>> When I say logs, I mean in the hadoop task tracker (localhost: 
>> 50030) and
>> drilling down into the logs.
>>
>> Do I need to manually perform a TableSplit or is this supposed to  
>> be done
>> automatically?
>>
>> If its something I need to do manually, can someone point me to  
>> some sample
>> code?
>>
>> If its supposed to be automatic and each mapper was supposed to get  
>> its own
>> set of rows,
>> should I write up a bug for this?  I using trunk 0.2.0 on hadoop  
>> trunk
>> 0.17.2.
>>
>> thanks,
>> Dru
>>


Re: newbie - map reduce not distributing

Posted by Jean-Daniel Cryans <jd...@gmail.com>.
Dru,

The regions will split when achieving a certain threshold so if you want
your computing to be distributed, you will have to have more data.

Regards,

J-D

On Wed, Jul 30, 2008 at 4:36 PM, Dru Jensen <dr...@gmail.com> wrote:

> Hello,
>
> I created a map/reduce process by extending the TableMap and TableReduce
> API but for some reason
> when I run multiple mappers, in the logs its showing that the same rows are
> being processed by each Mapper.
>
> When I say logs, I mean in the hadoop task tracker (localhost:50030) and
> drilling down into the logs.
>
> Do I need to manually perform a TableSplit or is this supposed to be done
> automatically?
>
> If its something I need to do manually, can someone point me to some sample
> code?
>
> If its supposed to be automatic and each mapper was supposed to get its own
> set of rows,
> should I write up a bug for this?  I using trunk 0.2.0 on hadoop trunk
> 0.17.2.
>
> thanks,
> Dru
>