Re: Identification of mapper slots

2013-10-14 Thread Rahul Jain
I assume you know the tradeoff here: If you do depend upon mapper slot # in
your implementation to speed it up, you are losing on code portability in
long term

That said, one way to achieve this is to use the JobConf API:

int partition = jobConf.getInt(JobContext.TASK_PARTITION, -1);

The framework assigns unique partition # to each mapper; this allows  them
to write to a distinct output file. Note that this is a global partition #,
not local to each node.

Also, in case you have mappers and reducers using the same cache, then add


jobConf.getBoolean(JobContext.TASK_ISMAP)...  check to indicate whether you
are executing in mapper or reducer context.


-Rahul







On Mon, Oct 14, 2013 at 2:49 PM, Hider, Sandy sandy.hi...@jhuapl.eduwrote:

 ** **

 In Hadoop under the mapred-site.conf  I can set the maximum number of
 mappers. For the sake of this email I will call the number of concurrent
 mappers: mapper slots.  

 ** **

 Is it possible to figure out from within the mapper which mapper slot it
 is running in? 

 ** **

 On this project this is important because each mapper has to fork off a
 Matlab runtime compiled executable.  The executable is passed in at runtime
 a cache to work in.  Setting up the cache when given an new directory takes
 a long time but can be used again quickly on future calls if provided the
 same location of the cache.   As it turns out when multiple mappers try to
 use the same cache they crash the executable.   So ideally if I could
 identify which mapper slot a mapper is running in, I can setup caches for
 each slot and avoid the cache creation time and still guarantee that no two
 mappers write to the same cache.  

 ** **

 Thanks for taking the time to read this,

 ** **

 Sandy

 ** **

 ** **



Re: Issue: Max block location exceeded for split error when running hive

2013-09-19 Thread Rahul Jain
I am assuming you have looked at this already:

https://issues.apache.org/jira/browse/MAPREDUCE-5186

You do have a workaround here to increase *mapreduce.job.max.split.locations
*value in hive configuration, or do we need more than that here ?

-Rahul


On Thu, Sep 19, 2013 at 11:00 AM, Murtaza Doctor murtazadoc...@gmail.comwrote:

 It used to throw a warning in 1.03 and now has become an IOException. I
 was more trying to figure out why it is exceeding the limit even though the
 replication factor is 3. Also Hive may use CombineInputSplit or some
 version of it, are we saying it will always exceed the limit of 10?


 On Thu, Sep 19, 2013 at 10:05 AM, Edward Capriolo 
 edlinuxg...@gmail.comwrote:

 We have this job submit property buried in hive that defaults to 10. We
 should make that configurable.


 On Wed, Sep 18, 2013 at 9:34 PM, Harsh J ha...@cloudera.com wrote:

 Do your input files carry a replication factor of 10+? That could be
 one cause behind this.

 On Thu, Sep 19, 2013 at 6:20 AM, Murtaza Doctor murtazadoc...@gmail.com
 wrote:
  Folks,
 
  Any one run into this issue before:
  java.io.IOException: Max block location exceeded for split: Paths:
  /foo/bar
  
  InputFormatClass: org.apache.hadoop.mapred.TextInputFormat
  splitsize: 15 maxsize: 10
  at
 
 org.apache.hadoop.mapreduce.split.JobSplitWriter.writeOldSplits(JobSplitWriter.java:162)
  at
 
 org.apache.hadoop.mapreduce.split.JobSplitWriter.createSplitFiles(JobSplitWriter.java:87)
  at
 
 org.apache.hadoop.mapreduce.JobSubmitter.writeOldSplits(JobSubmitter.java:501)
  at
 
 org.apache.hadoop.mapreduce.JobSubmitter.writeSplits(JobSubmitter.java:471)
  at
 
 org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:366)
  at org.apache.hadoop.mapreduce.Job$11.run(Job.java:1269)
  at org.apache.hadoop.mapreduce.Job$11.run(Job.java:1266)
  at java.security.AccessController.doPrivileged(Native Method)
  at javax.security.auth.Subject.doAs(Subject.java:415)
  at
 
 org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
  at org.apache.hadoop.mapreduce.Job.submit(Job.java:1266)
  at org.apache.hadoop.mapred.JobClient$1.run(JobClient.java:606)
  at org.apache.hadoop.mapred.JobClient$1.run(JobClient.java:601)
  at java.security.AccessController.doPrivileged(Native Method)
  at javax.security.auth.Subject.doAs(Subject.java:415)
  at
 
 org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
  at
 org.apache.hadoop.mapred.JobClient.submitJobInternal(JobClient.java:601)
  at org.apache.hadoop.mapred.JobClient.submitJob(JobClient.java:586)
  at
 org.apache.hadoop.hive.ql.exec.ExecDriver.execute(ExecDriver.java:447)
 
  When we set the property to something higher as suggested like:
  mapreduce.job.max.split.locations = more than on what it failed
  then the job runs successfully.
 
  I am trying to dig up additional documentation on this since the
 default
  seems to be 10, not sure how that limit was set.
  Additionally what is the recommended value and what factors does it
 depend
  on?
 
  We are running YARN, the actual query is Hive on CDH 4.3, with Hive
 version
  0.10
 
  Any pointers in this direction will be helpful.
 
  Regards,
  md



 --
 Harsh J






Re: Issue: Max block location exceeded for split error when running hive

2013-09-19 Thread Rahul Jain
Matt,

It would be better for you to do an global config update: set
*mapreduce.job.max.split.locations
*to at least the number of datanodes in your cluster, either in
hive-site.xml or mapred-site.xml. Either case, this is a sensible
configuration update if you are going to use CombineFileInputFormat to read
input data in hive.

-Rahul


On Thu, Sep 19, 2013 at 3:31 PM, Matt Davies m...@mattdavies.net wrote:

 What are the ramifications of setting a hard coded value in our scripts
 and then changing parameters which influence the input data size. I.e. I
 want to run across 1 day worth of data, then a different day I want to run
 against 30 days?




 On Thu, Sep 19, 2013 at 3:11 PM, Rahul Jain rja...@gmail.com wrote:

 I am assuming you have looked at this already:

 https://issues.apache.org/jira/browse/MAPREDUCE-5186

 You do have a workaround here to increase *mapreduce.job.max.split.locations
 *value in hive configuration, or do we need more than that here ?

 -Rahul


 On Thu, Sep 19, 2013 at 11:00 AM, Murtaza Doctor murtazadoc...@gmail.com
  wrote:

 It used to throw a warning in 1.03 and now has become an IOException. I
 was more trying to figure out why it is exceeding the limit even though the
 replication factor is 3. Also Hive may use CombineInputSplit or some
 version of it, are we saying it will always exceed the limit of 10?


 On Thu, Sep 19, 2013 at 10:05 AM, Edward Capriolo edlinuxg...@gmail.com
  wrote:

 We have this job submit property buried in hive that defaults to 10. We
 should make that configurable.


 On Wed, Sep 18, 2013 at 9:34 PM, Harsh J ha...@cloudera.com wrote:

 Do your input files carry a replication factor of 10+? That could be
 one cause behind this.

 On Thu, Sep 19, 2013 at 6:20 AM, Murtaza Doctor 
 murtazadoc...@gmail.com wrote:
  Folks,
 
  Any one run into this issue before:
  java.io.IOException: Max block location exceeded for split: Paths:
  /foo/bar
  
  InputFormatClass: org.apache.hadoop.mapred.TextInputFormat
  splitsize: 15 maxsize: 10
  at
 
 org.apache.hadoop.mapreduce.split.JobSplitWriter.writeOldSplits(JobSplitWriter.java:162)
  at
 
 org.apache.hadoop.mapreduce.split.JobSplitWriter.createSplitFiles(JobSplitWriter.java:87)
  at
 
 org.apache.hadoop.mapreduce.JobSubmitter.writeOldSplits(JobSubmitter.java:501)
  at
 
 org.apache.hadoop.mapreduce.JobSubmitter.writeSplits(JobSubmitter.java:471)
  at
 
 org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:366)
  at org.apache.hadoop.mapreduce.Job$11.run(Job.java:1269)
  at org.apache.hadoop.mapreduce.Job$11.run(Job.java:1266)
  at java.security.AccessController.doPrivileged(Native Method)
  at javax.security.auth.Subject.doAs(Subject.java:415)
  at
 
 org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
  at org.apache.hadoop.mapreduce.Job.submit(Job.java:1266)
  at org.apache.hadoop.mapred.JobClient$1.run(JobClient.java:606)
  at org.apache.hadoop.mapred.JobClient$1.run(JobClient.java:601)
  at java.security.AccessController.doPrivileged(Native Method)
  at javax.security.auth.Subject.doAs(Subject.java:415)
  at
 
 org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1408)
  at
 org.apache.hadoop.mapred.JobClient.submitJobInternal(JobClient.java:601)
  at org.apache.hadoop.mapred.JobClient.submitJob(JobClient.java:586)
  at
 org.apache.hadoop.hive.ql.exec.ExecDriver.execute(ExecDriver.java:447)
 
  When we set the property to something higher as suggested like:
  mapreduce.job.max.split.locations = more than on what it failed
  then the job runs successfully.
 
  I am trying to dig up additional documentation on this since the
 default
  seems to be 10, not sure how that limit was set.
  Additionally what is the recommended value and what factors does it
 depend
  on?
 
  We are running YARN, the actual query is Hive on CDH 4.3, with Hive
 version
  0.10
 
  Any pointers in this direction will be helpful.
 
  Regards,
  md



 --
 Harsh J








Re: What happens when you have fewer input files than mapper slots?

2013-03-19 Thread Rahul Jain
Which version of hadoop are you using ? MRV1 or MRV2 (yarn) ??

For MRv2 (yarn): you can pretty much achieve this using:

yarn.nodemanager.resource.memory-mb (system wide setting)
and
mapreduce.map.memory.mb  (job level setting)

e.g. if yarn.nodemanager.resource.memory-mb=100
and mapreduce.map.memory.mb= 40
a maximum of two mapper can run on a node at any time.

