[ 
https://issues.apache.org/jira/browse/RATIS-1281?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Hui Fei updated RATIS-1281:
---------------------------
    Description: 
This UT always timeout, even though I increase the time.
{code:java}
[ERROR] Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 60.201 
s <<< FAILURE! - in org.apache.ratis.grpc.TestRetryCacheWithGrpc
[ERROR] 
testRetryOnResourceUnavailableException(org.apache.ratis.grpc.TestRetryCacheWithGrpc)
  Time elapsed: 60.022 s  <<< ERROR!
org.junit.runners.model.TestTimedOutException: test timed out after 60000 
milliseconds
        at java.io.FileOutputStream.writeBytes(Native Method)
        at java.io.FileOutputStream.write(FileOutputStream.java:326)
        at java.io.BufferedOutputStream.write(BufferedOutputStream.java:122)
        at java.io.PrintStream.write(PrintStream.java:480)
        at 
org.apache.maven.surefire.booter.ForkingRunListener.writeTestOutput(ForkingRunListener.java:208)
        at 
org.apache.maven.surefire.report.ConsoleOutputCapture$ForwardingPrintStream.write(ConsoleOutputCapture.java:60)
        at sun.nio.cs.StreamEncoder.writeBytes(StreamEncoder.java:221)
        at sun.nio.cs.StreamEncoder.implFlushBuffer(StreamEncoder.java:291)
        at sun.nio.cs.StreamEncoder.implFlush(StreamEncoder.java:295)
        at sun.nio.cs.StreamEncoder.flush(StreamEncoder.java:141)
        at java.io.OutputStreamWriter.flush(OutputStreamWriter.java:229)
        at org.apache.log4j.helpers.QuietWriter.flush(QuietWriter.java:59)
        at org.apache.log4j.WriterAppender.subAppend(WriterAppender.java:324)
        at org.apache.log4j.WriterAppender.append(WriterAppender.java:162)
        at org.apache.log4j.AppenderSkeleton.doAppend(AppenderSkeleton.java:251)
        at 
org.apache.log4j.helpers.AppenderAttachableImpl.appendLoopOnAppenders(AppenderAttachableImpl.java:66)
        at org.apache.log4j.Category.callAppenders(Category.java:206)
        at org.apache.log4j.Category.forcedLog(Category.java:391)
        at org.apache.log4j.Category.log(Category.java:856)
        at org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:253)
        at 
org.apache.ratis.server.impl.RaftServerImpl.submitClientRequestAsync(RaftServerImpl.java:727)
        at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitClientRequestAsync$9(RaftServerProxy.java:415)
        at 
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$678/122031867.apply(Unknown
 Source)
        at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$null$7(RaftServerProxy.java:410)
        at 
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$603/413337014.get(Unknown 
Source)
        at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:117)
        at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$8(RaftServerProxy.java:410)
        at 
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$601/602116397.apply(Unknown
 Source)
        at 
java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:981)
        at 
java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2124)
        at 
org.apache.ratis.server.impl.RaftServerProxy.submitRequest(RaftServerProxy.java:409)
        at 
org.apache.ratis.server.impl.RaftServerProxy.submitClientRequestAsync(RaftServerProxy.java:415)
        at 
org.apache.ratis.grpc.TestRetryCacheWithGrpc.testRetryOnResourceUnavailableException(TestRetryCacheWithGrpc.java:84)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
        at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.lang.Thread.run(Thread.java:748)

In {code}
The first while clause in 
TestRetryCacheWithGrpc#testRetryOnResourceUnavailableException
{code:java}
while (!failure.get()) {
  long cid = callId;
  r = cluster.newRaftClientRequest(clientId, leaderProxy.getId(), callId++,
      new RaftTestUtil.SimpleMessage("message"));
  CompletableFuture<RaftClientReply> f = 
leaderProxy.submitClientRequestAsync(r);
  f.exceptionally(e -> {
      if (e.getCause() instanceof ResourceUnavailableException) {
        RetryCacheTestUtil.isFailed(RetryCacheTestUtil.get(leader, clientId, 
cid));
        failure.set(true);
      }
      return null;
  });
}

