You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "tkalkirill (via GitHub)" <gi...@apache.org> on 2023/04/06 08:54:08 UTC

[GitHub] [ignite-3] tkalkirill opened a new pull request, #1912: IGNITE-19231 Change thread pool for metastore raft group

tkalkirill opened a new pull request, #1912:
URL: https://github.com/apache/ignite-3/pull/1912

   https://issues.apache.org/jira/browse/IGNITE-19231


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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161435061


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftServer.java:
##########
@@ -76,6 +77,26 @@ boolean startRaftNode(
             RaftGroupOptions groupOptions
     );
 
+    /**
+     * Starts a Raft group bound to this cluster node.
+     *
+     * @param nodeId Raft node ID.
+     * @param configuration Raft configuration.
+     * @param evLsnr Listener for group membership and other events.
+     * @param lsnr Listener for state machine events.
+     * @param groupOptions Options to apply to the group.
+     * @param ownFsmCallerExecutorDisruptorConfig Configuration own (not shared) striped disruptor for FSMCaller service of raft node.

Review Comment:
   fix it



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161492668


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftGroupOptions.java:
##########
@@ -37,6 +39,9 @@ public class RaftGroupOptions {
     /** Raft meta storage factory. */
     private RaftMetaStorageFactory raftMetaStorageFactory;
 
+    /** Configuration own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor. */

Review Comment:
   Fix it



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161433003


##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3976,7 +3976,7 @@ private RaftGroupService createService(String groupId, TestPeer peer, NodeOption
 
         clusterService.start();
 
-        var service = new RaftGroupService(groupId, peer.getPeerId(), nodeOptions, rpcServer, nodeManager) {
+        var service = new RaftGroupService(groupId, peer.getPeerId(), nodeOptions, rpcServer, nodeManager, null) {

Review Comment:
   Why not.



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161437754


##########
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/RaftNodeDisruptorConfiguration.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft;
+
+/**
+ * Raft node disruptor configuration.
+ */
+public class RaftNodeDisruptorConfiguration {
+    private final String threadPostfix;
+
+    private final int stripes;
+
+    /**
+     * Constructor.
+     *
+     * @param threadPostfix Postfix the name of the disruptor threads.

Review Comment:
   fix it



##########
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/RaftNodeDisruptorConfiguration.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft;
+
+/**
+ * Raft node disruptor configuration.
+ */
+public class RaftNodeDisruptorConfiguration {
+    private final String threadPostfix;
+
+    private final int stripes;
+
+    /**
+     * Constructor.
+     *
+     * @param threadPostfix Postfix the name of the disruptor threads.
+     * @param stripes Number of disruptor stripes.
+     */
+    public RaftNodeDisruptorConfiguration(String threadPostfix, int stripes) {
+        this.threadPostfix = threadPostfix;
+        this.stripes = stripes;
+    }
+
+    /**
+     * Return postfix the name of the disruptor threads.

Review Comment:
   Fix it



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill merged pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill merged PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912


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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161437304


##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/JRaftUtils.java:
##########
@@ -54,7 +54,7 @@ public final class JRaftUtils {
      * @return true if bootstrap success
      */
     public static boolean bootstrap(final BootstrapOptions opts) throws InterruptedException {
-        final NodeImpl node = new NodeImpl("bootstrap", new PeerId("127.0.0.1", 0));
+        final NodeImpl node = new NodeImpl("bootstrap", new PeerId("127.0.0.1", 0), null);

Review Comment:
   Why not.



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161478460


##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java:
##########
@@ -571,9 +570,32 @@ public NodeImpl(
         this.confCtx = new ConfigurationCtx(this);
         this.wakingCandidate = null;
         this.applyCommittedFuture = new CompletableFuture<>();
-        this.ownFsmCallerExecutorDisruptorConfig = ownFsmCallerExecutorDisruptorConfig;
+        this.ownFsmCallerExecutorDisruptorConfig = null;
     }
 
+        public NodeImpl(

Review Comment:
   Fix it



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "rpuch (via GitHub)" <gi...@apache.org>.
rpuch commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161476054


##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java:
##########
@@ -571,9 +570,32 @@ public NodeImpl(
         this.confCtx = new ConfigurationCtx(this);
         this.wakingCandidate = null;
         this.applyCommittedFuture = new CompletableFuture<>();
-        this.ownFsmCallerExecutorDisruptorConfig = ownFsmCallerExecutorDisruptorConfig;
+        this.ownFsmCallerExecutorDisruptorConfig = null;
     }
 
+        public NodeImpl(

Review Comment:
   Same thing: let's reuse one constructor in the other. This seems to be especially important in this class, as the constructor has some non-trivial logic, not just assignments to fields, and it seems scary to duplicate this logic.



##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/RaftGroupService.java:
##########
@@ -76,26 +76,51 @@ public class RaftGroupService {
      * @param nodeOptions Node options.
      * @param rpcServer RPC server.
      * @param nodeManager Node manager.
-     * @param ownFsmCallerExecutorDisruptorConfig Configuration own striped disruptor for FSMCaller service of raft node, {@code null}
-     *      means use shared disruptor.
      */
     public RaftGroupService(
             final String groupId,
             final PeerId serverId,
             final NodeOptions nodeOptions,
             final RpcServer rpcServer,
-            final NodeManager nodeManager,
-            @Nullable RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig
+            final NodeManager nodeManager
     ) {
         super();
         this.groupId = groupId;
         this.serverId = serverId;
         this.nodeOptions = nodeOptions;
         this.rpcServer = rpcServer;
         this.nodeManager = nodeManager;
-        this.ownFsmCallerExecutorDisruptorConfig = ownFsmCallerExecutorDisruptorConfig;
+        this.ownFsmCallerExecutorDisruptorConfig = null;
     }
 
+        /**
+         * @param groupId Group Id.
+         * @param serverId Server id.
+         * @param nodeOptions Node options.
+         * @param rpcServer RPC server.
+         * @param nodeManager Node manager.
+         * @param ownFsmCallerExecutorDisruptorConfig Configuration own striped disruptor for FSMCaller service of raft node.
+         */
+        public RaftGroupService(
+                final String groupId,
+                final PeerId serverId,
+                final NodeOptions nodeOptions,
+                final RpcServer rpcServer,
+                final NodeManager nodeManager,
+                final RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig
+        ) {
+            super();

Review Comment:
   Sorry, I did not express this clearly: the idea was to reuse one constructor inside another. Looks like the only thing that prevents this is the `assert`, but it doesn't seem to be too important. By removing it, we can remove the duplication. WDYT?



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "rpuch (via GitHub)" <gi...@apache.org>.
rpuch commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161491291


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftGroupOptions.java:
##########
@@ -126,4 +131,20 @@ public RaftGroupOptions raftMetaStorageFactory(RaftMetaStorageFactory raftMetaSt
 
         return this;
     }
+
+    /**
+     * Returns configuration own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor.

Review Comment:
   ```suggestion
        * Returns configuration of own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor.
   ```



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftGroupOptions.java:
##########
@@ -37,6 +39,9 @@ public class RaftGroupOptions {
     /** Raft meta storage factory. */
     private RaftMetaStorageFactory raftMetaStorageFactory;
 
+    /** Configuration own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor. */

Review Comment:
   ```suggestion
       /** Configuration of own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor. */
   ```



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftGroupOptions.java:
##########
@@ -126,4 +131,20 @@ public RaftGroupOptions raftMetaStorageFactory(RaftMetaStorageFactory raftMetaSt
 
         return this;
     }
+
+    /**
+     * Returns configuration own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor.
+     */
+    public @Nullable RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig() {
+        return ownFsmCallerExecutorDisruptorConfig;
+    }
+
+    /**
+     * Sets configuration own striped disruptor for FSMCaller service of raft node.

Review Comment:
   ```suggestion
        * Sets configuration of own striped disruptor for FSMCaller service of raft node.
   ```



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161493195


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftGroupOptions.java:
##########
@@ -126,4 +131,20 @@ public RaftGroupOptions raftMetaStorageFactory(RaftMetaStorageFactory raftMetaSt
 
         return this;
     }
+
+    /**
+     * Returns configuration own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor.
+     */
+    public @Nullable RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig() {
+        return ownFsmCallerExecutorDisruptorConfig;
+    }
+
+    /**
+     * Sets configuration own striped disruptor for FSMCaller service of raft node.

Review Comment:
   fix it



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftGroupOptions.java:
##########
@@ -126,4 +131,20 @@ public RaftGroupOptions raftMetaStorageFactory(RaftMetaStorageFactory raftMetaSt
 
         return this;
     }
+
+    /**
+     * Returns configuration own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor.

Review Comment:
   fix it



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161480196


##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/RaftGroupService.java:
##########
@@ -76,26 +76,51 @@ public class RaftGroupService {
      * @param nodeOptions Node options.
      * @param rpcServer RPC server.
      * @param nodeManager Node manager.
-     * @param ownFsmCallerExecutorDisruptorConfig Configuration own striped disruptor for FSMCaller service of raft node, {@code null}
-     *      means use shared disruptor.
      */
     public RaftGroupService(
             final String groupId,
             final PeerId serverId,
             final NodeOptions nodeOptions,
             final RpcServer rpcServer,
-            final NodeManager nodeManager,
-            @Nullable RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig
+            final NodeManager nodeManager
     ) {
         super();
         this.groupId = groupId;
         this.serverId = serverId;
         this.nodeOptions = nodeOptions;
         this.rpcServer = rpcServer;
         this.nodeManager = nodeManager;
-        this.ownFsmCallerExecutorDisruptorConfig = ownFsmCallerExecutorDisruptorConfig;
+        this.ownFsmCallerExecutorDisruptorConfig = null;
     }
 
+        /**
+         * @param groupId Group Id.
+         * @param serverId Server id.
+         * @param nodeOptions Node options.
+         * @param rpcServer RPC server.
+         * @param nodeManager Node manager.
+         * @param ownFsmCallerExecutorDisruptorConfig Configuration own striped disruptor for FSMCaller service of raft node.
+         */
+        public RaftGroupService(
+                final String groupId,
+                final PeerId serverId,
+                final NodeOptions nodeOptions,
+                final RpcServer rpcServer,
+                final NodeManager nodeManager,
+                final RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig
+        ) {
+            super();

Review Comment:
   Fix it.



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161487364


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java:
##########
@@ -235,7 +236,36 @@ public <T extends RaftGroupService> CompletableFuture<T> startRaftGroupNode(
         }
 
         try {
-            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory);
+            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory, null);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    @Override
+    public CompletableFuture<RaftGroupService> startRaftGroupNode(
+            RaftNodeId nodeId,
+            PeersAndLearners configuration,
+            RaftGroupListener lsnr,
+            RaftGroupEventsListener eventsLsnr,
+            RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig

Review Comment:
   I'll try to fix it.



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161435624


##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java:
##########
@@ -225,6 +227,9 @@ public class NodeImpl implements Node, RaftServerService {
      */
     private volatile int electionTimeoutCounter;
 
+    /** Configuration own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor. */

Review Comment:
   fix it



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "rpuch (via GitHub)" <gi...@apache.org>.
rpuch commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161474223


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java:
##########
@@ -235,7 +236,36 @@ public <T extends RaftGroupService> CompletableFuture<T> startRaftGroupNode(
         }
 
         try {
-            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory);
+            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory, null);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    @Override
+    public CompletableFuture<RaftGroupService> startRaftGroupNode(
+            RaftNodeId nodeId,
+            PeersAndLearners configuration,
+            RaftGroupListener lsnr,
+            RaftGroupEventsListener eventsLsnr,
+            RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig

Review Comment:
   At least `RaftServer` is in the same module as `RaftGroupOptions`, so it seems that it is possible to pass `RaftNodeDisruptorConfiguration` to `RaftServer` inside of `RaftGroupOptions` easily. Does it take a lot of work? On the other hand, if this is done, `RaftServer` will become simpler right now.



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161432831


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java:
##########
@@ -175,6 +176,14 @@ public void onWrite(Iterator<CommandClosure<WriteCommand>> iterator) {
                 clo.result(null);
             } catch (IgniteInternalException e) {
                 clo.result(e);
+            } catch (CompletionException e) {
+                clo.result(e.getCause());
+            } catch (Throwable t) {

Review Comment:
   Of course I forgot to do that, thanks!



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] tkalkirill commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161445416


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java:
##########
@@ -235,7 +236,36 @@ public <T extends RaftGroupService> CompletableFuture<T> startRaftGroupNode(
         }
 
         try {
-            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory);
+            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory, null);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    @Override
+    public CompletableFuture<RaftGroupService> startRaftGroupNode(
+            RaftNodeId nodeId,
+            PeersAndLearners configuration,
+            RaftGroupListener lsnr,
+            RaftGroupEventsListener eventsLsnr,
+            RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig

Review Comment:
   Look, `RaftGroupOptions` is in module `ignite-raft` and `RaftManager` (as well as `RaftNodeDisruptorConfiguration`) are in module `ignite-raft-api`, I don't want to inflate the amount of work in this ticket, let's not do it in **IGNITE-18273**. 
   
   IGNITE-18273 already indicates what needs to be done and within it, we will move the configuration, I will mention this in it.



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "rpuch (via GitHub)" <gi...@apache.org>.
rpuch commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1161474223


##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java:
##########
@@ -235,7 +236,36 @@ public <T extends RaftGroupService> CompletableFuture<T> startRaftGroupNode(
         }
 
         try {
-            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory);
+            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory, null);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    @Override
+    public CompletableFuture<RaftGroupService> startRaftGroupNode(
+            RaftNodeId nodeId,
+            PeersAndLearners configuration,
+            RaftGroupListener lsnr,
+            RaftGroupEventsListener eventsLsnr,
+            RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig

Review Comment:
   At least `RaftServer` is in the same module as `RaftGroupOptions`, so it seems that it is possible to pass `RaftNodeDisruptorConfiguration` to `RaftServer` inside of `RaftGroupOptions` easily. Does it take a lot of work? On the other hand, `if this is done, `RaftServer` will become simpler right now.



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1912: IGNITE-19231 Change thread pool for metastore raft group

Posted by "rpuch (via GitHub)" <gi...@apache.org>.
rpuch commented on code in PR #1912:
URL: https://github.com/apache/ignite-3/pull/1912#discussion_r1160735665


##########
modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java:
##########
@@ -3976,7 +3976,7 @@ private RaftGroupService createService(String groupId, TestPeer peer, NodeOption
 
         clusterService.start();
 
-        var service = new RaftGroupService(groupId, peer.getPeerId(), nodeOptions, rpcServer, nodeManager) {
+        var service = new RaftGroupService(groupId, peer.getPeerId(), nodeOptions, rpcServer, nodeManager, null) {

Review Comment:
   How about having two constructors in `RaftGroupService`, one identical to the old one (by parameters), another with this new added parameter, to avoid adding this `, null` all over the place?



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java:
##########
@@ -175,6 +176,14 @@ public void onWrite(Iterator<CommandClosure<WriteCommand>> iterator) {
                 clo.result(null);
             } catch (IgniteInternalException e) {
                 clo.result(e);
+            } catch (CompletionException e) {
+                clo.result(e.getCause());
+            } catch (Throwable t) {

Review Comment:
   Shouldn't we rethrow it after logging? For example, after this change it would become a lot more difficult to notice that an `AssertionError` has happened.



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/server/RaftServer.java:
##########
@@ -76,6 +77,26 @@ boolean startRaftNode(
             RaftGroupOptions groupOptions
     );
 
+    /**
+     * Starts a Raft group bound to this cluster node.
+     *
+     * @param nodeId Raft node ID.
+     * @param configuration Raft configuration.
+     * @param evLsnr Listener for group membership and other events.
+     * @param lsnr Listener for state machine events.
+     * @param groupOptions Options to apply to the group.
+     * @param ownFsmCallerExecutorDisruptorConfig Configuration own (not shared) striped disruptor for FSMCaller service of raft node.

Review Comment:
   ```suggestion
        * @param ownFsmCallerExecutorDisruptorConfig Configuration of own (not shared) striped disruptor for FSMCaller service of raft node.
   ```



##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/JRaftUtils.java:
##########
@@ -54,7 +54,7 @@ public final class JRaftUtils {
      * @return true if bootstrap success
      */
     public static boolean bootstrap(final BootstrapOptions opts) throws InterruptedException {
-        final NodeImpl node = new NodeImpl("bootstrap", new PeerId("127.0.0.1", 0));
+        final NodeImpl node = new NodeImpl("bootstrap", new PeerId("127.0.0.1", 0), null);

Review Comment:
   I suggest adding another constructor to `NodeImpl` to avoid this `, null` all over the place



##########
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/RaftNodeDisruptorConfiguration.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft;
+
+/**
+ * Raft node disruptor configuration.
+ */
+public class RaftNodeDisruptorConfiguration {
+    private final String threadPostfix;
+
+    private final int stripes;
+
+    /**
+     * Constructor.
+     *
+     * @param threadPostfix Postfix the name of the disruptor threads.
+     * @param stripes Number of disruptor stripes.
+     */
+    public RaftNodeDisruptorConfiguration(String threadPostfix, int stripes) {
+        this.threadPostfix = threadPostfix;
+        this.stripes = stripes;
+    }
+
+    /**
+     * Return postfix the name of the disruptor threads.

Review Comment:
   ```suggestion
        * Return Disruptor threads' name postfix.
   ```



##########
modules/raft-api/src/main/java/org/apache/ignite/internal/raft/RaftNodeDisruptorConfiguration.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.raft;
+
+/**
+ * Raft node disruptor configuration.
+ */
+public class RaftNodeDisruptorConfiguration {
+    private final String threadPostfix;
+
+    private final int stripes;
+
+    /**
+     * Constructor.
+     *
+     * @param threadPostfix Postfix the name of the disruptor threads.

Review Comment:
   ```suggestion
        * @param threadPostfix Disruptor threads' name postfix.
   ```



##########
modules/raft/src/main/java/org/apache/ignite/internal/raft/Loza.java:
##########
@@ -235,7 +236,36 @@ public <T extends RaftGroupService> CompletableFuture<T> startRaftGroupNode(
         }
 
         try {
-            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory);
+            return startRaftGroupNodeInternal(nodeId, configuration, lsnr, eventsLsnr, groupOptions, raftServiceFactory, null);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    @Override
+    public CompletableFuture<RaftGroupService> startRaftGroupNode(
+            RaftNodeId nodeId,
+            PeersAndLearners configuration,
+            RaftGroupListener lsnr,
+            RaftGroupEventsListener eventsLsnr,
+            RaftNodeDisruptorConfiguration ownFsmCallerExecutorDisruptorConfig

Review Comment:
   In this class, we have overloads of this method that accept `RaftGroupOptions`. `RaftGroupOptions` was added to aggregate all options that are specific for a group, and it seems that the FSM caller disruptor configuration you are adding can be just added to that class. As a result, we'll not need so many overloads. The new configuration class can be either merged into `RaftGroupOptions`, or included as a sub-structure (as it seems to be passed to `NodeImpl`).
   
   The only catch here is that these overloads (having `RaftGroupOptions` among parameters) are not present on `RaftServer`, but it seems that we can just add them there. By doing this, we'll be able to get rid of that cast to `Loza` in `TableManager` as a by-product.
   
   I consulted @sashapolo on this matter, he seems to support this suggestion. He asked to also add a TODO mentioning https://issues.apache.org/jira/browse/IGNITE-18273 to the new overloads in `RaftService` (if this suggestion gets accepted).



##########
modules/raft/src/main/java/org/apache/ignite/raft/jraft/core/NodeImpl.java:
##########
@@ -225,6 +227,9 @@ public class NodeImpl implements Node, RaftServerService {
      */
     private volatile int electionTimeoutCounter;
 
+    /** Configuration own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor. */

Review Comment:
   ```suggestion
       /** Configuration of own striped disruptor for FSMCaller service of raft node, {@code null} means use shared disruptor. */
   ```



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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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