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/07/05 20:45:06 UTC

[GitHub] [ignite-3] vldpyatkov opened a new pull request, #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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

   …ReplicaListener interfaces


-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/core/src/main/java/org/apache/ignite/lang/ErrorGroups.java:
##########
@@ -44,4 +44,19 @@ public static class Table {
         /** Column not found. */
         public static int COLUMN_NOT_FOUND_ERR = TABLE_ERR_GROUP.registerErrorCode(4);
     }
+
+    /** Replicator error group. */
+    public static class Replicator {
+        /** Replicator error group. */
+        public static final ErrorGroup REPLICATOR_ERR_GROUP = ErrorGroup.newGroup("REP", 3);

Review Comment:
   Could you please merge fresh ignite3_tx? Denis already added group with code 3.
   `        public static final ErrorGroup TX_ERR_GROUP = ErrorGroup.newGroup("TX", 3);`



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()

Review Comment:
   In most cases, multi request leads to multi response, but nothing limits other way for some operation (or pass the response with null container, if the operation does not return any data).



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(resp -> {
+            assert resp instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", resp);
+
+            return (ReplicaResponse) resp;
+        });
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.
+     * @param request Request.
+     * @return Future to response.
+     */
+    public CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request) {
+        //TODO: Check replica is alive.
+        return sendToReplica(node, request);
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.
+     * @param request Request.
+     * @param storageId Storage id.
+     * @return Future to response.
+     */
+    public CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request, String storageId) {
+        //TODO: Check replica is alive.
+        return sendToReplica(node, request);
+
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws Exception {
+        //Free the resources.

Review Comment:
   I can remove Autoclosbale interface here.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);

Review Comment:
   But you should)) We need exception handling here.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.replicator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.message.CompleteOperationRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future.
