[
https://issues.apache.org/jira/browse/HDFS-7358?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14199077#comment-14199077
]
stack commented on HDFS-7358:
-----------------------------
See how we are 'Waiting for ack for: 42' twice in the below log snippet though
we wrote out a seq no. 43. At about same time the allocate/recycle numbering
goes 'off' at this time because its waiting on an ack that doesn't ever arrive
so there is an outstanding allocation with a corresponding recycle that will
never come (Should + one.releaseBuffer(byteArrayManager); be inside a finally
block?) If I run with one thread only, I don't see this issue. It is only with
two or more. My little program has 5 threads writing and calling sync.
I turned this feature off and see that we are skipping ack numbers from time to
time so this is problem is not brought on by this feature but you can't use
this feature till its fixed. Looking...
{code}
...
2014-11-05 11:16:47,293 DEBUG [sync.0] util.ByteArrayManager: allocate(65565):
count=43, aboveThreshold, [131072: 1/10, free=1], recycled? true
2014-11-05 11:16:47,293 DEBUG [sync.0] hdfs.DFSClient: DFSClient writeChunk
allocating new packet seqno=41,
src=/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083,
packetSize=65532, chunksPerPacket=127, bytesCurBlock=31232
2014-11-05 11:16:47,293 DEBUG [sync.0] hdfs.DFSClient: DFSClient flush() :
bytesCurBlock 32088 lastFlushOffset 31579
2014-11-05 11:16:47,293 DEBUG [sync.0] hdfs.DFSClient: Queued packet 41
2014-11-05 11:16:47,293 DEBUG [sync.0] hdfs.DFSClient: Waiting for ack for: 41
2014-11-05 11:16:47,293 DEBUG [DataStreamer for file
/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083
block BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
hdfs.DFSClient: DataStreamer block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940 sending
packet packet seqno:41 offsetInBlock:31232 lastPacketInBlock:false
lastByteOffsetInBlock: 32088
2014-11-05 11:16:47,294 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
hdfs.DFSClient: DFSClient seqno: 40 status: SUCCESS status: SUCCESS status:
SUCCESS downstreamAckTimeNanos: 487791
2014-11-05 11:16:47,294 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
util.ByteArrayManager: recycle: array.length=131072, [131072: 2/10, free=0],
freeQueue.offer, freeQueueSize=1
2014-11-05 11:16:47,294 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
hdfs.DFSClient: DFSClient seqno: 41 status: SUCCESS status: SUCCESS status:
SUCCESS downstreamAckTimeNanos: 465086
2014-11-05 11:16:47,294 DEBUG [sync.1] util.ByteArrayManager: allocate(65565):
count=44, aboveThreshold, [131072: 1/10, free=1], recycled? true
2014-11-05 11:16:47,295 DEBUG [sync.1] hdfs.DFSClient: DFSClient writeChunk
allocating new packet seqno=42,
src=/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083,
packetSize=65532, chunksPerPacket=127, bytesCurBlock=31744
2014-11-05 11:16:47,295 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
util.ByteArrayManager: recycle: array.length=131072, [131072: 2/10, free=0],
freeQueue.offer, freeQueueSize=1
2014-11-05 11:16:47,295 DEBUG [sync.1] hdfs.DFSClient: DFSClient flush() :
bytesCurBlock 32853 lastFlushOffset 32088
2014-11-05 11:16:47,295 DEBUG [sync.1] hdfs.DFSClient: Queued packet 42
2014-11-05 11:16:47,295 DEBUG [sync.1] hdfs.DFSClient: Waiting for ack for: 42
2014-11-05 11:16:47,295 DEBUG [DataStreamer for file
/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083
block BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
hdfs.DFSClient: DataStreamer block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940 sending
packet packet seqno:42 offsetInBlock:31744 lastPacketInBlock:false
lastByteOffsetInBlock: 32853
2014-11-05 11:16:47,295 DEBUG [sync.0] util.ByteArrayManager: allocate(65565):
count=45, aboveThreshold, [131072: 1/10, free=1], recycled? true
2014-11-05 11:16:47,295 DEBUG [sync.0] hdfs.DFSClient: DFSClient writeChunk
allocating new packet seqno=43,
src=/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083,
packetSize=65532, chunksPerPacket=127, bytesCurBlock=32768
2014-11-05 11:16:47,295 DEBUG [sync.0] hdfs.DFSClient: DFSClient flush() :
bytesCurBlock 32853 lastFlushOffset 32853
2014-11-05 11:16:47,295 DEBUG [sync.0] hdfs.DFSClient: Waiting for ack for: 42
2014-11-05 11:16:47,296 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
hdfs.DFSClient: DFSClient seqno: 42 status: SUCCESS status: SUCCESS status:
SUCCESS downstreamAckTimeNanos: 504509
2014-11-05 11:16:47,296 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
util.ByteArrayManager: recycle: array.length=131072, [131072: 2/10, free=0],
freeQueue.offer, freeQueueSize=1
2014-11-05 11:16:47,296 DEBUG [sync.0] util.ByteArrayManager: allocate(65565):
count=46, aboveThreshold, [131072: 1/10, free=1], recycled? true
2014-11-05 11:16:47,296 DEBUG [sync.0] hdfs.DFSClient: DFSClient writeChunk
allocating new packet seqno=44,
src=/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083,
packetSize=65532, chunksPerPacket=127, bytesCurBlock=32768
2014-11-05 11:16:47,296 DEBUG [sync.0] hdfs.DFSClient: DFSClient flush() :
bytesCurBlock 34208 lastFlushOffset 32853
2014-11-05 11:16:47,296 DEBUG [sync.0] hdfs.DFSClient: Queued packet 44
2014-11-05 11:16:47,297 DEBUG [sync.0] hdfs.DFSClient: Waiting for ack for: 44
2014-11-05 11:16:47,297 DEBUG [DataStreamer for file
/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083
block BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
hdfs.DFSClient: DataStreamer block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940 sending
packet packet seqno:44 offsetInBlock:32768 lastPacketInBlock:false
lastByteOffsetInBlock: 34208
2014-11-05 11:16:47,297 DEBUG [sync.1] util.ByteArrayManager: allocate(65565):
count=47, aboveThreshold, [131072: 2/10, free=0], recycled? false
2014-11-05 11:16:47,297 DEBUG [sync.1] hdfs.DFSClient: DFSClient writeChunk
allocating new packet seqno=45,
src=/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083,
packetSize=65532, chunksPerPacket=127, bytesCurBlock=33792
2014-11-05 11:16:47,297 DEBUG [sync.1] hdfs.DFSClient: DFSClient flush() :
bytesCurBlock 34383 lastFlushOffset 34208
2014-11-05 11:16:47,297 DEBUG [sync.1] hdfs.DFSClient: Queued packet 45
2014-11-05 11:16:47,297 DEBUG [sync.1] hdfs.DFSClient: Waiting for ack for: 45
2014-11-05 11:16:47,297 DEBUG [DataStreamer for file
/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083
block BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
hdfs.DFSClient: DataStreamer block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940 sending
packet packet seqno:45 offsetInBlock:33792 lastPacketInBlock:false
lastByteOffsetInBlock: 34383
2014-11-05 11:16:47,298 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
hdfs.DFSClient: DFSClient seqno: 44 status: SUCCESS status: SUCCESS status:
SUCCESS downstreamAckTimeNanos: 527654
2014-11-05 11:16:47,298 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
util.ByteArrayManager: recycle: array.length=131072, [131072: 3/10, free=0],
freeQueue.offer, freeQueueSize=1
2014-11-05 11:16:47,298 DEBUG [sync.0] util.ByteArrayManager: allocate(65565):
count=48, aboveThreshold, [131072: 2/10, free=1], recycled? true
2014-11-05 11:16:47,298 DEBUG [sync.0] hdfs.DFSClient: DFSClient writeChunk
allocating new packet seqno=46,
src=/user/stack/test-data/2256ed2b-6cc1-4144-88a5-227baf11842c/HLogPerformanceEvaluation/wals/hlog.1415215004083,
packetSize=65532, chunksPerPacket=127, bytesCurBlock=34304
2014-11-05 11:16:47,298 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
hdfs.DFSClient: DFSClient seqno: 45 status: SUCCESS status: SUCCESS status:
SUCCESS downstreamAckTimeNanos: 486422
2014-11-05 11:16:47,298 DEBUG [sync.0] hdfs.DFSClient: DFSClient flush() :
bytesCurBlock 35403 lastFlushOffset 34383
2014-11-05 11:16:47,298 DEBUG [ResponseProcessor for block
BP-410607956-10.20.84.26-1391491814882:blk_1075488801_1099513376940]
util.ByteArrayManager: recycle: array.length=131072, [131072: 3/10, free=0],
freeQueue.offer, freeQueueSize=1
2014-11-05 11:16:47,298 DEBUG [sync.0] hdfs.DFSClient: Queued packet 46
2014-11-05 11:16:47,298 DEBUG [sync.0] hdfs.DFSClient: Waiting for ack for: 46
...
{code}
> Clients may get stuck waiting when using ByteArrayManager
> ---------------------------------------------------------
>
> Key: HDFS-7358
> URL: https://issues.apache.org/jira/browse/HDFS-7358
> Project: Hadoop HDFS
> Issue Type: Bug
> Components: hdfs-client
> Reporter: Tsz Wo Nicholas Sze
> Assignee: Tsz Wo Nicholas Sze
> Attachments: h7358_20141104.patch, h7358_20141104_wait_timeout.patch
>
>
> [~stack] reported that clients might get stuck waiting when using
> ByteArrayManager; see [his
> comments|https://issues.apache.org/jira/browse/HDFS-7276?focusedCommentId=14197036&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14197036].
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)