hadoop-common-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Raghu Angadi <rang...@yahoo-inc.com>
Subject Re: Suggestions for making writing faster? DFSClient waiting while writing chunk
Date Mon, 11 May 2009 02:49:54 GMT

It should not be waiting unnecessarily. But the client has to, if any of 
the datanodes in the pipeline is not able to receive the as fast as 
client is writing. IOW writing goes as fast as the slowest of nodes 
involved in the pipeline (1 client and 3 datanodes).

But based on what your case is, you probably could benefit by increasing 
the buffer (number of unacked packets).. it would depend on where the 
datastream thread is blocked.

Raghu.

stack wrote:
> Writing a file, our application spends a load of time here:
> 
>         at java.lang.Object.wait(Native Method)
>         at java.lang.Object.wait(Object.java:485)
>         at
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:2964)
>         - locked <0x00007f11054c2b68> (a java.util.LinkedList)
>         - locked <0x00007f11054c24c0> (a
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
>         at
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:150)
>         at
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:132)
>         - locked <0x00007f11054c24c0> (a
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
>         at
> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:121)
>         - locked <0x00007f11054c24c0> (a
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
>         at
> org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:112)
>         at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
>         - locked <0x00007f11054c24c0> (a
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
>         at
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:49)
>         at java.io.DataOutputStream.write(DataOutputStream.java:90)
>         - locked <0x00007f1105694f28> (a
> org.apache.hadoop.fs.FSDataOutputStream)
>         at
> org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1020)
>         - locked <0x00007f1105694e98> (a
> org.apache.hadoop.io.SequenceFile$Writer)
>         at
> org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:984)
> 
> Here is the code from around line 2964 in writeChunk.
> 
>         // If queue is full, then wait till we can create  enough
> space
>         while (!closed && dataQueue.size() + ackQueue.size()  > maxPackets)
> {
>           try
> {
> 
> dataQueue.wait();
>           } catch (InterruptedException  e) {
> 
> }
> 
>         }
> 
> The queue of packets is full and we're waiting for it to be cleared.
> 
> Any suggestions for how I might get the DataStreamer to act more promptly
> clearing the package queue?
> 
> This is hadoop 0.20 branch.  Its a small cluster but relatively lightly
> loaded (so says ganglia).
> 
> Thanks,
> St.Ack
> 


Mime
View raw message