+     */
+    private final ConcurrentHashMap<UUID, CompletableFuture> ops = new ConcurrentHashMap<>();
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param replicaGrpId Replication group id.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String replicaGrpId,
+            ReplicaListener listener
+    ) {
+        this.replicaGrpId = replicaGrpId;
+        this.listener = listener;
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert replicaGrpId.equals(request.replicationGroupId()) : IgniteStringFormatter.format(
+                "Partition mismatch: request does not match the replica [reqPartId={}, replicaGrpId={}]", request.replicationGroupId(),
+                replicaGrpId);
+
+        //TODO:IGNITE-17378 Check replica is alive.
+
+        ReplicaResponse response;
+
+        try {
+            if (request instanceof CompleteOperationRequest) {
+                var opsRequest = (CompleteOperationRequest) request;
+
+                List<UUID> opIds = opsRequest.operationIds();
+
+                HashMap<UUID, Object> opRes = null;
+
+                if (opIds != null && !opIds.isEmpty()) {
+                    opRes = new HashMap<>(opIds.size());
+
+                    for (Entry<UUID, CompletableFuture> op : ops.entrySet()) {
+                        if (opIds.contains(op.getKey())) {
+                            opRes.put(op.getKey(), op.getValue().join());
+
+                            ops.remove(op.getKey());

Review Comment:
   I added another request/response procedure for clearing the futures.
   Now we can invoke a method ReplicaService#clearResult for clearing remote replica explicitly.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.replicator;
+
+import java.util.UUID;
+import org.apache.ignite.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.message.OperationType;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica listener.
+ * TODO:IGNITE-17258 Implement ReplicaListener.
+ */
+public class ReplicaListener {
+    /** Versioned partition storage. */
+    private MvPartitionStorage mvDataStorage;
+
+    // /** Transaction state storage. */
+    // private TxnStorage txnStorage;
+
+    // /** Inmemory lock storage. */
+    // private VolatileLockStorage volatileLockStorage;
+
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /**
+     * The listener constructor.
+     *
+     * @param mvDataStorage Partition storage.
+     * @param raftClient Raft client.
+     */
+    public ReplicaListener(MvPartitionStorage mvDataStorage, RaftGroupService raftClient) {
+        this.mvDataStorage = mvDataStorage;
+        this.raftClient = raftClient;
+    }
+
+    /**
+     * Applies an request for RW transaction.
+     *
+     * @param txId Transaction id.
+     * @param data      Data.
+     * @param opType    Operation type.
+     * @return Result.
+     */
+    public Object applyReadWrite(UUID txId, Object data, OperationType opType) {

Review Comment:
   How are you going to evaluate commitPartition from txId only?



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {

Review Comment:
   I will change the Javadoc, but I don't want parametrized this method with a listener, because we have only one implementation.
   If that method will be required in the future, I think, we will be able to extract a listener interface and add a new method for this case.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return getReplicaInternal(partId, raftClient);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Internal method for start a replica.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    private Replica getReplicaInternal(String partId, RaftGroupService raftClient) {
+        var replica = new Replica(partId, clusterNetSvc.topologyService(), new ReplicaListener(mvDataStorage, raftClient));
+
+        Replica previous = replicas.putIfAbsent(partId, replica);

Review Comment:
   Compute if absent is not allow understanding the swap happened or 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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {

Review Comment:
   You are right ROReplicaRequest and RWReplicaRequest have different set of attributes, but is there any reason to have two separate methods for them? Do you expect some generic handlers for all RO requests and other generic actions for all RW requests?



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;

Review Comment:
   Up to you.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);

Review Comment:
   I do nothing about that.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);

Review Comment:
   I did nothing about that.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaRequestLocator.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.replicator.message;
+
+import java.util.UUID;
+
+/**
+ * The class identifies a replication request as a part of a business transaction.
+ */
+public interface ReplicaRequestLocator {
+    /**
+     * Gets a replication group id.
+     *
+     * @return Replication group id.
+     */
+    String groupId();
+
+    /**
+     * Gets a transaction id. The id is the same for several requests in one transaction.
+     *
+     * @return Transaction id.
+     */
+    UUID transactionId();

Review Comment:
   That looks confusing. There are no transactions in CMG and MetaStorage.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaResponse.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.replicator.message;
+
+import java.util.UUID;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Replica response interface.
+ * TODO:IGNITE-17258 Add a specific response type for a replica listener. (@Transferable(ReplicaMessageGroup.TYPE_RESPONSE))
+ */
+public interface ReplicaResponse extends NetworkMessage {
+
+    /**
+     * Operation id in the replica with which the response is sent.
+     *
+     * @return Operation id.
+     */
+    UUID operationId();

Review Comment:
   As far as I understand there are three types of responses 
   -  Instant responses, e.g. read responses or write responses that already finish its replication. For such responses operationId doesn't have any sense.
   - Delayed responses, e.g. write responses that didn't finish its replication yet. For such responses operation id has sense.
   - Mixed responses, e.g. getAndRemove().



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;

Review Comment:
   I am not sure that the issue will be fixed in IGNITE-17255.
   There is a common problem for digit constants, not only for ignite-replica module.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.network.ClusterNode;
+
+/**
+ * Replica service.
+ */
+public interface ReplicaService extends AutoCloseable {
+    /** Replicator network message factory. */
+    static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Network timeout. */
+    static final int RPC_TIMEOUT = 3000;
+
+    /**
+     * Determines the replica server or not.
+     * If the replica is not a server, it delegates all command to a remote replica server.
+     *
+     * @return True when the replica is server, false when this is only client service.
+     */
+    boolean isServer();
+
+    /**
+     * Handles replication request.
+     *
+     * @param request Request.
+     * @return Future to response.

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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaResponse.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.replicator.message;
+
+import java.util.UUID;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Replica response interface.
+ * TODO:IGNITE-17258 Add a specific response type for a replica listener. (@Transferable(ReplicaMessageGroup.TYPE_RESPONSE))
+ */
+public interface ReplicaResponse extends NetworkMessage {
+
+    /**
+     * Operation id in the replica with which the response is sent.
+     *
+     * @return Operation id.
+     */
+    UUID operationId();

Review Comment:
   Divided responses to three types as you proposed.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.network.ClusterNode;
+
+/**
+ * Replica service.
+ */
+public interface ReplicaService extends AutoCloseable {
+    /** Replicator network message factory. */
+    static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Network timeout. */
+    static final int RPC_TIMEOUT = 3000;
+
+    /**
+     * Determines the replica server or not.
+     * If the replica is not a server, it delegates all command to a remote replica server.
+     *
+     * @return True when the replica is server, false when this is only client service.
+     */
+    boolean isServer();

Review Comment:
   Removed.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/README.md:
##########
@@ -0,0 +1,4 @@
+# Ignite replicator module.

Review Comment:
   I created the ticket and added TODO here.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return getReplicaInternal(partId, raftClient);

Review Comment:
   Renamed.
   I use internal method for avoiding too much nested braces, that grows out the code by weight.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/OperationType.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.replicator.message;
+
+import java.io.Serializable;
+
+/**
+ * Transaction operation type.
+ */
+public enum OperationType implements Serializable {

Review Comment:
   I checked it.
   I still think it is a good way.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.replicator;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.message.CleanupRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.WaiteOperationResultRequest;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future of operation result.
+     * The first id is a business transaction id within which the operation is handled.
+     * The second one is an id of operation (the id is locally generated when the replication request is received).
+     * Operation future is a leaf element for waiting the operation completion and receiving a result.
+     */
+    private final ConcurrentHashMap<UUID, ConcurrentHashMap<UUID, CompletableFuture>> ops = new ConcurrentHashMap<>();
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param replicaGrpId Replication group id.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String replicaGrpId,
+            ReplicaListener listener
+    ) {
+        this.replicaGrpId = replicaGrpId;
+        this.listener = listener;
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert replicaGrpId.equals(request.locator().groupId()) : IgniteStringFormatter.format(
+                "Partition mismatch: request does not match the replica [reqPartId={}, replicaGrpId={}]", request.locator().groupId(),
+                replicaGrpId);
+
+        //TODO:IGNITE-17378 Check replica is alive.
+
+        ReplicaResponse response;
+
+        try {
+            if (request instanceof WaiteOperationResultRequest) {
+                var opsRequest = (WaiteOperationResultRequest) request;

Review Comment:
   I believe that we may introduce private handleWaitOperatoinResultRequest(WaiteOperationResultRequest request) that besides better encapsulation will eliminate explicit cast.  



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.replicator;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.message.CleanupRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.WaiteOperationResultRequest;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future of operation result.
+     * The first id is a business transaction id within which the operation is handled.
+     * The second one is an id of operation (the id is locally generated when the replication request is received).
+     * Operation future is a leaf element for waiting the operation completion and receiving a result.
+     */
+    private final ConcurrentHashMap<UUID, ConcurrentHashMap<UUID, CompletableFuture>> ops = new ConcurrentHashMap<>();
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param replicaGrpId Replication group id.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String replicaGrpId,
+            ReplicaListener listener
+    ) {
+        this.replicaGrpId = replicaGrpId;
+        this.listener = listener;
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert replicaGrpId.equals(request.locator().groupId()) : IgniteStringFormatter.format(
+                "Partition mismatch: request does not match the replica [reqPartId={}, replicaGrpId={}]", request.locator().groupId(),
+                replicaGrpId);
+
+        //TODO:IGNITE-17378 Check replica is alive.
+
+        ReplicaResponse response;
+
+        try {
+            if (request instanceof WaiteOperationResultRequest) {
+                var opsRequest = (WaiteOperationResultRequest) request;

Review Comment:
   I believe that we may introduce private handleWaitOperatoinResultRequest(WaiteOperationResultRequest request) that besides better encapsulation will eliminate explicit class cast.  



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.replicator;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.message.CleanupRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.WaiteOperationResultRequest;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future of operation result.
+     * The first id is a business transaction id within which the operation is handled.
+     * The second one is an id of operation (the id is locally generated when the replication request is received).
+     * Operation future is a leaf element for waiting the operation completion and receiving a result.
+     */
+    private final ConcurrentHashMap<UUID, ConcurrentHashMap<UUID, CompletableFuture>> ops = new ConcurrentHashMap<>();
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param replicaGrpId Replication group id.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String replicaGrpId,
+            ReplicaListener listener
+    ) {
+        this.replicaGrpId = replicaGrpId;
+        this.listener = listener;
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert replicaGrpId.equals(request.locator().groupId()) : IgniteStringFormatter.format(
+                "Partition mismatch: request does not match the replica [reqPartId={}, replicaGrpId={}]", request.locator().groupId(),
+                replicaGrpId);
+
+        //TODO:IGNITE-17378 Check replica is alive.
+
+        ReplicaResponse response;
+
+        try {
+            if (request instanceof WaiteOperationResultRequest) {

Review Comment:
   Probably it's WaitOperation**s**ResultRequest.



-- 
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] vldpyatkov closed pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

Posted by GitBox <gi...@apache.org>.
vldpyatkov closed pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …
URL: https://github.com/apache/ignite-3/pull/927


-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(resp -> {
+            assert resp instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", resp);
+
+            return (ReplicaResponse) resp;
+        });
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.
+     * @param request Request.
+     * @return Future to response.
+     */
+    public CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request) {
+        //TODO: Check replica is alive.

Review Comment:
   Created https://issues.apache.org/jira/browse/IGNITE-17378



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)

Review Comment:
   This identifier required not only for logic in the code, I believe it helps to understand what happening in a debug.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;

Review Comment:
   It's a rule of thumb to pass only necessary objects if possible. What's the problem in passing replica list producer?



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {

Review Comment:
   But those types of transaction have different attributes (timestamp, id) as I said before.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {

Review Comment:
   I noted two exceptions which will be thrown by reasons: "node was stopped" and "replica with the same id is already existed" Also, added the both to the javadoc.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/CompleteOperationRequest.java:
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.replicator.message;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.network.annotations.Marshallable;
+import org.apache.ignite.network.annotations.Transferable;
+
+/**
+ * The request is sent from a coordinator to wait  operations complete.
+ */
+@Transferable(ReplicaMessageGroup.COMPLETE_OP_REQUEST)
+public interface CompleteOperationRequest extends ReplicaRequest {
+
+    /**
+     * Ids of operations that were started before, to wait to complete.
+     *
+     * @return List of operation ids.
+     */
+    @Marshallable
+    List<UUID> operationIds();

Review Comment:
   Does not matter.
   Did you mean it should be notes in javadoc?



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.exception.ReplicaAlreadyIsStartedException;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Manager to rule replicas.
+ * Only a single instance of the class exists in Ignite node.
+ * This class allow to start/stop/get a replica.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc) {
+        this.clusterNetSvc = clusterNetSvc;
+    }
+
+    /**
+     * Gets a replica by its replica group id.
+     *
+     * @param replicaGrpId Replication group id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     * @throws NodeStoppingException Is thrown when the node is stopping.
+     */
+    public Replica replica(String replicaGrpId) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            return replicas.get(replicaGrpId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts a replica. If a replica with the same partition id is already exist the method throws an exception.
+     *
+     * @param replicaGrpId   Replication group id.
+     * @param listener Replica listener.
+     * @return Replica.
+     * @throws NodeStoppingException Is thrown when the node is stopping.
+     * @throws ReplicaAlreadyIsStartedException Is thrown when a replica with the same replication group id already started.
+     */
+    public Replica startReplica(String replicaGrpId, ReplicaListener listener) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            return startReplicaInternal(replicaGrpId, listener);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Internal method for start a replica.
+     *
+     * @param replicaGrpId   Replication group id.
+     * @param listener Replica listener.
+     * @return Replica.
+     */
+    private Replica startReplicaInternal(String replicaGrpId, ReplicaListener listener) {
+        var replica = new Replica(replicaGrpId, listener);
+
+        Replica previous = replicas.putIfAbsent(replicaGrpId, replica);
+
+        if (previous != null) {
+            throw new ReplicaAlreadyIsStartedException(replicaGrpId);
+        }
+
+        return replica;
+    }
+
+    /**
+     * Stops a replica by the partition group id.
+     *
+     * @param replicaGrpId Replication group id.
+     * @return True if the replica is found and closed, false otherwise.
+     * @throws NodeStoppingException Is thrown when the node is stopping.
+     */
+    public boolean stopReplica(String replicaGrpId) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            return stopReplicaInternal(replicaGrpId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * An internal method for stopping a replica.
+     *
+     * @param replicaGrpId Replication group id.
+     * @return True if the replica is found and closed, false otherwise.
+     */
+    private boolean stopReplicaInternal(String replicaGrpId) {
+        Replica replica = replicas.remove(replicaGrpId);
+
+        if (replica == null) {
+            return false;
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void start() {
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.locator().groupId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.

Review Comment:
   sen**d**



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(resp -> {
+            assert resp instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", resp);
+
+            return (ReplicaResponse) resp;
+        });
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.
+     * @param request Request.
+     * @return Future to response.
+     */
+    public CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request) {

Review Comment:
   I do not know what will be happened, because invoke has no the exception description.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicatorUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This is an utility class for serialization cache tuples. It will be removed after another way for serialization is implemented into the
+ * network layer.
+ * TODO: Remove it after (IGNITE-14793)
+ */
+public class ReplicatorUtils {

Review Comment:
   It is not a similar one, the utility is the same as CommandUtils, but I cannot reuse it, because it is located into another module. 



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.

Review Comment:
   Changed the description.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(

Review Comment:
   I mean that it should be a matter of ReplicaListener handlers to check whether type is known or not. In other words it's all about moving given staff to replicaListener.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(

Review Comment:
   This comment does not clear to me.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.replicator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.message.CompleteOperationRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future.
+     */
+    private final ConcurrentHashMap<UUID, CompletableFuture> ops = new ConcurrentHashMap<>();
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param replicaGrpId Replication group id.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String replicaGrpId,
+            ReplicaListener listener
+    ) {
+        this.replicaGrpId = replicaGrpId;
+        this.listener = listener;
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert replicaGrpId.equals(request.replicationGroupId()) : IgniteStringFormatter.format(
+                "Partition mismatch: request does not match the replica [reqPartId={}, replicaGrpId={}]", request.replicationGroupId(),
+                replicaGrpId);
+
+        //TODO:IGNITE-17378 Check replica is alive.
+
+        ReplicaResponse response;
+
+        try {
+            if (request instanceof CompleteOperationRequest) {

Review Comment:
   It's rather AwaiteOperationResultRequest than Complete... because.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.replicator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.message.CompleteOperationRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future.
+     */
+    private final ConcurrentHashMap<UUID, CompletableFuture> ops = new ConcurrentHashMap<>();
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param replicaGrpId Replication group id.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String replicaGrpId,
+            ReplicaListener listener
+    ) {
+        this.replicaGrpId = replicaGrpId;
+        this.listener = listener;
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert replicaGrpId.equals(request.replicationGroupId()) : IgniteStringFormatter.format(
+                "Partition mismatch: request does not match the replica [reqPartId={}, replicaGrpId={}]", request.replicationGroupId(),
+                replicaGrpId);
+
+        //TODO:IGNITE-17378 Check replica is alive.
+
+        ReplicaResponse response;
+
+        try {
+            if (request instanceof CompleteOperationRequest) {
+                var opsRequest = (CompleteOperationRequest) request;
+
+                List<UUID> opIds = opsRequest.operationIds();
+
+                HashMap<UUID, Object> opRes = null;
+
+                if (opIds != null && !opIds.isEmpty()) {
+                    opRes = new HashMap<>(opIds.size());
+
+                    for (Entry<UUID, CompletableFuture> op : ops.entrySet()) {
+                        if (opIds.contains(op.getKey())) {
+                            opRes.put(op.getKey(), op.getValue().join());

Review Comment:
   What will happen if CompleteOperationRequest network timeout is less than future await time?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.replicator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.message.CompleteOperationRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future.
+     */
+    private final ConcurrentHashMap<UUID, CompletableFuture> ops = new ConcurrentHashMap<>();
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param replicaGrpId Replication group id.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String replicaGrpId,
+            ReplicaListener listener
+    ) {
+        this.replicaGrpId = replicaGrpId;
+        this.listener = listener;
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert replicaGrpId.equals(request.replicationGroupId()) : IgniteStringFormatter.format(
+                "Partition mismatch: request does not match the replica [reqPartId={}, replicaGrpId={}]", request.replicationGroupId(),
+                replicaGrpId);
+
+        //TODO:IGNITE-17378 Check replica is alive.
+
+        ReplicaResponse response;
+
+        try {
+            if (request instanceof CompleteOperationRequest) {
+                var opsRequest = (CompleteOperationRequest) request;
+
+                List<UUID> opIds = opsRequest.operationIds();
+
+                HashMap<UUID, Object> opRes = null;
+
+                if (opIds != null && !opIds.isEmpty()) {
+                    opRes = new HashMap<>(opIds.size());
+
+                    for (Entry<UUID, CompletableFuture> op : ops.entrySet()) {
+                        if (opIds.contains(op.getKey())) {
+                            opRes.put(op.getKey(), op.getValue().join());
+
+                            ops.remove(op.getKey());

Review Comment:
   Who and how will remove operation futures in case of coordinator failure?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.exception.ReplicaAlreadyIsStartedException;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Manager to rule replicas.
+ * Only a single instance of the class exists in Ignite node.
+ * This class allow to start/stop/get a replica.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc) {
+        this.clusterNetSvc = clusterNetSvc;
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param replicaGrpId Replication group id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     * @throws NodeStoppingException Is thrown when the node is stopping.
+     */
+    public Replica replica(String replicaGrpId) throws NodeStoppingException {
+        if (!busyLock.enterBusy()) {
+            throw new NodeStoppingException();
+        }
+
+        try {
+            return replicas.get(replicaGrpId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts a replica. If a replica with the same partition id is already exist the method throws an exception.
+     *
+     * @param replicaGrpId   Replication group id.
+     * @param listener Replica listener.
+     * @return Replica.
+     * @throws NodeStoppingException Is thrown when the node is stopping.
+     * @throws ReplicaAlreadyIsStartedException Is thrown when a replica with the same replication group id is started.

Review Comment:
   IsAlreadyStarted or just AlreadyStarted without is.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.exception.ReplicaAlreadyIsStartedException;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Manager to rule replicas.
+ * Only a single instance of the class exists in Ignite node.
+ * This class allow to start/stop/get a replica.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc) {
+        this.clusterNetSvc = clusterNetSvc;
+    }
+
+    /**
+     * Gets a replica by its partition id.

Review Comment:
   javadoc: partitionId -> replica group id



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.replicator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.message.CompleteOperationRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future.

Review Comment:
   Let's add more detailed explanation



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.exception.ReplicationException;
+import org.apache.ignite.internal.replicator.exception.ReplicationTimeoutException;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Response future.
+     * @throws NodeStoppingException Is thrown when the node is stopping.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) throws NodeStoppingException {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.replicationGroupId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(msg -> {
+            assert msg instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", msg);
+
+            return (ReplicaResponse) msg;
+        }).whenComplete((response, throwable) -> {
+            if (throwable != null) {
+                if (throwable instanceof TimeoutException) {

Review Comment:
   what if it's not TimeoutException?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaResponse.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.replicator.message;
+
+import java.util.UUID;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Replica response interface.
+ * TODO:IGNITE-17258 Add a specific response type for a replica listener. (@Transferable(ReplicaMessageGroup.TYPE_RESPONSE))
+ */
+public interface ReplicaResponse extends NetworkMessage {
+
+    /**
+     * Operation id in the replica with which the response is sent.
+     *
+     * @return Operation id.
+     */
+    UUID operationId();

Review Comment:
   Mmm, I'd actually expected that you will provide client side result awaiting logic. I mean that within clientSide ReplicaResponse there will be instant response and replication awaiting logic.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/exception/ReplicationError.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.replicator.exception;
+
+import java.util.HashMap;
+
+/**
+ * Codes of replication errors.
+ */
+public enum ReplicationError {

Review Comment:
   Please consolidate new exceptions with ErrorGroups related logic.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/CompleteOperationRequest.java:
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.replicator.message;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.network.annotations.Marshallable;
+import org.apache.ignite.network.annotations.Transferable;
+
+/**
+ * The request is sent from a coordinator to wait  operations complete.
+ */
+@Transferable(ReplicaMessageGroup.COMPLETE_OP_REQUEST)
+public interface CompleteOperationRequest extends ReplicaRequest {
+
+    /**
+     * Ids of operations that were started before, to wait to complete.
+     *
+     * @return List of operation ids.
+     */
+    @Marshallable
+    List<UUID> operationIds();

Review Comment:
   Order matters?



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.replicator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.message.CompleteOperationRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future.
+     */
+    private final ConcurrentHashMap<UUID, CompletableFuture> ops = new ConcurrentHashMap<>();
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param replicaGrpId Replication group id.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String replicaGrpId,
+            ReplicaListener listener
+    ) {
+        this.replicaGrpId = replicaGrpId;
+        this.listener = listener;
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert replicaGrpId.equals(request.replicationGroupId()) : IgniteStringFormatter.format(
+                "Partition mismatch: request does not match the replica [reqPartId={}, replicaGrpId={}]", request.replicationGroupId(),
+                replicaGrpId);
+
+        //TODO:IGNITE-17378 Check replica is alive.
+
+        ReplicaResponse response;
+
+        try {
+            if (request instanceof CompleteOperationRequest) {

Review Comment:
   Renamed.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()

Review Comment:
   So that means that given code should be updated, is that correct? Meaning that we should analyze ReplicaResponse inner data and return either mutiEntry or singleEntry message depending on ReplicaResponse attributes.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/MultiEntryResponse.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.replicator.message;
+
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.network.annotations.Marshallable;
+import org.apache.ignite.network.annotations.Transferable;
+
+/**
+ * Replica response.
+ */
+@Transferable(ReplicaMessageGroup.REPLICA_MULTI_RESPONSE)
+public interface MultiEntryResponse extends ReplicaResponse {

Review Comment:
   We can redesign it during implementing listener, but I am sure this way is right.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)

Review Comment:
   PartitionId is a rather bit string, and thre's already correlationId and paritionId in request, so it's always possible to evaluate response partitionId. So that, I still believe it's batter to remove given paramter.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)

Review Comment:
   PartitionId is a rather bit string, and thre's already correlationId and paritionId in request, so it's always possible to evaluate response partitionId. So that, I still believe it's batter to remove given parameter.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {

Review Comment:
    > but I don't want parametrized this method with a listener, because we have only one implementation.
   
   Please read my comments about different storages above.
   Besides that we will have multiple implementations - e.g. partition specific and meta storage specific.
   
   >If that method will be required in the future, 
   
   It's already clear that we need it right now and not in future.
   
   And what about
   >  Is that method thread-safe: what will happen if I'll try to create several listeners for given partId at the same time?



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.

Review Comment:
   My fault, you are right.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.replicator;
+
+import java.util.UUID;
+import org.apache.ignite.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.message.OperationType;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica listener.
+ * TODO:IGNITE-17258 Implement ReplicaListener.
+ */
+public class ReplicaListener {
+    /** Versioned partition storage. */
+    private MvPartitionStorage mvDataStorage;
+
+    // /** Transaction state storage. */
+    // private TxnStorage txnStorage;
+
+    // /** Inmemory lock storage. */
+    // private VolatileLockStorage volatileLockStorage;
+
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /**
+     * The listener constructor.
+     *
+     * @param mvDataStorage Partition storage.
+     * @param raftClient Raft client.
+     */
+    public ReplicaListener(MvPartitionStorage mvDataStorage, RaftGroupService raftClient) {
+        this.mvDataStorage = mvDataStorage;
+        this.raftClient = raftClient;
+    }
+
+    /**
+     * Applies an request for RW transaction.
+     *
+     * @param txId Transaction id.
+     * @param data      Data.
+     * @param opType    Operation type.
+     * @return Result.
+     */
+    public Object applyReadWrite(UUID txId, Object data, OperationType opType) {

Review Comment:
   >txId is not enough for requests processing
   Why do you think this?
   I think, all what you're required to handle an operation, you can get by txId and the operation type.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.replicator;
+
+import java.util.UUID;
+import org.apache.ignite.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.message.OperationType;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica listener.
+ * TODO:IGNITE-17258 Implement ReplicaListener.
+ */
+public class ReplicaListener {
+    /** Versioned partition storage. */
+    private MvPartitionStorage mvDataStorage;
+
+    // /** Transaction state storage. */
+    // private TxnStorage txnStorage;
+
+    // /** Inmemory lock storage. */
+    // private VolatileLockStorage volatileLockStorage;
+
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /**
+     * The listener constructor.
+     *
+     * @param mvDataStorage Partition storage.
+     * @param raftClient Raft client.
+     */
+    public ReplicaListener(MvPartitionStorage mvDataStorage, RaftGroupService raftClient) {
+        this.mvDataStorage = mvDataStorage;
+        this.raftClient = raftClient;
+    }
+
+    /**
+     * Applies an request for RW transaction.
+     *
+     * @param txId Transaction id.
+     * @param data      Data.
+     * @param opType    Operation type.
+     * @return Result.
+     */
+    public Object applyReadWrite(UUID txId, Object data, OperationType opType) {

Review Comment:
   >txId is not enough for requests processing
   
   Why do you think this?
   I think, all what you're required to handle an operation, you can get by txId and the operation type.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {

Review Comment:
   Yes, I think. It happens due to various serialization for multi rows and single rows request/response.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.replicator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.message.CompleteOperationRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future.

Review Comment:
   Extended javadoc.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.replicator;
+
+import java.util.UUID;
+import org.apache.ignite.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.message.OperationType;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica listener.
+ * TODO:IGNITE-17258 Implement ReplicaListener.
+ */
+public class ReplicaListener {
+    /** Versioned partition storage. */
+    private MvPartitionStorage mvDataStorage;
+
+    // /** Transaction state storage. */
+    // private TxnStorage txnStorage;
+
+    // /** Inmemory lock storage. */
+    // private VolatileLockStorage volatileLockStorage;
+
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /**
+     * The listener constructor.
+     *
+     * @param mvDataStorage Partition storage.
+     * @param raftClient Raft client.
+     */
+    public ReplicaListener(MvPartitionStorage mvDataStorage, RaftGroupService raftClient) {
+        this.mvDataStorage = mvDataStorage;
+        this.raftClient = raftClient;
+    }
+
+    /**
+     * Applies an request for RW transaction.
+     *
+     * @param txId Transaction id.
+     * @param data      Data.
+     * @param opType    Operation type.
+     * @return Result.
+     */
+    public Object applyReadWrite(UUID txId, Object data, OperationType opType) {

Review Comment:
   I still not sure which type we will return.
   But today it can be MultiEntryContainer or SingleEntryContainer



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;

Review Comment:
   There are many places where we pass some object to other one, but it is not mean we will use all methods of nested object.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/README.md:
##########
@@ -0,0 +1,4 @@
+# Ignite replicator module.

Review Comment:
   Let's create separate ticket for module description, current one is a bit poor.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {

Review Comment:
   
   > > but I don't want parametrized this method with a listener, because we have only one implementation.
   Please read my comments about different storages above.
   Besides that we will have multiple implementations - e.g. partition specific and meta storage specific.
   
   >If that method will be required in the future, 
   It's already clear that we need it right now and not in future.
   
   And what about
   >  Is that method thread-safe: what will happen if I'll try to create several listeners for given partId at the same time?



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert partitionId.equals(request.partitionId()) : IgniteStringFormatter.format(
+                "Request does not match to the replica [reqId={}, replicaId={}]", request.partitionId(), partitionId);
+
+        //TODO: Check replica is alive.
+        if (request instanceof ReadWriteRequest) {
+            return applyReadWriteCommandInternal((ReadWriteRequest) request);
+        }
+
+        return applyReadOnlyCommandInternal((ReadOnlyRequest) request);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws Exception {
+        //Close all resources.

Review Comment:
   Yep.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.replicator;
+
+import java.util.UUID;
+import org.apache.ignite.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.message.OperationType;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica listener.
+ * TODO:IGNITE-17258 Implement ReplicaListener.
+ */
+public class ReplicaListener {
+    /** Versioned partition storage. */
+    private MvPartitionStorage mvDataStorage;
+
+    // /** Transaction state storage. */
+    // private TxnStorage txnStorage;
+
+    // /** Inmemory lock storage. */
+    // private VolatileLockStorage volatileLockStorage;
+
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /**
+     * The listener constructor.
+     *
+     * @param mvDataStorage Partition storage.
+     * @param raftClient Raft client.
+     */
+    public ReplicaListener(MvPartitionStorage mvDataStorage, RaftGroupService raftClient) {
+        this.mvDataStorage = mvDataStorage;
+        this.raftClient = raftClient;
+    }
+
+    /**
+     * Applies an request for RW transaction.
+     *
+     * @param txId Transaction id.
+     * @param data      Data.
+     * @param opType    Operation type.
+     * @return Result.
+     */
+    public Object applyReadWrite(UUID txId, Object data, OperationType opType) {

Review Comment:
   Let's continue it conversation 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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaResponse.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.replicator.message;
+
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Replica response interface.
+ */
+public interface ReplicaResponse extends NetworkMessage {
+    /**
+     * Gets a partition id.
+     *
+     * @return Partition id.
+     */
+    String partitionId();

Review Comment:
   Do 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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert partitionId.equals(request.partitionId()) : IgniteStringFormatter.format(
+                "Request does not match to the replica [reqId={}, replicaId={}]", request.partitionId(), partitionId);

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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(resp -> {
+            assert resp instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", resp);
+
+            return (ReplicaResponse) resp;
+        });
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.

Review Comment:
   Done.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert partitionId.equals(request.partitionId()) : IgniteStringFormatter.format(
+                "Request does not match to the replica [reqId={}, replicaId={}]", request.partitionId(), partitionId);
+
+        //TODO: Check replica is alive.
+        if (request instanceof ReadWriteRequest) {
+            return applyReadWriteCommandInternal((ReadWriteRequest) request);
+        }
+
+        return applyReadOnlyCommandInternal((ReadOnlyRequest) request);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws Exception {
+        //Close all resources.

Review Comment:
   There are no resources yet.
   Do you mean the best way is refusing of AutoClosable interface until the resources will be appeared?



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.

Review Comment:
   Nope, It's expected that we will return generally speaking the result of an async replication and wait for the future to compete only if's necessary. 



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.replicator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.message.CompleteOperationRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteStringFormatter;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String replicaGrpId;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The map matches an operation id to the future.
+     */
+    private final ConcurrentHashMap<UUID, CompletableFuture> ops = new ConcurrentHashMap<>();
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param replicaGrpId Replication group id.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String replicaGrpId,
+            ReplicaListener listener
+    ) {
+        this.replicaGrpId = replicaGrpId;
+        this.listener = listener;
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert replicaGrpId.equals(request.replicationGroupId()) : IgniteStringFormatter.format(
+                "Partition mismatch: request does not match the replica [reqPartId={}, replicaGrpId={}]", request.replicationGroupId(),
+                replicaGrpId);
+
+        //TODO:IGNITE-17378 Check replica is alive.
+
+        ReplicaResponse response;
+
+        try {
+            if (request instanceof CompleteOperationRequest) {
+                var opsRequest = (CompleteOperationRequest) request;
+
+                List<UUID> opIds = opsRequest.operationIds();
+
+                HashMap<UUID, Object> opRes = null;
+
+                if (opIds != null && !opIds.isEmpty()) {
+                    opRes = new HashMap<>(opIds.size());
+
+                    for (Entry<UUID, CompletableFuture> op : ops.entrySet()) {
+                        if (opIds.contains(op.getKey())) {
+                            opRes.put(op.getKey(), op.getValue().join());
+
+                            ops.remove(op.getKey());

Review Comment:
   I think, it should be solve in transaction recovery procedure



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeoutException;
+import org.apache.ignite.internal.replicator.exception.ReplicationError;
+import org.apache.ignite.internal.replicator.exception.ReplicationException;
+import org.apache.ignite.internal.replicator.exception.ReplicationTimeoutException;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Response future.
+     * @throws NodeStoppingException Is thrown when the node is stopping.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) throws NodeStoppingException {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.replicationGroupId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(msg -> {
+            assert msg instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", msg);
+
+            return (ReplicaResponse) msg;
+        }).whenComplete((response, throwable) -> {
+            if (throwable != null) {
+                if (throwable instanceof TimeoutException) {

Review Comment:
   A common exception will be thrown.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaResponse.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.replicator.message;
+
+import java.util.UUID;
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Replica response interface.
+ * TODO:IGNITE-17258 Add a specific response type for a replica listener. (@Transferable(ReplicaMessageGroup.TYPE_RESPONSE))
+ */
+public interface ReplicaResponse extends NetworkMessage {
+
+    /**
+     * Operation id in the replica with which the response is sent.
+     *
+     * @return Operation id.
+     */
+    UUID operationId();

Review Comment:
   I added a method ReplicaService#waitForResult for waiting replication result.



-- 
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] vldpyatkov commented on pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

Posted by GitBox <gi...@apache.org>.
vldpyatkov commented on PR #927:
URL: https://github.com/apache/ignite-3/pull/927#issuecomment-1196782507

   Merged to ignite3_tx


-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {

Review Comment:
   Today here are only interfaces



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.

Review Comment:
   Because I think we return here to an instant response.
   Probably in the future, the response will be contained a future id, by that id the future will be available.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));

Review Comment:
   Here will be a future which connected with network communication. In case where the request executed locally, we have no the communication.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaMessageGroup.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.replicator.message;
+
+import org.apache.ignite.network.annotations.MessageGroup;
+
+/**
+ * Message group for the replication process.
+ */
+@MessageGroup(groupType = 8, groupName = "ReplicaMessages")
+public class ReplicaMessageGroup {

Review Comment:
   Just ok.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert partitionId.equals(request.partitionId()) : IgniteStringFormatter.format(
+                "Request does not match to the replica [reqId={}, replicaId={}]", request.partitionId(), partitionId);
+
+        //TODO: Check replica is alive.
+        if (request instanceof ReadWriteRequest) {

Review Comment:
   What's the point of splitting request into ReadWrite and Ro within **Replica**? I mean that it's all up to replicaListener to split requests by it's type. Are there any differences in handling RO and RW requests in Replica itself, except checking lease intervals for RW ones? 



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {

Review Comment:
   All I mean that we should propagate ReplicaRequest as is to ReplicaListner after some common pre processing in Replica.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {

Review Comment:
    > but I don't want parametrized this method with a listener, because we have only one implementation.
   Please read my comments about different storages above.
   Besides that we will have multiple implementations - e.g. partition specific and meta storage specific.
   
   >If that method will be required in the future, 
   It's already clear that we need it right now and not in future.
   
   And what about
   >  Is that method thread-safe: what will happen if I'll try to create several listeners for given partId at the same time?



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.replicator;
+
+import java.util.UUID;
+import org.apache.ignite.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.message.OperationType;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica listener.
+ * TODO:IGNITE-17258 Implement ReplicaListener.
+ */
+public class ReplicaListener {
+    /** Versioned partition storage. */
+    private MvPartitionStorage mvDataStorage;
+
+    // /** Transaction state storage. */
+    // private TxnStorage txnStorage;
+
+    // /** Inmemory lock storage. */
+    // private VolatileLockStorage volatileLockStorage;
+
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /**
+     * The listener constructor.
+     *
+     * @param mvDataStorage Partition storage.
+     * @param raftClient Raft client.
+     */
+    public ReplicaListener(MvPartitionStorage mvDataStorage, RaftGroupService raftClient) {
+        this.mvDataStorage = mvDataStorage;
+        this.raftClient = raftClient;
+    }
+
+    /**
+     * Applies an request for RW transaction.
+     *
+     * @param txId Transaction id.
+     * @param data      Data.
+     * @param opType    Operation type.
+     * @return Result.
+     */
+    public Object applyReadWrite(UUID txId, Object data, OperationType opType) {

Review Comment:
   I described it 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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.network.ClusterNode;
+
+/**
+ * Replica service.
+ */
+public interface ReplicaService extends AutoCloseable {
+    /** Replicator network message factory. */
+    static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Network timeout. */
+    static final int RPC_TIMEOUT = 3000;

Review Comment:
   I'd favor adding all such implementation specific stuff in corresponding implementation specific tickets. Let's remove it from given PR. Otherwise we will bogged down in timeout exception handling details.



##########
modules/replicator/src/main/java/org/apache/ignite/hlc/HybridTimestamp.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.hlc;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.tostring.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * A hybrid timestamp that combines physical clock and logical clock.
+ */
+public class HybridTimestamp implements Comparable<HybridTimestamp>, Serializable {

Review Comment:
   I don't think that we need HybridTimestamp in order to introduce ReplicaListener interface. I believe that timestamp will rather be a part of read only replica requests that an explicit parameter of applyReadOnly() method.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.network.ClusterNode;
+
+/**
+ * Replica service.
+ */
+public interface ReplicaService extends AutoCloseable {
+    /** Replicator network message factory. */
+    static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Network timeout. */
+    static final int RPC_TIMEOUT = 3000;
+
+    /**
+     * Determines the replica server or not.
+     * If the replica is not a server, it delegates all command to a remote replica server.
+     *
+     * @return True when the replica is server, false when this is only client service.
+     */
+    boolean isServer();

Review Comment:
   I still don't think that we need given method. 



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.network.ClusterNode;
+
+/**
+ * Replica service.
+ */
+public interface ReplicaService extends AutoCloseable {
+    /** Replicator network message factory. */
+    static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Network timeout. */
+    static final int RPC_TIMEOUT = 3000;
+
+    /**
+     * Determines the replica server or not.
+     * If the replica is not a server, it delegates all command to a remote replica server.
+     *
+     * @return True when the replica is server, false when this is only client service.
+     */
+    boolean isServer();
+
+    /**
+     * Handles replication request.
+     *
+     * @param request Request.
+     * @return Future to response.
+     */
+    CompletableFuture<ReplicaResponse> processRequest(ReplicaRequest request);
+
+
+    /**
+     * Invokes an action in replica.

Review Comment:
   Here and in other places I'd rather use javadocs similar to ones we have in MessagingService, e.g.
   "Node in which a request will be sent." -> "Recipient cluster node."
   Let's also clarify and denote guaranties explicitly, e.g. are invokes ordered?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.network.ClusterNode;
+
+/**
+ * Replica service.
+ */
+public interface ReplicaService extends AutoCloseable {
+    /** Replicator network message factory. */
+    static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Network timeout. */
+    static final int RPC_TIMEOUT = 3000;
+
+    /**
+     * Determines the replica server or not.
+     * If the replica is not a server, it delegates all command to a remote replica server.
+     *
+     * @return True when the replica is server, false when this is only client service.
+     */
+    boolean isServer();
+
+    /**
+     * Handles replication request.

Review Comment:
   Well, generally speaking that's not true, because lot's of ReplicaRequests aren't replication ones.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.network.ClusterNode;
+
+/**
+ * Replica service.
+ */
+public interface ReplicaService extends AutoCloseable {
+    /** Replicator network message factory. */
+    static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Network timeout. */
+    static final int RPC_TIMEOUT = 3000;
+
+    /**
+     * Determines the replica server or not.
+     * If the replica is not a server, it delegates all command to a remote replica server.
+     *
+     * @return True when the replica is server, false when this is only client service.
+     */
+    boolean isServer();
+
+    /**
+     * Handles replication request.
+     *
+     * @param request Request.
+     * @return Future to response.

Review Comment:
   Here and in other places, just "Response future".



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.network.ClusterNode;
+
+/**
+ * Replica service.
+ */
+public interface ReplicaService extends AutoCloseable {
+    /** Replicator network message factory. */
+    static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Network timeout. */
+    static final int RPC_TIMEOUT = 3000;
+
+    /**
+     * Determines the replica server or not.
+     * If the replica is not a server, it delegates all command to a remote replica server.
+     *
+     * @return True when the replica is server, false when this is only client service.
+     */
+    boolean isServer();
+
+    /**
+     * Handles replication request.
+     *
+     * @param request Request.
+     * @return Future to response.
+     */
+    CompletableFuture<ReplicaResponse> processRequest(ReplicaRequest request);
+
+
+    /**
+     * Invokes an action in replica.
+     *
+     * @param node Node in which a request will be sent.
+     * @param request Request.
+     * @return Future to response.
+     */
+    CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request);
+
+    /**
+     * Invokes an action in replica on the specific storage.
+     *
+     * @param node Node in which a request will be sent.
+     * @param request Request.
+     * @param storageId Storage id.
+     * @return Future to response.
+     */
+    CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request, String storageId);

Review Comment:
   Not sure whether we will use string as storageId, let's add todo with corresponding ticket here.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica server.
+ */
+public class Replica implements ReplicaService {
+    /** Raft client. */
+    private final RaftGroupService raftClient;

Review Comment:
   I'd favor adding all implementation specific details in corresponding tickets.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,268 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+import org.apache.ignite.raft.client.Peer;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica server.
+ */
+public class Replica implements ReplicaService {
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Logical topology service. */
+    private final ClusterManagementGroupManager logicalTopologyService;
+
+    /** Topology service. */
+    private final TopologyService topologyService;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param messagingService Message service.
+     * @param logicalTopologyService Logical topology service.
+     * @param raftGroupService Raft client.
+     * @param listener Replica listener.
+     */
+    public Replica(

Review Comment:
   Let's remove all implementation specific stuff for now.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.network.ClusterNode;
+
+/**
+ * Replica service.
+ */
+public interface ReplicaService extends AutoCloseable {
+    /** Replicator network message factory. */
+    static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Network timeout. */
+    static final int RPC_TIMEOUT = 3000;
+
+    /**
+     * Determines the replica server or not.
+     * If the replica is not a server, it delegates all command to a remote replica server.
+     *
+     * @return True when the replica is server, false when this is only client service.
+     */
+    boolean isServer();
+
+    /**
+     * Handles replication request.
+     *
+     * @param request Request.
+     * @return Future to response.
+     */
+    CompletableFuture<ReplicaResponse> processRequest(ReplicaRequest request);

Review Comment:
   That should be a method of Replica not ReplicaService.
   If it's not currently possible to specify possible exceptions in throws clause please add todo linked to any appropriate ticket, probably https://issues.apache.org/jira/browse/IGNITE-17258.



-- 
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] sanpwc commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert partitionId.equals(request.partitionId()) : IgniteStringFormatter.format(
+                "Request does not match to the replica [reqId={}, replicaId={}]", request.partitionId(), partitionId);

Review Comment:
   "Partition mismatch: request does not match the replica [request.partitionId={}, replica.partitionId={}]", request.partitionId(), partitionId



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert partitionId.equals(request.partitionId()) : IgniteStringFormatter.format(
+                "Request does not match to the replica [reqId={}, replicaId={}]", request.partitionId(), partitionId);
+
+        //TODO: Check replica is alive.
+        if (request instanceof ReadWriteRequest) {

Review Comment:
   What's the point of splitting request into ReadWrite and Ro within Replica? I believe that in future we will add checkLease() step for RW requests, however currently it's all up to ReplicaListener to process any kind of request within single entry point.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert partitionId.equals(request.partitionId()) : IgniteStringFormatter.format(
+                "Request does not match to the replica [reqId={}, replicaId={}]", request.partitionId(), partitionId);
+
+        //TODO: Check replica is alive.
+        if (request instanceof ReadWriteRequest) {
+            return applyReadWriteCommandInternal((ReadWriteRequest) request);
+        }
+
+        return applyReadOnlyCommandInternal((ReadOnlyRequest) request);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws Exception {
+        //Close all resources.

Review Comment:
   What kind of resources do you expect to be closed here?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {

Review Comment:
   I don't think that we need splitting for RW and RO neither in replica nor in Listener.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.replicator;
+
+import java.util.UUID;
+import org.apache.ignite.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.message.OperationType;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica listener.
+ * TODO:IGNITE-17258 Implement ReplicaListener.
+ */
+public class ReplicaListener {
+    /** Versioned partition storage. */
+    private MvPartitionStorage mvDataStorage;
+
+    // /** Transaction state storage. */
+    // private TxnStorage txnStorage;
+
+    // /** Inmemory lock storage. */
+    // private VolatileLockStorage volatileLockStorage;
+
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /**
+     * The listener constructor.
+     *
+     * @param mvDataStorage Partition storage.
+     * @param raftClient Raft client.
+     */
+    public ReplicaListener(MvPartitionStorage mvDataStorage, RaftGroupService raftClient) {
+        this.mvDataStorage = mvDataStorage;
+        this.raftClient = raftClient;
+    }
+
+    /**
+     * Applies an request for RW transaction.
+     *
+     * @param txId Transaction id.
+     * @param data      Data.
+     * @param opType    Operation type.
+     * @return Result.
+     */
+    public Object applyReadWrite(UUID txId, Object data, OperationType opType) {

Review Comment:
   As was mentioned in Replica, why do we need to split RW and RO handling?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.

Review Comment:
   As was previously mentioned not all ReplicaRequests are requests for the replication. 



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.replicator;
+
+import java.util.UUID;
+import org.apache.ignite.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.message.OperationType;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica listener.
+ * TODO:IGNITE-17258 Implement ReplicaListener.
+ */
+public class ReplicaListener {
+    /** Versioned partition storage. */
+    private MvPartitionStorage mvDataStorage;
+
+    // /** Transaction state storage. */
+    // private TxnStorage txnStorage;
+
+    // /** Inmemory lock storage. */
+    // private VolatileLockStorage volatileLockStorage;
+
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /**
+     * The listener constructor.
+     *
+     * @param mvDataStorage Partition storage.
+     * @param raftClient Raft client.
+     */
+    public ReplicaListener(MvPartitionStorage mvDataStorage, RaftGroupService raftClient) {
+        this.mvDataStorage = mvDataStorage;
+        this.raftClient = raftClient;
+    }
+
+    /**
+     * Applies an request for RW transaction.
+     *
+     * @param txId Transaction id.
+     * @param data      Data.
+     * @param opType    Operation type.
+     * @return Result.
+     */
+    public Object applyReadWrite(UUID txId, Object data, OperationType opType) {

Review Comment:
   txId is not enough for requests processing, e.g. for writeIntent resolution. Why do we need opType? class bases type checking seems better to me.
   All-in-all instead of applyReadWrite and applyReadOnly let's have ReplicaRequest processRequest(ReplicaRequest request). We may also consider merging Replica and ReplicaListener if it's occurred that Replica doesn't have enough generic actions to process.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.

Review Comment:
   It's not the replica service. 



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {

Review Comment:
   Here and in other places: javadoc ins't specific, What will happen if there's already replica with given partitionId? Is that method thread-safe: what will happen if I'll try to create several listeners for given partId at the same time? We should parameterize given method with ReplicaListener instead of RaftGroupService.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;

Review Comment:
   Auf, well let's say that it'll be fixed in IGNITE-17255



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return getReplicaInternal(partId, raftClient);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Internal method for start a replica.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    private Replica getReplicaInternal(String partId, RaftGroupService raftClient) {
+        var replica = new Replica(partId, clusterNetSvc.topologyService(), new ReplicaListener(mvDataStorage, raftClient));
+
+        Replica previous = replicas.putIfAbsent(partId, replica);

Review Comment:
   I'd rather use computeIfAbsent here in order not to create (and thus not to close) the new replica if there's one with samw partId.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.

Review Comment:
   Why it's not CompletableFuture<ReplicaResponse>? It's up to outer tx logic to whether await responses or not.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(resp -> {
+            assert resp instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", resp);
+
+            return (ReplicaResponse) resp;
+        });
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.

Review Comment:
   It's a recipient node.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/MultiEntryResponse.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.replicator.message;
+
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.network.annotations.Marshallable;
+import org.apache.ignite.network.annotations.Transferable;
+
+/**
+ * Replica response.
+ */
+@Transferable(ReplicaMessageGroup.REPLICA_MULTI_RESPONSE)
+public interface MultiEntryResponse extends ReplicaResponse {

Review Comment:
   I beleive that we should remove all ReplicaRequests and Responses for now, except ReplicaRequest and ReplicaResponse itself. We will bring them back during ReplicaListener Implementation.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(resp -> {
+            assert resp instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", resp);
+
+            return (ReplicaResponse) resp;
+        });
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.
+     * @param request Request.
+     * @return Future to response.
+     */
+    public CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request) {

Review Comment:
   What will happen if RPC_TIMEOUT expires? Please declare all possible exceptions. I's fine that we don't know full set of exceptions for now, but let's declare ones that are throws already: TimeoutException(?), IgniteInternalException, NodeStoppingExcpetion(If it's possible) etc.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaMessageGroup.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.replicator.message;
+
+import org.apache.ignite.network.annotations.MessageGroup;
+
+/**
+ * Message group for the replication process.
+ */
+@MessageGroup(groupType = 8, groupName = "ReplicaMessages")
+public class ReplicaMessageGroup {

Review Comment:
   After finalizing set of replica request and responses let's adjust given class.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()

Review Comment:
   Why it's expected that multiEntryRequest will always produce multiEntryResponse? E.g. I believe that sort of remove([]keys) and similar may have void response.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {

Review Comment:
   Neither do I think that we need splitting for single/multi entry requests. What's the point?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)

Review Comment:
   Do we really need to return paritionId backwards?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicatorUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.function.Consumer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteLogger;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This is an utility class for serialization cache tuples. It will be removed after another way for serialization is implemented into the
+ * network layer.
+ * TODO: Remove it after (IGNITE-14793)
+ */
+public class ReplicatorUtils {

Review Comment:
   Is it possible to reuse ByteUtils or similar instead of adding new one? If there some methods are missing in current vetrison of ByteUtils we might create them.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(

Review Comment:
   As was mentioned above, I believe it's up to Listener to check whether request has known type and throw Unsupported exception otherwise.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.replicator;
+
+import java.util.UUID;
+import org.apache.ignite.hlc.HybridTimestamp;
+import org.apache.ignite.internal.replicator.message.OperationType;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+
+/**
+ * Replica listener.
+ * TODO:IGNITE-17258 Implement ReplicaListener.
+ */
+public class ReplicaListener {
+    /** Versioned partition storage. */
+    private MvPartitionStorage mvDataStorage;
+
+    // /** Transaction state storage. */
+    // private TxnStorage txnStorage;
+
+    // /** Inmemory lock storage. */
+    // private VolatileLockStorage volatileLockStorage;
+
+    /** Raft client. */
+    private final RaftGroupService raftClient;
+
+    /**
+     * The listener constructor.
+     *
+     * @param mvDataStorage Partition storage.
+     * @param raftClient Raft client.
+     */
+    public ReplicaListener(MvPartitionStorage mvDataStorage, RaftGroupService raftClient) {
+        this.mvDataStorage = mvDataStorage;
+        this.raftClient = raftClient;
+    }
+
+    /**
+     * Applies an request for RW transaction.
+     *
+     * @param txId Transaction id.
+     * @param data      Data.
+     * @param opType    Operation type.
+     * @return Result.
+     */
+    public Object applyReadWrite(UUID txId, Object data, OperationType opType) {

Review Comment:
   Not sure whether it's a good idea to return object. BTW, what about Serializable?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);

Review Comment:
   What about exception handling?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {

Review Comment:
   That won't work, partitions might have different mvDataStorages and not only mvDataStorages actually. E.g. table1.partitionX will have RocksDbStorage1, table2.partitionY will have RocksDbStorage2, table3.parititionZ will have In-memory MVPartitionStorage, table4.parititionK will have PageMemory MVPartitionStorage etc.
   All-in-all storages will parameterize replicaListeners that will parameterize startReplica() method.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return getReplicaInternal(partId, raftClient);

Review Comment:
   Why it's **get**ReplicaInternal, it seems that startReplicaInternal suites better here.
   Btw why do we need Internal counterparts at all? 



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {
+            @Override
+            public void onReceived(NetworkMessage message, NetworkAddress senderAddr, @Nullable Long correlationId) {
+                if (!busyLock.enterBusy()) {
+                    throw new IgniteException(new NodeStoppingException());
+                }
+
+                try {
+                    assert message instanceof ReplicaRequest : IgniteStringFormatter.format("Unexpected message [message={}]", message);
+
+                    ReplicaRequest request = (ReplicaRequest) message;
+
+                    Replica replica = replicas.get(request.partitionId());
+
+                    if (replica == null) {
+                        //TODO:IGNITE-17255 Sent an exceptional response to the client side when the replica is absent.
+                    }
+
+                    ReplicaResponse resp = replica.processRequest(request);
+
+                    clusterNetSvc.messagingService().respond(senderAddr, resp, correlationId);
+                } finally {
+                    busyLock.leaveBusy();
+                }
+            }
+        });
+    }
+
+    /**
+     * Gets a replica by its partition id.
+     *
+     * @param partitionId Partition id.
+     * @return Instance of the replica or {@code null} if the replica is not started.
+     */
+    public Replica replica(String partitionId) {
+        if (!busyLock.enterBusy()) {
+            throw new IgniteException(new NodeStoppingException());
+        }
+
+        try {
+            return replicas.get(partitionId);
+        } finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * Starts replica server.
+     *
+     * @param partId     Partition id.
+     * @param raftClient Raft client.
+     * @return Replica.
+     */
+    public Replica startReplica(String partId, RaftGroupService raftClient) {

Review Comment:
   Javadoc ins't specific, What will happen if there's already replica with given partitionId? Is that method thread-safe: what will happen if I'll try to create several listeners for given partId at the same time? We should parameterize given method with ReplicaListener instead of RaftGroupService.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {

Review Comment:
   That should goes inside start method instead of constructor.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;

Review Comment:
   Despite the fact that I've earlier told that Manager seems fine here, I'd favor using replica list producer here. Otherwise it'll look like that it's fine to start and stop replicas from within client. 



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(resp -> {
+            assert resp instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", resp);
+
+            return (ReplicaResponse) resp;
+        });
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.
+     * @param request Request.
+     * @return Future to response.
+     */
+    public CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request) {
+        //TODO: Check replica is alive.
+        return sendToReplica(node, request);
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.
+     * @param request Request.
+     * @param storageId Storage id.
+     * @return Future to response.
+     */
+    public CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request, String storageId) {
+        //TODO: Check replica is alive.
+        return sendToReplica(node, request);
+
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws Exception {
+        //Free the resources.

Review Comment:
   What kind of resources do you expect to close here?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));

Review Comment:
   competedFuture, wht? I believe we may just propagate replica.ProcessRequests future.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaService.java:
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.MessagingService;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * The service is intended to execute requests on replicas.
+ * TODO:IGNITE-17255 Implement ReplicaService.
+ */
+public class ReplicaService implements AutoCloseable {
+    /** Network timeout. */
+    private static final int RPC_TIMEOUT = 3000;
+
+    /** Replica manager. */
+    private final ReplicaManager replicaManager;
+
+    /** Message service. */
+    private final MessagingService messagingService;
+
+    /** Local node. */
+    private final ClusterNode localNode;
+
+    /**
+     * The constructor of replica client.
+     *
+     * @param replicaManager Replica manager.
+     * @param messagingService Cluster message service.
+     * @param topologyService Topology service.
+     */
+    public ReplicaService(
+            ReplicaManager replicaManager,
+            MessagingService messagingService,
+            TopologyService topologyService
+    ) {
+        this.replicaManager = replicaManager;
+        this.messagingService = messagingService;
+
+        this.localNode = topologyService.localMember();
+    }
+
+    /**
+     * Sends request to the replica node.
+     *
+     * @param node Cluster node which holds a replica.
+     * @param req  Replica request.
+     * @return Future to response.
+     */
+    private CompletableFuture<ReplicaResponse> sendToReplica(ClusterNode node, ReplicaRequest req) {
+        if (localNode.equals(node)) {
+            Replica replica = replicaManager.replica(req.partitionId());
+
+            if (replica == null) {
+                //TODO:IGNITE-17255 Provide an exceptional response when the replica is absent.
+            }
+
+            return CompletableFuture.completedFuture(replica.processRequest(req));
+        }
+
+        return messagingService.invoke(node.address(), req, RPC_TIMEOUT).thenApply(resp -> {
+            assert resp instanceof ReplicaResponse : IgniteStringFormatter.format("Unexpected message response [resp={}]", resp);
+
+            return (ReplicaResponse) resp;
+        });
+    }
+
+    /**
+     * Passes a request to replication.
+     *
+     * @param node Cluster node.
+     * @param request Request.
+     * @return Future to response.
+     */
+    public CompletableFuture<ReplicaResponse> invoke(ClusterNode node, ReplicaRequest request) {
+        //TODO: Check replica is alive.

Review Comment:
   Please add corresponding ticket.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/OperationType.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.replicator.message;
+
+import java.io.Serializable;
+
+/**
+ * Transaction operation type.
+ */
+public enum OperationType implements Serializable {

Review Comment:
   I don't think that we need given enum, please check my comment above.



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaResponse.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.replicator.message;
+
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Replica response interface.
+ */
+public interface ReplicaResponse extends NetworkMessage {
+    /**
+     * Gets a partition id.
+     *
+     * @return Partition id.
+     */
+    String partitionId();

Review Comment:
   Do we really need replicationGroupId in response? Why?



##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/message/ReplicaRequest.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.replicator.message;
+
+import org.apache.ignite.network.NetworkMessage;
+
+/**
+ * Replica request.
+ */
+public interface ReplicaRequest extends NetworkMessage {
+    /**
+     * Gets a partition id.
+     *
+     * @return Partition id.
+     */
+    String partitionId();

Review Comment:
   MetaStorage ReplicaRequests won't have paritionId(), it's rather replicationGroupId(). My fault(( Sorry for that.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/Replica.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.replicator;
+
+import org.apache.ignite.internal.replicator.message.ReadOnlyMultyEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlyRequest;
+import org.apache.ignite.internal.replicator.message.ReadOnlySingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteMultiEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteRequest;
+import org.apache.ignite.internal.replicator.message.ReadWriteSingleEntryRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaMessagesFactory;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.replicator.message.container.MultiEntryContainer;
+import org.apache.ignite.internal.replicator.message.container.SingleEntryContainer;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.network.ClusterNode;
+import org.apache.ignite.network.TopologyService;
+
+/**
+ * Replica server.
+ * TODO:IGNITE-17257 Implement Replica server-side logic.
+ */
+public class Replica implements AutoCloseable {
+    /** Replicator network message factory. */
+    private static final ReplicaMessagesFactory REPLICA_MESSAGES_FACTORY = new ReplicaMessagesFactory();
+
+    /** Replica group identity, this id is the same as the considered partition's id. */
+    private final String partitionId;
+
+    /** Local cluster node. */
+    private final ClusterNode localNode;
+
+    /** Replica listener. */
+    private final ReplicaListener listener;
+
+    /**
+     * The constructor of replica server.
+     *
+     * @param partitionId Identity.
+     * @param topologyService Topology service.
+     * @param listener Replica listener.
+     */
+    public Replica(
+            String partitionId,
+            TopologyService topologyService,
+            ReplicaListener listener
+    ) {
+        this.partitionId = partitionId;
+        this.localNode = topologyService.localMember();
+        this.listener = listener;
+    }
+
+    /**
+     * It is an internal method to applies a RW request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadWriteCommandInternal(ReadWriteRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadWriteSingleEntryRequest) {
+            var request = (ReadWriteSingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadWriteMultiEntryRequest) {
+            var request = (ReadWriteMultiEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadWrite(req.txId(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * It is an internal method to applies a RO request to replicator.
+     *
+     * @param req Request to apply in replicator.
+     * @return Response.
+     */
+    private ReplicaResponse applyReadOnlyCommandInternal(ReadOnlyRequest req) {
+        //TODO: Check locks here.
+        if (req instanceof ReadOnlySingleEntryRequest) {
+            var request = (ReadOnlySingleEntryRequest) req;
+
+            var data = (SingleEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .singleEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else if (req instanceof ReadOnlyMultyEntryRequest) {
+            var request = (ReadOnlyMultyEntryRequest) req;
+
+            var data = (MultiEntryContainer) listener.applyReadOnly(req.timestamp(), request.entryContainer(), req.opType());
+
+            return REPLICA_MESSAGES_FACTORY
+                    .multiEntryResponse()
+                    .partitionId(partitionId)
+                    .entryContainer(data)
+                    .build();
+        } else {
+            throw new IgniteInternalException(
+                    IgniteStringFormatter.format("Unsupported request type [type={}]", req.getClass().getSimpleName()));
+        }
+    }
+
+    /**
+     * Process a replication request on the replica.
+     *
+     * @param request Request to replication.
+     * @return Response.
+     */
+    public ReplicaResponse processRequest(ReplicaRequest request) { // define proper set of exceptions that might be thrown.
+        assert partitionId.equals(request.partitionId()) : IgniteStringFormatter.format(
+                "Request does not match to the replica [reqId={}, replicaId={}]", request.partitionId(), partitionId);
+
+        //TODO: Check replica is alive.
+        if (request instanceof ReadWriteRequest) {

Review Comment:
   RW request contains transaction id and does not contain transaction timestamp.
   RO request contains a timestamp, but does not contain transaction id.
   By those requests, the requests have separated types.



-- 
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] vldpyatkov commented on a diff in pull request #927: IGNITE-17252 Introduce Replica, ReplicaServer(?), ReplicaService and …

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


##########
modules/replicator/src/main/java/org/apache/ignite/internal/replicator/ReplicaManager.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.replicator;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.replicator.message.ReplicaMessageGroup;
+import org.apache.ignite.internal.replicator.message.ReplicaRequest;
+import org.apache.ignite.internal.replicator.message.ReplicaResponse;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.apache.ignite.lang.IgniteStringFormatter;
+import org.apache.ignite.lang.NodeStoppingException;
+import org.apache.ignite.network.ClusterService;
+import org.apache.ignite.network.NetworkAddress;
+import org.apache.ignite.network.NetworkMessage;
+import org.apache.ignite.network.NetworkMessageHandler;
+import org.apache.ignite.raft.client.service.RaftGroupService;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * The replica service.
+ */
+public class ReplicaManager implements IgniteComponent {
+    /** Busy lock to stop synchronously. */
+    private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
+
+    /** Prevents double stopping the component. */
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /** Cluster network service. */
+    private final ClusterService clusterNetSvc;
+
+    /** Partition storage. */
+    private final MvPartitionStorage mvDataStorage;
+
+    /** Replicas. */
+    private final ConcurrentHashMap<String, Replica> replicas = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor for replica service.
+     *
+     * @param clusterNetSvc                 Cluster network service.
+     * @param mvDataStorage                 Partition storage.
+     */
+    public ReplicaManager(ClusterService clusterNetSvc, MvPartitionStorage mvDataStorage) {
+        this.clusterNetSvc = clusterNetSvc;
+        this.mvDataStorage = mvDataStorage;
+
+        clusterNetSvc.messagingService().addMessageHandler(ReplicaMessageGroup.class, new NetworkMessageHandler() {

Review Comment:
   Moved this action to start.



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