Uploaded image for project: 'Ignite'
  1. Ignite
  2. IGNITE-17582

Introduce mutipleReplica ReplicaRequests together with grouped TxCleanupReplicaRequest processing

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Open
    • Major
    • Resolution: Unresolved
    • None
    • None
    • None

    Description

      Motivation

      Within the scope of tx commit process, enlisted partitions are grouped together by primary replica cluster node, however during the tx finish processing brand new TxCleanupReplicaRequest is sent to each partition even colocated ones within same cluster node:

       

      public CompletableFuture<Void> cleanup(
              ClusterNode recipientNode,
              List<String> replicationGroupIds,
              UUID txId,
              boolean commit,
              HybridTimestamp commitTimestamp
      ) {
          // TODO: https://issues.apache.org/jira/browse/IGNITE-17582 Grouping replica requests.
          replicationGroupIds.forEach(groupId -> {
              try {
                  replicaService.invoke(
                          recipientNode,
                          FACTORY.txCleanupReplicaRequest()
                                  .groupId(groupId)
                                  .txId(txId)
                                  .commit(commit)
                                  .commitTimestamp(commitTimestamp)
                                  .build()
                  );
              } catch (NodeStoppingException e) {
                  throw new TransactionException("Failed to perform tx cleanup, node is stopping.");
              }
          });
      
          return null;
      } 

      Seems that we may introduce multipleReplicationGroups in addition to single ones that we already have in order to perform such collocated based requests.

      Definition of Done

      • General solution for groupping collocated replica requests is introduced.
      • TxCleanupReplicaRequests are grouped on recipient node basis.

      Implementation Notes

      Generally speaking, it's required to:

      • Introduce new interface  MutipleReplicasReplicaRequest with Collection<String> groupsId() in addition to already existing ReplicaRequest with single String groupId();
      • Implement proper multiple replicas fun out and aggregation(including expcetion handling) of the results within org.apache.ignite.network.NetworkMessageHandler#onReceived
      • Send such new TxCleanupReplicaRequests one per cluster node instead of one per replica, in other words rework org.apache.ignite.internal.tx.impl.TxManagerImpl#cleanup 

       

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              alapin Alexander Lapin
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated: