You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-user@hadoop.apache.org by Iván de Prado <iv...@gmail.com> on 2008/03/28 14:07:48 UTC

DFS get blocked when writing a file.

Hello, 

I'm working with Hadoop 0.16.1. I have an issue with the DFS. Sometimes
when writing to the HDFS it gets blocked. Sometimes it doesn't happen,
so it's not easily reproducible. 

My cluster have 4 nodes and one master with the NameNode and JobTracker.
This are the logs that appears when all gets blocked. Look to the block
blk_7857709233639057851 that seems to be the problematic one. It raises
the exception:

"Exception in receiveBlock for block  java.io.IOException: Trying to
change block file offset of block blk_7857709233639057851 to 33357824
but actual size of file is 33353728"

A bigger trace of the logs and a part of the stack trace:

hn3: 2008-03-28 07:34:44,499 INFO org.apache.hadoop.dfs.DataNode:
Receiving block blk_7857709233639057851 src: /172.16.3.2:46092
dest: /172.16.3.2:50010
hn3: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
Datanode 2 got response for connect ack  from downstream datanode with
firstbadlink as 
hn3: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
Datanode 2 forwarding connect ack to upstream firstbadlink is 
hn2: 2008-03-28 07:34:44,496 INFO org.apache.hadoop.dfs.DataNode:
Received block blk_8152094109584962620 of size 67108864 from /172.16.3.2
hn2: 2008-03-28 07:34:44,496 INFO org.apache.hadoop.dfs.DataNode:
PacketResponder 2 for block blk_8152094109584962620 terminating
hn2: 2008-03-28 07:34:44,500 INFO org.apache.hadoop.dfs.DataNode:
Receiving block blk_7857709233639057851 src: /172.16.3.5:35904
dest: /172.16.3.5:50010
hn2: 2008-03-28 07:34:44,502 INFO org.apache.hadoop.dfs.DataNode:
Datanode 1 got response for connect ack  from downstream datanode with
firstbadlink as 
hn2: 2008-03-28 07:34:44,502 INFO org.apache.hadoop.dfs.DataNode:
Datanode 1 forwarding connect ack to upstream firstbadlink is 
hn1: 2008-03-28 07:34:44,495 INFO org.apache.hadoop.dfs.DataNode:
Received block blk_8152094109584962620 of size 67108864 from /172.16.3.4
hn1: 2008-03-28 07:34:44,495 INFO org.apache.hadoop.dfs.DataNode:
PacketResponder 1 for block blk_8152094109584962620 terminating
hn4: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
Receiving block blk_7857709233639057851 src: /172.16.3.4:36887
dest: /172.16.3.4:50010
hn4: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
Datanode 0 forwarding connect ack to upstream firstbadlink is 
hn4: 2008-03-28 07:34:44,615 INFO org.apache.hadoop.dfs.DataNode:
Changing block file offset of block blk_7857709233639057851 from 4325376
to 4325376 meta file offset to 33799
hn3: 2008-03-28 07:34:45,304 INFO org.apache.hadoop.dfs.DataNode:
Changing block file offset of block blk_7857709233639057851 from
33353728 to 33357824 meta file offset to 260615
hn3: 2008-03-28 07:34:45,305 INFO org.apache.hadoop.dfs.DataNode:
Exception in receiveBlock for block  java.io.IOException: Trying to
change block file offset of block blk_7857709233639057851 to 33357824
but actual size of file is 33353728
hn1: 2008-03-28 07:35:31,835 INFO org.apache.hadoop.dfs.DataNode:
BlockReport of 564 blocks got processed in 128 msecs

Full thread dump Java HotSpot(TM) 64-Bit Server VM (10.0-b19 mixed
mode):

"ResponseProcessor for block blk_7857709233639057851" prio=10
tid=0x000000005c557800 nid=0x23ad waiting for monitor entry
[0x0000000040e15000..0x0000000040e15a10]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.hadoop.dfs.DFSClient$DFSOutputStream
$ResponseProcessor.run(DFSClient.java:1771)
        - waiting to lock <0x00002aaab43ad910> (a java.util.LinkedList)

"DataStreamer for file /user/properazzi/test/output/index/_0.cfs block
blk_7857709233639057851" prio=10 tid=0x000000005c59f000 nid=0x2392
runnable [0x0000000041219000..0x0000000041219d10]
   java.lang.Thread.State: RUNNABLE
        at java.net.SocketOutputStream.socketWrite0(Native Method)
        at
java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
        at
java.net.SocketOutputStream.write(SocketOutputStream.java:136)
        at
java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
        - locked <0x00002aaade9b8120> (a java.io.BufferedOutputStream)
        at java.io.DataOutputStream.write(DataOutputStream.java:90)
        - locked <0x00002aaade9b8148> (a java.io.DataOutputStream)
        at org.apache.hadoop.dfs.DFSClient$DFSOutputStream
$DataStreamer.run(DFSClient.java:1623)
        - locked <0x00002aaab43ad910> (a java.util.LinkedList)

"org.apache.hadoop.dfs.DFSClient$LeaseChecker@144aa0ce" daemon prio=10
tid=0x000000005c7f1000 nid=0x2254 waiting on condition
[0x0000000041118000..0x0000000041118a90]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
        at java.lang.Thread.sleep(Native Method)
        at org.apache.hadoop.dfs.DFSClient
$LeaseChecker.run(DFSClient.java:597)
        at java.lang.Thread.run(Thread.java:619)

"org.apache.hadoop.dfs.DFSClient$LeaseChecker@2d58f9d3" daemon prio=10
tid=0x000000005c4fec00 nid=0x224f waiting on condition
[0x0000000040f16000..0x0000000040f16c90]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
        at java.lang.Thread.sleep(Native Method)
        at org.apache.hadoop.dfs.DFSClient
$LeaseChecker.run(DFSClient.java:597)
        at java.lang.Thread.run(Thread.java:619)

"org.apache.hadoop.io.ObjectWritable Connection Culler" daemon prio=10
tid=0x000000005c7c5c00 nid=0x224d waiting on condition
[0x0000000040d14000..0x0000000040d14b90]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
        at java.lang.Thread.sleep(Native Method)
        at org.apache.hadoop.ipc.Client
