.18.1 jobtracker deadlock

2008-12-17 Thread Sagar Naik

Hi,

Found one Java-level deadlock:
=
SocketListener0-7:
 waiting to lock monitor 0x0845e1fc (object 0x54f95838, a 
org.apache.hadoop.mapred.JobTracker),

 which is held by IPC Server handler 0 on 54311
IPC Server handler 0 on 54311:
 waiting to lock monitor 0x4d671064 (object 0x57250a60, a 
org.apache.hadoop.mapred.JobInProgress),

 which is held by initJobs
initJobs:
 waiting to lock monitor 0x0845e1fc (object 0x54f95838, a 
org.apache.hadoop.mapred.JobTracker),

 which is held by IPC Server handler 0 on 54311

Java stack information for the threads listed above:
===
SocketListener0-7:
   at 
org.apache.hadoop.mapred.JobTracker.getClusterStatus(JobTracker.java:1826)
   - waiting to lock 0x54f95838 (a 
org.apache.hadoop.mapred.JobTracker)
   at 
org.apache.hadoop.mapred.jobtracker_jsp._jspService(jobtracker_jsp.java:135)
   at 
org.apache.jasper.runtime.HttpJspBase.service(HttpJspBase.java:94)

   at javax.servlet.http.HttpServlet.service(HttpServlet.java:802)
   at 
org.mortbay.jetty.servlet.ServletHolder.handle(ServletHolder.java:427)
   at 
org.mortbay.jetty.servlet.WebApplicationHandler.dispatch(WebApplicationHandler.java:475)
   at 
org.mortbay.jetty.servlet.ServletHandler.handle(ServletHandler.java:567)

   at org.mortbay.http.HttpContext.handle(HttpContext.java:1565)
   at 
org.mortbay.jetty.servlet.WebApplicationContext.handle(WebApplicationContext.java:635)

   at org.mortbay.http.HttpContext.handle(HttpContext.java:1517)
   at org.mortbay.http.HttpServer.service(HttpServer.java:954)
   at org.mortbay.http.HttpConnection.service(HttpConnection.java:814)
   at 
org.mortbay.http.HttpConnection.handleNext(HttpConnection.java:981)

   at org.mortbay.http.HttpConnection.handle(HttpConnection.java:831)
   at 
org.mortbay.http.SocketListener.handleConnection(SocketListener.java:244)

   at org.mortbay.util.ThreadedServer.handle(ThreadedServer.java:357)
   at org.mortbay.util.ThreadPool$PoolThread.run(ThreadPool.java:534)
IPC Server handler 0 on 54311:
   at 
org.apache.hadoop.mapred.JobInProgress.kill(JobInProgress.java:1451)
   - waiting to lock 0x57250a60 (a 
org.apache.hadoop.mapred.JobInProgress)

   at org.apache.hadoop.mapred.JobTracker.killJob(JobTracker.java:1843)
   - locked 0x54f95838 (a org.apache.hadoop.mapred.JobTracker)
   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
   at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)

   at java.lang.reflect.Method.invoke(Method.java:597)
   at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:452)
   at org.apache.hadoop.ipc.Server$Handler.run(Server.java:888)
initJobs:
   at 
org.apache.hadoop.mapred.JobTracker.finalizeJob(JobTracker.java:1015)
   - waiting to lock 0x54f95838 (a 
org.apache.hadoop.mapred.JobTracker)
   at 
org.apache.hadoop.mapred.JobInProgress.garbageCollect(JobInProgress.java:1656)

   - locked 0x57250a60 (a org.apache.hadoop.mapred.JobInProgress)
   at 
org.apache.hadoop.mapred.JobInProgress.kill(JobInProgress.java:1469)

   - locked 0x57250a60 (a org.apache.hadoop.mapred.JobInProgress)
   at 
org.apache.hadoop.mapred.JobTracker$JobInitThread.run(JobTracker.java:416)

   at java.lang.Thread.run(Thread.java:619)

Found 1 deadlock.



I found this condition. I will try to work on this

-Sagar



HOD questions

2008-12-17 Thread Craig Macdonald

Hello,

We have two HOD questions:

(1) For our current Torque PBS setup, the number of nodes requested by 
HOD (-l nodes=X) corresponds to the number of CPUs allocated, however 
these nodes can be spread across various partially or empty nodes. 
Unfortunately, HOD does not appear to honour the number of processors 
actually allocated by Torque PBS to that job.


For example, a current running HOD session can be viewed in qstat as:
104544.trmaster  user parallel HOD   4178 8  ----  288:0 R 01:48
  node29/2+node29/1+node29/0+node17/2+node17/1+node18/2+node18/1
  +node19/1

However, on inspection of the Jobtracker UI, it tells us that node19 has 
Max Map Tasks and Max Reduce Tasks both set to 2, when I think that 
for node19, it should only be allowed one map task.


I believe that for each node, HOD should determine (using the 
information in the $PBS_NODEFILE), how many CPUs for each node are 
allocated to the HOD job, and then set 
mapred.tasktracker.map.tasks.maximum appropriately on each node.


(2) In our InputFormat, we use the numSplits to tell us how many map 
tasks the job's files should be split into. However, HOD does not 
override the mapred.map.tasks property (nor the mapred.reduce.tasks), 
while they should be set dependent on the number of available task 
trackers and/or nodes in the HOD session.


Craig


OOME only with large datasets

2008-12-17 Thread Philip
I've been trying to trouble shoot an OOME we've been having.

When we run the job over a dataset that about 700GB (~9000 files) or larger
we will get an OOME on the map jobs.  However if we run the job over smaller
set of the data then everything works out fine.  So my question is: What
changes in Hadoop as the size of the input set increases?

We are on hadoop 0.18.0.

Here's is a stack trace produced by the job tracker.
java.lang.OutOfMemoryError: Java heap space at
java.util.Arrays.copyOf(Arrays.java:2882) at
java.lang.AbstractStringBuilder.expandCapacity(AbstractStringBuilder.java:100)
at java.lang.AbstractStringBuilder.append(AbstractStringBuilder.java:390) at
java.lang.StringBuffer.append(StringBuffer.java:224) at
com.sun.org.apache.xerces.internal.dom.DeferredDocumentImpl.getNodeValueString(DeferredDocumentImpl.java:1167)
at
com.sun.org.apache.xerces.internal.dom.DeferredDocumentImpl.getNodeValueString(DeferredDocumentImpl.java:1120)
at
com.sun.org.apache.xerces.internal.dom.DeferredTextImpl.synchronizeData(DeferredTextImpl.java:93)
at
com.sun.org.apache.xerces.internal.dom.CharacterDataImpl.getData(CharacterDataImpl.java:160)
at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:928)
at
org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:851)
at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:819) at
org.apache.hadoop.conf.Configuration.get(Configuration.java:278) at
org.apache.hadoop.conf.Configuration.getBoolean(Configuration.java:446) at
org.apache.hadoop.mapred.JobConf.getKeepFailedTaskFiles(JobConf.java:308) at
org.apache.hadoop.mapred.TaskTracker$TaskInProgress.setJobConf(TaskTracker.java:1506)
at
org.apache.hadoop.mapred.TaskTracker.launchTaskForJob(TaskTracker.java:727)
at org.apache.hadoop.mapred.TaskTracker.localizeJob(TaskTracker.java:721) at
org.apache.hadoop.mapred.TaskTracker.startNewTask(TaskTracker.java:1306) at
org.apache.hadoop.mapred.TaskTracker.offerService(TaskTracker.java:946) at
org.apache.hadoop.mapred.TaskTracker.run(TaskTracker.java:1343) at
org.apache.hadoop.mapred.TaskTracker.main(TaskTracker.java:2354)


Thanks,
Philip.


Re: Does datanode acts as readonly in case of DiskFull ?

2008-12-17 Thread Raghu Angadi

Sagar Naik wrote:

Hi ,

I would like to know what happens in case of DiskFull on a datanode
Does the datanode acts as block server only ?


Yes. I think so.

Does it rejects anymore Block creation request OR Namenode does not list 
it for new blocks


yes. NN will not allocate it any more blocks.

Did you notice anything different? (which is quite possible).

Raghu.



Hadoop 18
-Sagar




Datanode handling of single disk failure

2008-12-17 Thread Brian Bockelman

Hello all,

I'd like to take the datanode's capability to handle multiple  
directories to a somewhat-extreme, and get feedback on how well this  
might work.