{code}
There are 2 requests here, and because followers call 
blockWriteStateMachineData, the first request will be blocked, and 
ELEMENT_LIMIT_KEY is 1, the second request will fail and receive 
ResourceUnavailableException.

After quittng the first while clause. followers call 
unblockWriteStateMachineData

In the second while clause

{code}

while (failure.get()) {
  try {
  // retry until the request failed with ResourceUnavailableException succeeds.
  leaderProxy.submitClientRequestAsync(r).get();
  } catch (Exception e) {
    failure.set(false);
  }
}

{code}

There are 2 cases: 1. If the first request has been done, the 3rd request will 
be handled, and later requests will hit retrycache, and server will always 
return success. It couldn't get Exception and enter infinite loop. 2. If the 
first request hasn't been done, and the 3rd request has been reach the server, 
it will get   ResourceUnavailableException and set failure true, and quit the 
loop.

Mostly the case 1 occurs, and this UT nearly always fails

 

So i suggest that quit the loop when request handled successfully as the 
comments say
{quote}// retry until the request failed with ResourceUnavailableException 
succeeds.
{quote}

  was:
This UT always timeout, even though I increase the time.
{code:java}
[ERROR] Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 60.201 
s <<< FAILURE! - in org.apache.ratis.grpc.TestRetryCacheWithGrpc
[ERROR] 
testRetryOnResourceUnavailableException(org.apache.ratis.grpc.TestRetryCacheWithGrpc)
  Time elapsed: 60.022 s  <<< ERROR!
org.junit.runners.model.TestTimedOutException: test timed out after 60000 
milliseconds
        at java.io.FileOutputStream.writeBytes(Native Method)
        at java.io.FileOutputStream.write(FileOutputStream.java:326)
        at java.io.BufferedOutputStream.write(BufferedOutputStream.java:122)
        at java.io.PrintStream.write(PrintStream.java:480)
        at 
org.apache.maven.surefire.booter.ForkingRunListener.writeTestOutput(ForkingRunListener.java:208)
        at 
org.apache.maven.surefire.report.ConsoleOutputCapture$ForwardingPrintStream.write(ConsoleOutputCapture.java:60)
        at sun.nio.cs.StreamEncoder.writeBytes(StreamEncoder.java:221)
        at sun.nio.cs.StreamEncoder.implFlushBuffer(StreamEncoder.java:291)
        at sun.nio.cs.StreamEncoder.implFlush(StreamEncoder.java:295)
        at sun.nio.cs.StreamEncoder.flush(StreamEncoder.java:141)
        at java.io.OutputStreamWriter.flush(OutputStreamWriter.java:229)
        at org.apache.log4j.helpers.QuietWriter.flush(QuietWriter.java:59)
        at org.apache.log4j.WriterAppender.subAppend(WriterAppender.java:324)
        at org.apache.log4j.WriterAppender.append(WriterAppender.java:162)
        at org.apache.log4j.AppenderSkeleton.doAppend(AppenderSkeleton.java:251)
        at 
org.apache.log4j.helpers.AppenderAttachableImpl.appendLoopOnAppenders(AppenderAttachableImpl.java:66)
        at org.apache.log4j.Category.callAppenders(Category.java:206)
        at org.apache.log4j.Category.forcedLog(Category.java:391)
        at org.apache.log4j.Category.log(Category.java:856)
        at org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:253)
        at 
org.apache.ratis.server.impl.RaftServerImpl.submitClientRequestAsync(RaftServerImpl.java:727)
        at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitClientRequestAsync$9(RaftServerProxy.java:415)
        at 
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$678/122031867.apply(Unknown
 Source)
        at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$null$7(RaftServerProxy.java:410)
        at 
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$603/413337014.get(Unknown 
Source)
        at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:117)
        at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$8(RaftServerProxy.java:410)
        at 
org.apache.ratis.server.impl.RaftServerProxy$$Lambda$601/602116397.apply(Unknown
 Source)
        at 
java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:981)
        at 
java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2124)
        at 
org.apache.ratis.server.impl.RaftServerProxy.submitRequest(RaftServerProxy.java:409)
        at 