$ConnectionCuller.run(Client.java:423)


"main" prio=10 tid=0x000000005c417000 nid=0x223b waiting for monitor
entry [0x0000000040207000..0x0000000040209ed0]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.hadoop.dfs.DFSClient
$DFSOutputStream.writeChunk(DFSClient.java:2117)
        - waiting to lock <0x00002aaab43ad910> (a java.util.LinkedList)
        at
org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:141)
        at
org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:100)
        at
org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
        - locked <0x00002aaab43addd8> (a org.apache.hadoop.dfs.DFSClient
$DFSOutputStream)
        at org.apache.hadoop.fs.FSDataOutputStream
$PositionCache.write(FSDataOutputStream.java:41)
        at java.io.DataOutputStream.write(DataOutputStream.java:90)
        - locked <0x00002aaab43aef18> (a
org.apache.hadoop.fs.FSDataOutputStream)
        at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:47)
        at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:83)
        at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:157)
        at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:151)
        at
org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1028)
        at
org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1016)
        at
org.apache.hadoop.fs.FileSystem.moveFromLocalFile(FileSystem.java:1006)
        at
org.apache.hadoop.fs.FileSystem.completeLocalOutput(FileSystem.java:1077)
	...

Any Help with that? Ask for more information if needed. 

Thanks, and congratulations for your revolutionary project. 

Iván de Prado Alonso
http://ivandeprado.blogspot.com/




RE: DFS get blocked when writing a file.

Posted by Runping Qi <ru...@yahoo-inc.com>.
This is a know issue:
https://issues.apache.org/jira/browse/HADOOP-3033

Your best bet now is to use 0.16.2 release.

Runping


> -----Original Message-----
> From: Iván de Prado [mailto:ivan.prado.alonso@gmail.com]
> Sent: Friday, March 28, 2008 6:08 AM
> To: core-user@hadoop.apache.org
> Subject: DFS get blocked when writing a file.
> 
> Hello,
> 
> I'm working with Hadoop 0.16.1. I have an issue with the DFS. Sometimes
> when writing to the HDFS it gets blocked. Sometimes it doesn't happen,
> so it's not easily reproducible.
> 
> My cluster have 4 nodes and one master with the NameNode and JobTracker.
> This are the logs that appears when all gets blocked. Look to the block
> blk_7857709233639057851 that seems to be the problematic one. It raises
> the exception:
> 
> "Exception in receiveBlock for block  java.io.IOException: Trying to
> change block file offset of block blk_7857709233639057851 to 33357824
> but actual size of file is 33353728"
> 
> A bigger trace of the logs and a part of the stack trace:
> 
> hn3: 2008-03-28 07:34:44,499 INFO org.apache.hadoop.dfs.DataNode:
> Receiving block blk_7857709233639057851 src: /172.16.3.2:46092
> dest: /172.16.3.2:50010
> hn3: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 2 got response for connect ack  from downstream datanode with
> firstbadlink as
> hn3: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 2 forwarding connect ack to upstream firstbadlink is
> hn2: 2008-03-28 07:34:44,496 INFO org.apache.hadoop.dfs.DataNode:
> Received block blk_8152094109584962620 of size 67108864 from /172.16.3.2
> hn2: 2008-03-28 07:34:44,496 INFO org.apache.hadoop.dfs.DataNode:
> PacketResponder 2 for block blk_8152094109584962620 terminating
> hn2: 2008-03-28 07:34:44,500 INFO org.apache.hadoop.dfs.DataNode:
> Receiving block blk_7857709233639057851 src: /172.16.3.5:35904
> dest: /172.16.3.5:50010
> hn2: 2008-03-28 07:34:44,502 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 1 got response for connect ack  from downstream datanode with
> firstbadlink as
> hn2: 2008-03-28 07:34:44,502 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 1 forwarding connect ack to upstream firstbadlink is
> hn1: 2008-03-28 07:34:44,495 INFO org.apache.hadoop.dfs.DataNode:
> Received block blk_8152094109584962620 of size 67108864 from /172.16.3.4
> hn1: 2008-03-28 07:34:44,495 INFO org.apache.hadoop.dfs.DataNode:
> PacketResponder 1 for block blk_8152094109584962620 terminating
> hn4: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> Receiving block blk_7857709233639057851 src: /172.16.3.4:36887
> dest: /172.16.3.4:50010
> hn4: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 0 forwarding connect ack to upstream firstbadlink is
> hn4: 2008-03-28 07:34:44,615 INFO org.apache.hadoop.dfs.DataNode:
> Changing block file offset of block blk_7857709233639057851 from 4325376
> to 4325376 meta file offset to 33799
> hn3: 2008-03-28 07:34:45,304 INFO org.apache.hadoop.dfs.DataNode:
> Changing block file offset of block blk_7857709233639057851 from
> 33353728 to 33357824 meta file offset to 260615
> hn3: 2008-03-28 07:34:45,305 INFO org.apache.hadoop.dfs.DataNode:
> Exception in receiveBlock for block  java.io.IOException: Trying to
> change block file offset of block blk_7857709233639057851 to 33357824
> but actual size of file is 33353728
> hn1: 2008-03-28 07:35:31,835 INFO org.apache.hadoop.dfs.DataNode:
> BlockReport of 564 blocks got processed in 128 msecs
> 
> Full thread dump Java HotSpot(TM) 64-Bit Server VM (10.0-b19 mixed
> mode):
> 
> "ResponseProcessor for block blk_7857709233639057851" prio=10
> tid=0x000000005c557800 nid=0x23ad waiting for monitor entry
> [0x0000000040e15000..0x0000000040e15a10]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream
> $ResponseProcessor.run(DFSClient.java:1771)
>         - waiting to lock <0x00002aaab43ad910> (a java.util.LinkedList)
> 
> "DataStreamer for file /user/properazzi/test/output/index/_0.cfs block
> blk_7857709233639057851" prio=10 tid=0x000000005c59f000 nid=0x2392
> runnable [0x0000000041219000..0x0000000041219d10]
>    java.lang.Thread.State: RUNNABLE
>         at java.net.SocketOutputStream.socketWrite0(Native Method)
>         at
> java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
>         at
> java.net.SocketOutputStream.write(SocketOutputStream.java:136)
>         at
> java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
>         - locked <0x00002aaade9b8120> (a java.io.BufferedOutputStream)
>         at java.io.DataOutputStream.write(DataOutputStream.java:90)
>         - locked <0x00002aaade9b8148> (a java.io.DataOutputStream)
>         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream
> $DataStreamer.run(DFSClient.java:1623)
>         - locked <0x00002aaab43ad910> (a java.util.LinkedList)
> 
> "org.apache.hadoop.dfs.DFSClient$LeaseChecker@144aa0ce" daemon prio=10
> tid=0x000000005c7f1000 nid=0x2254 waiting on condition
> [0x0000000041118000..0x0000000041118a90]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
>         at java.lang.Thread.sleep(Native Method)
>         at org.apache.hadoop.dfs.DFSClient
> $LeaseChecker.run(DFSClient.java:597)
>         at java.lang.Thread.run(Thread.java:619)
> 
> "org.apache.hadoop.dfs.DFSClient$LeaseChecker@2d58f9d3" daemon prio=10
> tid=0x000000005c4fec00 nid=0x224f waiting on condition
> [0x0000000040f16000..0x0000000040f16c90]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
>         at java.lang.Thread.sleep(Native Method)
>         at org.apache.hadoop.dfs.DFSClient
> $LeaseChecker.run(DFSClient.java:597)
>         at java.lang.Thread.run(Thread.java:619)
> 
> "org.apache.hadoop.io.ObjectWritable Connection Culler" daemon prio=10
> tid=0x000000005c7c5c00 nid=0x224d waiting on condition
> [0x0000000040d14000..0x0000000040d14b90]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
>         at java.lang.Thread.sleep(Native Method)
>         at org.apache.hadoop.ipc.Client
> $ConnectionCuller.run(Client.java:423)
> 
> 
> "main" prio=10 tid=0x000000005c417000 nid=0x223b waiting for monitor
> entry [0x0000000040207000..0x0000000040209ed0]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at org.apache.hadoop.dfs.DFSClient
> $DFSOutputStream.writeChunk(DFSClient.java:2117)
>         - waiting to lock <0x00002aaab43ad910> (a java.util.LinkedList)
>         at
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java
> :141)
>         at
> org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:100)
>         at
> org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
>         - locked <0x00002aaab43addd8> (a org.apache.hadoop.dfs.DFSClient
> $DFSOutputStream)
>         at org.apache.hadoop.fs.FSDataOutputStream
> $PositionCache.write(FSDataOutputStream.java:41)
>         at java.io.DataOutputStream.write(DataOutputStream.java:90)
>         - locked <0x00002aaab43aef18> (a
> org.apache.hadoop.fs.FSDataOutputStream)
>         at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:47)
>         at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:83)
>         at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:157)
>         at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:151)
>         at
> org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1028)
>         at
> org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1016)
>         at
> org.apache.hadoop.fs.FileSystem.moveFromLocalFile(FileSystem.java:1006)
>         at
> org.apache.hadoop.fs.FileSystem.completeLocalOutput(FileSystem.java:1077)
> 	...
> 
> Any Help with that? Ask for more information if needed.
> 
> Thanks, and congratulations for your revolutionary project.
> 
> Iván de Prado Alonso
> http://ivandeprado.blogspot.com/
> 
> 