We have a few large RAID servers (12 to 48 disks) which we'd like to  
transition to Hadoop.  I'd like to mount each of the disks  
individually (i.e., /mnt/disk1, /mnt/disk2, ) and take advantage  
of Hadoop's replication - instead of pay the overhead to set up a RAID  
and still have to pay the overhead of replication.


However, we're a bit concerned about how well Hadoop might handle one  
of the directories disappearing from underneath it.  If a single  
volume, say, /mnt/disk1 starts returning I/O errors, is Hadoop smart  
enough to figure out that this whole volume is broken?  Or will we  
have to restart the datanode after any disk failure for it to search  
the directory realize everything is broken?  What happens if you start  
up the datanode with a data directory that it can't write into?


Is anyone running in this fashion (i.e., multiple data directories  
corresponding to different disk volumes ... even better if you're  
doing it with more than a few disks)?


Brian



Re: [video] visualization of the hadoop code history

2008-12-17 Thread Jeff Hammerbacher
Very cool stuff, but I don't see a reference anywhere to the author of the
visualization, which seems like poor form for a marketing video. I apologize
if I missed a reference somewhere.

Michael Ogawa at UC Davis wrote the code to generate that visualization and
open sourced it at http://code.google.com/p/codeswarm. For more, check out
http://vis.cs.ucdavis.edu/~ogawa/codeswarm.

On Wed, Dec 17, 2008 at 7:54 AM, Stefan Groschupf s...@101tec.com wrote:

 Owen O'Malley wrote:

 It is interesting, but it would be more interesting to track the authors
 of the patch rather than the committer. The two are rarely the same.


 Indeed.  There was a period of over a year where I wrote hardly anything
 but committed almost everything.  So I am vastly overrepresented in commits.



 Thanks for the feedback.

 The video was rendered from the svn log file (text version). If someone has
 a script that clean this file up and replace the committer name with the
 real patch author, we are happy to render the video again.


 Cheers,
 Stefan
 ~~~
 Hadoop training and consulting
 http://www.scaleunlimited.com
 http://www.101tec.com







Re: [video] visualization of the hadoop code history

2008-12-17 Thread Stefan Groschupf
Very cool stuff, but I don't see a reference anywhere to the author  
of the
visualization, which seems like poor form for a marketing video. I  
apologize

if I missed a reference somewhere.


Jeff, you missed it!
It is the first text screen at the end of the video.
It is actually a cool open source project with quite some contributors.

Stefan

~~~
Hadoop training and consulting
http://www.scaleunlimited.com
http://www.101tec.com





Re: [video] visualization of the hadoop code history

2008-12-17 Thread Stefan Groschupf

Owen O'Malley wrote:
It is interesting, but it would be more interesting to track the  
authors of the patch rather than the committer. The two are rarely  
the same.


Indeed.  There was a period of over a year where I wrote hardly  
anything but committed almost everything.  So I am vastly  
overrepresented in commits.



Thanks for the feedback.

The video was rendered from the svn log file (text version). If  
someone has a script that clean this file up and replace the committer  
name with the real patch author, we are happy to render the video again.



Cheers,
Stefan
~~~
Hadoop training and consulting
http://www.scaleunlimited.com
http://www.101tec.com






Re: [video] visualization of the hadoop code history

2008-12-17 Thread Jeff Hammerbacher
Ha, that's what I get for my short attention span. Rad stuff, sorry for
missing the recognition of the code_swarm utility.

On Wed, Dec 17, 2008 at 12:38 PM, Stefan Groschupf s...@101tec.com wrote:

 Very cool stuff, but I don't see a reference anywhere to the author of the
 visualization, which seems like poor form for a marketing video. I
 apologize
 if I missed a reference somewhere.


 Jeff, you missed it!
 It is the first text screen at the end of the video.
 It is actually a cool open source project with quite some contributors.


 Stefan

 ~~~
 Hadoop training and consulting
 http://www.scaleunlimited.com
 http://www.101tec.com






java.nio.channels.ClosedSelectorException

2008-12-17 Thread Brian Cho
Hi,

I've set up a Hadoop cluster but have a problem where multiple
datanodes and tasks stop responding. I first ran into the problem
using 0.19.0, but I also see the problem at 0.18.2. Java version is
1.6.0_11.

