Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-07 Thread via GitHub


sanpwc merged PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090


-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


sanpwc commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705582319


##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -295,7 +295,7 @@ public ReplicaManager(
 Executor requestsExecutor,
 LongSupplier idleSafeTimePropagationPeriodMsSupplier,
 FailureProcessor failureProcessor,
-Marshaller raftCommandsMarshaller,
+@Nullable Marshaller raftCommandsMarshaller,

Review Comment:
   And I still do not understand what's the purpose of
   ```
   if (snapshotFactory != null) {
   raftGroupOptions.snapshotStorageFactory(snapshotFactory);
   }
   ```
   What's default value for raftGroupOptions.snapshotStorageFactory?



-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


sanpwc commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705582319


##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -295,7 +295,7 @@ public ReplicaManager(
 Executor requestsExecutor,
 LongSupplier idleSafeTimePropagationPeriodMsSupplier,
 FailureProcessor failureProcessor,
-Marshaller raftCommandsMarshaller,
+@Nullable Marshaller raftCommandsMarshaller,

Review Comment:
   And I still do not understand what's the purpose of
   ```
   if (snapshotFactory != null) {
   raftGroupOptions.snapshotStorageFactory(snapshotFactory);
   }
   ```



-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


sanpwc commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705580705


##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -295,7 +295,7 @@ public ReplicaManager(
 Executor requestsExecutor,
 LongSupplier idleSafeTimePropagationPeriodMsSupplier,
 FailureProcessor failureProcessor,
-Marshaller raftCommandsMarshaller,
+@Nullable Marshaller raftCommandsMarshaller,

Review Comment:
   All I'm talking about is the fact that if you mark both fields as @Nullable 
in ReplicaManager constructor you should also mark the fields both in 
ReplicaManager and RaftGroupOptions. 



-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


sanpwc commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705578453


##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -762,16 +762,13 @@ public CompletableFuture startReplica(
 
 /**
  * Creates and start new replica.
- * TODO: must be deleted or be private after 
https://issues.apache.org/jira/browse/IGNITE-22373
  *
  * @param replicaGrpId Replication group id.
  * @param storageIndexTracker Storage index tracker.
  * @param newReplicaListenerFut Future that returns ready ReplicaListener 
for replica creation.
  * @return Future that promises ready new replica when done.
  */