Re: DFS get blocked when writing a file.

Posted by Iván de Prado <iv...@gmail.com>.
Thanks very much for the help. 

I will investigate more about that. 

Iván

El lun, 31-03-2008 a las 11:11 -0700, Raghu Angadi escribió:
> Iván,
> 
> Whether this was expected or an error depends on what happened on the 
> client. This could happen and would not be a bug if client was killed 
> for some other reason for e.g. But if client is also similarly surprised 
>   then its a different case.
> 
> You could grep for this block in NameNode log and client. If you are 
> still interested in looking into this, I would suggest opening a jira.
> 
> Raghu.
> 
> Iván de Prado wrote:
> > Thanks, 
> > 
> > I have tried with the trunk version and now the exception "Trying to
> > change block file offset of block blk_... to ... but actual size of file
> > is ..." has disappeared and the jobs don't seems to get blocked.
> > 
> > But I have another "Broken Pipe" and "EOF" exceptions in the dfs logs.
> > They seems similar to https://issues.apache.org/jira/browse/HADOOP-2042
> > ticket. The Jobs ends but not sure if they are executed smoothly. are
> > these exceptions normal? As example, the exceptions for the block
> > (6801211507359331627) appears in two nodes (I have 2 as replication) and
> > looks like:
> > 
> > hn2: 2008-03-31 05:03:13,736 INFO org.apache.hadoop.dfs.DataNode:
> > Datanode 0 forwarding connect ack to upstream firstbadlink is 
> > hn2: 2008-03-31 05:03:14,507 INFO org.apache.hadoop.dfs.DataNode:
> > Receiving block blk_6801211507359331627 src: /172.16.3.6:38218
> > dest: /172.16.3.6:50010
> > 
> > hn2: 2008-03-31 05:04:14,528 INFO org.apache.hadoop.dfs.DataNode:
> > Exception in receiveBlock for block blk_6801211507359331627
> > java.io.EOFException
> > hn2: 2008-03-31 05:04:14,528 INFO org.apache.hadoop.dfs.DataNode:
> > PacketResponder 0 for block blk_6801211507359331627 Interrupted.
> > hn2: 2008-03-31 05:04:14,528 INFO org.apache.hadoop.dfs.DataNode:
> > PacketResponder 0 for block blk_6801211507359331627 terminating
> > hn2: 2008-03-31 05:04:14,530 INFO org.apache.hadoop.dfs.DataNode:
> > writeBlock blk_6801211507359331627 received exception
> > java.io.EOFException
> > hn2: 2008-03-31 05:04:14,530 ERROR org.apache.hadoop.dfs.DataNode:
> > 172.16.3.4:50010:DataXceiver: java.io.EOFException
> > hn2:    at java.io.DataInputStream.readInt(DataInputStream.java:375)
> > hn2:    at org.apache.hadoop.dfs.DataNode
> > $BlockReceiver.receiveBlock(DataNode.java:2243)
> > hn2:    at org.apache.hadoop.dfs.DataNode
> > $DataXceiver.writeBlock(DataNode.java:1157)
> > hn2:    at org.apache.hadoop.dfs.DataNode
> > $DataXceiver.run(DataNode.java:938)
> > hn2:    at java.lang.Thread.run(Thread.java:619)
> > 
> > hn4: 2008-03-31 05:03:13,590 INFO org.apache.hadoop.dfs.DataNode:
> > Datanode 0 forwarding connect ack to upstream firstbadlink is 
> > hn4: 2008-03-31 05:03:14,506 INFO org.apache.hadoop.dfs.DataNode:
> > Receiving block blk_6801211507359331627 src: /172.16.3.6:41112
> > dest: /172.16.3.6:50010
> > 
> > hn4: 2008-03-31 05:03:26,825 INFO org.apache.hadoop.dfs.DataNode:
> > Exception in receiveBlock for block blk_6801211507359331627
> > java.io.EOFException
> > 
> > hn4: 2008-03-31 05:04:14,524 INFO org.apache.hadoop.dfs.DataNode:
> > PacketResponder blk_6801211507359331627 1 Exception
> > java.net.SocketException: Broken pipe
> > hn4:    at java.net.SocketOutputStream.socketWrite0(Native Method)
> > hn4:    at
> > java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
> > hn4:    at
> > java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> > hn4:    at java.io.DataOutputStream.writeLong(DataOutputStream.java:207)
> > hn4:    at org.apache.hadoop.dfs.DataNode
> > $PacketResponder.run(DataNode.java:1825)
> > hn4:    at java.lang.Thread.run(Thread.java:619)
> > hn4: 
> > hn4: 2008-03-31 05:04:14,525 INFO org.apache.hadoop.dfs.DataNode:
> > PacketResponder 1 for block blk_6801211507359331627 terminating
> > hn4: 2008-03-31 05:04:14,525 INFO org.apache.hadoop.dfs.DataNode:
> > writeBlock blk_6801211507359331627 received exception
> > java.io.EOFException
> > hn4: 2008-03-31 05:04:14,526 ERROR org.apache.hadoop.dfs.DataNode:
> > 172.16.3.6:50010:DataXceiver: java.io.EOFException
> > hn4:    at java.io.DataInputStream.readInt(DataInputStream.java:375)
> > hn4:    at org.apache.hadoop.dfs.DataNode
> > $BlockReceiver.receiveBlock(DataNode.java:2243)
> > hn4:    at org.apache.hadoop.dfs.DataNode
> > $DataXceiver.writeBlock(DataNode.java:1157)
> > hn4:    at org.apache.hadoop.dfs.DataNode
> > $DataXceiver.run(DataNode.java:938)
> > hn4:    at java.lang.Thread.run(Thread.java:619)
> > hn4: 
> > 
> > Many thanks, 
> > 
> > Iván de Prado Alonso
> > http://ivandeprado.blogspot.com/


Re: DFS get blocked when writing a file.

Posted by Raghu Angadi <ra...@yahoo-inc.com>.
Iván,

Whether this was expected or an error depends on what happened on the 
client. This could happen and would not be a bug if client was killed 
for some other reason for e.g. But if client is also similarly surprised 
  then its a different case.

You could grep for this block in NameNode log and client. If you are 
still interested in looking into this, I would suggest opening a jira.

Raghu.

Iván de Prado wrote:
> Thanks, 
> 
> I have tried with the trunk version and now the exception "Trying to
> change block file offset of block blk_... to ... but actual size of file
> is ..." has disappeared and the jobs don't seems to get blocked.
> 
> But I have another "Broken Pipe" and "EOF" exceptions in the dfs logs.
> They seems similar to https://issues.apache.org/jira/browse/HADOOP-2042
> ticket. The Jobs ends but not sure if they are executed smoothly. are
> these exceptions normal? As example, the exceptions for the block
> (6801211507359331627) appears in two nodes (I have 2 as replication) and
> looks like:
> 
> hn2: 2008-03-31 05:03:13,736 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 0 forwarding connect ack to upstream firstbadlink is 
> hn2: 2008-03-31 05:03:14,507 INFO org.apache.hadoop.dfs.DataNode:
> Receiving block blk_6801211507359331627 src: /172.16.3.6:38218
> dest: /172.16.3.6:50010
> 
> hn2: 2008-03-31 05:04:14,528 INFO org.apache.hadoop.dfs.DataNode:
> Exception in receiveBlock for block blk_6801211507359331627
> java.io.EOFException
> hn2: 2008-03-31 05:04:14,528 INFO org.apache.hadoop.dfs.DataNode:
> PacketResponder 0 for block blk_6801211507359331627 Interrupted.
> hn2: 2008-03-31 05:04:14,528 INFO org.apache.hadoop.dfs.DataNode:
> PacketResponder 0 for block blk_6801211507359331627 terminating
> hn2: 2008-03-31 05:04:14,530 INFO org.apache.hadoop.dfs.DataNode:
> writeBlock blk_6801211507359331627 received exception
> java.io.EOFException
> hn2: 2008-03-31 05:04:14,530 ERROR org.apache.hadoop.dfs.DataNode:
> 172.16.3.4:50010:DataXceiver: java.io.EOFException
> hn2:    at java.io.DataInputStream.readInt(DataInputStream.java:375)
> hn2:    at org.apache.hadoop.dfs.DataNode
> $BlockReceiver.receiveBlock(DataNode.java:2243)
> hn2:    at org.apache.hadoop.dfs.DataNode
> $DataXceiver.writeBlock(DataNode.java:1157)
> hn2:    at org.apache.hadoop.dfs.DataNode
> $DataXceiver.run(DataNode.java:938)
> hn2:    at java.lang.Thread.run(Thread.java:619)
> 
> hn4: 2008-03-31 05:03:13,590 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 0 forwarding connect ack to upstream firstbadlink is 
> hn4: 2008-03-31 05:03:14,506 INFO org.apache.hadoop.dfs.DataNode:
> Receiving block blk_6801211507359331627 src: /172.16.3.6:41112
> dest: /172.16.3.6:50010
> 
> hn4: 2008-03-31 05:03:26,825 INFO org.apache.hadoop.dfs.DataNode:
> Exception in receiveBlock for block blk_6801211507359331627
> java.io.EOFException
> 
> hn4: 2008-03-31 05:04:14,524 INFO org.apache.hadoop.dfs.DataNode:
> PacketResponder blk_6801211507359331627 1 Exception
> java.net.SocketException: Broken pipe
> hn4:    at java.net.SocketOutputStream.socketWrite0(Native Method)
> hn4:    at
> java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
> hn4:    at
> java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> hn4:    at java.io.DataOutputStream.writeLong(DataOutputStream.java:207)
> hn4:    at org.apache.hadoop.dfs.DataNode
> $PacketResponder.run(DataNode.java:1825)
> hn4:    at java.lang.Thread.run(Thread.java:619)
> hn4: 
> hn4: 2008-03-31 05:04:14,525 INFO org.apache.hadoop.dfs.DataNode:
> PacketResponder 1 for block blk_6801211507359331627 terminating
> hn4: 2008-03-31 05:04:14,525 INFO org.apache.hadoop.dfs.DataNode:
> writeBlock blk_6801211507359331627 received exception
> java.io.EOFException
> hn4: 2008-03-31 05:04:14,526 ERROR org.apache.hadoop.dfs.DataNode:
> 172.16.3.6:50010:DataXceiver: java.io.EOFException
> hn4:    at java.io.DataInputStream.readInt(DataInputStream.java:375)
> hn4:    at org.apache.hadoop.dfs.DataNode
> $BlockReceiver.receiveBlock(DataNode.java:2243)
> hn4:    at org.apache.hadoop.dfs.DataNode
> $DataXceiver.writeBlock(DataNode.java:1157)
> hn4:    at org.apache.hadoop.dfs.DataNode
> $DataXceiver.run(DataNode.java:938)
> hn4:    at java.lang.Thread.run(Thread.java:619)
> hn4: 
> 
> Many thanks, 
> 
> Iván de Prado Alonso
> http://ivandeprado.blogspot.com/

