Re: Suggestions for making writing faster? DFSClient waiting while writing chunk

2009-05-12 Thread stack
On Mon, May 11, 2009 at 9:43 PM, Raghu Angadi  wrote:

> stack wrote:
>
>> Thanks Raghu:
>>
>> Here is where it gets stuck:  [...]
>>
>
> Is that where it normally stuck? That implies it is spending unusually long
> time at the end of writing a block, which should not be the case.


I studied datanode as you suggested.  This sent be back to the client
application and indeed, we were spending time finalizing blocks because
block size had been set way down in the application.  Write-rate is
reasonable again.

Thanks for the pointers Raghu,
St.Ack


Re: Suggestions for making writing faster? DFSClient waiting while writing chunk

2009-05-11 Thread Raghu Angadi

stack wrote:

Thanks Raghu:

Here is where it gets stuck:  [...]


Is that where it normally stuck? That implies it is spending unusually 
long time at the end of writing a block, which should not be the case. 
Also increasing maxPackets mostly won't help in this case.


Checking what datanodes' threads for this block are doing will be very 
helpful.. it is possible they are stuck trying to 'finalize the block'.. 
which could have blocked for various other reasons...  The final ack is 
sent only after finalizing a block.


Raghu.



"DataStreamer for file
/hbasetrunk2/.logs/aa0-000-13.u.powerset.com_1241988169615_60021/hlog.dat.1242020985471
block blk_-1659539029802462400_12649" daemon prio=10 tid=0x7f10ac00
nid=0x660 in Object.wait() [0x43a33000..0x43a33c80]
   java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at java.lang.Object.wait(Object.java:485)
at
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2322)
- locked <0x7f10e2b0c588> (a java.util.LinkedList)

Which is the wait in the below in the middle of DataStream.run:

  // Is this block full?
  if (one.lastPacketInBlock) {
synchronized (ackQueue) {
  while (!hasError && ackQueue.size() != 0 && clientRunning) {
try {
  ackQueue.wait();   // wait for acks to arrive from
datanodes
} catch (InterruptedException  e) {
}
  }
}

Sounds like, if we set the replication down from 3 to 2 it should write a
little faster.

Regards increasing size of ackqueue, are you thinking maxPackage?  Currently
its hardcoded at 80 -- a queue of 5MB (packets are 64k).  You thinking I
should experiment with that?  I suppose that won't hel w/ much w/ getting my
writes on the datanode.  Maybe I should be digging on datanode side to
figure why its slow getting back to the client?

Thanks,
St.Ack




On Sun, May 10, 2009 at 7:49 PM, Raghu Angadi  wrote:


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 <0x7f11054c2b68> (a java.util.LinkedList)
   - locked <0x7f11054c24c0> (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 <0x7f11054c24c0> (a
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
   at
org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:121)
   - locked <0x7f11054c24c0> (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 <0x7f11054c24c0> (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 <0x7f1105694f28> (a
org.apache.hadoop.fs.FSDataOutputStream)
   at
org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1020)
   - locked <0x7f1105694e98> (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








Re: Suggestions for making writing faster? DFSClient waiting while writing chunk

2009-05-10 Thread stack
Thanks Raghu:

Here is where it gets stuck:

"DataStreamer for file
/hbasetrunk2/.logs/aa0-000-13.u.powerset.com_1241988169615_60021/hlog.dat.1242020985471
block blk_-1659539029802462400_12649" daemon prio=10 tid=0x7f10ac00
nid=0x660 in Object.wait() [0x43a33000..0x43a33c80]
   java.lang.Thread.State: WAITING (on object monitor)
at java.lang.Object.wait(Native Method)
at java.lang.Object.wait(Object.java:485)
at
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2322)
- locked <0x7f10e2b0c588> (a java.util.LinkedList)

Which is the wait in the below in the middle of DataStream.run:

  // Is this block full?
  if (one.lastPacketInBlock) {
synchronized (ackQueue) {
  while (!hasError && ackQueue.size() != 0 && clientRunning) {
try {
  ackQueue.wait();   // wait for acks to arrive from
datanodes
} catch (InterruptedException  e) {
}
  }
}

Sounds like, if we set the replication down from 3 to 2 it should write a
little faster.

Regards increasing size of ackqueue, are you thinking maxPackage?  Currently
its hardcoded at 80 -- a queue of 5MB (packets are 64k).  You thinking I
should experiment with that?  I suppose that won't hel w/ much w/ getting my
writes on the datanode.  Maybe I should be digging on datanode side to
figure why its slow getting back to the client?

Thanks,
St.Ack




On Sun, May 10, 2009 at 7:49 PM, Raghu Angadi  wrote:

>
> 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 <0x7f11054c2b68> (a java.util.LinkedList)
>>- locked <0x7f11054c24c0> (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 <0x7f11054c24c0> (a
>> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
>>at
>> org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:121)
>>- locked <0x7f11054c24c0> (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 <0x7f11054c24c0> (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 <0x7f1105694f28> (a
>> org.apache.hadoop.fs.FSDataOutputStream)
>>at
>> org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1020)
>>- locked <0x7f1105694e98> (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
>>
>>
>


Re: Suggestions for making writing faster? DFSClient waiting while writing chunk

2009-05-10 Thread Raghu Angadi


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 <0x7f11054c2b68> (a java.util.LinkedList)
- locked <0x7f11054c24c0> (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 <0x7f11054c24c0> (a
org.apache.hadoop.hdfs.DFSClient$DFSOutputStream)
at
org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:121)
- locked <0x7f11054c24c0> (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 <0x7f11054c24c0> (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 <0x7f1105694f28> (a
org.apache.hadoop.fs.FSDataOutputStream)
at
org.apache.hadoop.io.SequenceFile$Writer.append(SequenceFile.java:1020)
- locked <0x7f1105694e98> (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