For MRv1, The equivalent way will be to control mapper slots on each
machine:
mapred.tasktracker.map.tasks.maximum,  of course this does not give you
'per job' control. on mappers.

In addition in both cases, you can use a scheduler with 'pools / queues'
capability in addition to restrict the overall use of grid resource. Do
read fair scheduler and capacity scheduler documentation...


-Rahul




On Tue, Mar 19, 2013 at 1:55 PM, jeremy p athomewithagroove...@gmail.comwrote:

 Short version : let's say you have 20 nodes, and each node has 10 mapper
 slots.  You start a job with 20 very small input files.  How is the work
 distributed to the cluster?  Will it be even, with each node spawning one
 mapper task?  Is there any way of predicting or controlling how the work
 will be distributed?

 Long version : My cluster is currently used for two different jobs.  The
 cluster is currently optimized for Job A, so each node has a maximum of 18
 mapper slots.  However, I also need to run Job B.  Job B is VERY
 cpu-intensive, so we really only want one mapper to run on a node at any
 given time.  I've done a bunch of research, and it doesn't seem like Hadoop
 gives you any way to set the maximum number of mappers per node on a
 per-job basis.  I'm at my wit's end here, and considering some rather
 egregious workarounds.  If you can think of anything that can help me, I'd
 very much appreciate it.

 Thanks!

 --Jeremy



Re: Time taken for launching Application Master

2013-01-20 Thread Rahul Jain
Check your node manager logs to understand the bottleneck first. When we
had a similar issue on recent version of hadoop, which includes fix for
MAPREDUCE-4068: we rearranged our job jar file to reduce time spent on
'expanding' the job jar file by the node manager(s).

-Rahul

On Sun, Jan 20, 2013 at 10:34 PM, Krishna Kishore Bonagiri 
write2kish...@gmail.com wrote:

 Hi,
I am seeing that from the time ApplicationMaster is sumitted by my
 Client to the ASM part of RM, it is taking around 7 seconds for AM to get
 started. Is there a way to reduce that time, I mean to speed it up?

 Thanks,
 Kishore



Re: YARN Pi example job stuck at 0%(No MR tasks are started by ResourceManager)

2012-07-30 Thread Rahul Jain
The inability to look at map-reduce logs for failed logs is due to number
of open issues in yarn; see my recent comment here:
https://issues.apache.org/jira/browse/MAPREDUCE-4428?focusedCommentId=13412995page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13412995

You can workaround this by enabling log aggregation and manually copying
job logs from HDFS log location. Of course that is a painful way till the
yarn log collection and history bugs are resolved in an upcoming release.

-Rahul


 12/07/27 09:38:27 INFO mapred.ResourceMgrDelegate: Submitted application
 application_1343365114818_0002 to ResourceManager at ihub-an-l1/
 172.31.192.151:8040
 12/07/27 09:38:27 INFO mapreduce.Job: The url to track the job:
 http://ihub-an-l1:/proxy/application_1343365114818_0002/
 12/07/27 09:38:27 INFO mapreduce.Job: Running job: job_1343365114818_0002

 No Map-Reduce task are started by the cluster. I dont see any errors
 anywhere in the application. Please help me in resolving this problem.

 Thanks,
 Anil Gupta



Re: YARN Pi example job stuck at 0%(No MR tasks are started by ResourceManager)

2012-07-30 Thread Rahul Jain
Yes, do ensure that JAVA_HOME is properly set on all nodes through hadoop
env or the login shell.

Regarding agregated logs,  they will be found in hdfs under the directory
set through 'yarn.nodemanager.remote-app-log-dir' , a subdirectory for each
job.

-Rahul



On Mon, Jul 30, 2012 at 8:12 PM, abhiTowson cal
abhishek.dod...@gmail.comwrote:

 Hi anil,

 Adding property resolved issue for me, and i also made this change

 vim hadoop-env.sh

 export JAVA_HOME=/usr/lib/java-1.6.0/jdk1.6.0_33
 if [ $JAVA_HOME !=  ]; then
   #echo run java in $JAVA_HOME
   JAVA_HOME=$JAVA_HOME
 fi

 if [ $JAVA_HOME =  ]; then
   echo Error: JAVA_HOME is not set.
   exit 1
 fi

 JAVA=$JAVA_HOME/bin/java
 JAVA_HEAP_MAX=-Xmx1000m

 Regards
 Abhishek


 On Mon, Jul 30, 2012 at 10:47 PM, anil gupta anilgupt...@gmail.com
 wrote:
  Hi Abhishek,
 
  Did you mean that adding yarn.resourcemanager.resource-tracker.address
  along with yarn.log-aggregation-enable in my configuration will resolve
 the
  problem in which map-reduce job fails at 0% with the following error: In
  the web page of
 
 http://data-node:8042/node/containerlogs/container_1343687008058_0003_01_01/rootthe
  page says:
  Failed redirect for container_1343687008058_0003_01_01  Failed while
  trying to construct the redirect url to the log server. Log Server url
 may
  not be configured. Unknown container. Container either has not started or
  has already completed or doesn't belong to this node at all.
  Please let me know.
 
  Thanks,
  Anil Gupta
 
  On Mon, Jul 30, 2012 at 7:30 PM, abhiTowson cal
  abhishek.dod...@gmail.comwrote:
 
  hi anil,
 
  Adding these help me resolve the issue for me
  yarn.resourcemanager.resource-tracker.address
 
  Regards
  Abhishek
 
  On Mon, Jul 30, 2012 at 7:56 PM, anil gupta anilgupt...@gmail.com
 wrote:
   Hi Rahul,
  
   Thanks for your response. I can certainly enable the
   yarn.log-aggregation-enable to true. But after enabling this what
 manual
   steps i will have to take to run jobs. Could you please elaborate.
  
   Thanks,
   Anil
  
   On Mon, Jul 30, 2012 at 4:26 PM, Rahul Jain rja...@gmail.com wrote:
  
   The inability to look at map-reduce logs for failed logs is due to
  number
   of open issues in yarn; see my recent comment here:
  
  
 
 https://issues.apache.org/jira/browse/MAPREDUCE-4428?focusedCommentId=13412995page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13412995
  
   You can workaround this by enabling log aggregation and manually
 copying
   job logs from HDFS log location. Of course that is a painful way till
  the
   yarn log collection and history bugs are resolved in an upcoming
  release.
  
   -Rahul
  
  
