You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ratis.apache.org by GitBox <gi...@apache.org> on 2020/05/23 17:54:14 UTC

[GitHub] [incubator-ratis] runzhiwang opened a new pull request #107: RATIS-867. Fix failed UT: TestMetaServer#testListLogs

runzhiwang opened a new pull request #107:
URL: https://github.com/apache/incubator-ratis/pull/107


   What's the problem ?
   {code:java}
   ava.lang.AssertionError: expected:<19> but was:<20>
   	at org.apache.ratis.logservice.server.TestMetaServer.testJMXCount(TestMetaServer.java:339)
   	at org.apache.ratis.logservice.server.TestMetaServer.testListLogs(TestMetaServer.java:331)
   {code}
   
   What's the reason ?
   1. when create log, it will call [RaftClientImpl::sendRequestWithRetry](https://github.com/apache/incubator-ratis/blob/master/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java#L285), if throw TimeoutIOException, it will retry at [final RaftClientReply reply = sendRequest(request)](https://github.com/apache/incubator-ratis/blob/master/ratis-client/src/main/java/org/apache/ratis/client/impl/RaftClientImpl.java#L296), So JMXCount will increase many times at [timerContext = metricRegistry.timer(type.name()).time()](https://github.com/apache/incubator-ratis/blob/master/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/MetaStateMachine.java#L224] when retry happens.  Then JMXCount i.e. 20 not equal to createCount i.e. 19
   
   2. The TimeoutIOException is as follows:
   
   {code:java}
   org.apache.ratis.protocol.TimeoutIOException: deadline exceeded after 2.999977899s. [buffered_nanos=1460409, remote_addr=localhost/127.0.0.1:9001]
   {code}
   
   3.  The reason of TimeoutIOException is when create log, it will call [client.groupAdd](https://github.com/apache/incubator-ratis/blob/master/ratis-logservice/src/main/java/org/apache/ratis/logservice/server/MetaStateMachine.java#L325). When add a new group, the following code [file.write(jvmName.getBytes(StandardCharsets.UTF_8))](https://github.com/apache/incubator-ratis/blob/master/ratis-server/src/main/java/org/apache/ratis/server/storage/RaftStorageDirectory.java#L342) and [((FileOutputStream)out).getChannel().force(true)](https://github.com/apache/incubator-ratis/blob/master/ratis-common/src/main/java/org/apache/ratis/util/AtomicFileOutputStream.java) will both force sync data to disk. The force sync to disk sometimes cost more than 2 seconds, but the timeout threshold of groupAdd is 3 seconds, so TimeoutIOException happens.
   
   How to fix ?
   Increase the timeout threshold from 3 seconds to 15 seconds.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] bshashikant commented on pull request #107: RATIS-867. Fix failed UT: TestMetaServer#testListLogs

Posted by GitBox <gi...@apache.org>.
bshashikant commented on pull request #107:
URL: https://github.com/apache/incubator-ratis/pull/107#issuecomment-637481766


   Thanks @runzhiwang for the contribution. I have committed this.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] bshashikant commented on a change in pull request #107: RATIS-867. Fix failed UT: TestMetaServer#testListLogs

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #107:
URL: https://github.com/apache/incubator-ratis/pull/107#discussion_r433734477



##########
File path: ratis-logservice/src/test/java/org/apache/ratis/logservice/server/TestMetaServer.java
##########
@@ -61,36 +63,52 @@
     static AtomicInteger createCount = new AtomicInteger();
     static AtomicInteger deleteCount = new AtomicInteger();
     static AtomicInteger listCount = new AtomicInteger();
-    LogServiceClient client = new LogServiceClient(cluster.getMetaIdentity()){
-        @Override public LogStream createLog(LogName logName) throws IOException {
+    static LogServiceClient client = null;
+
+    @BeforeClass
+    public static void beforeClass() {
+        cluster = new LogServiceCluster(3);
+        cluster.createWorkers(3);
+        workers = cluster.getWorkers();
+        assert(workers.size() == 3);
+
+        RaftProperties properties = new RaftProperties();
+        RaftClientConfigKeys.Rpc.setRequestTimeout(properties, TimeDuration.valueOf(15, TimeUnit.SECONDS));
+
+
+        cluster.getMasters().parallelStream().forEach(master ->
+            ((MetaStateMachine)master.getMetaStateMachine()).setProperties(properties));
+
+        client = new LogServiceClient(cluster.getMetaIdentity(), properties) {
+          @Override public LogStream createLog(LogName logName) throws IOException {

Review comment:
       Can you plz fix the formatting here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on pull request #107: RATIS-867. Fix failed UT: TestMetaServer#testListLogs

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #107:
URL: https://github.com/apache/incubator-ratis/pull/107#issuecomment-633106487


   @bshashikant @lokeshj1703 Could you help review this patch ? Thank you very much.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] runzhiwang commented on pull request #107: RATIS-867. Fix failed UT: TestMetaServer#testListLogs

Posted by GitBox <gi...@apache.org>.
runzhiwang commented on pull request #107:
URL: https://github.com/apache/incubator-ratis/pull/107#issuecomment-637446125


   @bshashikant updated


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-ratis] bshashikant merged pull request #107: RATIS-867. Fix failed UT: TestMetaServer#testListLogs

Posted by GitBox <gi...@apache.org>.
bshashikant merged pull request #107:
URL: https://github.com/apache/incubator-ratis/pull/107


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org