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

Tsz-wo Sze commented on HDDS-10750:
-----------------------------------

BTW, [~smeng] and I found in HDDS-10108 that 
TestRandomKeyGenerator.testKeyLargerThan2GB() may fail with recent Ratis 
snapshots.  The failure happened in both the master branch and the hsync branch.
{code}
+++ b/pom.xml
@@ -75,10 +75,10 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xs
     <!-- HDDS Rocks Native dependency version-->
     <hdds.rocks.native.version>${hdds.version}</hdds.rocks.native.version>
     <!-- Apache Ratis version -->
-    <ratis.version>3.0.1</ratis.version>
+    <ratis.version>3.1.0-29bba59-SNAPSHOT</ratis.version>
 
     <!-- Apache Ratis thirdparty version -->
-    <ratis.thirdparty.version>1.0.5</ratis.thirdparty.version>
+    <ratis.thirdparty.version>1.0.6</ratis.thirdparty.version>
 
     <!-- Apache Ranger plugin version -->
     <ranger.version>2.3.0</ranger.version>
{code}


> Intermittent fork timeout while stopping Ratis server
> -----------------------------------------------------
>
>                 Key: HDDS-10750
>                 URL: https://issues.apache.org/jira/browse/HDDS-10750
>             Project: Apache Ozone
>          Issue Type: Sub-task
>            Reporter: Attila Doroszlai
>            Priority: Critical
>         Attachments: 2024-04-21T16-53-06_683-jvmRun1.dump, 
> 2024-05-03T11-31-12_561-jvmRun1.dump, 
> org.apache.hadoop.ozone.client.rpc.TestECKeyOutputStreamWithZeroCopy-output.txt,
>  org.apache.hadoop.ozone.container.TestECContainerRecovery-output.txt
>
>
> {code:title=https://github.com/adoroszlai/ozone-build-results/blob/master/2024/04/21/30803/it-client/output.log}
> [INFO] Running 
> org.apache.hadoop.ozone.client.rpc.TestECKeyOutputStreamWithZeroCopy
> [INFO] 
> [INFO] Results:
> ...
> ... There was a timeout or other error in the fork
> {code}
> {code}
> "main" 
>    java.lang.Thread.State: WAITING
>         at java.lang.Object.wait(Native Method)
>         at java.util.concurrent.ForkJoinTask.doInvoke(ForkJoinTask.java:405)
>         ...
>         at 
> org.apache.hadoop.ozone.MiniOzoneClusterImpl.stopDatanodes(MiniOzoneClusterImpl.java:473)
>         at 
> org.apache.hadoop.ozone.MiniOzoneClusterImpl.stop(MiniOzoneClusterImpl.java:414)
>         at 
> org.apache.hadoop.ozone.MiniOzoneClusterImpl.shutdown(MiniOzoneClusterImpl.java:400)
>         at 
> org.apache.hadoop.ozone.client.rpc.AbstractTestECKeyOutputStream.shutdown(AbstractTestECKeyOutputStream.java:160)
> "ForkJoinPool.commonPool-worker-7" 
>    java.lang.Thread.State: TIMED_WAITING
>         ...
>         at 
> java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1475)
>         at 
> org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:144)
>         at 
> org.apache.ratis.util.ConcurrentUtils.shutdownAndWait(ConcurrentUtils.java:136)
>         at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$close$9(RaftServerProxy.java:438)
>         ...
>         at 
> org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:304)
>         at 
> org.apache.ratis.server.impl.RaftServerProxy.close(RaftServerProxy.java:415)
>         at 
> org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis.stop(XceiverServerRatis.java:603)
>         at 
> org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer.stop(OzoneContainer.java:484)
>         at 
> org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.close(DatanodeStateMachine.java:447)
>         at 
> org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine.stopDaemon(DatanodeStateMachine.java:637)
>         at 
> org.apache.hadoop.ozone.HddsDatanodeService.stop(HddsDatanodeService.java:550)
>         at 
> org.apache.hadoop.ozone.MiniOzoneClusterImpl.stopDatanode(MiniOzoneClusterImpl.java:479)
>         at 
> org.apache.hadoop.ozone.MiniOzoneClusterImpl$$Lambda$2077/645273703.accept(Unknown
>  Source)
> "c7edee5d-bf3c-45a7-a783-e11562f208dc-impl-thread2" 
>    java.lang.Thread.State: WAITING
>         ...
>         at 
> java.util.concurrent.CompletableFuture.join(CompletableFuture.java:1947)
>         at 
> org.apache.ratis.server.impl.RaftServerImpl.lambda$close$3(RaftServerImpl.java:543)
>         at 
> org.apache.ratis.server.impl.RaftServerImpl$$Lambda$1925/263251010.run(Unknown
>  Source)
>         at 
> org.apache.ratis.util.LifeCycle.lambda$checkStateAndClose$7(LifeCycle.java:306)
>         at org.apache.ratis.util.LifeCycle$$Lambda$1204/655954062.get(Unknown 
> Source)
>         at 
> org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:326)
>         at 
> org.apache.ratis.util.LifeCycle.checkStateAndClose(LifeCycle.java:304)
>         at 
> org.apache.ratis.server.impl.RaftServerImpl.close(RaftServerImpl.java:525)
> {code}



--
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