Re: DFS get blocked when writing a file.

Posted by Iván de Prado <iv...@gmail.com>.
Thanks, 

I have tried with the trunk version and now the exception "Trying to
change block file offset of block blk_... to ... but actual size of file
is ..." has disappeared and the jobs don't seems to get blocked.

But I have another "Broken Pipe" and "EOF" exceptions in the dfs logs.
They seems similar to https://issues.apache.org/jira/browse/HADOOP-2042
ticket. The Jobs ends but not sure if they are executed smoothly. are
these exceptions normal? As example, the exceptions for the block
(6801211507359331627) appears in two nodes (I have 2 as replication) and
looks like:

hn2: 2008-03-31 05:03:13,736 INFO org.apache.hadoop.dfs.DataNode:
Datanode 0 forwarding connect ack to upstream firstbadlink is 
hn2: 2008-03-31 05:03:14,507 INFO org.apache.hadoop.dfs.DataNode:
Receiving block blk_6801211507359331627 src: /172.16.3.6:38218
dest: /172.16.3.6:50010

hn2: 2008-03-31 05:04:14,528 INFO org.apache.hadoop.dfs.DataNode:
Exception in receiveBlock for block blk_6801211507359331627
java.io.EOFException
hn2: 2008-03-31 05:04:14,528 INFO org.apache.hadoop.dfs.DataNode:
PacketResponder 0 for block blk_6801211507359331627 Interrupted.
hn2: 2008-03-31 05:04:14,528 INFO org.apache.hadoop.dfs.DataNode:
PacketResponder 0 for block blk_6801211507359331627 terminating
hn2: 2008-03-31 05:04:14,530 INFO org.apache.hadoop.dfs.DataNode:
writeBlock blk_6801211507359331627 received exception
java.io.EOFException
hn2: 2008-03-31 05:04:14,530 ERROR org.apache.hadoop.dfs.DataNode:
172.16.3.4:50010:DataXceiver: java.io.EOFException
hn2:    at java.io.DataInputStream.readInt(DataInputStream.java:375)
hn2:    at org.apache.hadoop.dfs.DataNode
$BlockReceiver.receiveBlock(DataNode.java:2243)
hn2:    at org.apache.hadoop.dfs.DataNode
$DataXceiver.writeBlock(DataNode.java:1157)
hn2:    at org.apache.hadoop.dfs.DataNode
$DataXceiver.run(DataNode.java:938)
hn2:    at java.lang.Thread.run(Thread.java:619)

hn4: 2008-03-31 05:03:13,590 INFO org.apache.hadoop.dfs.DataNode:
Datanode 0 forwarding connect ack to upstream firstbadlink is 
hn4: 2008-03-31 05:03:14,506 INFO org.apache.hadoop.dfs.DataNode:
Receiving block blk_6801211507359331627 src: /172.16.3.6:41112
dest: /172.16.3.6:50010

hn4: 2008-03-31 05:03:26,825 INFO org.apache.hadoop.dfs.DataNode:
Exception in receiveBlock for block blk_6801211507359331627
java.io.EOFException

hn4: 2008-03-31 05:04:14,524 INFO org.apache.hadoop.dfs.DataNode:
PacketResponder blk_6801211507359331627 1 Exception
java.net.SocketException: Broken pipe
hn4:    at java.net.SocketOutputStream.socketWrite0(Native Method)
hn4:    at
java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
hn4:    at
java.net.SocketOutputStream.write(SocketOutputStream.java:136)
hn4:    at java.io.DataOutputStream.writeLong(DataOutputStream.java:207)
hn4:    at org.apache.hadoop.dfs.DataNode
$PacketResponder.run(DataNode.java:1825)
hn4:    at java.lang.Thread.run(Thread.java:619)
hn4: 
hn4: 2008-03-31 05:04:14,525 INFO org.apache.hadoop.dfs.DataNode:
PacketResponder 1 for block blk_6801211507359331627 terminating
hn4: 2008-03-31 05:04:14,525 INFO org.apache.hadoop.dfs.DataNode:
writeBlock blk_6801211507359331627 received exception
java.io.EOFException
hn4: 2008-03-31 05:04:14,526 ERROR org.apache.hadoop.dfs.DataNode:
172.16.3.6:50010:DataXceiver: java.io.EOFException
hn4:    at java.io.DataInputStream.readInt(DataInputStream.java:375)
hn4:    at org.apache.hadoop.dfs.DataNode
$BlockReceiver.receiveBlock(DataNode.java:2243)
hn4:    at org.apache.hadoop.dfs.DataNode
$DataXceiver.writeBlock(DataNode.java:1157)
hn4:    at org.apache.hadoop.dfs.DataNode
$DataXceiver.run(DataNode.java:938)
hn4:    at java.lang.Thread.run(Thread.java:619)
hn4: 