org.apache.ratis.server.impl.RaftServerProxy.submitClientRequestAsync(RaftServerProxy.java:415)
        at 
org.apache.ratis.grpc.TestRetryCacheWithGrpc.testRetryOnResourceUnavailableException(TestRetryCacheWithGrpc.java:84)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
        at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
        at 
org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.lang.Thread.run(Thread.java:748)

In {code}
The first while clause in 
TestRetryCacheWithGrpc#testRetryOnResourceUnavailableException

{code}

while (!failure.get()) {
  long cid = callId;
  r = cluster.newRaftClientRequest(clientId, leaderProxy.getId(), callId++,
      new RaftTestUtil.SimpleMessage("message"));
  CompletableFuture<RaftClientReply> f = 
leaderProxy.submitClientRequestAsync(r);
  f.exceptionally(e -> {
      if (e.getCause() instanceof ResourceUnavailableException) {
        RetryCacheTestUtil.isFailed(RetryCacheTestUtil.get(leader, clientId, 
cid));
        failure.set(true);
      }
      return null;
  });
}

{code}

There are 2 requests here, and because followers call 
blockWriteStateMachineData, the first request will be blocked, and 
ELEMENT_LIMIT_KEY is 1, the second request will fail and receive 
ResourceUnavailableException.

After quittng the first while clause. followers call 
unblockWriteStateMachineData

In the second while clause:  1. If the first request has been done, the 3rd 
request will be handled, and later requests will hit retrycache, and server 
will always return success. It couldn't get Exception and enter infinite loop. 
2. If the first request hasn't been done, and the 3rd request has been reach 
the server, it will get   ResourceUnavailableException and set failure true, 
and quit the loop.

Mostly the case 1 occurs, and this UT nearly always fail

 

So i suggest that quit the loop when request handled successfully as the 
comments say

{quote}

// retry until the request failed with ResourceUnavailableException succeeds.

{quote}