12/07/27 09:38:27 INFO mapred.ResourceMgrDelegate: Submitted
  application
application_1343365114818_0002 to ResourceManager at ihub-an-l1/
172.31.192.151:8040
12/07/27 09:38:27 INFO mapreduce.Job: The url to track the job:
http://ihub-an-l1:/proxy/application_1343365114818_0002/
12/07/27 09:38:27 INFO mapreduce.Job: Running job:
  job_1343365114818_0002
   
No Map-Reduce task are started by the cluster. I dont see any
 errors
anywhere in the application. Please help me in resolving this
 problem.
   
Thanks,
Anil Gupta
   
  
  
  
  
   --
   Thanks  Regards,
   Anil Gupta
 
 
 
 
  --
  Thanks  Regards,
  Anil Gupta



Re: Hive Thrift help

2012-04-16 Thread Rahul Jain
I am assuming you read thru:

https://cwiki.apache.org/Hive/hiveserver.html

The server comes up on port 10,000 by default, did you verify that it is
actually listening on the port ?  You can also connect to hive server using
web browser to confirm its status.

-Rahul

On Mon, Apr 16, 2012 at 1:53 PM, Michael Wang michael.w...@meredith.comwrote:

 we need to connect to HIVE from Microstrategy reports, and it requires the
 Hive Thrift server. But I
 tried to start it, and it just hangs as below.
 # hive --service hiveserver
 Starting Hive Thrift Server
 Any ideas?
 Thanks,
 Michael

 This electronic message, including any attachments, may contain
 proprietary, confidential or privileged information for the sole use of the
 intended recipient(s). You are hereby notified that any unauthorized
 disclosure, copying, distribution, or use of this message is prohibited. If
 you have received this message in error, please immediately notify the
 sender by reply e-mail and delete it.



Re: Map Task Capacity Not Changing

2011-12-16 Thread Rahul Jain
You might be suffering from HADOOP-7822; I'd suggest you verify your pid
files and fix the problem by hand if it is the same issue.

-Rahul

On Fri, Dec 16, 2011 at 2:40 PM, Joey Krabacher jkrabac...@gmail.comwrote:

 Turns out my tasktrackers(on the datanodes) are not starting properly

 so I guess they are taking some alternate route??

 because they are up and running...even though when I run
 stop-mapred.sh it says data01: no tasktracker to stop

 --Joey

 On Thu, Dec 15, 2011 at 5:37 PM, James Warren
 james.war...@stanfordalumni.org wrote:
  (moving to mapreduce-user@, bcc'ing common-user@)
 
  Hi Joey -
 
  You'll want to change the value on all of your servers running
 tasktrackers
  and then restart each tasktracker to reread the configuration.
 
  cheers,
  -James
 
  On Thu, Dec 15, 2011 at 3:30 PM, Joey Krabacher jkrabac...@gmail.com
 wrote:
 
  I have looked up how to up this value on the web and have tried all
  suggestions to no avail.
 
  Any help would be great.
 
  Here is some background:
 
  Version: 0.20.2, r911707
  Compiled: Fri Feb 19 08:07:34 UTC 2010 by chrisdo
 
  Nodes: 5
  Current Map Task Capacity : 10  --- this is what I want to increase.
 
  What I have tried :
 
  Adding
property
 namemapred.tasktracker.map.tasks.maximum/name
 value8/value
 finaltrue/final
   /property
  to mapred-site.xml on NameNode.  I also added this to one of the
  datanodes for the hell of it and that didn't work either.
 
  Thanks.
 



Re: How do I connect Java Visual VM to a remote task?

2011-10-17 Thread Rahul Jain
The easy way to debug such problems in our experience is to use 'jmap' to
take a few snapshots of one of the tasktrackers (child tasks) and analyze
them under a profiler tool such as jprofiler, yourkit etc. This should give
you pretty good indication of objects that are using up most heap memory.

You can add JVM options to suspend child tasks upon startup and attach using
debugger etc. but that is more painful in a distributed environment.


-Rahul

On Mon, Oct 17, 2011 at 11:34 AM, W.P. McNeill bill...@gmail.com wrote:

 I'm investigating a bug where my mapper and reducer tasks run out of
 memory.
 It only reproduces when I run on large data sets, so the best way to dig in
 is to launch my job with sufficiently large inputs on the cluster and
 monitor the memory characteristics of the failing JVMs remotely. Java
 Visual
 VM looks like the tool I want to use. Specifically I want to use it to do
 heap dumps on my tasks. I can't figure out how to set up the listening end
 on the cluster nodes, however.

 Here is what I have tried:

 1. *Turn on JMX remote for the tasks*...I added the following options to
 mapred.child.java.opts:
 com.sun.management.jmxremote,

 com.sun.management.jmxremote.port=8004,com.sun.management.jmxremote.authenticate=false,com.sun.management.jmxremote.ssl
 = false.

 This does not work because there is contention for the JMX remote port when
 multiple tasks run on the same node. All but the first task fail at JVM
 initialization time, causing the job to fail before I can see the repro.

 2. *Use jstatd*...I tried running jstatd in the background on my cluster
 nodes. It launches and runs, but when I try to connect using Visual VM,
 nothing happens.

 I am going to try adding -XX:-HeapDumpOnOutOfMemoryError, which will at
 least give me post-mortem information. Does anyone know where the heap dump
 file will  be written?

 Has anyone debugged a similar setup? What tools did you use?



Re: ChainMapper and ChainReducer: Are the key/value pairs distributed to the nodes of the cluster before each Map phase?

2011-04-29 Thread Rahul Jain
Your latter statement is correct:

 if the output of the Map1 phase (or Reduce phase) is immediately inserted
to Map2 phase (or Map3 Phase) within the same node, without any
distribution.

ChainMappers / ChainReducers are just convenience classes to allow reuse of
mapper code  whether executing as part of a sequence or executing
standalone. These do not force the system to do any additional distribution,
grouping, sorting etc.

-Rahul

2011/4/29 Panayotis Antonopoulos antonopoulos...@hotmail.com


 Hello,
 Let' say we have a MR job that uses ChainMapper and ChainReducer like in
 the following diagram:
 Input-Map1-Map2-Reduce-Map3-Output

 The input is split and distributed to the nodes of the cluster before being
 processed by Map1 phase.
 Also, before the Reduce phase the key/value pairs are also distributed to
 the Reducers according to the Partitions made by the Partitioner.

 I expected that the same thing (distribution of the keys) would happen
 before Map2 and Map3 phases but after reading Pro Hadoop Book I strongly
 doubt it.

 I would like to ask you if the key/value pairs emitted by the Map1 phase
 (or those emitted by the Reduce phase) are distributed to the nodes of the
 cluster before being processed by the next Map phase,
 or if the output of the Map1 phase (or Reduce phase) is immediately
 inserted to Map2 phase (or Map3 Phase) within the same node, without any
 distribution.

 Thank you in advance!
 Panagiotis Antonopoulos



Re: Reduce java.lang.OutOfMemoryError

2011-02-16 Thread Rahul Jain
If you google for such memory failures, you'll find the mapreduce tunable
that'll help you:

mapred.job.shuffle.input.buffer.percent ; it is well known that the default
values in hadoop config

don't work well for large data systems

-Rahul


On Wed, Feb 16, 2011 at 10:36 AM, James Seigel ja...@tynt.com wrote:

 Good luck.

 Let me know how it goes.

 James

 Sent from my mobile. Please excuse the typos.

 On 2011-02-16, at 11:11 AM, Kelly Burkhart kelly.burkh...@gmail.com
 wrote:

  OK, the job was preferring the config file on my local machine which
  is not part of the cluster over the cluster config files.  That seems
  completely broken to me; my config was basically empty other than
  containing the location of the cluster and my job apparently used
  defaults rather than the cluster config.  It doesn't make sense to me
  to keep configuration files synchronized on every machine that may
  access the cluster.
 
  I'm running again; we'll see if it completes this time.
 
  -K
 
  On Wed, Feb 16, 2011 at 10:30 AM, James Seigel ja...@tynt.com wrote:
  Hrmmm. Well as you've pointed out. 200m is quite small and is probably
  the cause.
 
  Now thEre might be some overriding settings in something you are using
  to launch or something.
 
  You could set those values in the config to not be overridden in the
  main conf then see what tries to override it in the logs
 
  Cheers
  James
 
  Sent from my mobile. Please excuse the typos.
 
  On 2011-02-16, at 9:21 AM, Kelly Burkhart kelly.burkh...@gmail.com
 wrote:
 
  I should have mentioned this in my last email: I thought of that so I
  logged into every machine in the cluster; each machine's
  mapred-site.xml has the same md5sum.
 
  On Wed, Feb 16, 2011 at 10:15 AM, James Seigel ja...@tynt.com wrote:
  He might not have that conf distributed out to each machine
 
 
  Sent from my mobile. Please excuse the typos.
 
  On 2011-02-16, at 9:10 AM, Kelly Burkhart kelly.burkh...@gmail.com
 wrote:
 
  Our clust admin (who's out of town today) has mapred.child.java.opts
  set to -Xmx1280 in mapred-site.xml.  However, if I go to the job
  configuration page for a job I'm running right now, it claims this
  option is set to -Xmx200m.  There are other settings in
  mapred-site.xml that are different too.  Why would map/reduce jobs
 not
  respect the mapred-site.xml file?
 
  -K
 
  On Wed, Feb 16, 2011 at 9:43 AM, Jim Falgout 
 jim.falg...@pervasive.com wrote:
  You can set the amount of memory used by the reducer using the
 mapreduce.reduce.java.opts property. Set it in mapred-site.xml or override
 it in your job. You can set it to something like: -Xm512M to increase the
 amount of memory used by the JVM spawned for the reducer task.
 
  -Original Message-
  From: Kelly Burkhart [mailto:kelly.burkh...@gmail.com]
  Sent: Wednesday, February 16, 2011 9:12 AM
  To: common-user@hadoop.apache.org
  Subject: Re: Reduce java.lang.OutOfMemoryError
 
  I have had it fail with a single reducer and with 100 reducers.
  Ultimately it needs to be funneled to a single reducer though.
 
  -K
 
  On Wed, Feb 16, 2011 at 9:02 AM, real great..
  greatness.hardn...@gmail.com wrote:
  Hi,
  How many reducers are you using currently?
  Try increasing the number or reducers.
  Let me know if it helps.
 
  On Wed, Feb 16, 2011 at 8:30 PM, Kelly Burkhart 
 kelly.burkh...@gmail.comwrote:
 
  Hello, I'm seeing frequent fails in reduce jobs with errors
 similar
  to
  this:
 
 
  2011-02-15 15:21:10,163 INFO org.apache.hadoop.mapred.ReduceTask:
  header: attempt_201102081823_0175_m_002153_0, compressed len:
 172492,
  decompressed len: 172488
  2011-02-15 15:21:10,163 FATAL org.apache.hadoop.mapred.TaskRunner:
  attempt_201102081823_0175_r_34_0 : Map output copy failure :
  java.lang.OutOfMemoryError: Java heap space
 at
 
 org.apache.hadoop.mapred.ReduceTask$ReduceCopier$MapOutputCopier.shuf
  fleInMemory(ReduceTask.java:1508)
 at
 
 org.apache.hadoop.mapred.ReduceTask$ReduceCopier$MapOutputCopier.getM
  apOutput(ReduceTask.java:1408)
 at
 
 org.apache.hadoop.mapred.ReduceTask$ReduceCopier$MapOutputCopier.copy
  Output(ReduceTask.java:1261)
 at
 
 org.apache.hadoop.mapred.ReduceTask$ReduceCopier$MapOutputCopier.run(
  ReduceTask.java:1195)
 
  2011-02-15 15:21:10,163 INFO org.apache.hadoop.mapred.ReduceTask:
  Shuffling 172488 bytes (172492 raw bytes) into RAM from
  attempt_201102081823_0175_m_002153_0
  2011-02-15 15:21:10,424 INFO org.apache.hadoop.mapred.ReduceTask:
  header: attempt_201102081823_0175_m_002118_0, compressed len:
 161944,
  decompressed len: 161940
  2011-02-15 15:21:10,424 INFO org.apache.hadoop.mapred.ReduceTask:
  header: attempt_201102081823_0175_m_001704_0, compressed len:
 228365,
  decompressed len: 228361
  2011-02-15 15:21:10,424 INFO org.apache.hadoop.mapred.ReduceTask:
  Task
  attempt_201102081823_0175_r_34_0: Failed fetch #1 from
  attempt_201102081823_0175_m_002153_0
  2011-02-15 15:21:10,424 FATAL 

Re: Help: How to increase amont maptasks per job ?

2011-01-07 Thread Rahul Jain
Also make sure you've enough input files for the next stage mappers to work
with...

Read thru the input splits part of tutorial:
http://wiki.apache.org/hadoop/HadoopMapReduce

If the last stage had only 4 reducers running, they'd generate 4 output
files. This will limit the # of mappers started in the next stage to 4,
unless you tune your input split parameters or write a custom input split.

Hope this helps, there is lot more literature on this on the web and hadoop
books released till date.

-Rahul


On Fri, Jan 7, 2011 at 1:19 PM, Ted Yu yuzhih...@gmail.com wrote:

 Set higher values for mapred.tasktracker.map.tasks.maximum (and
 mapred.tasktracker.reduce.tasks.maximum) in mapred-site.xml

 On Fri, Jan 7, 2011 at 12:58 PM, Tali K ncherr...@hotmail.com wrote:

 
 
 
 
  We have a jobs which runs in several map/reduce stages.  In the first
 job,
  a large number of map tasks -82  are initiated, as expected.
  And that cause all nodes to be used.
   In a
  later job, where we are still dealing with large amounts of
   data, only 4 map tasks are initiated, and that caused to use only 4
 nodes.
  This stage is actually the
  workhorse of the job, and requires much more processing power than the
  initial stage.
   We are trying to understand why only a few map tasks are
  being used, as we are not getting the full advantage of our cluster.
 
 
 
 



Re: Can MapReduce run simultaneous producer/consumer processes?

2011-01-06 Thread Rahul Jain
In case the producer / consumer don't require sorting to happen, take a look
at ChainMapper:

http://hadoop.apache.org/common/docs/r0.20.1/api/org/apache/hadoop/mapred/lib/ChainMapper.html

If you do want the stuff to happen after sorting, take a look at:

http://hadoop.apache.org/mapreduce/docs/current/api/org/apache/hadoop/mapreduce/lib/chain/ChainReducer.html

More esoteric cases will require you to do separate map reduces, at least
with the current hadoop framework.


On Thu, Jan 6, 2011 at 2:27 PM, W.P. McNeill bill...@gmail.com wrote:

 Say I have two MapReduce processes, A and B.  The two are algorithmically
 dissimilar, so they have to be implemented as separate MapReduce processes.
  The output of A is used as the input of B, so A has to run first.
  However,
 B doesn't need to take all of A's output as input, only a partition of it.
  So in theory A and B could run at the same time in a producer/consumer
 arrangement, where B would start to work as soon as A had produced some
 output but before A had completed.  Obviously, this could be a big
 parallelization win.

 Is this possible in MapReduce?  I know at the most basic level it is
 not–there is no synchronization mechanism that allows the same HDFS
 directory to be used for both input and output–but is there some
 abstraction
 layer on top that allows it?  I've been digging around, and I think the
 answer is No but I want to be sure.

 More specifically, the only abstraction layer I'm aware of that chains
 together MapReduce processes is Cascade, and I think it requires the reduce
 steps to be serialized, but again I'm not sure because I've only read the
 documentation and haven't actually played with it.



Re: How to build multiple inverted indexes?

2010-07-29 Thread Rahul Jain
Hadoop does not prevent you from writing key value pair multiple times in
the same map iteration if that is what is your roadblock.

You can call collector.collect() multiple times with same or distinct key
value pairs within a single map iteration.

-Rahul

On Thu, Jul 29, 2010 at 8:10 AM, ailinykh ailin...@gmail.com wrote:


 Hello, everybody!
 I have a bunch of records. Each record has key, and two fields A,B - R(k,
 A,B)
 I want to build two inverted indexes, one per each field. As output I
 expect
 two files
 IndexA =(A1- [k1,k2,k3..]),(A2 -[k1,k2,k4...]) ...
 IndexB =(B1- [k1,k2,k3..]),(B2-[k1,k2,k4...]) ...

 Hadoop mapper allows to write out only one key value pair. Of cause, I can
 run it twice, but I don't want to read the same record twice. What is the
 best way to handle this situation?

 Thank you,
  Andrey
 --
 View this message in context:
 http://old.nabble.com/How-to-build-multiple-inverted-indexes--tp29297476p29297476.html
 Sent from the Hadoop core-user mailing list archive at Nabble.com.




Re: reading distributed cache returns null pointer

2010-07-08 Thread Rahul Jain
I am not sure why you are using getFileClassPaths() API to access files...
here is what works for us:

Add the file(s) to distributed cache using:
DistributedCache.addCacheFile(p.toUri(), conf);

Read the files on the mapper using:

URI[] uris = DistributedCache.getCacheFiles(conf);
// access one of the files:
paths[0] = new Path(uris[0].getPath());
// now follow hadoop or local file APIs to access the file...


Did you try the above and did it not work ?

-Rahul

On Thu, Jul 8, 2010 at 12:04 PM, abc xyz fabc_xyz...@yahoo.com wrote:

 Hello all,

 As a new user of hadoop, I am having some problems with understanding some
 things. I am writing a program to load a file to the distributed cache and
 read
 this file in each mapper. In my driver program, I have added the file to my
 distributed cache using:

Path p=new
 Path(hdfs://localhost:9100/user/denimLive/denim/DCache/Orders.txt);
 DistributedCache.addCacheFile(p.toUri(), conf);

 In the configure method of the mapper, I am reading the file from cache
 using:
 Path[] cacheFiles=DistributedCache.getFileClassPaths(conf);
 BufferedReader joinReader=new BufferedReader(new
 FileReader(cacheFiles[0].toString()));

 however, the cacheFiles variable has null value in it.

 There is something mentioned on the Yahoo tutorial for hadoop about
 distributed
 cache which I do not understand:

 As a cautionary note: If you use the local JobRunner in Hadoop (i.e., what
 happens if you call JobClient.runJob()in a program with no or an empty
 hadoop-conf.xmlaccessible), then no local data directory is created; the
 getLocalCacheFiles()call will return an empty set of results. Unit test
 code
 should take this into account.

 what does this mean? I am executing my program in pseudo-distributed mode
 on
 windows using Eclipse.

 Any suggestion in this regard is highly valued.

 Thanks  in advance.






Re: reading distributed cache returns null pointer

2010-07-08 Thread Rahul Jain
Yes, distributed cache writes files to the local file system for each mapper
/ reducer. So you should be able to access the file(s) using local file
system APIs.

 If the files were staying in HDFS there would be no point to using
distributed cache since all mappers already have access to the global HDFS
directories :).

-Rahul

On Thu, Jul 8, 2010 at 3:03 PM, abc xyz fabc_xyz...@yahoo.com wrote:

 Hi Rahul,
 Thanks. It worked. I was using getFileClassPaths() to get the paths to the
 files
 in the cache and then use this path to access the file. It should have
 worked
 but I don't know why that doesn't produce the required result.

 I added the file HDFS file DCache/Orders.txt to my distributed cache. After
 calling DistributedCache.getCacheFiles(conf); in the configure method of
 the
 mapper node, if I read the file now from the returned path (which happens
 to be
 DCache/Orders.txt) using the Hadoop API , would the file be read from the
 local
 directory of the mapper node? More specifically I am doing this:


FileSystemhdfs=FileSystem.get(conf);
 URI[] uris=DistributedCache.getCacheFiles(conf);
 Path my_path=new Path(uris[0].getPath());

if(hdfs.exists(my_path))
{
FSDataInputStreamfs=hdfs.open(my_path);
while((str=fs.readLine())!=null)
  System.out.println(str);
 }

 Thanks


 
 From: Rahul Jain rja...@gmail.com
 To: common-user@hadoop.apache.org
 Sent: Thu, July 8, 2010 8:15:58 PM
 Subject: Re: reading distributed cache returns null pointer

 I am not sure why you are using getFileClassPaths() API to access files...
 here is what works for us:

 Add the file(s) to distributed cache using:
 DistributedCache.addCacheFile(p.toUri(), conf);

 Read the files on the mapper using:

 URI[] uris = DistributedCache.getCacheFiles(conf);
 // access one of the files:
 paths[0] = new Path(uris[0].getPath());
 // now follow hadoop or local file APIs to access the file...


 Did you try the above and did it not work ?

 -Rahul

 On Thu, Jul 8, 2010 at 12:04 PM, abc xyz fabc_xyz...@yahoo.com wrote:

  Hello all,
 
  As a new user of hadoop, I am having some problems with understanding
 some
  things. I am writing a program to load a file to the distributed cache
 and
  read
  this file in each mapper. In my driver program, I have added the file to
 my
  distributed cache using:
 
 Path p=new
  Path(hdfs://localhost:9100/user/denimLive/denim/DCache/Orders.txt);
  DistributedCache.addCacheFile(p.toUri(), conf);
 
  In the configure method of the mapper, I am reading the file from cache
  using:
  Path[] cacheFiles=DistributedCache.getFileClassPaths(conf);
  BufferedReader joinReader=new BufferedReader(new
  FileReader(cacheFiles[0].toString()));
 
  however, the cacheFiles variable has null value in it.
 
  There is something mentioned on the Yahoo tutorial for hadoop about
  distributed
  cache which I do not understand:
 
  As a cautionary note: If you use the local JobRunner in Hadoop (i.e.,
 what
  happens if you call JobClient.runJob()in a program with no or an empty
  hadoop-conf.xmlaccessible), then no local data directory is created; the
  getLocalCacheFiles()call will return an empty set of results. Unit test
  code
  should take this into account.
 
  what does this mean? I am executing my program in pseudo-distributed mode
  on
  windows using Eclipse.
 
  Any suggestion in this regard is highly valued.
 
  Thanks  in advance.
 
 
 
 







Re: Hadoop JobTracker Hanging

2010-06-22 Thread Rahul Jain
There are two issues which were fixed in 0.21.0  and can cause job tracker
to run out of memory:

https://issues.apache.org/jira/browse/MAPREDUCE-1316

and

https://issues.apache.org/jira/browse/MAPREDUCE-841

We've been hit by MAPREDUCE-841  (large jobConf objects with large number of
tasks, especially when running pig jobs) a number of times in hadoop 0.20.1,
0.20.2+.

The current workarounds are:

a) Be careful about what you store in jobConf object
b)  Understand and control the largest number of mappers/reducers that can
be queued at any time for processing.
c) Provide lot of RAM to jobTracker

We use (c) to save on debugging man hours most of the time :).

-Rahul

On Tue, Jun 22, 2010 at 8:53 AM, Allen Wittenauer
awittena...@linkedin.comwrote:


 On Jun 22, 2010, at 3:17 AM, Steve Loughran wrote:
 
  I'm surprised its the JT that is OOM-ing, anecdotally its the NN and 2ary
 NN that use more, especially if the files are many and the blocksize small.
 the JT should not be tracking that much data over time

 Pre-0.20.2, there are definitely bugs with how the JT history is handled,
 causing some memory leakage.

 The other fairly common condition is if you have way too many tasks per
 job.  This is usually an indication that your data layout is way out of
 whack (too little data in too many files) or that you should be using
 CombinedFileInputFormat.