-@VisibleForTesting
-@Deprecated
-public CompletableFuture startReplica(
+private CompletableFuture startReplica(

Review Comment:
   I mean that formally speaking this method is still @Deprecared and 
@VisibleForTesting, thus it's incorrect to remove given annotations.



-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


JAkutenshi commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705543214


##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -295,7 +295,7 @@ public ReplicaManager(
 Executor requestsExecutor,
 LongSupplier idleSafeTimePropagationPeriodMsSupplier,
 FailureProcessor failureProcessor,
-Marshaller raftCommandsMarshaller,
+@Nullable Marshaller raftCommandsMarshaller,

Review Comment:
   Previously there were empty snapshot storage implementation, but later I 
found that there are places where `RaftGroupOptions#defaults` is used and all 
discussed fields are handled with null-checks like in 
`org.apache.ignite.internal.raft.server.RaftGroupOptions#snapshotStorageFactory()`
 lines 480-482 for marshaller and 493-495 for storage factory. There is a point 
why I had been decided to use already existed mechanisms.
   
   On the other hand in tests there several (mostly dependencies) issues with 
especially the factory and marshaller, on of the most known is [IGNITE-2 
with thread local marshaller in a 
test](https://issues.apache.org/jira/browse/IGNITE-2). With the current 
solution we're reusing internal raft code and rely on it and get rid of 
dependency issues in tests.
   
   >only commandsMarshaller is marked as https://github.com/nullable, but 
snapshotStorageFactory doesn't.
   
   About that, we can look inside `RaftGroupOptions` and see that fields 
inconsistently marked as `Nullable`, but they aren't initializing in the 
constructor and the only way to set the value is setters that may not be 
called. The most fair way is to mark the snapshots factory as nullable too and 
remove null checks in `ReplicaManager`, relying totally on raft internals.



-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


JAkutenshi commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705543214


##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -295,7 +295,7 @@ public ReplicaManager(
 Executor requestsExecutor,
 LongSupplier idleSafeTimePropagationPeriodMsSupplier,
 FailureProcessor failureProcessor,
-Marshaller raftCommandsMarshaller,
+@Nullable Marshaller raftCommandsMarshaller,

Review Comment:
   Previously there were empty snapshot storage implementation, but later I 
found that there are places where `RaftGroupOptions#defaults` is used and all 
discussed fields are handled with null-checks like in 
`org.apache.ignite.internal.raft.server.RaftGroupOptions#snapshotStorageFactory()`
 lines 480-482 for marshaller and 493-495 for storage factory. There is a point 
why I had been decided to use already existed mechanisms.
   
   On the other hand in tests there several (mostly dependencies) issues with 
especially the factory and marshaller, on of the most known is [IGNITE-2 
with thread local marshaller in a 
test](https://issues.apache.org/jira/browse/IGNITE-2). With the current 
solution we're reusing internal raft code and rely on it and get rid of 
dependency issues in tests.
   
   >only commandsMarshaller is marked as https://github.com/nullable, but 
snapshotStorageFactory doesn't.
   
   About that, we can look inside `RaftGroupOptions` and see that fields 
inconsistently marked as `@Nullable`, but they aren't initializing in the 
constructor and the only way to set the value is setters that may not be 
called. The most fair way is to mark the snapshots factory as nullable too and 
remove null checks in `ReplicaManager`, relying totally on raft internals.



-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


JAkutenshi commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705498553


##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -762,16 +762,13 @@ public CompletableFuture startReplica(
 
 /**
  * Creates and start new replica.
- * TODO: must be deleted or be private after 
https://issues.apache.org/jira/browse/IGNITE-22373
  *
  * @param replicaGrpId Replication group id.
  * @param storageIndexTracker Storage index tracker.
  * @param newReplicaListenerFut Future that returns ready ReplicaListener 
for replica creation.
  * @return Future that promises ready new replica when done.
  */
-@VisibleForTesting
-@Deprecated
-public CompletableFuture startReplica(
+private CompletableFuture startReplica(

Review Comment:
   Yes, it will be removed an the current state of 22373's patch [reflects 
it](https://github.com/apache/ignite-3/pull/4144/files#diff-69e9a603ac4198f678f9d816bfa70292521abcc8a22373e3114062e4fc2cL734-L820).



##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -762,16 +762,13 @@ public CompletableFuture startReplica(
 
 /**
  * Creates and start new replica.
- * TODO: must be deleted or be private after 
https://issues.apache.org/jira/browse/IGNITE-22373
  *
  * @param replicaGrpId Replication group id.
  * @param storageIndexTracker Storage index tracker.
  * @param newReplicaListenerFut Future that returns ready ReplicaListener 
for replica creation.
  * @return Future that promises ready new replica when done.
  */
-@VisibleForTesting
-@Deprecated
-public CompletableFuture startReplica(
+private CompletableFuture startReplica(

Review Comment:
   Yes, it will be removed and the current state of 22373's patch [reflects 
it](https://github.com/apache/ignite-3/pull/4144/files#diff-69e9a603ac4198f678f9d816bfa70292521abcc8a22373e3114062e4fc2cL734-L820).



-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


sanpwc commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705476530


##
modules/table/src/testFixtures/java/org/apache/ignite/distributed/ItTxTestCluster.java:
##
@@ -673,112 +696,72 @@ public TableViewInternal startTable(String tableName, 
SchemaDescriptor schemaDes
 mock(IndexMetaStorage.class)
 );
 
-CompletableFuture partitionReadyFuture = 
raftServers.get(assignment).startRaftGroupNode(
-new RaftNodeId(grpId, configuration.peer(assignment)),
-configuration,
-partitionListener,
-RaftGroupEventsListener.noopLsnr,
-topologyAwareRaftGroupServiceFactory
-).thenAccept(
-raftSvc -> {
-PartitionReplicaListener listener = 
newReplicaListener(
-mvPartStorage,
-raftSvc,
-txManagers.get(assignment),
-Runnable::run,
-partId,
-tableId,
-() -> Map.of(pkLocker.id(), pkLocker),
-pkStorage,
-Map::of,
-clockServices.get(assignment),
-safeTime,
-txStateStorage,
-transactionStateResolver,
-storageUpdateHandler,
-new 
DummyValidationSchemasSource(schemaManager),
-
nodeResolver.getByConsistentId(assignment),
-new AlwaysSyncedSchemaSyncService(),
-catalogService,
-placementDriver,
-nodeResolver,
-cursorRegistries.get(assignment),
-schemaManager
-);
-
-replicaManagers.get(assignment).startReplica(
-new TablePartitionId(tableId, partId),
-storageIndexTracker,
-completedFuture(listener)
-);
-}
+Function 
createReplicaListener = raftClient -> newReplicaListener(
+mvPartStorage,
+raftClient,
+txManagers.get(assignment),
+Runnable::run,
+partId,
+tableId,
+() -> Map.of(pkLocker.id(), pkLocker),
+pkStorage,
+Map::of,
+clockServices.get(assignment),
+safeTime,
+txStateStorage,
+transactionStateResolver,
+storageUpdateHandler,
+new DummyValidationSchemasSource(schemaManager),
+nodeResolver.getByConsistentId(assignment),
+new AlwaysSyncedSchemaSyncService(),
+catalogService,
+placementDriver,
+nodeResolver,
+cursorRegistries.get(assignment),
+schemaManager
 );
 
+CompletableFuture partitionReadyFuture = 
replicaManagers.get(assignment)
+.startReplica(
+RaftGroupEventsListener.noopLsnr,
+partitionListener,
+false,
+null,
+createReplicaListener,
+storageIndexTracker,
+grpId,
+configuration
+)
+.thenAccept(unused -> { });
+
 partitionReadyFutures.add(partitionReadyFuture);
 }
 
-PeersAndLearners membersConf = 
PeersAndLearners.fromConsistentIds(partAssignments);
-
-if (startClient) {
-RaftGroupService service = RaftGroupServiceImpl
-.start(grpId, client, FACTORY, raftConfig, 
membersConf, true, executor, commandsMarshaller)
-.get(5, TimeUnit.SECONDS);
+// waiting for started replicas otherwise we would have NPE on 
{@link R

Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


sanpwc commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705476530


##
modules/table/src/testFixtures/java/org/apache/ignite/distributed/ItTxTestCluster.java:
##
@@ -673,112 +696,72 @@ public TableViewInternal startTable(String tableName, 
SchemaDescriptor schemaDes
 mock(IndexMetaStorage.class)
 );
 
-CompletableFuture partitionReadyFuture = 
raftServers.get(assignment).startRaftGroupNode(
-new RaftNodeId(grpId, configuration.peer(assignment)),
-configuration,
-partitionListener,
-RaftGroupEventsListener.noopLsnr,
-topologyAwareRaftGroupServiceFactory
-).thenAccept(
-raftSvc -> {
-PartitionReplicaListener listener = 
newReplicaListener(
-mvPartStorage,
-raftSvc,
-txManagers.get(assignment),
-Runnable::run,
-partId,
-tableId,
-() -> Map.of(pkLocker.id(), pkLocker),
-pkStorage,
-Map::of,
-clockServices.get(assignment),
-safeTime,
-txStateStorage,
-transactionStateResolver,
-storageUpdateHandler,
-new 
DummyValidationSchemasSource(schemaManager),
-
nodeResolver.getByConsistentId(assignment),
-new AlwaysSyncedSchemaSyncService(),
-catalogService,
-placementDriver,
-nodeResolver,
-cursorRegistries.get(assignment),
-schemaManager
-);
-
-replicaManagers.get(assignment).startReplica(
-new TablePartitionId(tableId, partId),
-storageIndexTracker,
-completedFuture(listener)
-);
-}
+Function 
createReplicaListener = raftClient -> newReplicaListener(
+mvPartStorage,
+raftClient,
+txManagers.get(assignment),
+Runnable::run,
+partId,
+tableId,
+() -> Map.of(pkLocker.id(), pkLocker),
+pkStorage,
+Map::of,
+clockServices.get(assignment),
+safeTime,
+txStateStorage,
+transactionStateResolver,
+storageUpdateHandler,
+new DummyValidationSchemasSource(schemaManager),
+nodeResolver.getByConsistentId(assignment),
+new AlwaysSyncedSchemaSyncService(),
+catalogService,
+placementDriver,
+nodeResolver,
+cursorRegistries.get(assignment),
+schemaManager
 );
 
+CompletableFuture partitionReadyFuture = 
replicaManagers.get(assignment)
+.startReplica(
+RaftGroupEventsListener.noopLsnr,
+partitionListener,
+false,
+null,
+createReplicaListener,
+storageIndexTracker,
+grpId,
+configuration
+)
+.thenAccept(unused -> { });
+
 partitionReadyFutures.add(partitionReadyFuture);
 }
 
-PeersAndLearners membersConf = 
PeersAndLearners.fromConsistentIds(partAssignments);
-
-if (startClient) {
-RaftGroupService service = RaftGroupServiceImpl
-.start(grpId, client, FACTORY, raftConfig, 
membersConf, true, executor, commandsMarshaller)
-.get(5, TimeUnit.SECONDS);
+// waiting for started replicas otherwise we would have NPE on 
{@link R

Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


sanpwc commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705473067


##
modules/table/src/testFixtures/java/org/apache/ignite/distributed/ItTxTestCluster.java:
##
@@ -673,112 +696,72 @@ public TableViewInternal startTable(String tableName, 
SchemaDescriptor schemaDes
 mock(IndexMetaStorage.class)
 );
 
-CompletableFuture partitionReadyFuture = 
raftServers.get(assignment).startRaftGroupNode(
-new RaftNodeId(grpId, configuration.peer(assignment)),
-configuration,
-partitionListener,
-RaftGroupEventsListener.noopLsnr,
-topologyAwareRaftGroupServiceFactory
-).thenAccept(
-raftSvc -> {
-PartitionReplicaListener listener = 
newReplicaListener(
-mvPartStorage,
-raftSvc,
-txManagers.get(assignment),
-Runnable::run,
-partId,
-tableId,
-() -> Map.of(pkLocker.id(), pkLocker),
-pkStorage,
-Map::of,
-clockServices.get(assignment),
-safeTime,
-txStateStorage,
-transactionStateResolver,
-storageUpdateHandler,
-new 
DummyValidationSchemasSource(schemaManager),
-
nodeResolver.getByConsistentId(assignment),
-new AlwaysSyncedSchemaSyncService(),
-catalogService,
-placementDriver,
-nodeResolver,
-cursorRegistries.get(assignment),
-schemaManager
-);
-
-replicaManagers.get(assignment).startReplica(
-new TablePartitionId(tableId, partId),
-storageIndexTracker,
-completedFuture(listener)
-);
-}
+Function 
createReplicaListener = raftClient -> newReplicaListener(
+mvPartStorage,
+raftClient,
+txManagers.get(assignment),
+Runnable::run,
+partId,
+tableId,
+() -> Map.of(pkLocker.id(), pkLocker),
+pkStorage,
+Map::of,
+clockServices.get(assignment),
+safeTime,
+txStateStorage,
+transactionStateResolver,
+storageUpdateHandler,
+new DummyValidationSchemasSource(schemaManager),
+nodeResolver.getByConsistentId(assignment),
+new AlwaysSyncedSchemaSyncService(),
+catalogService,
+placementDriver,
+nodeResolver,
+cursorRegistries.get(assignment),
+schemaManager
 );
 
+CompletableFuture partitionReadyFuture = 
replicaManagers.get(assignment)
+.startReplica(
+RaftGroupEventsListener.noopLsnr,
+partitionListener,
+false,
+null,
+createReplicaListener,
+storageIndexTracker,
+grpId,
+configuration
+)
+.thenAccept(unused -> { });

Review Comment:
   I'd rather make partitionReadyFutures `List>`



-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


sanpwc commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705447303


##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -762,16 +762,13 @@ public CompletableFuture startReplica(
 
 /**
  * Creates and start new replica.
- * TODO: must be deleted or be private after 
https://issues.apache.org/jira/browse/IGNITE-22373
  *
  * @param replicaGrpId Replication group id.
  * @param storageIndexTracker Storage index tracker.
  * @param newReplicaListenerFut Future that returns ready ReplicaListener 
for replica creation.
  * @return Future that promises ready new replica when done.
  */
-@VisibleForTesting
-@Deprecated
-public CompletableFuture startReplica(
+private CompletableFuture startReplica(

Review Comment:
   It's still @Deprecated @VisibleForTesting and should be removed in 
https://issues.apache.org/jira/browse/IGNITE-22373. The only change for now is 
public -> private.



-- 
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-unsubscr...@ignite.apache.org

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



Re: [PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-08-06 Thread via GitHub


sanpwc commented on code in PR #4090:
URL: https://github.com/apache/ignite-3/pull/4090#discussion_r1705442763


##
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##
@@ -295,7 +295,7 @@ public ReplicaManager(
 Executor requestsExecutor,
 LongSupplier idleSafeTimePropagationPeriodMsSupplier,
 FailureProcessor failureProcessor,
-Marshaller raftCommandsMarshaller,
+@Nullable Marshaller raftCommandsMarshaller,

Review Comment:
   Annotating both raftCommandsMarshaller and snapshotStorageFactory with 
@Nullable is not consistent.
   First of all
   
   1. In RaftGroupOptions only commandsMarshaller is marked as @Nullable, but 
snapshotStorageFactory doesn't.
   2. Aforementioned fields in ReplicaManager itself aren't annotated.
   3. What's the point in !null wrapped assignment like 
   ```
   if (snapshotFactory != null) {
   raftGroupOptions.snapshotStorageFactory(snapshotFactory);
   }
   ```
   ?



-- 
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-unsubscr...@ignite.apache.org

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



[PR] IGNITE-22691 Delete startReplica(ReplicationGroupId, PendingComparableValuesTracker, CompletableFuture) [ignite-3]

2024-07-15 Thread via GitHub


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

   JIRA Ticket: [IGNITE-22691 | 
](https://issues.apache.org/jira/browse/IGNITE-22691)
   
   ## The goal
   
   ## The reason
   
   ## The solution
   
   ---
   
   Thank you for submitting the pull request.
   
   To streamline the review process of the patch and ensure better code quality
   we ask both an author and a reviewer to verify the following:
   
   ### The Review Checklist
   - [ ] **Formal criteria:** TC status, codestyle, mandatory documentation. 
Also make sure to complete the following:  
   \- There is a single JIRA ticket related to the pull request.  
   \- The web-link to the pull request is attached to the JIRA ticket.  
   \- The JIRA ticket has the Patch Available state.  
   \- The description of the JIRA ticket explains WHAT was made, WHY and HOW.  
   \- The pull request title is treated as the final commit message. The 
following pattern must be used: IGNITE- Change summary where  - number 
of JIRA issue.
   - [ ] **Design:** new code conforms with the design principles of the 
components it is added to.
   - [ ] **Patch quality:** patch cannot be split into smaller pieces, its size 
must be reasonable.
   - [ ] **Code quality:** code is clean and readable, necessary developer 
documentation is added if needed.
   - [ ] **Tests code quality:** test set covers positive/negative scenarios, 
happy/edge cases. Tests are effective in terms of execution time and resources.
   
   ### Notes
   - [Apache Ignite Coding 
Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Java+Code+Style+Guide)


-- 
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-unsubscr...@ignite.apache.org

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