> TestRetryCacheWithGrpc#testRetryOnResourceUnavailableException fails
> --------------------------------------------------------------------
>
>                 Key: RATIS-1281
>                 URL: https://issues.apache.org/jira/browse/RATIS-1281
>             Project: Ratis
>          Issue Type: Bug
>          Components: test
>    Affects Versions: 2.0.0
>            Reporter: Hui Fei
>            Assignee: Hui Fei
>            Priority: Major
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> This UT always timeout, even though I increase the time.
> {code:java}
> [ERROR] Tests run: 1, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 
> 60.201 s <<< FAILURE! - in org.apache.ratis.grpc.TestRetryCacheWithGrpc
> [ERROR] 
> testRetryOnResourceUnavailableException(org.apache.ratis.grpc.TestRetryCacheWithGrpc)
>   Time elapsed: 60.022 s  <<< ERROR!
> org.junit.runners.model.TestTimedOutException: test timed out after 60000 
> milliseconds
>       at java.io.FileOutputStream.writeBytes(Native Method)
>       at java.io.FileOutputStream.write(FileOutputStream.java:326)
>       at java.io.BufferedOutputStream.write(BufferedOutputStream.java:122)
>       at java.io.PrintStream.write(PrintStream.java:480)
>       at 
> org.apache.maven.surefire.booter.ForkingRunListener.writeTestOutput(ForkingRunListener.java:208)
>       at 
> org.apache.maven.surefire.report.ConsoleOutputCapture$ForwardingPrintStream.write(ConsoleOutputCapture.java:60)
>       at sun.nio.cs.StreamEncoder.writeBytes(StreamEncoder.java:221)
>       at sun.nio.cs.StreamEncoder.implFlushBuffer(StreamEncoder.java:291)
>       at sun.nio.cs.StreamEncoder.implFlush(StreamEncoder.java:295)
>       at sun.nio.cs.StreamEncoder.flush(StreamEncoder.java:141)
>       at java.io.OutputStreamWriter.flush(OutputStreamWriter.java:229)
>       at org.apache.log4j.helpers.QuietWriter.flush(QuietWriter.java:59)
>       at org.apache.log4j.WriterAppender.subAppend(WriterAppender.java:324)
>       at org.apache.log4j.WriterAppender.append(WriterAppender.java:162)
>       at org.apache.log4j.AppenderSkeleton.doAppend(AppenderSkeleton.java:251)
>       at 
> org.apache.log4j.helpers.AppenderAttachableImpl.appendLoopOnAppenders(AppenderAttachableImpl.java:66)
>       at org.apache.log4j.Category.callAppenders(Category.java:206)
>       at org.apache.log4j.Category.forcedLog(Category.java:391)
>       at org.apache.log4j.Category.log(Category.java:856)
>       at org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:253)
>       at 
> org.apache.ratis.server.impl.RaftServerImpl.submitClientRequestAsync(RaftServerImpl.java:727)
>       at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$submitClientRequestAsync$9(RaftServerProxy.java:415)
>       at 
> org.apache.ratis.server.impl.RaftServerProxy$$Lambda$678/122031867.apply(Unknown
>  Source)
>       at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$null$7(RaftServerProxy.java:410)
>       at 
> org.apache.ratis.server.impl.RaftServerProxy$$Lambda$603/413337014.get(Unknown
>  Source)
>       at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:117)
>       at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$submitRequest$8(RaftServerProxy.java:410)
>       at 
> org.apache.ratis.server.impl.RaftServerProxy$$Lambda$601/602116397.apply(Unknown
>  Source)
>       at 
> java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:981)
>       at 
> java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2124)
>       at 
> org.apache.ratis.server.impl.RaftServerProxy.submitRequest(RaftServerProxy.java:409)
>       at 
> org.apache.ratis.server.impl.RaftServerProxy.submitClientRequestAsync(RaftServerProxy.java:415)
>       at 
> org.apache.ratis.grpc.TestRetryCacheWithGrpc.testRetryOnResourceUnavailableException(TestRetryCacheWithGrpc.java:84)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
>       at 
> org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
>       at 
> org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
>       at 
> org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
>       at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:298)
>       at 
> org.junit.internal.runners.statements.FailOnTimeout$CallableStatement.call(FailOnTimeout.java:292)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at java.lang.Thread.run(Thread.java:748)
> In {code}
> The first while clause in 
> TestRetryCacheWithGrpc#testRetryOnResourceUnavailableException
> {code:java}
> while (!failure.get()) {
>   long cid = callId;
>   r = cluster.newRaftClientRequest(clientId, leaderProxy.getId(), callId++,
>       new RaftTestUtil.SimpleMessage("message"));
>   CompletableFuture<RaftClientReply> f = 
> leaderProxy.submitClientRequestAsync(r);
>   f.exceptionally(e -> {
>       if (e.getCause() instanceof ResourceUnavailableException) {
>         RetryCacheTestUtil.isFailed(RetryCacheTestUtil.get(leader, clientId, 
> cid));
>         failure.set(true);
>       }
>       return null;
>   });
> }
> {code}
> There are 2 requests here, and because followers call 
> blockWriteStateMachineData, the first request will be blocked, and 
> ELEMENT_LIMIT_KEY is 1, the second request will fail and receive 
> ResourceUnavailableException.
> After quittng the first while clause. followers call 
> unblockWriteStateMachineData
> In the second while clause
> {code}
> while (failure.get()) {
>   try {
>   // retry until the request failed with ResourceUnavailableException 
> succeeds.
>   leaderProxy.submitClientRequestAsync(r).get();
>   } catch (Exception e) {
>     failure.set(false);
>   }
> }
> {code}
> There are 2 cases: 1. If the first request has been done, the 3rd request 
> will be handled, and later requests will hit retrycache, and server will 
> always return success. It couldn't get Exception and enter infinite loop. 2. 
> If the first request hasn't been done, and the 3rd request has been reach the 
> server, it will get   ResourceUnavailableException and set failure true, and 
> quit the loop.
> Mostly the case 1 occurs, and this UT nearly always fails
>  
> So i suggest that quit the loop when request handled successfully as the 
> comments say
> {quote}// retry until the request failed with ResourceUnavailableException 
> succeeds.
> {quote}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to