Many thanks, 

Iván de Prado Alonso
http://ivandeprado.blogspot.com/


El vie, 28-03-2008 a las 07:42 -0700, Raghu Angadi escribió:
> > "Exception in receiveBlock for block  java.io.IOException: Trying to
>  > change block file offset of block blk_7857709233639057851 to 33357824
>  > but actual size of file is 33353728"
> 
> This was fixed in HADOOP-3033. You can try running latest 0.16 branch 
> (svn...hadoop/core/branches/branch-016). 0.16.2 release is scheduled for 
> early next week.
> 
> This exception does not fully explain blocked client. If the client 
> blocks again with latest 0.16 branch, could you include stacktraces on 
> datanodes also? You could file a jira so that it is convenient to attach 
> logs and stacktrace.
> 
> Raghu.
> 
> Iván de Prado wrote:
> > Hello, 
> > 
> > I'm working with Hadoop 0.16.1. I have an issue with the DFS. Sometimes
> > when writing to the HDFS it gets blocked. Sometimes it doesn't happen,
> > so it's not easily reproducible. 
> > 
> > My cluster have 4 nodes and one master with the NameNode and JobTracker.
> > This are the logs that appears when all gets blocked. Look to the block
> > blk_7857709233639057851 that seems to be the problematic one. It raises
> > the exception:
> > 
> > "Exception in receiveBlock for block  java.io.IOException: Trying to
> > change block file offset of block blk_7857709233639057851 to 33357824
> > but actual size of file is 33353728"
> > 
> > A bigger trace of the logs and a part of the stack trace:
> > 
> > hn3: 2008-03-28 07:34:44,499 INFO org.apache.hadoop.dfs.DataNode:
> > Receiving block blk_7857709233639057851 src: /172.16.3.2:46092
> > dest: /172.16.3.2:50010
> > hn3: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> > Datanode 2 got response for connect ack  from downstream datanode with
> > firstbadlink as 
> > hn3: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> > Datanode 2 forwarding connect ack to upstream firstbadlink is 
> > hn2: 2008-03-28 07:34:44,496 INFO org.apache.hadoop.dfs.DataNode:
> > Received block blk_8152094109584962620 of size 67108864 from /172.16.3.2
> > hn2: 2008-03-28 07:34:44,496 INFO org.apache.hadoop.dfs.DataNode:
> > PacketResponder 2 for block blk_8152094109584962620 terminating
> > hn2: 2008-03-28 07:34:44,500 INFO org.apache.hadoop.dfs.DataNode:
> > Receiving block blk_7857709233639057851 src: /172.16.3.5:35904
> > dest: /172.16.3.5:50010
> > hn2: 2008-03-28 07:34:44,502 INFO org.apache.hadoop.dfs.DataNode:
> > Datanode 1 got response for connect ack  from downstream datanode with
> > firstbadlink as 
> > hn2: 2008-03-28 07:34:44,502 INFO org.apache.hadoop.dfs.DataNode:
> > Datanode 1 forwarding connect ack to upstream firstbadlink is 
> > hn1: 2008-03-28 07:34:44,495 INFO org.apache.hadoop.dfs.DataNode:
> > Received block blk_8152094109584962620 of size 67108864 from /172.16.3.4
> > hn1: 2008-03-28 07:34:44,495 INFO org.apache.hadoop.dfs.DataNode:
> > PacketResponder 1 for block blk_8152094109584962620 terminating
> > hn4: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> > Receiving block blk_7857709233639057851 src: /172.16.3.4:36887
> > dest: /172.16.3.4:50010
> > hn4: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> > Datanode 0 forwarding connect ack to upstream firstbadlink is 
> > hn4: 2008-03-28 07:34:44,615 INFO org.apache.hadoop.dfs.DataNode:
> > Changing block file offset of block blk_7857709233639057851 from 4325376
> > to 4325376 meta file offset to 33799
> > hn3: 2008-03-28 07:34:45,304 INFO org.apache.hadoop.dfs.DataNode:
> > Changing block file offset of block blk_7857709233639057851 from
> > 33353728 to 33357824 meta file offset to 260615
> > hn3: 2008-03-28 07:34:45,305 INFO org.apache.hadoop.dfs.DataNode:
> > Exception in receiveBlock for block  java.io.IOException: Trying to
> > change block file offset of block blk_7857709233639057851 to 33357824
> > but actual size of file is 33353728
> > hn1: 2008-03-28 07:35:31,835 INFO org.apache.hadoop.dfs.DataNode:
> > BlockReport of 564 blocks got processed in 128 msecs
> > 
> > Full thread dump Java HotSpot(TM) 64-Bit Server VM (10.0-b19 mixed
> > mode):
> > 
> > "ResponseProcessor for block blk_7857709233639057851" prio=10
> > tid=0x000000005c557800 nid=0x23ad waiting for monitor entry
> > [0x0000000040e15000..0x0000000040e15a10]
> >    java.lang.Thread.State: BLOCKED (on object monitor)
> >         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream
> > $ResponseProcessor.run(DFSClient.java:1771)
> >         - waiting to lock <0x00002aaab43ad910> (a java.util.LinkedList)
> > 
> > "DataStreamer for file /user/properazzi/test/output/index/_0.cfs block
> > blk_7857709233639057851" prio=10 tid=0x000000005c59f000 nid=0x2392
> > runnable [0x0000000041219000..0x0000000041219d10]
> >    java.lang.Thread.State: RUNNABLE
> >         at java.net.SocketOutputStream.socketWrite0(Native Method)
> >         at
> > java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
> >         at
> > java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> >         at
> > java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
> >         - locked <0x00002aaade9b8120> (a java.io.BufferedOutputStream)
> >         at java.io.DataOutputStream.write(DataOutputStream.java:90)
> >         - locked <0x00002aaade9b8148> (a java.io.DataOutputStream)
> >         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream
> > $DataStreamer.run(DFSClient.java:1623)
> >         - locked <0x00002aaab43ad910> (a java.util.LinkedList)
> > 
> > "org.apache.hadoop.dfs.DFSClient$LeaseChecker@144aa0ce" daemon prio=10
> > tid=0x000000005c7f1000 nid=0x2254 waiting on condition
> > [0x0000000041118000..0x0000000041118a90]
> >    java.lang.Thread.State: TIMED_WAITING (sleeping)
> >         at java.lang.Thread.sleep(Native Method)
> >         at org.apache.hadoop.dfs.DFSClient
> > $LeaseChecker.run(DFSClient.java:597)
> >         at java.lang.Thread.run(Thread.java:619)
> > 
> > "org.apache.hadoop.dfs.DFSClient$LeaseChecker@2d58f9d3" daemon prio=10
> > tid=0x000000005c4fec00 nid=0x224f waiting on condition
> > [0x0000000040f16000..0x0000000040f16c90]
> >    java.lang.Thread.State: TIMED_WAITING (sleeping)
> >         at java.lang.Thread.sleep(Native Method)
> >         at org.apache.hadoop.dfs.DFSClient
> > $LeaseChecker.run(DFSClient.java:597)
> >         at java.lang.Thread.run(Thread.java:619)
> > 
> > "org.apache.hadoop.io.ObjectWritable Connection Culler" daemon prio=10
> > tid=0x000000005c7c5c00 nid=0x224d waiting on condition
> > [0x0000000040d14000..0x0000000040d14b90]
> >    java.lang.Thread.State: TIMED_WAITING (sleeping)
> >         at java.lang.Thread.sleep(Native Method)
> >         at org.apache.hadoop.ipc.Client
> > $ConnectionCuller.run(Client.java:423)
> > 
> > 
> > "main" prio=10 tid=0x000000005c417000 nid=0x223b waiting for monitor
> > entry [0x0000000040207000..0x0000000040209ed0]
> >    java.lang.Thread.State: BLOCKED (on object monitor)
> >         at org.apache.hadoop.dfs.DFSClient
> > $DFSOutputStream.writeChunk(DFSClient.java:2117)
> >         - waiting to lock <0x00002aaab43ad910> (a java.util.LinkedList)
> >         at
> > org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:141)
> >         at
> > org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:100)
> >         at
> > org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
> >         - locked <0x00002aaab43addd8> (a org.apache.hadoop.dfs.DFSClient
> > $DFSOutputStream)
> >         at org.apache.hadoop.fs.FSDataOutputStream
> > $PositionCache.write(FSDataOutputStream.java:41)
> >         at java.io.DataOutputStream.write(DataOutputStream.java:90)
> >         - locked <0x00002aaab43aef18> (a
> > org.apache.hadoop.fs.FSDataOutputStream)
> >         at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:47)
> >         at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:83)
> >         at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:157)
> >         at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:151)
> >         at
> > org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1028)
> >         at
> > org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1016)
> >         at
> > org.apache.hadoop.fs.FileSystem.moveFromLocalFile(FileSystem.java:1006)
> >         at
> > org.apache.hadoop.fs.FileSystem.completeLocalOutput(FileSystem.java:1077)
> > 	...
> > 
> > Any Help with that? Ask for more information if needed. 
> > 
> > Thanks, and congratulations for your revolutionary project. 
> > 
> > Iván de Prado Alonso
> > http://ivandeprado.blogspot.com/
> > 
> > 
> > 
> 


Re: DFS get blocked when writing a file.

Posted by Raghu Angadi <ra...@yahoo-inc.com>.
 > "Exception in receiveBlock for block  java.io.IOException: Trying to
 > change block file offset of block blk_7857709233639057851 to 33357824
 > but actual size of file is 33353728"

This was fixed in HADOOP-3033. You can try running latest 0.16 branch 
(svn...hadoop/core/branches/branch-016). 0.16.2 release is scheduled for 
early next week.

This exception does not fully explain blocked client. If the client 
blocks again with latest 0.16 branch, could you include stacktraces on 
datanodes also? You could file a jira so that it is convenient to attach 
logs and stacktrace.

Raghu.

Iván de Prado wrote:
> Hello, 
> 
> I'm working with Hadoop 0.16.1. I have an issue with the DFS. Sometimes
> when writing to the HDFS it gets blocked. Sometimes it doesn't happen,
> so it's not easily reproducible. 
> 
> My cluster have 4 nodes and one master with the NameNode and JobTracker.
> This are the logs that appears when all gets blocked. Look to the block
> blk_7857709233639057851 that seems to be the problematic one. It raises
> the exception:
> 
> "Exception in receiveBlock for block  java.io.IOException: Trying to
> change block file offset of block blk_7857709233639057851 to 33357824
> but actual size of file is 33353728"
> 
> A bigger trace of the logs and a part of the stack trace:
> 
> hn3: 2008-03-28 07:34:44,499 INFO org.apache.hadoop.dfs.DataNode:
> Receiving block blk_7857709233639057851 src: /172.16.3.2:46092
> dest: /172.16.3.2:50010
> hn3: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 2 got response for connect ack  from downstream datanode with
> firstbadlink as 
> hn3: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 2 forwarding connect ack to upstream firstbadlink is 
> hn2: 2008-03-28 07:34:44,496 INFO org.apache.hadoop.dfs.DataNode:
> Received block blk_8152094109584962620 of size 67108864 from /172.16.3.2
> hn2: 2008-03-28 07:34:44,496 INFO org.apache.hadoop.dfs.DataNode:
> PacketResponder 2 for block blk_8152094109584962620 terminating
> hn2: 2008-03-28 07:34:44,500 INFO org.apache.hadoop.dfs.DataNode:
> Receiving block blk_7857709233639057851 src: /172.16.3.5:35904
> dest: /172.16.3.5:50010
> hn2: 2008-03-28 07:34:44,502 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 1 got response for connect ack  from downstream datanode with
> firstbadlink as 
> hn2: 2008-03-28 07:34:44,502 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 1 forwarding connect ack to upstream firstbadlink is 
> hn1: 2008-03-28 07:34:44,495 INFO org.apache.hadoop.dfs.DataNode:
> Received block blk_8152094109584962620 of size 67108864 from /172.16.3.4
> hn1: 2008-03-28 07:34:44,495 INFO org.apache.hadoop.dfs.DataNode:
> PacketResponder 1 for block blk_8152094109584962620 terminating
> hn4: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> Receiving block blk_7857709233639057851 src: /172.16.3.4:36887
> dest: /172.16.3.4:50010
> hn4: 2008-03-28 07:34:44,501 INFO org.apache.hadoop.dfs.DataNode:
> Datanode 0 forwarding connect ack to upstream firstbadlink is 
> hn4: 2008-03-28 07:34:44,615 INFO org.apache.hadoop.dfs.DataNode:
> Changing block file offset of block blk_7857709233639057851 from 4325376
> to 4325376 meta file offset to 33799
> hn3: 2008-03-28 07:34:45,304 INFO org.apache.hadoop.dfs.DataNode:
> Changing block file offset of block blk_7857709233639057851 from
> 33353728 to 33357824 meta file offset to 260615
> hn3: 2008-03-28 07:34:45,305 INFO org.apache.hadoop.dfs.DataNode:
> Exception in receiveBlock for block  java.io.IOException: Trying to
> change block file offset of block blk_7857709233639057851 to 33357824
> but actual size of file is 33353728
> hn1: 2008-03-28 07:35:31,835 INFO org.apache.hadoop.dfs.DataNode:
> BlockReport of 564 blocks got processed in 128 msecs
> 
> Full thread dump Java HotSpot(TM) 64-Bit Server VM (10.0-b19 mixed
> mode):
> 
> "ResponseProcessor for block blk_7857709233639057851" prio=10
> tid=0x000000005c557800 nid=0x23ad waiting for monitor entry
> [0x0000000040e15000..0x0000000040e15a10]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream
> $ResponseProcessor.run(DFSClient.java:1771)
>         - waiting to lock <0x00002aaab43ad910> (a java.util.LinkedList)
> 
> "DataStreamer for file /user/properazzi/test/output/index/_0.cfs block
> blk_7857709233639057851" prio=10 tid=0x000000005c59f000 nid=0x2392
> runnable [0x0000000041219000..0x0000000041219d10]
>    java.lang.Thread.State: RUNNABLE
>         at java.net.SocketOutputStream.socketWrite0(Native Method)
>         at
> java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
>         at
> java.net.SocketOutputStream.write(SocketOutputStream.java:136)
>         at
> java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
>         - locked <0x00002aaade9b8120> (a java.io.BufferedOutputStream)
>         at java.io.DataOutputStream.write(DataOutputStream.java:90)
>         - locked <0x00002aaade9b8148> (a java.io.DataOutputStream)
>         at org.apache.hadoop.dfs.DFSClient$DFSOutputStream
> $DataStreamer.run(DFSClient.java:1623)
>         - locked <0x00002aaab43ad910> (a java.util.LinkedList)
> 
> "org.apache.hadoop.dfs.DFSClient$LeaseChecker@144aa0ce" daemon prio=10
> tid=0x000000005c7f1000 nid=0x2254 waiting on condition
> [0x0000000041118000..0x0000000041118a90]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
>         at java.lang.Thread.sleep(Native Method)
>         at org.apache.hadoop.dfs.DFSClient
> $LeaseChecker.run(DFSClient.java:597)
>         at java.lang.Thread.run(Thread.java:619)
> 
> "org.apache.hadoop.dfs.DFSClient$LeaseChecker@2d58f9d3" daemon prio=10
> tid=0x000000005c4fec00 nid=0x224f waiting on condition
> [0x0000000040f16000..0x0000000040f16c90]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
>         at java.lang.Thread.sleep(Native Method)
>         at org.apache.hadoop.dfs.DFSClient
> $LeaseChecker.run(DFSClient.java:597)
>         at java.lang.Thread.run(Thread.java:619)
> 
> "org.apache.hadoop.io.ObjectWritable Connection Culler" daemon prio=10
> tid=0x000000005c7c5c00 nid=0x224d waiting on condition
> [0x0000000040d14000..0x0000000040d14b90]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
>         at java.lang.Thread.sleep(Native Method)
>         at org.apache.hadoop.ipc.Client
> $ConnectionCuller.run(Client.java:423)
> 
> 
> "main" prio=10 tid=0x000000005c417000 nid=0x223b waiting for monitor
> entry [0x0000000040207000..0x0000000040209ed0]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at org.apache.hadoop.dfs.DFSClient
> $DFSOutputStream.writeChunk(DFSClient.java:2117)
>         - waiting to lock <0x00002aaab43ad910> (a java.util.LinkedList)
>         at
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:141)
>         at
> org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:100)
>         at
> org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86)
>         - locked <0x00002aaab43addd8> (a org.apache.hadoop.dfs.DFSClient
> $DFSOutputStream)
>         at org.apache.hadoop.fs.FSDataOutputStream
> $PositionCache.write(FSDataOutputStream.java:41)
>         at java.io.DataOutputStream.write(DataOutputStream.java:90)
>         - locked <0x00002aaab43aef18> (a
> org.apache.hadoop.fs.FSDataOutputStream)
>         at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:47)
>         at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:83)
>         at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:157)
>         at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:151)
>         at
> org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1028)
>         at
> org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:1016)
>         at
> org.apache.hadoop.fs.FileSystem.moveFromLocalFile(FileSystem.java:1006)
>         at
> org.apache.hadoop.fs.FileSystem.completeLocalOutput(FileSystem.java:1077)
> 	...
> 
> Any Help with that? Ask for more information if needed. 
> 
> Thanks, and congratulations for your revolutionary project. 
> 
> Iván de Prado Alonso
> http://ivandeprado.blogspot.com/
> 
> 
>