Looking at the logs, the first sign of trouble seems to be either
java.nio.channels.ClosedSelectorException or java.io.IOException:
Filesystem closed. I've attached the last 4KB from failed map tasks
taken from the web interface (0.18.2), and the end of a snippet of a
datanode log (0.19.0).

Task nodes stop responding after the exception. The datanode does not
fail right away, but rather continues to raise null pointer exceptions
while failing to communicate with the namenode. Eventually the
namenode decides the datanode is Dead. The exceptions are not raised
every time, but are fairly frequent (for example, after running 5
straight m/r jobs, I had 8 failed datanodes). Also, we've seen this
happen on a 16-node setup, as well as a 4-node setup (different
hardware).

Has anyone run into similar problems before? What would be your
suggestion to try and troubleshoot this? I'm pretty much stumped as to
why this is happening.

Thank you,
Brian Cho







Task Logs: 'attempt_200812161652_0002_m_03_0'


stdout logs



stderr logs

Exception in thread
org.apache.hadoop.dfs.dfsclient$leasechec...@a9ae05
java.nio.channels.ClosedSelectorException
at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:66)
at sun.nio.ch.SelectorImpl.selectNow(SelectorImpl.java:88)
at sun.nio.ch.Util.releaseTemporarySelector(Util.java:135)
at sun.nio.ch.SocketAdaptor.connect(SocketAdaptor.java:118)
at 
org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:300)
at org.apache.hadoop.ipc.Client$Connection.access$1700(Client.java:177)
at org.apache.hadoop.ipc.Client.getConnection(Client.java:789)
at org.apache.hadoop.ipc.Client.call(Client.java:704)
at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
at org.apache.hadoop.dfs.$Proxy1.renewLease(Unknown Source)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
at java.lang.reflect.Method.invoke(Method.java:597)
at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
at org.apache.hadoop.dfs.$Proxy1.renewLease(Unknown Source)
at org.apache.hadoop.dfs.DFSClient$LeaseChecker.run(DFSClient.java:782)
at java.lang.Thread.run(Thread.java:619)



syslog logs

2008-12-16 16:54:16,767 INFO org.apache.hadoop.metrics.jvm.JvmMetrics:
Initializing JVM Metrics with processName=MAP, sessionId=
2008-12-16 16:54:16,886 INFO org.apache.hadoop.mapred.MapTask: numReduceTasks: 0
2008-12-16 17:09:35,889 WARN org.apache.hadoop.mapred.TaskRunner:
Parent died.  Exiting attempt_200812161652_0002_m_03_0








Task Logs: 'attempt_200812161652_0002_m_03_0'


stdout logs



stderr logs

Exception in thread
org.apache.hadoop.dfs.dfsclient$leasechec...@a9ae05
java.nio.channels.ClosedSelectorException
at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:66)
at sun.nio.ch.SelectorImpl.selectNow(SelectorImpl.java:88)
at sun.nio.ch.Util.releaseTemporarySelector(Util.java:135)
at sun.nio.ch.SocketAdaptor.connect(SocketAdaptor.java:118)
at 
org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:300)
at org.apache.hadoop.ipc.Client$Connection.access$1700(Client.java:177)
at org.apache.hadoop.ipc.Client.getConnection(Client.java:789)
at org.apache.hadoop.ipc.Client.call(Client.java:704)
at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
at org.apache.hadoop.dfs.$Proxy1.renewLease(Unknown Source)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
at java.lang.reflect.Method.invoke(Method.java:597)
at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:82)
at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:59)
at org.apache.hadoop.dfs.$Proxy1.renewLease(Unknown Source)
at org.apache.hadoop.dfs.DFSClient$LeaseChecker.run(DFSClient.java:782)
at java.lang.Thread.run(Thread.java:619)



syslog logs

2008-12-16 16:54:16,767 INFO org.apache.hadoop.metrics.jvm.JvmMetrics:
Initializing JVM Metrics with processName=MAP, sessionId=
2008-12-16 16:54:16,886 INFO org.apache.hadoop.mapred.MapTask: 

Warning on turning on ipv6 on your Hadoop clusters

2008-12-17 Thread Runping Qi


If you may have turned on ipv6 on your hadoop cluster, it may cause severe
performance hit!

When I ran the gridmix2 benchmark on a newly constructed cluster, it took
30% more time than the baseline time that was obtained on a similar cluster.

I noticed that some task processes on some machines took 3+ minutes to
initialize.
After examining these processes in details, I found that they were stuck at
socket initialization tile, as shown in the following stack:

main prio=10 tid=0x0805b400 nid=0x4681 runnable [0xf7fbb000..0xf7fbc208]
   java.lang.Thread.State: RUNNABLE
at java.net.PlainSocketImpl.initProto(Native Method)
at java.net.PlainSocketImpl.clinit(PlainSocketImpl.java:84)
at java.net.Socket.setImpl(Socket.java:434)
at java.net.Socket.init(Socket.java:68)
at sun.nio.ch.SocketAdaptor.init(SocketAdaptor.java:50)
at sun.nio.ch.SocketAdaptor.create(SocketAdaptor.java:55)
at sun.nio.ch.SocketChannelImpl.socket(SocketChannelImpl.java:105)
- locked 0xf17a38c8 (a java.lang.Object)
at 
org.apache.hadoop.net.StandardSocketFactory.createSocket(StandardSocketFacto
ry.java:58)
at 
org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:298)
- locked 0xf1795db0 (a org.apache.hadoop.ipc.Client$Connection)
at org.apache.hadoop.ipc.Client$Connection.access$1700(Client.java:178)
at org.apache.hadoop.ipc.Client.getConnection(Client.java:820)
at org.apache.hadoop.ipc.Client.call(Client.java:705)
at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
at org.apache.hadoop.mapred.$Proxy0.getProtocolVersion(Unknown Source)
at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:348)
at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:335)
at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:372)
at 
org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:2188)


I did a search on the web and found that that was due to a known bug for
Java related to ipv6.

More information about the bug can be found at the following two pages:

http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6483406
http://edocs.bea.com/jrockit/releases/5026x/relnotes/relnotes.html

{quote}
Slow startup because of a hang in java.net.PlainSocketImpl.initProto(),
which typically is called when creating the first Socket orServerSocket.
In BEA JRockit 5.0 R26 the network stack is configured so that IPv6 is
used in preference to IPv4 when it is present.
During initialization of the network stack, the network code connects a
socket to its own loopback interface to set up some data   structures.
Blocking this connection (e.g. with a firewall) will cause the
initialization code to wait for a socket timeout, after which   the system
falls back on using IPv4.


{quote}

Suggested Workaround:

Either set -Djava.net.preferIPv4Stack=true for the child process option,
which forces Java to use IPv4
instead, or you disable IPv6 entirely in the system. The proper fix is to
allow IPv6 traffic from localhost to localhost.
For more information, see the Sun documentation:
http://java.sun.com/j2se/1.4.2/docs/guide/net/ipv6_guide/#ipv6-networking



Runping




DiskUsage ('du -sk') probably hangs Datanode

2008-12-17 Thread Sagar Naik


I see createBlockException and Abandoning block quite often
When I check the datanode, they are running. I can browse file system 
from that datanode:50075
However, I also notice tht a  du forked off from the DN. This 'du' run 
anywhere from 6mins to 30 mins.


During this time no logs are generated . DN appears in S1 state and the 
'du' in D state.


Is it possible tht jvm has bug or hdd is bad.
I m using /usr/java/jdk1.6.0_07/bin/java and planing to move onto 11

However, I start noticing this after DFS is 50% (on avg) full

Pl help me with some pointers

Hadoop version : .18.1

-Sagar








Re: DiskUsage ('du -sk') probably hangs Datanode

2008-12-17 Thread Brian Bockelman

Hey Sagar,

If the 'du' is in the D state, then that probably means bad things  
for your hardware.


I recommend looking in dmesg and /var/log/messages for anything  
interesting, as well as perform a hard-drive diagnostic test (may be  
as simple as a SMART tests) to see if there's an issue.


I can't say for sure, but the 'du' is probably not hanging the  
Datanode; it's probably a symptom of larger problems.


Brian

On Dec 17, 2008, at 8:29 PM, Sagar Naik wrote:



I see createBlockException and Abandoning block quite often
When I check the datanode, they are running. I can browse file  
system from that datanode:50075
However, I also notice tht a  du forked off from the DN. This 'du'  
run anywhere from 6mins to 30 mins.


During this time no logs are generated . DN appears in S1 state and  
the 'du' in D state.


Is it possible tht jvm has bug or hdd is bad.
I m using /usr/java/jdk1.6.0_07/bin/java and planing to move onto 11

However, I start noticing this after DFS is 50% (on avg) full

Pl help me with some pointers

Hadoop version : .18.1

-Sagar









Re: Output.collect uses toString for custom key class. Is it possible to change this?

2008-12-17 Thread Aaron Kimball
NullWritable has a get() method that returns the singleton instance of the
NullWritable.
- Aaron

On Tue, Dec 16, 2008 at 9:30 AM, David Coe david@chalklabs.net wrote:

 Owen O'Malley wrote:
 
  On Dec 16, 2008, at 9:14 AM, David Coe wrote:
 
  Does the SequenceFileOutputFormat work with NullWritable as the value?
 
  Yes.

 Owen O'Malley wrote:
  It means you are trying to write a null value. Your reduce is doing
  something like:
 
  output.collect(key, null);
 
  In TextOutputFormat, that is ok and just skips it.
  SequenceFileOutputFormat doesn't like nulls.
 
  -- Owen
 Since the SequenceFileOutputFormat doesn't like nulls, how would I use
 NullWritable?  Obviously output.collect(key, null) isn't working.  If I
 change it to output.collect(key, new IntWritable()) I get the result I
 want (plus an int that I don't), but output.collect(key, new
 NullWritable()) does not work.

 Thanks again.

 David




Re: DiskUsage ('du -sk') probably hangs Datanode

2008-12-17 Thread Sagar Naik

Brian Bockelman wrote:

Hey Sagar,

If the 'du' is in the D state, then that probably means bad things 
for your hardware.


I recommend looking in dmesg and /var/log/messages for anything 
interesting, as well as perform a hard-drive diagnostic test (may be 
as simple as a SMART tests) to see if there's an issue.


I can't say for sure, but the 'du' is probably not hanging the 
Datanode; it's probably a symptom of larger problems.



Thanks Brian
I will start SMART tests
Pl tell me what direction I should look in case of larger problems.



Brian

On Dec 17, 2008, at 8:29 PM, Sagar Naik wrote:



I see createBlockException and Abandoning block quite often
When I check the datanode, they are running. I can browse file system 
from that datanode:50075
However, I also notice tht a  du forked off from the DN. This 'du' 
run anywhere from 6mins to 30 mins.


During this time no logs are generated . DN appears in S1 state and 
the 'du' in D state.


Is it possible tht jvm has bug or hdd is bad.
I m using /usr/java/jdk1.6.0_07/bin/java and planing to move onto 11

However, I start noticing this after DFS is 50% (on avg) full

Pl help me with some pointers

Hadoop version : .18.1

-Sagar











Copy data between HDFS instances...

2008-12-17 Thread C G
Hi All:

I am setting up 2 grids, each with its own HDFS.  The grids are unaware of each 
other but exist on the same network.

I'd like to copy data from one HDFS to the other.  Is there a way to do this 
simply, or do I need to cobble together scripts to copy from HDFS on one side 
and pipe to a dfs -cp on the other side?

I tried something like this:

 hadoop dfs -ls hdfs://grid1NameNode:portNo/

from grid2 trying to ls on grid1 but got a wrong FS error message.  I also 
tried:

hadoop dfs -ls hdfs://grid1NameNode:portNo/foo

on grid2 where /foo exists on grid1 and got 0 files found.

I assume there is some way to do this and I just don't have the right command 
line magic.  This is Hadoop 0.15.0.

Any help appreciated.

Thanks,
C G




  



Re: Copy data between HDFS instances...

2008-12-17 Thread lohit
try
hadoop distcp

more info here
http://hadoop.apache.org/core/docs/current/distcp.html
 Documentation is for current release, but looking hadoop distcp should print 
out help message.


Thanks,
Lohit

- Original Message 
From: C G parallel...@yahoo.com
To: core-user@hadoop.apache.org
Sent: Wednesday, December 17, 2008 7:18:51 PM
Subject: Copy data between HDFS instances...

