You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/11/28 14:09:15 UTC

[GitHub] [ignite-3] sashapolo opened a new pull request, #1383: IGNITE-18265 Update Raft packages structure

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

   * Renamed `ignite-raft-client` to `ignite-raft-api`
   * Got rid of JRaft classes leaking from `ignite-raft` as much as possible
   
   https://issues.apache.org/jira/browse/IGNITE-18265


-- 
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] kgusakov commented on a diff in pull request #1383: IGNITE-18265 Update Raft packages structure

Posted by GitBox <gi...@apache.org>.
kgusakov commented on code in PR #1383:
URL: https://github.com/apache/ignite-3/pull/1383#discussion_r1036174962


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -769,7 +775,7 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
                                         );
 
                                         try {
-                                            raftMgr.startRaftGroupNode(
+                                            ((Loza) raftMgr).startRaftGroupNode(

Review Comment:
   Am I right, that in a future we want to use `RaftManager` interface instead? If yes, could you add it to IGNITE-18273 or create another one issue to commit this intention?



##########
modules/affinity/pom.xml:
##########
@@ -33,39 +33,17 @@
     <version>3.0.0-SNAPSHOT</version>
 
     <dependencies>
-        <dependency>

Review Comment:
   Is it just cleanup of legacy unactual dependencies? Just want to clarify.



##########
modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ItMetaStorageServiceTest.java:
##########
@@ -908,38 +895,32 @@ public void testCursorsCleanup() throws Exception {
             return cursor;
         });
 
-        List<Peer> peers = List.of(new Peer(cluster.get(0).topologyService().localMember().name()));
+        List<String> peers = List.of(cluster.get(0).topologyService().localMember().name());
 
-        RaftGroupService metaStorageRaftGrpSvc = RaftGroupServiceImpl.start(
-                INSTANCE,
-                cluster.get(1),
-                FACTORY,
-                10_000,
-                peers,
-                true,
-                200,
-                executor
-        ).get(3, TimeUnit.SECONDS);
+        MetaStorageService metaStorageSvc2 = raftManagers.get(1)
+                .startRaftGroupService(INSTANCE, peers, List.of())
+                .thenApply(service -> new MetaStorageServiceImpl(service, NODE_ID_1, NODE_ID_1))
+                .get(3, TimeUnit.SECONDS);
 
-        try {
-            MetaStorageService metaStorageSvc2 = new MetaStorageServiceImpl(metaStorageRaftGrpSvc, NODE_ID_1, NODE_ID_1);
+        Cursor<Entry> cursorNode0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
 
-            Cursor<Entry> cursorNode0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
+        assertTrue(cursorNode0.hasNext());
 
-            Cursor<Entry> cursor2Node0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
+        Cursor<Entry> cursor2Node0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
 
-            final Cursor<Entry> cursorNode1 = metaStorageSvc2.range(EXPECTED_RESULT_ENTRY.key(), null);
+        assertTrue(cursor2Node0.hasNext());
 
-            metaStorageSvc.closeCursors(NODE_ID_0).get();
+        Cursor<Entry> cursorNode1 = metaStorageSvc2.range(EXPECTED_RESULT_ENTRY.key(), null);
 
-            assertThrows(NoSuchElementException.class, () -> cursorNode0.iterator().next());
+        assertTrue(cursorNode1.hasNext());
 
-            assertThrows(NoSuchElementException.class, () -> cursor2Node0.iterator().next());
+        metaStorageSvc.closeCursors(NODE_ID_0).get();
 
-            assertEquals(EXPECTED_RESULT_ENTRY, (cursorNode1.iterator().next()));
-        } finally {
-            metaStorageRaftGrpSvc.shutdown();

Review Comment:
   Shouldn't we leave this shutdown in place? I know, that it is "no-op" at the moment, but it is a part of RaftGroupService contract and can be extended by any logic in future.



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/MockedStructuresTest.java:
##########
@@ -511,7 +508,7 @@ private TableManager createTableManager() {
                 workDir,
                 msm,
                 schemaManager,
-                view -> new LocalLogStorageFactory(),
+                null,

Review Comment:
   Are we sure, that it isn't dangerous? Maybe ask @rpuch ?



##########
modules/runner/build.gradle:
##########
@@ -32,6 +32,7 @@ dependencies {
     implementation project(':ignite-api')
     implementation project(':ignite-vault')
     implementation project(':ignite-network')
+    implementation project(':ignite-raft-api')

Review Comment:
   What about the same changes in pom.xml?



##########
modules/table/build.gradle:
##########
@@ -33,7 +33,7 @@ dependencies {
     implementation project(':ignite-network-api')
     implementation project(':ignite-raft')
     implementation project(':ignite-replicator')
-    implementation project(':ignite-raft-client')
+    implementation project(':ignite-raft-api')

Review Comment:
   pom.xml?



-- 
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] sashapolo commented on a diff in pull request #1383: IGNITE-18265 Update Raft packages structure

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #1383:
URL: https://github.com/apache/ignite-3/pull/1383#discussion_r1037212449


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -769,7 +775,7 @@ private void updateAssignmentInternal(ConfigurationNotificationEvent<byte[]> ass
                                         );
 
                                         try {
-                                            raftMgr.startRaftGroupNode(
+                                            ((Loza) raftMgr).startRaftGroupNode(

Review Comment:
   yes, I will add it to the ticket



-- 
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 #1383: IGNITE-18265 Update Raft packages structure

Posted by GitBox <gi...@apache.org>.
tkalkirill merged PR #1383:
URL: https://github.com/apache/ignite-3/pull/1383


-- 
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] sashapolo commented on a diff in pull request #1383: IGNITE-18265 Update Raft packages structure

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #1383:
URL: https://github.com/apache/ignite-3/pull/1383#discussion_r1037212068


##########
modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ItMetaStorageServiceTest.java:
##########
@@ -908,38 +895,32 @@ public void testCursorsCleanup() throws Exception {
             return cursor;
         });
 
-        List<Peer> peers = List.of(new Peer(cluster.get(0).topologyService().localMember().name()));
+        List<String> peers = List.of(cluster.get(0).topologyService().localMember().name());
 
-        RaftGroupService metaStorageRaftGrpSvc = RaftGroupServiceImpl.start(
-                INSTANCE,
-                cluster.get(1),
-                FACTORY,
-                10_000,
-                peers,
-                true,
-                200,
-                executor
-        ).get(3, TimeUnit.SECONDS);
+        MetaStorageService metaStorageSvc2 = raftManagers.get(1)
+                .startRaftGroupService(INSTANCE, peers, List.of())
+                .thenApply(service -> new MetaStorageServiceImpl(service, NODE_ID_1, NODE_ID_1))
+                .get(3, TimeUnit.SECONDS);
 
-        try {
-            MetaStorageService metaStorageSvc2 = new MetaStorageServiceImpl(metaStorageRaftGrpSvc, NODE_ID_1, NODE_ID_1);
+        Cursor<Entry> cursorNode0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
 
-            Cursor<Entry> cursorNode0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
+        assertTrue(cursorNode0.hasNext());
 
-            Cursor<Entry> cursor2Node0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
+        Cursor<Entry> cursor2Node0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
 
-            final Cursor<Entry> cursorNode1 = metaStorageSvc2.range(EXPECTED_RESULT_ENTRY.key(), null);
+        assertTrue(cursor2Node0.hasNext());
 
-            metaStorageSvc.closeCursors(NODE_ID_0).get();
+        Cursor<Entry> cursorNode1 = metaStorageSvc2.range(EXPECTED_RESULT_ENTRY.key(), null);
 
-            assertThrows(NoSuchElementException.class, () -> cursorNode0.iterator().next());
+        assertTrue(cursorNode1.hasNext());
 
-            assertThrows(NoSuchElementException.class, () -> cursor2Node0.iterator().next());
+        metaStorageSvc.closeCursors(NODE_ID_0).get();
 
-            assertEquals(EXPECTED_RESULT_ENTRY, (cursorNode1.iterator().next()));
-        } finally {
-            metaStorageRaftGrpSvc.shutdown();

Review Comment:
   my bad, let's bring it back



-- 
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] sashapolo commented on a diff in pull request #1383: IGNITE-18265 Update Raft packages structure

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #1383:
URL: https://github.com/apache/ignite-3/pull/1383#discussion_r1037211521


##########
modules/affinity/pom.xml:
##########
@@ -33,39 +33,17 @@
     <version>3.0.0-SNAPSHOT</version>
 
     <dependencies>
-        <dependency>

Review Comment:
   yes



-- 
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] sashapolo commented on a diff in pull request #1383: IGNITE-18265 Update Raft packages structure

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #1383:
URL: https://github.com/apache/ignite-3/pull/1383#discussion_r1037213791


##########
modules/runner/build.gradle:
##########
@@ -32,6 +32,7 @@ dependencies {
     implementation project(':ignite-api')
     implementation project(':ignite-vault')
     implementation project(':ignite-network')
+    implementation project(':ignite-raft-api')

Review Comment:
   maven build works correctly without this dependency as it is transitive



##########
modules/table/build.gradle:
##########
@@ -33,7 +33,7 @@ dependencies {
     implementation project(':ignite-network-api')
     implementation project(':ignite-raft')
     implementation project(':ignite-replicator')
-    implementation project(':ignite-raft-client')
+    implementation project(':ignite-raft-api')

Review Comment:
   please see my comment above



-- 
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] sashapolo commented on a diff in pull request #1383: IGNITE-18265 Update Raft packages structure

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #1383:
URL: https://github.com/apache/ignite-3/pull/1383#discussion_r1037213161


##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/MockedStructuresTest.java:
##########
@@ -511,7 +508,7 @@ private TableManager createTableManager() {
                 workDir,
                 msm,
                 schemaManager,
-                view -> new LocalLogStorageFactory(),
+                null,

Review Comment:
   This callback is only invoked if an in-memory Raft node has been configured



-- 
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] sashapolo commented on a diff in pull request #1383: IGNITE-18265 Update Raft packages structure

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #1383:
URL: https://github.com/apache/ignite-3/pull/1383#discussion_r1037224500


##########
modules/metastorage-client/src/integrationTest/java/org/apache/ignite/internal/metastorage/client/ItMetaStorageServiceTest.java:
##########
@@ -908,38 +895,32 @@ public void testCursorsCleanup() throws Exception {
             return cursor;
         });
 
-        List<Peer> peers = List.of(new Peer(cluster.get(0).topologyService().localMember().name()));
+        List<String> peers = List.of(cluster.get(0).topologyService().localMember().name());
 
-        RaftGroupService metaStorageRaftGrpSvc = RaftGroupServiceImpl.start(
-                INSTANCE,
-                cluster.get(1),
-                FACTORY,
-                10_000,
-                peers,
-                true,
-                200,
-                executor
-        ).get(3, TimeUnit.SECONDS);
+        MetaStorageService metaStorageSvc2 = raftManagers.get(1)
+                .startRaftGroupService(INSTANCE, peers, List.of())
+                .thenApply(service -> new MetaStorageServiceImpl(service, NODE_ID_1, NODE_ID_1))
+                .get(3, TimeUnit.SECONDS);
 
-        try {
-            MetaStorageService metaStorageSvc2 = new MetaStorageServiceImpl(metaStorageRaftGrpSvc, NODE_ID_1, NODE_ID_1);
+        Cursor<Entry> cursorNode0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
 
-            Cursor<Entry> cursorNode0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
+        assertTrue(cursorNode0.hasNext());
 
-            Cursor<Entry> cursor2Node0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
+        Cursor<Entry> cursor2Node0 = metaStorageSvc.range(EXPECTED_RESULT_ENTRY.key(), null);
 
-            final Cursor<Entry> cursorNode1 = metaStorageSvc2.range(EXPECTED_RESULT_ENTRY.key(), null);
+        assertTrue(cursor2Node0.hasNext());
 
-            metaStorageSvc.closeCursors(NODE_ID_0).get();
+        Cursor<Entry> cursorNode1 = metaStorageSvc2.range(EXPECTED_RESULT_ENTRY.key(), null);
 
-            assertThrows(NoSuchElementException.class, () -> cursorNode0.iterator().next());
+        assertTrue(cursorNode1.hasNext());
 
-            assertThrows(NoSuchElementException.class, () -> cursor2Node0.iterator().next());
+        metaStorageSvc.closeCursors(NODE_ID_0).get();
 
-            assertEquals(EXPECTED_RESULT_ENTRY, (cursorNode1.iterator().next()));
-        } finally {
-            metaStorageRaftGrpSvc.shutdown();

Review Comment:
   fixed



-- 
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] kgusakov commented on a diff in pull request #1383: IGNITE-18265 Update Raft packages structure

Posted by GitBox <gi...@apache.org>.
kgusakov commented on code in PR #1383:
URL: https://github.com/apache/ignite-3/pull/1383#discussion_r1037215661


##########
modules/runner/build.gradle:
##########
@@ -32,6 +32,7 @@ dependencies {
     implementation project(':ignite-api')
     implementation project(':ignite-vault')
     implementation project(':ignite-network')
+    implementation project(':ignite-raft-api')

Review Comment:
   Ok, 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