This is an automated email from the ASF dual-hosted git repository.

vpyatkov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new d4de4f6a71 IGNITE-18456 Explain treading model in corresponding 
README.md file for MetaStorageManager (#1493)
d4de4f6a71 is described below

commit d4de4f6a714df05905db9eafa4e530c55adc207d
Author: Vladislav Pyatkov <[email protected]>
AuthorDate: Thu Jan 12 16:11:32 2023 +0400

    IGNITE-18456 Explain treading model in corresponding README.md file for 
MetaStorageManager (#1493)
---
 modules/metastorage/README.md                      | 45 ++++++++++++++++++++++
 .../metastorage/impl/MetaStorageManagerImpl.java   |  3 +-
 2 files changed, 46 insertions(+), 2 deletions(-)

diff --git a/modules/metastorage/README.md b/modules/metastorage/README.md
new file mode 100644
index 0000000000..27a752e5d2
--- /dev/null
+++ b/modules/metastorage/README.md
@@ -0,0 +1,45 @@
+# Metastorage
+
+The module for storing and accessing metadata. This module is linked to 
another one:
+
+- metastorage-api - The module contains classes that other components can use 
to access the metastorage service.
+
+To avoid data loss, the storage is distributed using the RAFT consensus 
algorithm. Every cluster node has access to such distributed
+storage, but the majority of members are learners, and only a small number are 
voters in terms of RAFT algorithm. Typically, only a small
+number of the cluster's voting nodes make up the raft-group (number of nodes 
must be odd, either 3 or 5.). The remaining nodes listen to
+metadata updates but do not vote (they called learners in terms of RAFT).
+
+## Threading model
+
+There is only one thread specifically for notifying watchers. The thread is 
created in KeyValueStorage from the executor:
+
+```java
+private ExecutorService watchExecutor = Executors.newSingleThreadExecutor(new 
NamedThreadFactory(NamedThreadFactory
+        .threadPrefix(nodeName, "metastorage-watch-executor"),LOG));
+```
+
+Additionally, the storage internally has two thread executors for creating a 
snapshot:
+
+```java
+private ExecutorService snapshotExecutor = Executors.newFixedThreadPool(2,new 
NamedThreadFactory(NamedThreadFactory
+        .threadPrefix(nodeName, "metastorage-snapshot-executor"),LOG));
+```
+
+To distinguish to which node the specific thread belongs, each executor has a 
node name in its prefix.
+
+### Interface methods
+
+Futures are returned by a number of Metastorage Manager operations, like 
get(), getAll(), invoke(), etc. Those futures are completed when
+the corresponding RAFT command is completed in the Metastorage group. As the 
entire replication process takes place in RAFT threads, this
+result appears in the RAFT client executor with prefix 
<NODE_NAME>%Raft-Group-Client. See RAFT module for more information about its
+threading model.
+
+Although some methods return futures, they are often run synchronously. 
Futures are dependent on asynchronous Metastorage initialization,
+since starting an RAFT group requires time. In other words, we have a chance 
to complete those features in the RAFT client executor thread (
+prefix NODE NAME>%Raft-Group-Client).
+
+### Using common pool
+
+The component uses common ForkJoinPool on start (in fact, it is not necessary, 
because all components start asynchronously in the same
+ForkJoinPool). The using of the common pool is dangerous, because the pool can 
be occupied by another threads that hosted on the same JVM 
+(TODO: IGNITE-18505 Avoid using common pool on start components).
diff --git 
a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
 
b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
index 172b65692e..6848877983 100644
--- 
a/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
+++ 
b/modules/metastorage/src/main/java/org/apache/ignite/internal/metastorage/impl/MetaStorageManagerImpl.java
@@ -199,8 +199,7 @@ public class MetaStorageManagerImpl implements 
MetaStorageManager {
     @Override
     public void start() {
         this.metaStorageSvcFut = cmgMgr.metaStorageNodes()
-                // use default executor to avoid blocking CMG manager threads
-                .thenComposeAsync(metaStorageNodes -> {
+                .thenCompose(metaStorageNodes -> {
                     if (!busyLock.enterBusy()) {
                         return CompletableFuture.failedFuture(new 
NodeStoppingException());
                     }

Reply via email to