Hi All:

I am setting up 2 grids, each with its own HDFS.  The grids are unaware of each 
other but exist on the same network.

I'd like to copy data from one HDFS to the other.  Is there a way to do this 
simply, or do I need to cobble together scripts to copy from HDFS on one side 
and pipe to a dfs -cp on the other side?

I tried something like this:

 hadoop dfs -ls hdfs://grid1NameNode:portNo/

from grid2 trying to ls on grid1 but got a wrong FS error message.  I also 
tried:

hadoop dfs -ls hdfs://grid1NameNode:portNo/foo

on grid2 where /foo exists on grid1 and got 0 files found.

I assume there is some way to do this and I just don't have the right command 
line magic.  This is Hadoop 0.15.0.

Any help appreciated.

Thanks,
C G


Re: HOD questions

2008-12-17 Thread Hemanth Yamijala

Craig,

Hello,

We have two HOD questions:

(1) For our current Torque PBS setup, the number of nodes requested by 
HOD (-l nodes=X) corresponds to the number of CPUs allocated, however 
these nodes can be spread across various partially or empty nodes. 
Unfortunately, HOD does not appear to honour the number of processors 
actually allocated by Torque PBS to that job.


Just FYI, at Yahoo! we've set torque to allocate separate nodes for the 
number specified to HOD. In other words, the number corresponds to the 
number of nodes, not processors. This has proved simpler to manage. I 
forget right now, but I think you can make Torque behave like this (to 
not treat processors as individual nodes).

For example, a current running HOD session can be viewed in qstat as:
104544.trmaster  user parallel HOD   4178 8  ----  288:0 R 
01:48

  node29/2+node29/1+node29/0+node17/2+node17/1+node18/2+node18/1
  +node19/1

However, on inspection of the Jobtracker UI, it tells us that node19 
has Max Map Tasks and Max Reduce Tasks both set to 2, when I think 
that for node19, it should only be allowed one map task.


While HOD does not do this automatically, please note that since you are 
bringing up a Map/Reduce cluster on the allocated nodes, you can submit 
map/reduce parameters with which to bring up the cluster when allocating 
jobs. The relevant options are --gridservice-mapred.server-params (or -M 
in shorthand). Please refer to
http://hadoop.apache.org/core/docs/r0.19.0/hod_user_guide.html#Options+for+Configuring+Hadoop 
for details.


I believe that for each node, HOD should determine (using the 
information in the $PBS_NODEFILE), how many CPUs for each node are 
allocated to the HOD job, and then set 
mapred.tasktracker.map.tasks.maximum appropriately on each node.


(2) In our InputFormat, we use the numSplits to tell us how many map 
tasks the job's files should be split into. However, HOD does not 
override the mapred.map.tasks property (nor the mapred.reduce.tasks), 
while they should be set dependent on the number of available task 
trackers and/or nodes in the HOD session.


Can this not be submitted via the Hadoop job's configuration ? Again, 
HOD cannot do this automatically currently. But you could use the 
hod.client-params to set up a client side hadoop-site.xml that would 
work like this for all jobs submitted to the cluster.


Hope this helps some.

Thanks
Hemanth


Re: Output.collect uses toString for custom key class. Is it possible to change this?

2008-12-17 Thread Owen O'Malley


On Dec 16, 2008, at 9:30 AM, David Coe wrote:


Since the SequenceFileOutputFormat doesn't like nulls, how would I use
NullWritable?  Obviously output.collect(key, null) isn't working.   
If I

change it to output.collect(key, new IntWritable()) I get the result I
want (plus an int that I don't), but output.collect(key, new
NullWritable()) does not work.


Sorry, I answered you literally. You can write a SequenceFile with  
NullWritables as the values, but you really want optional nulls. I'd  
probably define a Wrapper class like GenericWritable. It would look  
something like:


class NullableWriableT extends Writable implements Writable {
  private T instance;
  private boolean isNull;
  public void setNull(boolean isNull) {
this.isNull = isNull;
  }
  public void readFields(DataInput in) throws IOException {
read isNull;
if (!isNull) {
   instance.readFields(in);
  }
  public void write(DataOutput out) throws IOException {
write isNull;
if (!isNull) {
   instance.write(out);
}
  }
}

-- Owen