[ 
https://issues.apache.org/jira/browse/HDDS-12313?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17926281#comment-17926281
 ] 

Attila Doroszlai commented on HDDS-12313:
-----------------------------------------

Also, in 1/100 runs:

{code}
testConcurrentExceptionHandling(int, int) timed out after 300 seconds
...
        Suppressed: java.lang.InterruptedException
                at java.lang.Object.wait(Native Method)
                at java.lang.Thread.join(Thread.java:1257)
                at java.lang.Thread.join(Thread.java:1331)
                at 
org.apache.hadoop.fs.ozone.TestHSync.runConcurrentWriteHSyncWithException(TestHSync.java:907)
                at 
org.apache.hadoop.fs.ozone.TestHSync.testConcurrentExceptionHandling(TestHSync.java:852)
{code}

* 
https://github.com/adoroszlai/ozone-build-results/blob/master/2025/01/06/35755/it-filesystem/hadoop-ozone/integration-test/org.apache.hadoop.fs.ozone.TestHSync.txt
* 
https://github.com/adoroszlai/ozone-build-results/blob/master/2025/02/08/36459/it-filesystem/hadoop-ozone/integration-test/org.apache.hadoop.fs.ozone.TestHSync.txt

> Intermittent fork timeout in TestHSync#testConcurrentExceptionHandling
> ----------------------------------------------------------------------
>
>                 Key: HDDS-12313
>                 URL: https://issues.apache.org/jira/browse/HDDS-12313
>             Project: Apache Ozone
>          Issue Type: Sub-task
>    Affects Versions: 2.0.0
>            Reporter: Attila Doroszlai
>            Assignee: Wei-Chiu Chuang
>            Priority: Major
>
> {{KeyOutputStream#close()}} is blocked, waiting for write lock:
> {code}
> "main" 
>    java.lang.Thread.State: WAITING
>         at sun.misc.Unsafe.park(Native Method)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870)
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
>         at 
> java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:209)
>         at 
> java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285)
>         at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.doInWriteLock(KeyOutputStream.java:258)
>         at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.close(KeyOutputStream.java:647)
>         at 
> org.apache.hadoop.ozone.client.io.OzoneOutputStream.close(OzoneOutputStream.java:105)
>         at 
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.close(OzoneFSOutputStream.java:70)
>         at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:77)
>         at 
> org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
>         at 
> org.apache.hadoop.fs.ozone.TestHSync.testConcurrentExceptionHandling(TestHSync.java:853)
> {code}
> Write lock is held by thread waiting for free buffer in BufferPool:
> {code}
> "Syncer-6" 
>    java.lang.Thread.State: WAITING
>         at sun.misc.Unsafe.park(Native Method)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at 
> org.apache.hadoop.hdds.scm.storage.BufferPool.allocateBuffer(BufferPool.java:98)
>         at 
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.allocateNewBufferIfNeeded(BlockOutputStream.java:429)
>         at 
> org.apache.hadoop.hdds.scm.storage.BlockOutputStream.write(BlockOutputStream.java:386)
>         at 
> org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.write(BlockOutputStreamEntry.java:186)
>         at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.writeToOutputStream(KeyOutputStream.java:307)
>         at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.handleWrite(KeyOutputStream.java:283)
>         at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.lambda$write$1(KeyOutputStream.java:249)
>         at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream$$Lambda$1871/1147820709.run(Unknown
>  Source)
>         at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.doInWriteLock(KeyOutputStream.java:260)
>         at 
> org.apache.hadoop.ozone.client.io.KeyOutputStream.write(KeyOutputStream.java:248)
>         at 
> org.apache.hadoop.ozone.client.io.OzoneOutputStream.write(OzoneOutputStream.java:94)
>         at 
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.lambda$write$1(OzoneFSOutputStream.java:58)
>         at 
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream$$Lambda$1870/2092940497.run(Unknown
>  Source)
>         at 
> org.apache.hadoop.hdds.tracing.TracingUtil.executeInSpan(TracingUtil.java:184)
>         at 
> org.apache.hadoop.hdds.tracing.TracingUtil.executeInNewSpan(TracingUtil.java:149)
>         at 
> org.apache.hadoop.fs.ozone.OzoneFSOutputStream.write(OzoneFSOutputStream.java:54)
>         at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:62)
>         at java.io.DataOutputStream.write(DataOutputStream.java:107)
>         at java.io.FilterOutputStream.write(FilterOutputStream.java:97)
>         at 
> org.apache.hadoop.fs.ozone.TestHSync.lambda$runConcurrentWriteHSyncWithException$9(TestHSync.java:873)
>         at 
> org.apache.hadoop.fs.ozone.TestHSync$$Lambda$2189/1164139233.run(Unknown 
> Source)
> {code}
> 5/100 runs:
> https://github.com/adoroszlai/ozone/actions/runs/13271802577
> * 
> https://github.com/adoroszlai/ozone-build-results/blob/master/2024/12/08/35245/it-filesystem/2024-12-08T14-37-06_480-jvmRun1.dump
> * 
> https://github.com/adoroszlai/ozone-build-results/blob/master/2025/01/27/36193/it-filesystem/2025-01-27T07-34-21_545-jvmRun1.dump
> * 
> https://github.com/adoroszlai/ozone-build-results/blob/master/2025/01/29/36234/it-filesystem/2025-01-29T14-38-56_041-jvmRun1.dump
> * 
> https://github.com/adoroszlai/ozone-build-results/blob/master/2025/02/10/36510/it-filesystem/2025-02-10T19-18-58_666-jvmRun1.dump
> * 
> https://github.com/adoroszlai/ozone-build-results/blob/master/2025/02/11/36535/it-filesystem/2025-02-11T09-36-12_792-jvmRun1.dump



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to