You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Alexey Scherbakov (Jira)" <ji...@apache.org> on 2021/03/02 09:27:00 UTC

[jira] [Commented] (IGNITE-14110) Create networking module

    [ https://issues.apache.org/jira/browse/IGNITE-14110?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17293545#comment-17293545 ] 

Alexey Scherbakov commented on IGNITE-14110:
--------------------------------------------

[~sergeychugunov]

I've reviewed a patch and have some comments and ideas for further improvement.

1. NetworkMessage looks redundant. Only used in NetworkMessageHandler and can be replaced with 
 onReceived(NetworkMember sender, Object msg). I see no point in creating the wrapper object, sending methods also don't use it.

2. org.apache.ignite.network.NetworkClusterFactory#startScaleCubeBasedCluster - doesn't look like correct factory implementation.
 NetworkClusterFactory must be interface with a createCluster method. ScaleCubeNetworkClusterFactory must implement it.

3. Most crucial to me - I think we are lacking a method like *sendWithResponse* for sending a message with the response, so called RPC style communication. Without it a user should implement it's own request/response management (as it was in ignite 2), which is very inconvenient and produces a lot of boilerplate code. I've added a stub implementation and used it in my PR with raft client, see [1]. Actually, scalecube supports it out of the box, so it shouldn't be difficult to implement fully.

Important thing to keep in mind: if some error has occured on server side and expected response can't be generated, the ErrorResponse(int code, String msg) should be delivered to sender and used to complete a future with an exception containing a code and a message.

4. guaranteedSend is implemented badly, it's synchronous despite of returning a future.
 I've tried to fix it in my PR [2]. Also I suggest to rename it to *send* to match sendWithResponse. Actually, send is similar to send with void response, probably we don't need it at all.

5. It seems ITScaleCubeNetworkClusterMessagingTest has a race, see [3]

6. It seems current implementation executes message handler in netty's NIO thread, which doesn't look correct to me. We should have an ability to route specific messages to specific executors, or if such routing not configured, execute message handler in default executor.

[1] [https://github.com/apache/ignite-3/pull/59/files#diff-7d48a577745f9c83969e32d33e3f66f664833c7559f75769b54feaf3072ec850R93]

[2] [https://github.com/apache/ignite-3/pull/59/files#diff-7d48a577745f9c83969e32d33e3f66f664833c7559f75769b54feaf3072ec850R88]

[3] [https://ci.ignite.apache.org/viewLog.html?buildId=5897478&buildTypeId=ignite3_Tests_IntegrationTests]

 

> Create networking module
> ------------------------
>
>                 Key: IGNITE-14110
>                 URL: https://issues.apache.org/jira/browse/IGNITE-14110
>             Project: Ignite
>          Issue Type: Sub-task
>            Reporter: Anton Kalashnikov
>            Assignee: Anton Kalashnikov
>            Priority: Major
>              Labels: iep-66
>             Fix For: 3.0.0-alpha2
>
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> It needs to create a networking module with some API and simple implementation for further improvment



--
This message was sent by Atlassian Jira
(v8.3.4#803005)