You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ignite.apache.org by Ivan Bessonov <be...@gmail.com> on 2020/05/21 12:37:43 UTC

[DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Hello Igniters,

I'd like to discuss with you changes related to [1] and [2]. Both issues
are mostly the same so
let's discuss the core idea.

*Motivation.*

There are certain environments that don't allow Ignite server nodes to open
TCP connections to
thick clients, e.g. K8S, AWS Lambda or Azure Functions. To operate in such
environments, the
server needs a way to request a client to open an "inverse" communication
connection to it.

I've prepared a PR (still in progress) that introduces new mechanism of
opening connection and
related configuration.

*Main idea*

This mechanism is called "communication via discovery" or "inverse
connection", it works as
follows:
 - server that needs to connect to "unreachable" thick client sends a
specific Discovery message
   (inverse communication request) to that client;
 - client node upon receiving the request opens communication connection to
that server;
 - server sees connection opened by client and proceeds with its task (that
required opening
   connection to the client).

Working name for new configuration parameter for this feature is
environmentType, it is an
enum with two values (again, working names): STANDALONE (default) and
VIRTUALIZED.
It is used as a hint to server to speed-up establishing of connections:
when server sees a client
with VIRTUALIZED environmentType it doesn't try to open connection to it
and sends inverse
communication request right away.
If environmentType is STANDALONE then server tries to open a connection in
a regular way
(iterating over all client addresses) and sends request only if all
attempts failed.

There is a concern about naming of the configuration as it catches only one
use-case: when we
deal with some kind of virtualized environment like K8S.
There are other options I've encountered in private discussion:
- connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
- networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
- communicationViaDiscovery - ALWAYS / FALLBACK
- isReachableFromAllNodes (true/false flag)
- initiateConnectionsOnThisNode (true/false flag).

*Limitations*

The feature cannot be used along with pairedConnection setting as this
setting implies
establishing connections in both directions. Also current implementation
supports opening only
client-to-server connections. Other types of connections like
client-to-client or server-to-server
will be implemented in separate tickets.

[1] https://issues.apache.org/jira/browse/IGNITE-12438
[2] https://issues.apache.org/jira/browse/IGNITE-13013

-- 
Sincerely yours,
Ivan Bessonov

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Raymond Wilson <ra...@trimble.com>.
Thanks Ivan and Denis for clarifying things.

I think we are in good shape :)

Thanks,
Raymond


On Tue, Jun 30, 2020 at 10:29 AM Denis Magda <dm...@apache.org> wrote:

> Hi Raymond,
>
> You will not come across any internode-communication issues with your
> deployment configuration as long as the servers and clients are running
> withing the K8 environment.
>
> The issue, discussed here, takes place if one of the following happens:
>
>    - The clients are deployed in K8 while the servers are running on
>    virtual machines (or vice versa).
>    - A serverless function attempts to use a thick client that by the
>    current design creates a ServerSocket connection:
>    https://issues.apache.org/jira/browse/IGNITE-13013
>
> As for the continuous queries, I have the following use case that can
> easily fail. Let's say your 10 servers are running on virtual machines
> while a thick client is managed by K8. The client registers a CQ in the
> cluster and all 10 servers at some point in time will need to send an
> update notification to the client. To do that, they have to open a
> connection with the client, and here the things can fall apart.
>
> -
> Denis
>
>
> On Fri, Jun 26, 2020 at 11:54 PM Raymond Wilson <
> raymond_wilson@trimble.com>
> wrote:
>
> > I have just caught up with this discussion and wanted to outline a set of
> > use
> > cases we have that rely on server nodes communicating with client nodes.
> >
> > Firstly, I'd like to confirm my mental model of server & client nodes
> > within
> > a grid (ignoring thin clients for now):
> >
> > A grid contains a set of nodes somewhat arbitrarily labelled 'server' and
> > 'client' where the distinction of a 'server' node is that it is
> responsible
> > for containing data (in-memory only, or also with persistence). Apart
> from
> > that distinction, all nodes are essentially peers in the grid and may use
> > the messaging fabric, compute layer and other grid features on an equal
> > footing.
> >
> > In our solution we leverage these capabilities to build and orchestrate
> > complex analytics queries that utilise compute functions that are
> initiated
> > in three distinct ways: client -> client, client -> server and server ->
> > client, and where all three styles of initiation are using within a
> single
> > analytics request made to the grid it self. I can go into more detail
> about
> > the exact sequencing of these activities if you like, but it may be
> > sufficient to know they are used to reason about the problem statement
> and
> > proposals outlined here.
> >
> > Our infrastructure is deployed to Kubernetes using EKS on AWS, and all
> > three
> > relationships between client and server nodes noted above function well
> > (caveat: we do see odd things though such as long pauses on critical
> worker
> > threads, and occasional empty topology warnings when locating client
> nodes
> > to send requests to). We also use continuous queries in three contexts
> (all
> > within server nodes).
> >
> > If this thread is suggesting changing the functional relationship between
> > server and client nodes then this may have impacts on our architecture
> and
> > implementation that we will need to consider.
> >
> > This thread has highlighted issues with K8s deployments and also CQ
> issues.
> > The suggestion is that Server to Client just doesn't work on K8s, which
> > does
> > not agree with our experience of it working. I'd also like to understand
> > better the bounds of the issue with CQ: When does it not work and what
> are
> > the symptoms we would see if there was an issue with the way we are using
> > it, or the K8s infrastructure we deploy to?
> >
> > Thanks,
> > Raymond.
> >
> >
> >
> >
> > --
> > Sent from: http://apache-ignite-developers.2346864.n4.nabble.com/
> >
>


-- 
<http://www.trimble.com/>
Raymond Wilson
Solution Architect, Civil Construction Software Systems (CCSS)
11 Birmingham Drive | Christchurch, New Zealand
+64-21-2013317 Mobile
raymond_wilson@trimble.com

<https://worksos.trimble.com/?utm_source=Trimble&utm_medium=emailsign&utm_campaign=Launch>

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Denis Magda <dm...@apache.org>.
Hi Raymond,

You will not come across any internode-communication issues with your
deployment configuration as long as the servers and clients are running
withing the K8 environment.

The issue, discussed here, takes place if one of the following happens:

   - The clients are deployed in K8 while the servers are running on
   virtual machines (or vice versa).
   - A serverless function attempts to use a thick client that by the
   current design creates a ServerSocket connection:
   https://issues.apache.org/jira/browse/IGNITE-13013

As for the continuous queries, I have the following use case that can
easily fail. Let's say your 10 servers are running on virtual machines
while a thick client is managed by K8. The client registers a CQ in the
cluster and all 10 servers at some point in time will need to send an
update notification to the client. To do that, they have to open a
connection with the client, and here the things can fall apart.

-
Denis


On Fri, Jun 26, 2020 at 11:54 PM Raymond Wilson <ra...@trimble.com>
wrote:

> I have just caught up with this discussion and wanted to outline a set of
> use
> cases we have that rely on server nodes communicating with client nodes.
>
> Firstly, I'd like to confirm my mental model of server & client nodes
> within
> a grid (ignoring thin clients for now):
>
> A grid contains a set of nodes somewhat arbitrarily labelled 'server' and
> 'client' where the distinction of a 'server' node is that it is responsible
> for containing data (in-memory only, or also with persistence). Apart from
> that distinction, all nodes are essentially peers in the grid and may use
> the messaging fabric, compute layer and other grid features on an equal
> footing.
>
> In our solution we leverage these capabilities to build and orchestrate
> complex analytics queries that utilise compute functions that are initiated
> in three distinct ways: client -> client, client -> server and server ->
> client, and where all three styles of initiation are using within a single
> analytics request made to the grid it self. I can go into more detail about
> the exact sequencing of these activities if you like, but it may be
> sufficient to know they are used to reason about the problem statement and
> proposals outlined here.
>
> Our infrastructure is deployed to Kubernetes using EKS on AWS, and all
> three
> relationships between client and server nodes noted above function well
> (caveat: we do see odd things though such as long pauses on critical worker
> threads, and occasional empty topology warnings when locating client nodes
> to send requests to). We also use continuous queries in three contexts (all
> within server nodes).
>
> If this thread is suggesting changing the functional relationship between
> server and client nodes then this may have impacts on our architecture and
> implementation that we will need to consider.
>
> This thread has highlighted issues with K8s deployments and also CQ issues.
> The suggestion is that Server to Client just doesn't work on K8s, which
> does
> not agree with our experience of it working. I'd also like to understand
> better the bounds of the issue with CQ: When does it not work and what are
> the symptoms we would see if there was an issue with the way we are using
> it, or the K8s infrastructure we deploy to?
>
> Thanks,
> Raymond.
>
>
>
>
> --
> Sent from: http://apache-ignite-developers.2346864.n4.nabble.com/
>

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Ivan Bessonov <be...@gmail.com>.
Ivan,

Currently we have no requirement to maintain all possible connections
opened. Every node can have arbitrary number of connections to every
other node (it's configurable with "connectionsPerNode" setting).

Also, we can't expect that client would magically open connection when
we need it, that's the main issue. Changing this approach is out of
scope and I can't guarantee that it can or will be implemented this way.


пн, 29 июн. 2020 г. в 17:30, Ivan Pavlukhin <vo...@gmail.com>:

> Ivan,
>
> It seems that if a server notices that an existing connection to a
> client cannot be used anymore then the server can expect that the
> client will establish a new one. Is it just out of current iteration
> scope? Or are there still other fundamental problems?
>
> 2020-06-29 16:32 GMT+03:00, Ivan Bessonov <be...@gmail.com>:
> > Hi Ivan,
> >
> > sure, TCP connections are lazy. So, if a connection is not already opened
> > then node (trying to send a message) will initiate connection opening.
> > It's also possible that the opened connection is spontaneously closed for
> > some reason. Otherwise you are right, everything is as you described.
> >
> > There's also a tie breaker when two nodes connect to each other at the
> > same time. Only one of them will succeed and it depends on internal
> > discovery order, which you can't control basically.
> >
> > пн, 29 июн. 2020 г. в 16:01, Ivan Pavlukhin <vo...@gmail.com>:
> >
> >> Hi Ivan,
> >>
> >> Sorry for a possibly naive question. As I understand we are talking
> >> about order of establishing client-server connections. And I suppose
> >> that in some environments (e.g. cloud) servers cannot directly
> >> establish connections with clients. But TCP connections are
> >> bidirectional and we still can send messages in both directions. Could
> >> you please provide an example case in which servers have to initiate
> >> new connections to clients?
> >>
> >> 2020-06-29 13:08 GMT+03:00, Ivan Bessonov <be...@gmail.com>:
> >> > Hi igniters, Hi Raymond,
> >> >
> >> > that was a really good point. I will try to address it as much as I
> >> > can.
> >> >
> >> > First of all, this new mode will be configurable for now. As Val
> >> suggested,
> >> > "TcpCommunicationSpi#forceClientToServerConnections" will be a new
> >> > setting to trigger this behavior. Disabled by default.
> >> >
> >> > About issues with K8S deployments - I'm not an expert, but from what
> >> > I've
> >> > heard, sometimes servers and client nodes are not in the same
> >> environments.
> >> > For example, there is an Ignite cluster and user tries to start client
> >> node
> >> > in
> >> > isolated K8S pod. In this case clients cannot properly resolve their
> >> > own
> >> > addresses
> >> > and send it to servers, making it impossible for servers to connect to
> >> such
> >> > clients.
> >> > Or, in other words, clients are used as if they were thin.
> >> >
> >> > In your case everything is fine, clients and servers share the same
> >> network
> >> > and can resolve each other's addresses.
> >> >
> >> > Now, CQ issue [1]. You can pass a custom event filter when you
> register
> >> > a
> >> > new
> >> > continuous query. But, depending on the setup, the class of this
> filter
> >> may
> >> > not
> >> > be in the classpath of the server node that holds the data and invokes
> >> that
> >> > filter.
> >> > There are two solutions to the problem:
> >> > - server fails to resolve class name and fails to register CQ;
> >> > - or server can have p2p deployment enabled. Let's assume that it was
> a
> >> > client
> >> > node that requested CQ. In this case the server will try to download
> >> > "class" file
> >> > directly from the node that sent the filter object in the first place.
> >> Due
> >> > to a poor
> >> > design decision it will be done synchronously while registering the
> >> query,
> >> > and
> >> > query registration is happening in "discovery" thread. In normal
> >> > circumstances
> >> > the server will load the class and finish query registration, it's
> just
> >> > a
> >> > little bit slow.
> >> >
> >> > Second case is not compatible with a new
> >> > "forceClientToServerConnections"
> >> > setting. I'm not sure that I need to go into all technical details,
> but
> >> the
> >> > result of
> >> > such procedure is a cluster that cannot process any discovery messages
> >> > during
> >> > TCP connection timeout, we're talking about tens of seconds or maybe
> >> > even
> >> > several minutes depending on the settings and the environment. All
> this
> >> > time the
> >> > server will be in a "deadlock" state inside of the "discovery" thread.
> >> > It
> >> > means that
> >> > some cluster operations will be unavailable during this period, like
> >> > new
> >> > node joining
> >> > or starting a new cache. Node failures will not be processed properly
> >> > as
> >> > well. For
> >> > me it's hard to predict real behavior until we reproduce the situation
> >> in a
> >> > live
> >> > environment. I saw this in tests only.
> >> >
> >> > I hope that my message clarifies the situation, or at least doesn't
> >> > cause
> >> > more
> >> > confusion. These changes will not affect your infrastructure or your
> >> Ignite
> >> > installations, they are aimed at adding more flexibility to other ways
> >> > of
> >> > using Ignite.
> >> >
> >> > [1] https://issues.apache.org/jira/browse/IGNITE-13156
> >> >
> >> >
> >> >
> >> > сб, 27 июн. 2020 г. в 09:54, Raymond Wilson <
> raymond_wilson@trimble.com
> >> >:
> >> >
> >> >> I have just caught up with this discussion and wanted to outline a
> set
> >> of
> >> >> use
> >> >> cases we have that rely on server nodes communicating with client
> >> >> nodes.
> >> >>
> >> >> Firstly, I'd like to confirm my mental model of server & client nodes
> >> >> within
> >> >> a grid (ignoring thin clients for now):
> >> >>
> >> >> A grid contains a set of nodes somewhat arbitrarily labelled 'server'
> >> and
> >> >> 'client' where the distinction of a 'server' node is that it is
> >> >> responsible
> >> >> for containing data (in-memory only, or also with persistence). Apart
> >> >> from
> >> >> that distinction, all nodes are essentially peers in the grid and may
> >> use
> >> >> the messaging fabric, compute layer and other grid features on an
> >> >> equal
> >> >> footing.
> >> >>
> >> >> In our solution we leverage these capabilities to build and
> >> >> orchestrate
> >> >> complex analytics queries that utilise compute functions that are
> >> >> initiated
> >> >> in three distinct ways: client -> client, client -> server and server
> >> >> ->
> >> >> client, and where all three styles of initiation are using within a
> >> >> single
> >> >> analytics request made to the grid it self. I can go into more detail
> >> >> about
> >> >> the exact sequencing of these activities if you like, but it may be
> >> >> sufficient to know they are used to reason about the problem
> statement
> >> >> and
> >> >> proposals outlined here.
> >> >>
> >> >> Our infrastructure is deployed to Kubernetes using EKS on AWS, and
> all
> >> >> three
> >> >> relationships between client and server nodes noted above function
> >> >> well
> >> >> (caveat: we do see odd things though such as long pauses on critical
> >> >> worker
> >> >> threads, and occasional empty topology warnings when locating client
> >> >> nodes
> >> >> to send requests to). We also use continuous queries in three
> contexts
> >> >> (all
> >> >> within server nodes).
> >> >>
> >> >> If this thread is suggesting changing the functional relationship
> >> between
> >> >> server and client nodes then this may have impacts on our
> architecture
> >> >> and
> >> >> implementation that we will need to consider.
> >> >>
> >> >> This thread has highlighted issues with K8s deployments and also CQ
> >> >> issues.
> >> >> The suggestion is that Server to Client just doesn't work on K8s,
> >> >> which
> >> >> does
> >> >> not agree with our experience of it working. I'd also like to
> >> >> understand
> >> >> better the bounds of the issue with CQ: When does it not work and
> what
> >> >> are
> >> >> the symptoms we would see if there was an issue with the way we are
> >> using
> >> >> it, or the K8s infrastructure we deploy to?
> >> >>
> >> >> Thanks,
> >> >> Raymond.
> >> >>
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> Sent from: http://apache-ignite-developers.2346864.n4.nabble.com/
> >> >>
> >> >
> >> >
> >> > --
> >> > Sincerely yours,
> >> > Ivan Bessonov
> >> >
> >>
> >>
> >> --
> >>
> >> Best regards,
> >> Ivan Pavlukhin
> >>
> >
> >
> > --
> > Sincerely yours,
> > Ivan Bessonov
> >
>
>
> --
>
> Best regards,
> Ivan Pavlukhin
>


-- 
Sincerely yours,
Ivan Bessonov

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Ivan Pavlukhin <vo...@gmail.com>.
Ivan,

It seems that if a server notices that an existing connection to a
client cannot be used anymore then the server can expect that the
client will establish a new one. Is it just out of current iteration
scope? Or are there still other fundamental problems?

2020-06-29 16:32 GMT+03:00, Ivan Bessonov <be...@gmail.com>:
> Hi Ivan,
>
> sure, TCP connections are lazy. So, if a connection is not already opened
> then node (trying to send a message) will initiate connection opening.
> It's also possible that the opened connection is spontaneously closed for
> some reason. Otherwise you are right, everything is as you described.
>
> There's also a tie breaker when two nodes connect to each other at the
> same time. Only one of them will succeed and it depends on internal
> discovery order, which you can't control basically.
>
> пн, 29 июн. 2020 г. в 16:01, Ivan Pavlukhin <vo...@gmail.com>:
>
>> Hi Ivan,
>>
>> Sorry for a possibly naive question. As I understand we are talking
>> about order of establishing client-server connections. And I suppose
>> that in some environments (e.g. cloud) servers cannot directly
>> establish connections with clients. But TCP connections are
>> bidirectional and we still can send messages in both directions. Could
>> you please provide an example case in which servers have to initiate
>> new connections to clients?
>>
>> 2020-06-29 13:08 GMT+03:00, Ivan Bessonov <be...@gmail.com>:
>> > Hi igniters, Hi Raymond,
>> >
>> > that was a really good point. I will try to address it as much as I
>> > can.
>> >
>> > First of all, this new mode will be configurable for now. As Val
>> suggested,
>> > "TcpCommunicationSpi#forceClientToServerConnections" will be a new
>> > setting to trigger this behavior. Disabled by default.
>> >
>> > About issues with K8S deployments - I'm not an expert, but from what
>> > I've
>> > heard, sometimes servers and client nodes are not in the same
>> environments.
>> > For example, there is an Ignite cluster and user tries to start client
>> node
>> > in
>> > isolated K8S pod. In this case clients cannot properly resolve their
>> > own
>> > addresses
>> > and send it to servers, making it impossible for servers to connect to
>> such
>> > clients.
>> > Or, in other words, clients are used as if they were thin.
>> >
>> > In your case everything is fine, clients and servers share the same
>> network
>> > and can resolve each other's addresses.
>> >
>> > Now, CQ issue [1]. You can pass a custom event filter when you register
>> > a
>> > new
>> > continuous query. But, depending on the setup, the class of this filter
>> may
>> > not
>> > be in the classpath of the server node that holds the data and invokes
>> that
>> > filter.
>> > There are two solutions to the problem:
>> > - server fails to resolve class name and fails to register CQ;
>> > - or server can have p2p deployment enabled. Let's assume that it was a
>> > client
>> > node that requested CQ. In this case the server will try to download
>> > "class" file
>> > directly from the node that sent the filter object in the first place.
>> Due
>> > to a poor
>> > design decision it will be done synchronously while registering the
>> query,
>> > and
>> > query registration is happening in "discovery" thread. In normal
>> > circumstances
>> > the server will load the class and finish query registration, it's just
>> > a
>> > little bit slow.
>> >
>> > Second case is not compatible with a new
>> > "forceClientToServerConnections"
>> > setting. I'm not sure that I need to go into all technical details, but
>> the
>> > result of
>> > such procedure is a cluster that cannot process any discovery messages
>> > during
>> > TCP connection timeout, we're talking about tens of seconds or maybe
>> > even
>> > several minutes depending on the settings and the environment. All this
>> > time the
>> > server will be in a "deadlock" state inside of the "discovery" thread.
>> > It
>> > means that
>> > some cluster operations will be unavailable during this period, like
>> > new
>> > node joining
>> > or starting a new cache. Node failures will not be processed properly
>> > as
>> > well. For
>> > me it's hard to predict real behavior until we reproduce the situation
>> in a
>> > live
>> > environment. I saw this in tests only.
>> >
>> > I hope that my message clarifies the situation, or at least doesn't
>> > cause
>> > more
>> > confusion. These changes will not affect your infrastructure or your
>> Ignite
>> > installations, they are aimed at adding more flexibility to other ways
>> > of
>> > using Ignite.
>> >
>> > [1] https://issues.apache.org/jira/browse/IGNITE-13156
>> >
>> >
>> >
>> > сб, 27 июн. 2020 г. в 09:54, Raymond Wilson <raymond_wilson@trimble.com
>> >:
>> >
>> >> I have just caught up with this discussion and wanted to outline a set
>> of
>> >> use
>> >> cases we have that rely on server nodes communicating with client
>> >> nodes.
>> >>
>> >> Firstly, I'd like to confirm my mental model of server & client nodes
>> >> within
>> >> a grid (ignoring thin clients for now):
>> >>
>> >> A grid contains a set of nodes somewhat arbitrarily labelled 'server'
>> and
>> >> 'client' where the distinction of a 'server' node is that it is
>> >> responsible
>> >> for containing data (in-memory only, or also with persistence). Apart
>> >> from
>> >> that distinction, all nodes are essentially peers in the grid and may
>> use
>> >> the messaging fabric, compute layer and other grid features on an
>> >> equal
>> >> footing.
>> >>
>> >> In our solution we leverage these capabilities to build and
>> >> orchestrate
>> >> complex analytics queries that utilise compute functions that are
>> >> initiated
>> >> in three distinct ways: client -> client, client -> server and server
>> >> ->
>> >> client, and where all three styles of initiation are using within a
>> >> single
>> >> analytics request made to the grid it self. I can go into more detail
>> >> about
>> >> the exact sequencing of these activities if you like, but it may be
>> >> sufficient to know they are used to reason about the problem statement
>> >> and
>> >> proposals outlined here.
>> >>
>> >> Our infrastructure is deployed to Kubernetes using EKS on AWS, and all
>> >> three
>> >> relationships between client and server nodes noted above function
>> >> well
>> >> (caveat: we do see odd things though such as long pauses on critical
>> >> worker
>> >> threads, and occasional empty topology warnings when locating client
>> >> nodes
>> >> to send requests to). We also use continuous queries in three contexts
>> >> (all
>> >> within server nodes).
>> >>
>> >> If this thread is suggesting changing the functional relationship
>> between
>> >> server and client nodes then this may have impacts on our architecture
>> >> and
>> >> implementation that we will need to consider.
>> >>
>> >> This thread has highlighted issues with K8s deployments and also CQ
>> >> issues.
>> >> The suggestion is that Server to Client just doesn't work on K8s,
>> >> which
>> >> does
>> >> not agree with our experience of it working. I'd also like to
>> >> understand
>> >> better the bounds of the issue with CQ: When does it not work and what
>> >> are
>> >> the symptoms we would see if there was an issue with the way we are
>> using
>> >> it, or the K8s infrastructure we deploy to?
>> >>
>> >> Thanks,
>> >> Raymond.
>> >>
>> >>
>> >>
>> >>
>> >> --
>> >> Sent from: http://apache-ignite-developers.2346864.n4.nabble.com/
>> >>
>> >
>> >
>> > --
>> > Sincerely yours,
>> > Ivan Bessonov
>> >
>>
>>
>> --
>>
>> Best regards,
>> Ivan Pavlukhin
>>
>
>
> --
> Sincerely yours,
> Ivan Bessonov
>


-- 

Best regards,
Ivan Pavlukhin

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Ivan Bessonov <be...@gmail.com>.
Hi Ivan,

sure, TCP connections are lazy. So, if a connection is not already opened
then node (trying to send a message) will initiate connection opening.
It's also possible that the opened connection is spontaneously closed for
some reason. Otherwise you are right, everything is as you described.

There's also a tie breaker when two nodes connect to each other at the
same time. Only one of them will succeed and it depends on internal
discovery order, which you can't control basically.

пн, 29 июн. 2020 г. в 16:01, Ivan Pavlukhin <vo...@gmail.com>:

> Hi Ivan,
>
> Sorry for a possibly naive question. As I understand we are talking
> about order of establishing client-server connections. And I suppose
> that in some environments (e.g. cloud) servers cannot directly
> establish connections with clients. But TCP connections are
> bidirectional and we still can send messages in both directions. Could
> you please provide an example case in which servers have to initiate
> new connections to clients?
>
> 2020-06-29 13:08 GMT+03:00, Ivan Bessonov <be...@gmail.com>:
> > Hi igniters, Hi Raymond,
> >
> > that was a really good point. I will try to address it as much as I can.
> >
> > First of all, this new mode will be configurable for now. As Val
> suggested,
> > "TcpCommunicationSpi#forceClientToServerConnections" will be a new
> > setting to trigger this behavior. Disabled by default.
> >
> > About issues with K8S deployments - I'm not an expert, but from what I've
> > heard, sometimes servers and client nodes are not in the same
> environments.
> > For example, there is an Ignite cluster and user tries to start client
> node
> > in
> > isolated K8S pod. In this case clients cannot properly resolve their own
> > addresses
> > and send it to servers, making it impossible for servers to connect to
> such
> > clients.
> > Or, in other words, clients are used as if they were thin.
> >
> > In your case everything is fine, clients and servers share the same
> network
> > and can resolve each other's addresses.
> >
> > Now, CQ issue [1]. You can pass a custom event filter when you register a
> > new
> > continuous query. But, depending on the setup, the class of this filter
> may
> > not
> > be in the classpath of the server node that holds the data and invokes
> that
> > filter.
> > There are two solutions to the problem:
> > - server fails to resolve class name and fails to register CQ;
> > - or server can have p2p deployment enabled. Let's assume that it was a
> > client
> > node that requested CQ. In this case the server will try to download
> > "class" file
> > directly from the node that sent the filter object in the first place.
> Due
> > to a poor
> > design decision it will be done synchronously while registering the
> query,
> > and
> > query registration is happening in "discovery" thread. In normal
> > circumstances
> > the server will load the class and finish query registration, it's just a
> > little bit slow.
> >
> > Second case is not compatible with a new "forceClientToServerConnections"
> > setting. I'm not sure that I need to go into all technical details, but
> the
> > result of
> > such procedure is a cluster that cannot process any discovery messages
> > during
> > TCP connection timeout, we're talking about tens of seconds or maybe even
> > several minutes depending on the settings and the environment. All this
> > time the
> > server will be in a "deadlock" state inside of the "discovery" thread. It
> > means that
> > some cluster operations will be unavailable during this period, like new
> > node joining
> > or starting a new cache. Node failures will not be processed properly as
> > well. For
> > me it's hard to predict real behavior until we reproduce the situation
> in a
> > live
> > environment. I saw this in tests only.
> >
> > I hope that my message clarifies the situation, or at least doesn't cause
> > more
> > confusion. These changes will not affect your infrastructure or your
> Ignite
> > installations, they are aimed at adding more flexibility to other ways of
> > using Ignite.
> >
> > [1] https://issues.apache.org/jira/browse/IGNITE-13156
> >
> >
> >
> > сб, 27 июн. 2020 г. в 09:54, Raymond Wilson <raymond_wilson@trimble.com
> >:
> >
> >> I have just caught up with this discussion and wanted to outline a set
> of
> >> use
> >> cases we have that rely on server nodes communicating with client nodes.
> >>
> >> Firstly, I'd like to confirm my mental model of server & client nodes
> >> within
> >> a grid (ignoring thin clients for now):
> >>
> >> A grid contains a set of nodes somewhat arbitrarily labelled 'server'
> and
> >> 'client' where the distinction of a 'server' node is that it is
> >> responsible
> >> for containing data (in-memory only, or also with persistence). Apart
> >> from
> >> that distinction, all nodes are essentially peers in the grid and may
> use
> >> the messaging fabric, compute layer and other grid features on an equal
> >> footing.
> >>
> >> In our solution we leverage these capabilities to build and orchestrate
> >> complex analytics queries that utilise compute functions that are
> >> initiated
> >> in three distinct ways: client -> client, client -> server and server ->
> >> client, and where all three styles of initiation are using within a
> >> single
> >> analytics request made to the grid it self. I can go into more detail
> >> about
> >> the exact sequencing of these activities if you like, but it may be
> >> sufficient to know they are used to reason about the problem statement
> >> and
> >> proposals outlined here.
> >>
> >> Our infrastructure is deployed to Kubernetes using EKS on AWS, and all
> >> three
> >> relationships between client and server nodes noted above function well
> >> (caveat: we do see odd things though such as long pauses on critical
> >> worker
> >> threads, and occasional empty topology warnings when locating client
> >> nodes
> >> to send requests to). We also use continuous queries in three contexts
> >> (all
> >> within server nodes).
> >>
> >> If this thread is suggesting changing the functional relationship
> between
> >> server and client nodes then this may have impacts on our architecture
> >> and
> >> implementation that we will need to consider.
> >>
> >> This thread has highlighted issues with K8s deployments and also CQ
> >> issues.
> >> The suggestion is that Server to Client just doesn't work on K8s, which
> >> does
> >> not agree with our experience of it working. I'd also like to understand
> >> better the bounds of the issue with CQ: When does it not work and what
> >> are
> >> the symptoms we would see if there was an issue with the way we are
> using
> >> it, or the K8s infrastructure we deploy to?
> >>
> >> Thanks,
> >> Raymond.
> >>
> >>
> >>
> >>
> >> --
> >> Sent from: http://apache-ignite-developers.2346864.n4.nabble.com/
> >>
> >
> >
> > --
> > Sincerely yours,
> > Ivan Bessonov
> >
>
>
> --
>
> Best regards,
> Ivan Pavlukhin
>


-- 
Sincerely yours,
Ivan Bessonov

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Ivan Pavlukhin <vo...@gmail.com>.
Hi Ivan,

Sorry for a possibly naive question. As I understand we are talking
about order of establishing client-server connections. And I suppose
that in some environments (e.g. cloud) servers cannot directly
establish connections with clients. But TCP connections are
bidirectional and we still can send messages in both directions. Could
you please provide an example case in which servers have to initiate
new connections to clients?

2020-06-29 13:08 GMT+03:00, Ivan Bessonov <be...@gmail.com>:
> Hi igniters, Hi Raymond,
>
> that was a really good point. I will try to address it as much as I can.
>
> First of all, this new mode will be configurable for now. As Val suggested,
> "TcpCommunicationSpi#forceClientToServerConnections" will be a new
> setting to trigger this behavior. Disabled by default.
>
> About issues with K8S deployments - I'm not an expert, but from what I've
> heard, sometimes servers and client nodes are not in the same environments.
> For example, there is an Ignite cluster and user tries to start client node
> in
> isolated K8S pod. In this case clients cannot properly resolve their own
> addresses
> and send it to servers, making it impossible for servers to connect to such
> clients.
> Or, in other words, clients are used as if they were thin.
>
> In your case everything is fine, clients and servers share the same network
> and can resolve each other's addresses.
>
> Now, CQ issue [1]. You can pass a custom event filter when you register a
> new
> continuous query. But, depending on the setup, the class of this filter may
> not
> be in the classpath of the server node that holds the data and invokes that
> filter.
> There are two solutions to the problem:
> - server fails to resolve class name and fails to register CQ;
> - or server can have p2p deployment enabled. Let's assume that it was a
> client
> node that requested CQ. In this case the server will try to download
> "class" file
> directly from the node that sent the filter object in the first place. Due
> to a poor
> design decision it will be done synchronously while registering the query,
> and
> query registration is happening in "discovery" thread. In normal
> circumstances
> the server will load the class and finish query registration, it's just a
> little bit slow.
>
> Second case is not compatible with a new "forceClientToServerConnections"
> setting. I'm not sure that I need to go into all technical details, but the
> result of
> such procedure is a cluster that cannot process any discovery messages
> during
> TCP connection timeout, we're talking about tens of seconds or maybe even
> several minutes depending on the settings and the environment. All this
> time the
> server will be in a "deadlock" state inside of the "discovery" thread. It
> means that
> some cluster operations will be unavailable during this period, like new
> node joining
> or starting a new cache. Node failures will not be processed properly as
> well. For
> me it's hard to predict real behavior until we reproduce the situation in a
> live
> environment. I saw this in tests only.
>
> I hope that my message clarifies the situation, or at least doesn't cause
> more
> confusion. These changes will not affect your infrastructure or your Ignite
> installations, they are aimed at adding more flexibility to other ways of
> using Ignite.
>
> [1] https://issues.apache.org/jira/browse/IGNITE-13156
>
>
>
> сб, 27 июн. 2020 г. в 09:54, Raymond Wilson <ra...@trimble.com>:
>
>> I have just caught up with this discussion and wanted to outline a set of
>> use
>> cases we have that rely on server nodes communicating with client nodes.
>>
>> Firstly, I'd like to confirm my mental model of server & client nodes
>> within
>> a grid (ignoring thin clients for now):
>>
>> A grid contains a set of nodes somewhat arbitrarily labelled 'server' and
>> 'client' where the distinction of a 'server' node is that it is
>> responsible
>> for containing data (in-memory only, or also with persistence). Apart
>> from
>> that distinction, all nodes are essentially peers in the grid and may use
>> the messaging fabric, compute layer and other grid features on an equal
>> footing.
>>
>> In our solution we leverage these capabilities to build and orchestrate
>> complex analytics queries that utilise compute functions that are
>> initiated
>> in three distinct ways: client -> client, client -> server and server ->
>> client, and where all three styles of initiation are using within a
>> single
>> analytics request made to the grid it self. I can go into more detail
>> about
>> the exact sequencing of these activities if you like, but it may be
>> sufficient to know they are used to reason about the problem statement
>> and
>> proposals outlined here.
>>
>> Our infrastructure is deployed to Kubernetes using EKS on AWS, and all
>> three
>> relationships between client and server nodes noted above function well
>> (caveat: we do see odd things though such as long pauses on critical
>> worker
>> threads, and occasional empty topology warnings when locating client
>> nodes
>> to send requests to). We also use continuous queries in three contexts
>> (all
>> within server nodes).
>>
>> If this thread is suggesting changing the functional relationship between
>> server and client nodes then this may have impacts on our architecture
>> and
>> implementation that we will need to consider.
>>
>> This thread has highlighted issues with K8s deployments and also CQ
>> issues.
>> The suggestion is that Server to Client just doesn't work on K8s, which
>> does
>> not agree with our experience of it working. I'd also like to understand
>> better the bounds of the issue with CQ: When does it not work and what
>> are
>> the symptoms we would see if there was an issue with the way we are using
>> it, or the K8s infrastructure we deploy to?
>>
>> Thanks,
>> Raymond.
>>
>>
>>
>>
>> --
>> Sent from: http://apache-ignite-developers.2346864.n4.nabble.com/
>>
>
>
> --
> Sincerely yours,
> Ivan Bessonov
>


-- 

Best regards,
Ivan Pavlukhin

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Ivan Bessonov <be...@gmail.com>.
Hi igniters, Hi Raymond,

that was a really good point. I will try to address it as much as I can.

First of all, this new mode will be configurable for now. As Val suggested,
"TcpCommunicationSpi#forceClientToServerConnections" will be a new
setting to trigger this behavior. Disabled by default.

About issues with K8S deployments - I'm not an expert, but from what I've
heard, sometimes servers and client nodes are not in the same environments.
For example, there is an Ignite cluster and user tries to start client node
in
isolated K8S pod. In this case clients cannot properly resolve their own
addresses
and send it to servers, making it impossible for servers to connect to such
clients.
Or, in other words, clients are used as if they were thin.

In your case everything is fine, clients and servers share the same network
and can resolve each other's addresses.

Now, CQ issue [1]. You can pass a custom event filter when you register a
new
continuous query. But, depending on the setup, the class of this filter may
not
be in the classpath of the server node that holds the data and invokes that
filter.
There are two solutions to the problem:
- server fails to resolve class name and fails to register CQ;
- or server can have p2p deployment enabled. Let's assume that it was a
client
node that requested CQ. In this case the server will try to download
"class" file
directly from the node that sent the filter object in the first place. Due
to a poor
design decision it will be done synchronously while registering the query,
and
query registration is happening in "discovery" thread. In normal
circumstances
the server will load the class and finish query registration, it's just a
little bit slow.

Second case is not compatible with a new "forceClientToServerConnections"
setting. I'm not sure that I need to go into all technical details, but the
result of
such procedure is a cluster that cannot process any discovery messages
during
TCP connection timeout, we're talking about tens of seconds or maybe even
several minutes depending on the settings and the environment. All this
time the
server will be in a "deadlock" state inside of the "discovery" thread. It
means that
some cluster operations will be unavailable during this period, like new
node joining
or starting a new cache. Node failures will not be processed properly as
well. For
me it's hard to predict real behavior until we reproduce the situation in a
live
environment. I saw this in tests only.

I hope that my message clarifies the situation, or at least doesn't cause
more
confusion. These changes will not affect your infrastructure or your Ignite
installations, they are aimed at adding more flexibility to other ways of
using Ignite.

[1] https://issues.apache.org/jira/browse/IGNITE-13156



сб, 27 июн. 2020 г. в 09:54, Raymond Wilson <ra...@trimble.com>:

> I have just caught up with this discussion and wanted to outline a set of
> use
> cases we have that rely on server nodes communicating with client nodes.
>
> Firstly, I'd like to confirm my mental model of server & client nodes
> within
> a grid (ignoring thin clients for now):
>
> A grid contains a set of nodes somewhat arbitrarily labelled 'server' and
> 'client' where the distinction of a 'server' node is that it is responsible
> for containing data (in-memory only, or also with persistence). Apart from
> that distinction, all nodes are essentially peers in the grid and may use
> the messaging fabric, compute layer and other grid features on an equal
> footing.
>
> In our solution we leverage these capabilities to build and orchestrate
> complex analytics queries that utilise compute functions that are initiated
> in three distinct ways: client -> client, client -> server and server ->
> client, and where all three styles of initiation are using within a single
> analytics request made to the grid it self. I can go into more detail about
> the exact sequencing of these activities if you like, but it may be
> sufficient to know they are used to reason about the problem statement and
> proposals outlined here.
>
> Our infrastructure is deployed to Kubernetes using EKS on AWS, and all
> three
> relationships between client and server nodes noted above function well
> (caveat: we do see odd things though such as long pauses on critical worker
> threads, and occasional empty topology warnings when locating client nodes
> to send requests to). We also use continuous queries in three contexts (all
> within server nodes).
>
> If this thread is suggesting changing the functional relationship between
> server and client nodes then this may have impacts on our architecture and
> implementation that we will need to consider.
>
> This thread has highlighted issues with K8s deployments and also CQ issues.
> The suggestion is that Server to Client just doesn't work on K8s, which
> does
> not agree with our experience of it working. I'd also like to understand
> better the bounds of the issue with CQ: When does it not work and what are
> the symptoms we would see if there was an issue with the way we are using
> it, or the K8s infrastructure we deploy to?
>
> Thanks,
> Raymond.
>
>
>
>
> --
> Sent from: http://apache-ignite-developers.2346864.n4.nabble.com/
>


-- 
Sincerely yours,
Ivan Bessonov

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Raymond Wilson <ra...@trimble.com>.
I have just caught up with this discussion and wanted to outline a set of use
cases we have that rely on server nodes communicating with client nodes.

Firstly, I'd like to confirm my mental model of server & client nodes within
a grid (ignoring thin clients for now): 

A grid contains a set of nodes somewhat arbitrarily labelled 'server' and
'client' where the distinction of a 'server' node is that it is responsible
for containing data (in-memory only, or also with persistence). Apart from
that distinction, all nodes are essentially peers in the grid and may use
the messaging fabric, compute layer and other grid features on an equal
footing.

In our solution we leverage these capabilities to build and orchestrate
complex analytics queries that utilise compute functions that are initiated
in three distinct ways: client -> client, client -> server and server ->
client, and where all three styles of initiation are using within a single
analytics request made to the grid it self. I can go into more detail about
the exact sequencing of these activities if you like, but it may be
sufficient to know they are used to reason about the problem statement and
proposals outlined here.

Our infrastructure is deployed to Kubernetes using EKS on AWS, and all three
relationships between client and server nodes noted above function well
(caveat: we do see odd things though such as long pauses on critical worker
threads, and occasional empty topology warnings when locating client nodes
to send requests to). We also use continuous queries in three contexts (all
within server nodes).

If this thread is suggesting changing the functional relationship between
server and client nodes then this may have impacts on our architecture and
implementation that we will need to consider.

This thread has highlighted issues with K8s deployments and also CQ issues.
The suggestion is that Server to Client just doesn't work on K8s, which does
not agree with our experience of it working. I'd also like to understand
better the bounds of the issue with CQ: When does it not work and what are
the symptoms we would see if there was an issue with the way we are using
it, or the K8s infrastructure we deploy to?

Thanks,
Raymond.




--
Sent from: http://apache-ignite-developers.2346864.n4.nabble.com/

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Sergey Chugunov <se...@gmail.com>.
Val,

I like your suggestion about naming, it describes the purpose of the
configuration the best.

On Tue, Jun 16, 2020 at 5:18 PM Ivan Bessonov <be...@gmail.com> wrote:

> Hi,
>
> I created new issue that describes CQ problem in more details: [1]
> I'm fine with experimental status and new property naming.
>
> [1] https://issues.apache.org/jira/browse/IGNITE-13156
>
> вт, 16 июн. 2020 г. в 02:20, Valentin Kulichenko <
> valentin.kulichenko@gmail.com>:
>
> > Folks,
> >
> > Thanks for providing the detailed clarifications. Let's add the
> parameter,
> > mark the new feature as experimental, and target for making it the
> default
> > mode in Ignite 3.0.
> >
> > I still don't think we can come up with a naming that is really
> intuitive,
> > but let's try to simplify it as much as possible. How about this:
> >
> > TcpCommunicationSpi#forceClientToServerConnections -- false by default,
> > true if the new mode needs to be enabled.
> >
> > Let me know your thoughts.
> >
> > -Val
> >
> > On Wed, Jun 10, 2020 at 4:10 PM Denis Magda <dm...@apache.org> wrote:
> >
> > > Sergey,
> > >
> > > Thanks for the detailed explanation and for covering all corner cases.
> > >
> > > Considering the improvement's criticality, I would continue moving in
> the
> > > initial direction and add that particular configuration property.
> > > Potentially, we can put more effort throughout an Ignite 3.0 timeframe
> > and
> > > remove the property altogether. @Valentin Kulichenko
> > > <vk...@gridgain.com>, could you please suggest any alternate
> > naming?
> > >
> > > Btw, what are the specifics of the issue with continuous queries? It
> will
> > > be ideal if we could release this new communication option in the GA
> > state
> > > in 2.9.
> > >
> > > -
> > > Denis
> > >
> > >
> > > On Wed, Jun 10, 2020 at 1:22 AM Sergey Chugunov <
> > sergey.chugunov@gmail.com
> > > >
> > > wrote:
> > >
> > > > Denis, Val,
> > > >
> > > > Idea of prohibiting servers to open connections to clients and force
> > > > clients to always open "inverse connections" to servers looks
> > promising.
> > > To
> > > > be clear, by "inverse connections" I mean here that server needed to
> > > > communicate with client requests client to open a connection back
> > instead
> > > > of opening connection by itself using addresses published by the
> > client.
> > > >
> > > > If we apply the idea it will indeed allow us to simplify our
> > > configuration
> > > > (no need for new configuration property), another advantage is
> clients
> > > > won't need to publish their addresses anymore (with one side note
> I'll
> > > > cover at the end), it will also simplify our code.
> > > >
> > > > However applying it with current implementation of inverse connection
> > > > request (when request goes across all ring) may bring significant
> delay
> > > of
> > > > opening first connection depending on cluster size and relative
> > positions
> > > > between server that needs to communicate with client (target server)
> > and
> > > > client's router node.
> > > >
> > > > It is possible to overcome this by sending inverse connection request
> > not
> > > > via discovery but directly to router server node via communication
> and
> > > > convert to discovery message only on the router. We'll still have two
> > > hops
> > > > of communication request instead of one plus discovery worker on
> client
> > > may
> > > > be busy working on other stuff slowing down handling of connection
> > > request.
> > > > But it should be fine.
> > > >
> > > > However with this solution it is hard to implement failover of router
> > > node:
> > > > let me describe it in more details.
> > > > In case of router node failure target server won't be able to
> determine
> > > if
> > > > client received inverse comm request successfully and (even worse)
> > won't
> > > be
> > > > able to figure out new router for the client without waiting for
> > > discovery
> > > > event of the client reconnect.
> > > > And this brings us to the following choise: we either wait for
> > discovery
> > > > event about client reconnect (this is deadlock-prone as current
> > protocol
> > > of
> > > > CQ registration opens comm connection to client right from discovery
> > > thread
> > > > in some cases; if we wait for new discovery event from discovery
> > thread,
> > > it
> > > > is a deadlock) or we fail opening the connection by timeout thus
> adding
> > > new
> > > > scenarios when opening connection may fail.
> > > >
> > > > Thus implementing communication model "clients connect to servers,
> > > servers
> > > > never connect to clients" efficiently requires more work on different
> > > parts
> > > > of our functionality and rigorous testing of readiness of our code
> for
> > > more
> > > > communication connection failures.
> > > >
> > > > So to me the least risky decision is not to delete new configuration
> > but
> > > > leave it with experimental status. If we find out that direct request
> > > > (server -> router server -> target client) implementation works well
> > and
> > > > doesn't bring much complexity in failover scenarios we'll remove that
> > > > configuration and prohibit servers to open connections to clients by
> > > > default.
> > > >
> > > > Side note: there are rare but yet possible scenarios where client
> node
> > > > needs to open communication connection to other client node. If we
> let
> > > > clients not to publish their addresses these scenarios will stop
> > working
> > > > without additional logic like sending data through router node. As
> far
> > > as I
> > > > know client-client connectivity is involved in p2p class deployment
> > > > scenarios, does anyone know about other cases?
> > > >
> > > > --
> > > > Thanks,
> > > > Sergey Chugunov
> > > >
> > > > On Wed, Jun 3, 2020 at 5:37 PM Denis Magda <dm...@apache.org>
> wrote:
> > > >
> > > > > Ivan,
> > > > >
> > > > > It feels like Val is driving us in the right direction. Is there
> any
> > > > reason
> > > > > for keeping the current logic when servers can open connections to
> > > > clients?
> > > > >
> > > > > -
> > > > > Denis
> > > > >
> > > > >
> > > > > On Thu, May 21, 2020 at 4:48 PM Valentin Kulichenko <
> > > > > valentin.kulichenko@gmail.com> wrote:
> > > > >
> > > > > > Ivan,
> > > > > >
> > > > > > Have you considered eliminating server to client connections
> > > > altogether?
> > > > > > Or, at the very least making the "client to server only" mode the
> > > > default
> > > > > > one?
> > > > > >
> > > > > > All the suggested names are confusing and not intuitive, and I
> > doubt
> > > we
> > > > > > will be able to find a good one. A server initiating a TCP
> > connection
> > > > > with
> > > > > > a client is confusing in the first place and creates a usability
> > > issue.
> > > > > We
> > > > > > now want to solve it by introducing an additional configuration
> > > > > > parameter, and therefore additional complexity. I don't think
> this
> > is
> > > > the
> > > > > > right approach.
> > > > > >
> > > > > > What are the drawbacks of permanently switching to
> client-to-server
> > > > > > connections? Is there any value provided by the server-to-client
> > > > option?
> > > > > >
> > > > > > As for pair connections, I'm not sure I understand why there is a
> > > > > > limitation. As far as I know, the idea behind this feature is
> that
> > we
> > > > > > maintain two connections between two nodes instead of one, so
> that
> > > > every
> > > > > > connection is used for communication in a single direction only.
> > Why
> > > > does
> > > > > > it matter which node initiates the connection? Why can't one of
> the
> > > > nodes
> > > > > > (e.g., a client) initiate both connections, and then use them
> > > > > accordingly?
> > > > > > Correct me if I'm wrong, but I don't see why we can't do this.
> > > > > >
> > > > > > -Val
> > > > > >
> > > > > > On Thu, May 21, 2020 at 1:58 PM Denis Magda <dm...@apache.org>
> > > wrote:
> > > > > >
> > > > > > > Ivan,
> > > > > > >
> > > > > > > Considering that the setting controls the way a communication
> SPI
> > > > > > > connection is open I would add the new parameter to
> > > CommunicationSpi
> > > > > > > interface naming it as follows:
> > > > > > >
> > > > > > > >
> > > > > > > > CommunicationSpi.connectionInitiationMode
> > > > > > > > {
> > > > > > > >     BIDIRECTIONAL, //both clients and servers initiate a
> > > connection
> > > > > > > > initiation procedure
> > > > > > > >     CLIENTS_TO_SERVERS //servers cannot open a connection to
> > > > clients,
> > > > > > > only
> > > > > > > > clients can do that
> > > > > > > > }
> > > > > > >
> > > > > > >
> > > > > > > The problem with the environment type approach is that private
> > > > networks
> > > > > > of
> > > > > > > bare-metal environments usually impose restrictions similar to
> > > > virtual
> > > > > > > environments powered by Kubernetes. Thus,
> > > environmentType.VIRTUALIZED
> > > > > > > doesn't cover all the cases and I'm struggling to come up with
> a
> > > > > > universal
> > > > > > > alternative.
> > > > > > >
> > > > > > > -
> > > > > > > Denis
> > > > > > >
> > > > > > >
> > > > > > > On Thu, May 21, 2020 at 5:38 AM Ivan Bessonov <
> > > bessonov.ip@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hello Igniters,
> > > > > > > >
> > > > > > > > I'd like to discuss with you changes related to [1] and [2].
> > Both
> > > > > > issues
> > > > > > > > are mostly the same so
> > > > > > > > let's discuss the core idea.
> > > > > > > >
> > > > > > > > *Motivation.*
> > > > > > > >
> > > > > > > > There are certain environments that don't allow Ignite server
> > > nodes
> > > > > to
> > > > > > > open
> > > > > > > > TCP connections to
> > > > > > > > thick clients, e.g. K8S, AWS Lambda or Azure Functions. To
> > > operate
> > > > in
> > > > > > > such
> > > > > > > > environments, the
> > > > > > > > server needs a way to request a client to open an "inverse"
> > > > > > communication
> > > > > > > > connection to it.
> > > > > > > >
> > > > > > > > I've prepared a PR (still in progress) that introduces new
> > > > mechanism
> > > > > of
> > > > > > > > opening connection and
> > > > > > > > related configuration.
> > > > > > > >
> > > > > > > > *Main idea*
> > > > > > > >
> > > > > > > > This mechanism is called "communication via discovery" or
> > > "inverse
> > > > > > > > connection", it works as
> > > > > > > > follows:
> > > > > > > >  - server that needs to connect to "unreachable" thick client
> > > > sends a
> > > > > > > > specific Discovery message
> > > > > > > >    (inverse communication request) to that client;
> > > > > > > >  - client node upon receiving the request opens communication
> > > > > > connection
> > > > > > > to
> > > > > > > > that server;
> > > > > > > >  - server sees connection opened by client and proceeds with
> > its
> > > > task
> > > > > > > (that
> > > > > > > > required opening
> > > > > > > >    connection to the client).
> > > > > > > >
> > > > > > > > Working name for new configuration parameter for this feature
> > is
> > > > > > > > environmentType, it is an
> > > > > > > > enum with two values (again, working names): STANDALONE
> > (default)
> > > > and
> > > > > > > > VIRTUALIZED.
> > > > > > > > It is used as a hint to server to speed-up establishing of
> > > > > connections:
> > > > > > > > when server sees a client
> > > > > > > > with VIRTUALIZED environmentType it doesn't try to open
> > > connection
> > > > to
> > > > > > it
> > > > > > > > and sends inverse
> > > > > > > > communication request right away.
> > > > > > > > If environmentType is STANDALONE then server tries to open a
> > > > > connection
> > > > > > > in
> > > > > > > > a regular way
> > > > > > > > (iterating over all client addresses) and sends request only
> if
> > > all
> > > > > > > > attempts failed.
> > > > > > > >
> > > > > > > > There is a concern about naming of the configuration as it
> > > catches
> > > > > only
> > > > > > > one
> > > > > > > > use-case: when we
> > > > > > > > deal with some kind of virtualized environment like K8S.
> > > > > > > > There are other options I've encountered in private
> discussion:
> > > > > > > > - connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
> > > > > > > > - networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
> > > > > > > > - communicationViaDiscovery - ALWAYS / FALLBACK
> > > > > > > > - isReachableFromAllNodes (true/false flag)
> > > > > > > > - initiateConnectionsOnThisNode (true/false flag).
> > > > > > > >
> > > > > > > > *Limitations*
> > > > > > > >
> > > > > > > > The feature cannot be used along with pairedConnection
> setting
> > as
> > > > > this
> > > > > > > > setting implies
> > > > > > > > establishing connections in both directions. Also current
> > > > > > implementation
> > > > > > > > supports opening only
> > > > > > > > client-to-server connections. Other types of connections like
> > > > > > > > client-to-client or server-to-server
> > > > > > > > will be implemented in separate tickets.
> > > > > > > >
> > > > > > > > [1] https://issues.apache.org/jira/browse/IGNITE-12438
> > > > > > > > [2] https://issues.apache.org/jira/browse/IGNITE-13013
> > > > > > > >
> > > > > > > > --
> > > > > > > > Sincerely yours,
> > > > > > > > Ivan Bessonov
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
> --
> Sincerely yours,
> Ivan Bessonov
>

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Ivan Bessonov <be...@gmail.com>.
Hi,

I created new issue that describes CQ problem in more details: [1]
I'm fine with experimental status and new property naming.

[1] https://issues.apache.org/jira/browse/IGNITE-13156

вт, 16 июн. 2020 г. в 02:20, Valentin Kulichenko <
valentin.kulichenko@gmail.com>:

> Folks,
>
> Thanks for providing the detailed clarifications. Let's add the parameter,
> mark the new feature as experimental, and target for making it the default
> mode in Ignite 3.0.
>
> I still don't think we can come up with a naming that is really intuitive,
> but let's try to simplify it as much as possible. How about this:
>
> TcpCommunicationSpi#forceClientToServerConnections -- false by default,
> true if the new mode needs to be enabled.
>
> Let me know your thoughts.
>
> -Val
>
> On Wed, Jun 10, 2020 at 4:10 PM Denis Magda <dm...@apache.org> wrote:
>
> > Sergey,
> >
> > Thanks for the detailed explanation and for covering all corner cases.
> >
> > Considering the improvement's criticality, I would continue moving in the
> > initial direction and add that particular configuration property.
> > Potentially, we can put more effort throughout an Ignite 3.0 timeframe
> and
> > remove the property altogether. @Valentin Kulichenko
> > <vk...@gridgain.com>, could you please suggest any alternate
> naming?
> >
> > Btw, what are the specifics of the issue with continuous queries? It will
> > be ideal if we could release this new communication option in the GA
> state
> > in 2.9.
> >
> > -
> > Denis
> >
> >
> > On Wed, Jun 10, 2020 at 1:22 AM Sergey Chugunov <
> sergey.chugunov@gmail.com
> > >
> > wrote:
> >
> > > Denis, Val,
> > >
> > > Idea of prohibiting servers to open connections to clients and force
> > > clients to always open "inverse connections" to servers looks
> promising.
> > To
> > > be clear, by "inverse connections" I mean here that server needed to
> > > communicate with client requests client to open a connection back
> instead
> > > of opening connection by itself using addresses published by the
> client.
> > >
> > > If we apply the idea it will indeed allow us to simplify our
> > configuration
> > > (no need for new configuration property), another advantage is clients
> > > won't need to publish their addresses anymore (with one side note I'll
> > > cover at the end), it will also simplify our code.
> > >
> > > However applying it with current implementation of inverse connection
> > > request (when request goes across all ring) may bring significant delay
> > of
> > > opening first connection depending on cluster size and relative
> positions
> > > between server that needs to communicate with client (target server)
> and
> > > client's router node.
> > >
> > > It is possible to overcome this by sending inverse connection request
> not
> > > via discovery but directly to router server node via communication and
> > > convert to discovery message only on the router. We'll still have two
> > hops
> > > of communication request instead of one plus discovery worker on client
> > may
> > > be busy working on other stuff slowing down handling of connection
> > request.
> > > But it should be fine.
> > >
> > > However with this solution it is hard to implement failover of router
> > node:
> > > let me describe it in more details.
> > > In case of router node failure target server won't be able to determine
> > if
> > > client received inverse comm request successfully and (even worse)
> won't
> > be
> > > able to figure out new router for the client without waiting for
> > discovery
> > > event of the client reconnect.
> > > And this brings us to the following choise: we either wait for
> discovery
> > > event about client reconnect (this is deadlock-prone as current
> protocol
> > of
> > > CQ registration opens comm connection to client right from discovery
> > thread
> > > in some cases; if we wait for new discovery event from discovery
> thread,
> > it
> > > is a deadlock) or we fail opening the connection by timeout thus adding
> > new
> > > scenarios when opening connection may fail.
> > >
> > > Thus implementing communication model "clients connect to servers,
> > servers
> > > never connect to clients" efficiently requires more work on different
> > parts
> > > of our functionality and rigorous testing of readiness of our code for
> > more
> > > communication connection failures.
> > >
> > > So to me the least risky decision is not to delete new configuration
> but
> > > leave it with experimental status. If we find out that direct request
> > > (server -> router server -> target client) implementation works well
> and
> > > doesn't bring much complexity in failover scenarios we'll remove that
> > > configuration and prohibit servers to open connections to clients by
> > > default.
> > >
> > > Side note: there are rare but yet possible scenarios where client node
> > > needs to open communication connection to other client node. If we let
> > > clients not to publish their addresses these scenarios will stop
> working
> > > without additional logic like sending data through router node. As far
> > as I
> > > know client-client connectivity is involved in p2p class deployment
> > > scenarios, does anyone know about other cases?
> > >
> > > --
> > > Thanks,
> > > Sergey Chugunov
> > >
> > > On Wed, Jun 3, 2020 at 5:37 PM Denis Magda <dm...@apache.org> wrote:
> > >
> > > > Ivan,
> > > >
> > > > It feels like Val is driving us in the right direction. Is there any
> > > reason
> > > > for keeping the current logic when servers can open connections to
> > > clients?
> > > >
> > > > -
> > > > Denis
> > > >
> > > >
> > > > On Thu, May 21, 2020 at 4:48 PM Valentin Kulichenko <
> > > > valentin.kulichenko@gmail.com> wrote:
> > > >
> > > > > Ivan,
> > > > >
> > > > > Have you considered eliminating server to client connections
> > > altogether?
> > > > > Or, at the very least making the "client to server only" mode the
> > > default
> > > > > one?
> > > > >
> > > > > All the suggested names are confusing and not intuitive, and I
> doubt
> > we
> > > > > will be able to find a good one. A server initiating a TCP
> connection
> > > > with
> > > > > a client is confusing in the first place and creates a usability
> > issue.
> > > > We
> > > > > now want to solve it by introducing an additional configuration
> > > > > parameter, and therefore additional complexity. I don't think this
> is
> > > the
> > > > > right approach.
> > > > >
> > > > > What are the drawbacks of permanently switching to client-to-server
> > > > > connections? Is there any value provided by the server-to-client
> > > option?
> > > > >
> > > > > As for pair connections, I'm not sure I understand why there is a
> > > > > limitation. As far as I know, the idea behind this feature is that
> we
> > > > > maintain two connections between two nodes instead of one, so that
> > > every
> > > > > connection is used for communication in a single direction only.
> Why
> > > does
> > > > > it matter which node initiates the connection? Why can't one of the
> > > nodes
> > > > > (e.g., a client) initiate both connections, and then use them
> > > > accordingly?
> > > > > Correct me if I'm wrong, but I don't see why we can't do this.
> > > > >
> > > > > -Val
> > > > >
> > > > > On Thu, May 21, 2020 at 1:58 PM Denis Magda <dm...@apache.org>
> > wrote:
> > > > >
> > > > > > Ivan,
> > > > > >
> > > > > > Considering that the setting controls the way a communication SPI
> > > > > > connection is open I would add the new parameter to
> > CommunicationSpi
> > > > > > interface naming it as follows:
> > > > > >
> > > > > > >
> > > > > > > CommunicationSpi.connectionInitiationMode
> > > > > > > {
> > > > > > >     BIDIRECTIONAL, //both clients and servers initiate a
> > connection
> > > > > > > initiation procedure
> > > > > > >     CLIENTS_TO_SERVERS //servers cannot open a connection to
> > > clients,
> > > > > > only
> > > > > > > clients can do that
> > > > > > > }
> > > > > >
> > > > > >
> > > > > > The problem with the environment type approach is that private
> > > networks
> > > > > of
> > > > > > bare-metal environments usually impose restrictions similar to
> > > virtual
> > > > > > environments powered by Kubernetes. Thus,
> > environmentType.VIRTUALIZED
> > > > > > doesn't cover all the cases and I'm struggling to come up with a
> > > > > universal
> > > > > > alternative.
> > > > > >
> > > > > > -
> > > > > > Denis
> > > > > >
> > > > > >
> > > > > > On Thu, May 21, 2020 at 5:38 AM Ivan Bessonov <
> > bessonov.ip@gmail.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hello Igniters,
> > > > > > >
> > > > > > > I'd like to discuss with you changes related to [1] and [2].
> Both
> > > > > issues
> > > > > > > are mostly the same so
> > > > > > > let's discuss the core idea.
> > > > > > >
> > > > > > > *Motivation.*
> > > > > > >
> > > > > > > There are certain environments that don't allow Ignite server
> > nodes
> > > > to
> > > > > > open
> > > > > > > TCP connections to
> > > > > > > thick clients, e.g. K8S, AWS Lambda or Azure Functions. To
> > operate
> > > in
> > > > > > such
> > > > > > > environments, the
> > > > > > > server needs a way to request a client to open an "inverse"
> > > > > communication
> > > > > > > connection to it.
> > > > > > >
> > > > > > > I've prepared a PR (still in progress) that introduces new
> > > mechanism
> > > > of
> > > > > > > opening connection and
> > > > > > > related configuration.
> > > > > > >
> > > > > > > *Main idea*
> > > > > > >
> > > > > > > This mechanism is called "communication via discovery" or
> > "inverse
> > > > > > > connection", it works as
> > > > > > > follows:
> > > > > > >  - server that needs to connect to "unreachable" thick client
> > > sends a
> > > > > > > specific Discovery message
> > > > > > >    (inverse communication request) to that client;
> > > > > > >  - client node upon receiving the request opens communication
> > > > > connection
> > > > > > to
> > > > > > > that server;
> > > > > > >  - server sees connection opened by client and proceeds with
> its
> > > task
> > > > > > (that
> > > > > > > required opening
> > > > > > >    connection to the client).
> > > > > > >
> > > > > > > Working name for new configuration parameter for this feature
> is
> > > > > > > environmentType, it is an
> > > > > > > enum with two values (again, working names): STANDALONE
> (default)
> > > and
> > > > > > > VIRTUALIZED.
> > > > > > > It is used as a hint to server to speed-up establishing of
> > > > connections:
> > > > > > > when server sees a client
> > > > > > > with VIRTUALIZED environmentType it doesn't try to open
> > connection
> > > to
> > > > > it
> > > > > > > and sends inverse
> > > > > > > communication request right away.
> > > > > > > If environmentType is STANDALONE then server tries to open a
> > > > connection
> > > > > > in
> > > > > > > a regular way
> > > > > > > (iterating over all client addresses) and sends request only if
> > all
> > > > > > > attempts failed.
> > > > > > >
> > > > > > > There is a concern about naming of the configuration as it
> > catches
> > > > only
> > > > > > one
> > > > > > > use-case: when we
> > > > > > > deal with some kind of virtualized environment like K8S.
> > > > > > > There are other options I've encountered in private discussion:
> > > > > > > - connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
> > > > > > > - networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
> > > > > > > - communicationViaDiscovery - ALWAYS / FALLBACK
> > > > > > > - isReachableFromAllNodes (true/false flag)
> > > > > > > - initiateConnectionsOnThisNode (true/false flag).
> > > > > > >
> > > > > > > *Limitations*
> > > > > > >
> > > > > > > The feature cannot be used along with pairedConnection setting
> as
> > > > this
> > > > > > > setting implies
> > > > > > > establishing connections in both directions. Also current
> > > > > implementation
> > > > > > > supports opening only
> > > > > > > client-to-server connections. Other types of connections like
> > > > > > > client-to-client or server-to-server
> > > > > > > will be implemented in separate tickets.
> > > > > > >
> > > > > > > [1] https://issues.apache.org/jira/browse/IGNITE-12438
> > > > > > > [2] https://issues.apache.org/jira/browse/IGNITE-13013
> > > > > > >
> > > > > > > --
> > > > > > > Sincerely yours,
> > > > > > > Ivan Bessonov
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


-- 
Sincerely yours,
Ivan Bessonov

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Valentin Kulichenko <va...@gmail.com>.
Folks,

Thanks for providing the detailed clarifications. Let's add the parameter,
mark the new feature as experimental, and target for making it the default
mode in Ignite 3.0.

I still don't think we can come up with a naming that is really intuitive,
but let's try to simplify it as much as possible. How about this:

TcpCommunicationSpi#forceClientToServerConnections -- false by default,
true if the new mode needs to be enabled.

Let me know your thoughts.

-Val

On Wed, Jun 10, 2020 at 4:10 PM Denis Magda <dm...@apache.org> wrote:

> Sergey,
>
> Thanks for the detailed explanation and for covering all corner cases.
>
> Considering the improvement's criticality, I would continue moving in the
> initial direction and add that particular configuration property.
> Potentially, we can put more effort throughout an Ignite 3.0 timeframe and
> remove the property altogether. @Valentin Kulichenko
> <vk...@gridgain.com>, could you please suggest any alternate naming?
>
> Btw, what are the specifics of the issue with continuous queries? It will
> be ideal if we could release this new communication option in the GA state
> in 2.9.
>
> -
> Denis
>
>
> On Wed, Jun 10, 2020 at 1:22 AM Sergey Chugunov <sergey.chugunov@gmail.com
> >
> wrote:
>
> > Denis, Val,
> >
> > Idea of prohibiting servers to open connections to clients and force
> > clients to always open "inverse connections" to servers looks promising.
> To
> > be clear, by "inverse connections" I mean here that server needed to
> > communicate with client requests client to open a connection back instead
> > of opening connection by itself using addresses published by the client.
> >
> > If we apply the idea it will indeed allow us to simplify our
> configuration
> > (no need for new configuration property), another advantage is clients
> > won't need to publish their addresses anymore (with one side note I'll
> > cover at the end), it will also simplify our code.
> >
> > However applying it with current implementation of inverse connection
> > request (when request goes across all ring) may bring significant delay
> of
> > opening first connection depending on cluster size and relative positions
> > between server that needs to communicate with client (target server) and
> > client's router node.
> >
> > It is possible to overcome this by sending inverse connection request not
> > via discovery but directly to router server node via communication and
> > convert to discovery message only on the router. We'll still have two
> hops
> > of communication request instead of one plus discovery worker on client
> may
> > be busy working on other stuff slowing down handling of connection
> request.
> > But it should be fine.
> >
> > However with this solution it is hard to implement failover of router
> node:
> > let me describe it in more details.
> > In case of router node failure target server won't be able to determine
> if
> > client received inverse comm request successfully and (even worse) won't
> be
> > able to figure out new router for the client without waiting for
> discovery
> > event of the client reconnect.
> > And this brings us to the following choise: we either wait for discovery
> > event about client reconnect (this is deadlock-prone as current protocol
> of
> > CQ registration opens comm connection to client right from discovery
> thread
> > in some cases; if we wait for new discovery event from discovery thread,
> it
> > is a deadlock) or we fail opening the connection by timeout thus adding
> new
> > scenarios when opening connection may fail.
> >
> > Thus implementing communication model "clients connect to servers,
> servers
> > never connect to clients" efficiently requires more work on different
> parts
> > of our functionality and rigorous testing of readiness of our code for
> more
> > communication connection failures.
> >
> > So to me the least risky decision is not to delete new configuration but
> > leave it with experimental status. If we find out that direct request
> > (server -> router server -> target client) implementation works well and
> > doesn't bring much complexity in failover scenarios we'll remove that
> > configuration and prohibit servers to open connections to clients by
> > default.
> >
> > Side note: there are rare but yet possible scenarios where client node
> > needs to open communication connection to other client node. If we let
> > clients not to publish their addresses these scenarios will stop working
> > without additional logic like sending data through router node. As far
> as I
> > know client-client connectivity is involved in p2p class deployment
> > scenarios, does anyone know about other cases?
> >
> > --
> > Thanks,
> > Sergey Chugunov
> >
> > On Wed, Jun 3, 2020 at 5:37 PM Denis Magda <dm...@apache.org> wrote:
> >
> > > Ivan,
> > >
> > > It feels like Val is driving us in the right direction. Is there any
> > reason
> > > for keeping the current logic when servers can open connections to
> > clients?
> > >
> > > -
> > > Denis
> > >
> > >
> > > On Thu, May 21, 2020 at 4:48 PM Valentin Kulichenko <
> > > valentin.kulichenko@gmail.com> wrote:
> > >
> > > > Ivan,
> > > >
> > > > Have you considered eliminating server to client connections
> > altogether?
> > > > Or, at the very least making the "client to server only" mode the
> > default
> > > > one?
> > > >
> > > > All the suggested names are confusing and not intuitive, and I doubt
> we
> > > > will be able to find a good one. A server initiating a TCP connection
> > > with
> > > > a client is confusing in the first place and creates a usability
> issue.
> > > We
> > > > now want to solve it by introducing an additional configuration
> > > > parameter, and therefore additional complexity. I don't think this is
> > the
> > > > right approach.
> > > >
> > > > What are the drawbacks of permanently switching to client-to-server
> > > > connections? Is there any value provided by the server-to-client
> > option?
> > > >
> > > > As for pair connections, I'm not sure I understand why there is a
> > > > limitation. As far as I know, the idea behind this feature is that we
> > > > maintain two connections between two nodes instead of one, so that
> > every
> > > > connection is used for communication in a single direction only. Why
> > does
> > > > it matter which node initiates the connection? Why can't one of the
> > nodes
> > > > (e.g., a client) initiate both connections, and then use them
> > > accordingly?
> > > > Correct me if I'm wrong, but I don't see why we can't do this.
> > > >
> > > > -Val
> > > >
> > > > On Thu, May 21, 2020 at 1:58 PM Denis Magda <dm...@apache.org>
> wrote:
> > > >
> > > > > Ivan,
> > > > >
> > > > > Considering that the setting controls the way a communication SPI
> > > > > connection is open I would add the new parameter to
> CommunicationSpi
> > > > > interface naming it as follows:
> > > > >
> > > > > >
> > > > > > CommunicationSpi.connectionInitiationMode
> > > > > > {
> > > > > >     BIDIRECTIONAL, //both clients and servers initiate a
> connection
> > > > > > initiation procedure
> > > > > >     CLIENTS_TO_SERVERS //servers cannot open a connection to
> > clients,
> > > > > only
> > > > > > clients can do that
> > > > > > }
> > > > >
> > > > >
> > > > > The problem with the environment type approach is that private
> > networks
> > > > of
> > > > > bare-metal environments usually impose restrictions similar to
> > virtual
> > > > > environments powered by Kubernetes. Thus,
> environmentType.VIRTUALIZED
> > > > > doesn't cover all the cases and I'm struggling to come up with a
> > > > universal
> > > > > alternative.
> > > > >
> > > > > -
> > > > > Denis
> > > > >
> > > > >
> > > > > On Thu, May 21, 2020 at 5:38 AM Ivan Bessonov <
> bessonov.ip@gmail.com
> > >
> > > > > wrote:
> > > > >
> > > > > > Hello Igniters,
> > > > > >
> > > > > > I'd like to discuss with you changes related to [1] and [2]. Both
> > > > issues
> > > > > > are mostly the same so
> > > > > > let's discuss the core idea.
> > > > > >
> > > > > > *Motivation.*
> > > > > >
> > > > > > There are certain environments that don't allow Ignite server
> nodes
> > > to
> > > > > open
> > > > > > TCP connections to
> > > > > > thick clients, e.g. K8S, AWS Lambda or Azure Functions. To
> operate
> > in
> > > > > such
> > > > > > environments, the
> > > > > > server needs a way to request a client to open an "inverse"
> > > > communication
> > > > > > connection to it.
> > > > > >
> > > > > > I've prepared a PR (still in progress) that introduces new
> > mechanism
> > > of
> > > > > > opening connection and
> > > > > > related configuration.
> > > > > >
> > > > > > *Main idea*
> > > > > >
> > > > > > This mechanism is called "communication via discovery" or
> "inverse
> > > > > > connection", it works as
> > > > > > follows:
> > > > > >  - server that needs to connect to "unreachable" thick client
> > sends a
> > > > > > specific Discovery message
> > > > > >    (inverse communication request) to that client;
> > > > > >  - client node upon receiving the request opens communication
> > > > connection
> > > > > to
> > > > > > that server;
> > > > > >  - server sees connection opened by client and proceeds with its
> > task
> > > > > (that
> > > > > > required opening
> > > > > >    connection to the client).
> > > > > >
> > > > > > Working name for new configuration parameter for this feature is
> > > > > > environmentType, it is an
> > > > > > enum with two values (again, working names): STANDALONE (default)
> > and
> > > > > > VIRTUALIZED.
> > > > > > It is used as a hint to server to speed-up establishing of
> > > connections:
> > > > > > when server sees a client
> > > > > > with VIRTUALIZED environmentType it doesn't try to open
> connection
> > to
> > > > it
> > > > > > and sends inverse
> > > > > > communication request right away.
> > > > > > If environmentType is STANDALONE then server tries to open a
> > > connection
> > > > > in
> > > > > > a regular way
> > > > > > (iterating over all client addresses) and sends request only if
> all
> > > > > > attempts failed.
> > > > > >
> > > > > > There is a concern about naming of the configuration as it
> catches
> > > only
> > > > > one
> > > > > > use-case: when we
> > > > > > deal with some kind of virtualized environment like K8S.
> > > > > > There are other options I've encountered in private discussion:
> > > > > > - connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
> > > > > > - networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
> > > > > > - communicationViaDiscovery - ALWAYS / FALLBACK
> > > > > > - isReachableFromAllNodes (true/false flag)
> > > > > > - initiateConnectionsOnThisNode (true/false flag).
> > > > > >
> > > > > > *Limitations*
> > > > > >
> > > > > > The feature cannot be used along with pairedConnection setting as
> > > this
> > > > > > setting implies
> > > > > > establishing connections in both directions. Also current
> > > > implementation
> > > > > > supports opening only
> > > > > > client-to-server connections. Other types of connections like
> > > > > > client-to-client or server-to-server
> > > > > > will be implemented in separate tickets.
> > > > > >
> > > > > > [1] https://issues.apache.org/jira/browse/IGNITE-12438
> > > > > > [2] https://issues.apache.org/jira/browse/IGNITE-13013
> > > > > >
> > > > > > --
> > > > > > Sincerely yours,
> > > > > > Ivan Bessonov
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Denis Magda <dm...@apache.org>.
Sergey,

Thanks for the detailed explanation and for covering all corner cases.

Considering the improvement's criticality, I would continue moving in the
initial direction and add that particular configuration property.
Potentially, we can put more effort throughout an Ignite 3.0 timeframe and
remove the property altogether. @Valentin Kulichenko
<vk...@gridgain.com>, could you please suggest any alternate naming?

Btw, what are the specifics of the issue with continuous queries? It will
be ideal if we could release this new communication option in the GA state
in 2.9.

-
Denis


On Wed, Jun 10, 2020 at 1:22 AM Sergey Chugunov <se...@gmail.com>
wrote:

> Denis, Val,
>
> Idea of prohibiting servers to open connections to clients and force
> clients to always open "inverse connections" to servers looks promising. To
> be clear, by "inverse connections" I mean here that server needed to
> communicate with client requests client to open a connection back instead
> of opening connection by itself using addresses published by the client.
>
> If we apply the idea it will indeed allow us to simplify our configuration
> (no need for new configuration property), another advantage is clients
> won't need to publish their addresses anymore (with one side note I'll
> cover at the end), it will also simplify our code.
>
> However applying it with current implementation of inverse connection
> request (when request goes across all ring) may bring significant delay of
> opening first connection depending on cluster size and relative positions
> between server that needs to communicate with client (target server) and
> client's router node.
>
> It is possible to overcome this by sending inverse connection request not
> via discovery but directly to router server node via communication and
> convert to discovery message only on the router. We'll still have two hops
> of communication request instead of one plus discovery worker on client may
> be busy working on other stuff slowing down handling of connection request.
> But it should be fine.
>
> However with this solution it is hard to implement failover of router node:
> let me describe it in more details.
> In case of router node failure target server won't be able to determine if
> client received inverse comm request successfully and (even worse) won't be
> able to figure out new router for the client without waiting for discovery
> event of the client reconnect.
> And this brings us to the following choise: we either wait for discovery
> event about client reconnect (this is deadlock-prone as current protocol of
> CQ registration opens comm connection to client right from discovery thread
> in some cases; if we wait for new discovery event from discovery thread, it
> is a deadlock) or we fail opening the connection by timeout thus adding new
> scenarios when opening connection may fail.
>
> Thus implementing communication model "clients connect to servers, servers
> never connect to clients" efficiently requires more work on different parts
> of our functionality and rigorous testing of readiness of our code for more
> communication connection failures.
>
> So to me the least risky decision is not to delete new configuration but
> leave it with experimental status. If we find out that direct request
> (server -> router server -> target client) implementation works well and
> doesn't bring much complexity in failover scenarios we'll remove that
> configuration and prohibit servers to open connections to clients by
> default.
>
> Side note: there are rare but yet possible scenarios where client node
> needs to open communication connection to other client node. If we let
> clients not to publish their addresses these scenarios will stop working
> without additional logic like sending data through router node. As far as I
> know client-client connectivity is involved in p2p class deployment
> scenarios, does anyone know about other cases?
>
> --
> Thanks,
> Sergey Chugunov
>
> On Wed, Jun 3, 2020 at 5:37 PM Denis Magda <dm...@apache.org> wrote:
>
> > Ivan,
> >
> > It feels like Val is driving us in the right direction. Is there any
> reason
> > for keeping the current logic when servers can open connections to
> clients?
> >
> > -
> > Denis
> >
> >
> > On Thu, May 21, 2020 at 4:48 PM Valentin Kulichenko <
> > valentin.kulichenko@gmail.com> wrote:
> >
> > > Ivan,
> > >
> > > Have you considered eliminating server to client connections
> altogether?
> > > Or, at the very least making the "client to server only" mode the
> default
> > > one?
> > >
> > > All the suggested names are confusing and not intuitive, and I doubt we
> > > will be able to find a good one. A server initiating a TCP connection
> > with
> > > a client is confusing in the first place and creates a usability issue.
> > We
> > > now want to solve it by introducing an additional configuration
> > > parameter, and therefore additional complexity. I don't think this is
> the
> > > right approach.
> > >
> > > What are the drawbacks of permanently switching to client-to-server
> > > connections? Is there any value provided by the server-to-client
> option?
> > >
> > > As for pair connections, I'm not sure I understand why there is a
> > > limitation. As far as I know, the idea behind this feature is that we
> > > maintain two connections between two nodes instead of one, so that
> every
> > > connection is used for communication in a single direction only. Why
> does
> > > it matter which node initiates the connection? Why can't one of the
> nodes
> > > (e.g., a client) initiate both connections, and then use them
> > accordingly?
> > > Correct me if I'm wrong, but I don't see why we can't do this.
> > >
> > > -Val
> > >
> > > On Thu, May 21, 2020 at 1:58 PM Denis Magda <dm...@apache.org> wrote:
> > >
> > > > Ivan,
> > > >
> > > > Considering that the setting controls the way a communication SPI
> > > > connection is open I would add the new parameter to CommunicationSpi
> > > > interface naming it as follows:
> > > >
> > > > >
> > > > > CommunicationSpi.connectionInitiationMode
> > > > > {
> > > > >     BIDIRECTIONAL, //both clients and servers initiate a connection
> > > > > initiation procedure
> > > > >     CLIENTS_TO_SERVERS //servers cannot open a connection to
> clients,
> > > > only
> > > > > clients can do that
> > > > > }
> > > >
> > > >
> > > > The problem with the environment type approach is that private
> networks
> > > of
> > > > bare-metal environments usually impose restrictions similar to
> virtual
> > > > environments powered by Kubernetes. Thus, environmentType.VIRTUALIZED
> > > > doesn't cover all the cases and I'm struggling to come up with a
> > > universal
> > > > alternative.
> > > >
> > > > -
> > > > Denis
> > > >
> > > >
> > > > On Thu, May 21, 2020 at 5:38 AM Ivan Bessonov <bessonov.ip@gmail.com
> >
> > > > wrote:
> > > >
> > > > > Hello Igniters,
> > > > >
> > > > > I'd like to discuss with you changes related to [1] and [2]. Both
> > > issues
> > > > > are mostly the same so
> > > > > let's discuss the core idea.
> > > > >
> > > > > *Motivation.*
> > > > >
> > > > > There are certain environments that don't allow Ignite server nodes
> > to
> > > > open
> > > > > TCP connections to
> > > > > thick clients, e.g. K8S, AWS Lambda or Azure Functions. To operate
> in
> > > > such
> > > > > environments, the
> > > > > server needs a way to request a client to open an "inverse"
> > > communication
> > > > > connection to it.
> > > > >
> > > > > I've prepared a PR (still in progress) that introduces new
> mechanism
> > of
> > > > > opening connection and
> > > > > related configuration.
> > > > >
> > > > > *Main idea*
> > > > >
> > > > > This mechanism is called "communication via discovery" or "inverse
> > > > > connection", it works as
> > > > > follows:
> > > > >  - server that needs to connect to "unreachable" thick client
> sends a
> > > > > specific Discovery message
> > > > >    (inverse communication request) to that client;
> > > > >  - client node upon receiving the request opens communication
> > > connection
> > > > to
> > > > > that server;
> > > > >  - server sees connection opened by client and proceeds with its
> task
> > > > (that
> > > > > required opening
> > > > >    connection to the client).
> > > > >
> > > > > Working name for new configuration parameter for this feature is
> > > > > environmentType, it is an
> > > > > enum with two values (again, working names): STANDALONE (default)
> and
> > > > > VIRTUALIZED.
> > > > > It is used as a hint to server to speed-up establishing of
> > connections:
> > > > > when server sees a client
> > > > > with VIRTUALIZED environmentType it doesn't try to open connection
> to
> > > it
> > > > > and sends inverse
> > > > > communication request right away.
> > > > > If environmentType is STANDALONE then server tries to open a
> > connection
> > > > in
> > > > > a regular way
> > > > > (iterating over all client addresses) and sends request only if all
> > > > > attempts failed.
> > > > >
> > > > > There is a concern about naming of the configuration as it catches
> > only
> > > > one
> > > > > use-case: when we
> > > > > deal with some kind of virtualized environment like K8S.
> > > > > There are other options I've encountered in private discussion:
> > > > > - connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
> > > > > - networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
> > > > > - communicationViaDiscovery - ALWAYS / FALLBACK
> > > > > - isReachableFromAllNodes (true/false flag)
> > > > > - initiateConnectionsOnThisNode (true/false flag).
> > > > >
> > > > > *Limitations*
> > > > >
> > > > > The feature cannot be used along with pairedConnection setting as
> > this
> > > > > setting implies
> > > > > establishing connections in both directions. Also current
> > > implementation
> > > > > supports opening only
> > > > > client-to-server connections. Other types of connections like
> > > > > client-to-client or server-to-server
> > > > > will be implemented in separate tickets.
> > > > >
> > > > > [1] https://issues.apache.org/jira/browse/IGNITE-12438
> > > > > [2] https://issues.apache.org/jira/browse/IGNITE-13013
> > > > >
> > > > > --
> > > > > Sincerely yours,
> > > > > Ivan Bessonov
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Sergey Chugunov <se...@gmail.com>.
Denis, Val,

Idea of prohibiting servers to open connections to clients and force
clients to always open "inverse connections" to servers looks promising. To
be clear, by "inverse connections" I mean here that server needed to
communicate with client requests client to open a connection back instead
of opening connection by itself using addresses published by the client.

If we apply the idea it will indeed allow us to simplify our configuration
(no need for new configuration property), another advantage is clients
won't need to publish their addresses anymore (with one side note I'll
cover at the end), it will also simplify our code.

However applying it with current implementation of inverse connection
request (when request goes across all ring) may bring significant delay of
opening first connection depending on cluster size and relative positions
between server that needs to communicate with client (target server) and
client's router node.

It is possible to overcome this by sending inverse connection request not
via discovery but directly to router server node via communication and
convert to discovery message only on the router. We'll still have two hops
of communication request instead of one plus discovery worker on client may
be busy working on other stuff slowing down handling of connection request.
But it should be fine.

However with this solution it is hard to implement failover of router node:
let me describe it in more details.
In case of router node failure target server won't be able to determine if
client received inverse comm request successfully and (even worse) won't be
able to figure out new router for the client without waiting for discovery
event of the client reconnect.
And this brings us to the following choise: we either wait for discovery
event about client reconnect (this is deadlock-prone as current protocol of
CQ registration opens comm connection to client right from discovery thread
in some cases; if we wait for new discovery event from discovery thread, it
is a deadlock) or we fail opening the connection by timeout thus adding new
scenarios when opening connection may fail.

Thus implementing communication model "clients connect to servers, servers
never connect to clients" efficiently requires more work on different parts
of our functionality and rigorous testing of readiness of our code for more
communication connection failures.

So to me the least risky decision is not to delete new configuration but
leave it with experimental status. If we find out that direct request
(server -> router server -> target client) implementation works well and
doesn't bring much complexity in failover scenarios we'll remove that
configuration and prohibit servers to open connections to clients by
default.

Side note: there are rare but yet possible scenarios where client node
needs to open communication connection to other client node. If we let
clients not to publish their addresses these scenarios will stop working
without additional logic like sending data through router node. As far as I
know client-client connectivity is involved in p2p class deployment
scenarios, does anyone know about other cases?

--
Thanks,
Sergey Chugunov

On Wed, Jun 3, 2020 at 5:37 PM Denis Magda <dm...@apache.org> wrote:

> Ivan,
>
> It feels like Val is driving us in the right direction. Is there any reason
> for keeping the current logic when servers can open connections to clients?
>
> -
> Denis
>
>
> On Thu, May 21, 2020 at 4:48 PM Valentin Kulichenko <
> valentin.kulichenko@gmail.com> wrote:
>
> > Ivan,
> >
> > Have you considered eliminating server to client connections altogether?
> > Or, at the very least making the "client to server only" mode the default
> > one?
> >
> > All the suggested names are confusing and not intuitive, and I doubt we
> > will be able to find a good one. A server initiating a TCP connection
> with
> > a client is confusing in the first place and creates a usability issue.
> We
> > now want to solve it by introducing an additional configuration
> > parameter, and therefore additional complexity. I don't think this is the
> > right approach.
> >
> > What are the drawbacks of permanently switching to client-to-server
> > connections? Is there any value provided by the server-to-client option?
> >
> > As for pair connections, I'm not sure I understand why there is a
> > limitation. As far as I know, the idea behind this feature is that we
> > maintain two connections between two nodes instead of one, so that every
> > connection is used for communication in a single direction only. Why does
> > it matter which node initiates the connection? Why can't one of the nodes
> > (e.g., a client) initiate both connections, and then use them
> accordingly?
> > Correct me if I'm wrong, but I don't see why we can't do this.
> >
> > -Val
> >
> > On Thu, May 21, 2020 at 1:58 PM Denis Magda <dm...@apache.org> wrote:
> >
> > > Ivan,
> > >
> > > Considering that the setting controls the way a communication SPI
> > > connection is open I would add the new parameter to CommunicationSpi
> > > interface naming it as follows:
> > >
> > > >
> > > > CommunicationSpi.connectionInitiationMode
> > > > {
> > > >     BIDIRECTIONAL, //both clients and servers initiate a connection
> > > > initiation procedure
> > > >     CLIENTS_TO_SERVERS //servers cannot open a connection to clients,
> > > only
> > > > clients can do that
> > > > }
> > >
> > >
> > > The problem with the environment type approach is that private networks
> > of
> > > bare-metal environments usually impose restrictions similar to virtual
> > > environments powered by Kubernetes. Thus, environmentType.VIRTUALIZED
> > > doesn't cover all the cases and I'm struggling to come up with a
> > universal
> > > alternative.
> > >
> > > -
> > > Denis
> > >
> > >
> > > On Thu, May 21, 2020 at 5:38 AM Ivan Bessonov <be...@gmail.com>
> > > wrote:
> > >
> > > > Hello Igniters,
> > > >
> > > > I'd like to discuss with you changes related to [1] and [2]. Both
> > issues
> > > > are mostly the same so
> > > > let's discuss the core idea.
> > > >
> > > > *Motivation.*
> > > >
> > > > There are certain environments that don't allow Ignite server nodes
> to
> > > open
> > > > TCP connections to
> > > > thick clients, e.g. K8S, AWS Lambda or Azure Functions. To operate in
> > > such
> > > > environments, the
> > > > server needs a way to request a client to open an "inverse"
> > communication
> > > > connection to it.
> > > >
> > > > I've prepared a PR (still in progress) that introduces new mechanism
> of
> > > > opening connection and
> > > > related configuration.
> > > >
> > > > *Main idea*
> > > >
> > > > This mechanism is called "communication via discovery" or "inverse
> > > > connection", it works as
> > > > follows:
> > > >  - server that needs to connect to "unreachable" thick client sends a
> > > > specific Discovery message
> > > >    (inverse communication request) to that client;
> > > >  - client node upon receiving the request opens communication
> > connection
> > > to
> > > > that server;
> > > >  - server sees connection opened by client and proceeds with its task
> > > (that
> > > > required opening
> > > >    connection to the client).
> > > >
> > > > Working name for new configuration parameter for this feature is
> > > > environmentType, it is an
> > > > enum with two values (again, working names): STANDALONE (default) and
> > > > VIRTUALIZED.
> > > > It is used as a hint to server to speed-up establishing of
> connections:
> > > > when server sees a client
> > > > with VIRTUALIZED environmentType it doesn't try to open connection to
> > it
> > > > and sends inverse
> > > > communication request right away.
> > > > If environmentType is STANDALONE then server tries to open a
> connection
> > > in
> > > > a regular way
> > > > (iterating over all client addresses) and sends request only if all
> > > > attempts failed.
> > > >
> > > > There is a concern about naming of the configuration as it catches
> only
> > > one
> > > > use-case: when we
> > > > deal with some kind of virtualized environment like K8S.
> > > > There are other options I've encountered in private discussion:
> > > > - connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
> > > > - networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
> > > > - communicationViaDiscovery - ALWAYS / FALLBACK
> > > > - isReachableFromAllNodes (true/false flag)
> > > > - initiateConnectionsOnThisNode (true/false flag).
> > > >
> > > > *Limitations*
> > > >
> > > > The feature cannot be used along with pairedConnection setting as
> this
> > > > setting implies
> > > > establishing connections in both directions. Also current
> > implementation
> > > > supports opening only
> > > > client-to-server connections. Other types of connections like
> > > > client-to-client or server-to-server
> > > > will be implemented in separate tickets.
> > > >
> > > > [1] https://issues.apache.org/jira/browse/IGNITE-12438
> > > > [2] https://issues.apache.org/jira/browse/IGNITE-13013
> > > >
> > > > --
> > > > Sincerely yours,
> > > > Ivan Bessonov
> > > >
> > >
> >
>

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Denis Magda <dm...@apache.org>.
Ivan,

It feels like Val is driving us in the right direction. Is there any reason
for keeping the current logic when servers can open connections to clients?

-
Denis


On Thu, May 21, 2020 at 4:48 PM Valentin Kulichenko <
valentin.kulichenko@gmail.com> wrote:

> Ivan,
>
> Have you considered eliminating server to client connections altogether?
> Or, at the very least making the "client to server only" mode the default
> one?
>
> All the suggested names are confusing and not intuitive, and I doubt we
> will be able to find a good one. A server initiating a TCP connection with
> a client is confusing in the first place and creates a usability issue. We
> now want to solve it by introducing an additional configuration
> parameter, and therefore additional complexity. I don't think this is the
> right approach.
>
> What are the drawbacks of permanently switching to client-to-server
> connections? Is there any value provided by the server-to-client option?
>
> As for pair connections, I'm not sure I understand why there is a
> limitation. As far as I know, the idea behind this feature is that we
> maintain two connections between two nodes instead of one, so that every
> connection is used for communication in a single direction only. Why does
> it matter which node initiates the connection? Why can't one of the nodes
> (e.g., a client) initiate both connections, and then use them accordingly?
> Correct me if I'm wrong, but I don't see why we can't do this.
>
> -Val
>
> On Thu, May 21, 2020 at 1:58 PM Denis Magda <dm...@apache.org> wrote:
>
> > Ivan,
> >
> > Considering that the setting controls the way a communication SPI
> > connection is open I would add the new parameter to CommunicationSpi
> > interface naming it as follows:
> >
> > >
> > > CommunicationSpi.connectionInitiationMode
> > > {
> > >     BIDIRECTIONAL, //both clients and servers initiate a connection
> > > initiation procedure
> > >     CLIENTS_TO_SERVERS //servers cannot open a connection to clients,
> > only
> > > clients can do that
> > > }
> >
> >
> > The problem with the environment type approach is that private networks
> of
> > bare-metal environments usually impose restrictions similar to virtual
> > environments powered by Kubernetes. Thus, environmentType.VIRTUALIZED
> > doesn't cover all the cases and I'm struggling to come up with a
> universal
> > alternative.
> >
> > -
> > Denis
> >
> >
> > On Thu, May 21, 2020 at 5:38 AM Ivan Bessonov <be...@gmail.com>
> > wrote:
> >
> > > Hello Igniters,
> > >
> > > I'd like to discuss with you changes related to [1] and [2]. Both
> issues
> > > are mostly the same so
> > > let's discuss the core idea.
> > >
> > > *Motivation.*
> > >
> > > There are certain environments that don't allow Ignite server nodes to
> > open
> > > TCP connections to
> > > thick clients, e.g. K8S, AWS Lambda or Azure Functions. To operate in
> > such
> > > environments, the
> > > server needs a way to request a client to open an "inverse"
> communication
> > > connection to it.
> > >
> > > I've prepared a PR (still in progress) that introduces new mechanism of
> > > opening connection and
> > > related configuration.
> > >
> > > *Main idea*
> > >
> > > This mechanism is called "communication via discovery" or "inverse
> > > connection", it works as
> > > follows:
> > >  - server that needs to connect to "unreachable" thick client sends a
> > > specific Discovery message
> > >    (inverse communication request) to that client;
> > >  - client node upon receiving the request opens communication
> connection
> > to
> > > that server;
> > >  - server sees connection opened by client and proceeds with its task
> > (that
> > > required opening
> > >    connection to the client).
> > >
> > > Working name for new configuration parameter for this feature is
> > > environmentType, it is an
> > > enum with two values (again, working names): STANDALONE (default) and
> > > VIRTUALIZED.
> > > It is used as a hint to server to speed-up establishing of connections:
> > > when server sees a client
> > > with VIRTUALIZED environmentType it doesn't try to open connection to
> it
> > > and sends inverse
> > > communication request right away.
> > > If environmentType is STANDALONE then server tries to open a connection
> > in
> > > a regular way
> > > (iterating over all client addresses) and sends request only if all
> > > attempts failed.
> > >
> > > There is a concern about naming of the configuration as it catches only
> > one
> > > use-case: when we
> > > deal with some kind of virtualized environment like K8S.
> > > There are other options I've encountered in private discussion:
> > > - connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
> > > - networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
> > > - communicationViaDiscovery - ALWAYS / FALLBACK
> > > - isReachableFromAllNodes (true/false flag)
> > > - initiateConnectionsOnThisNode (true/false flag).
> > >
> > > *Limitations*
> > >
> > > The feature cannot be used along with pairedConnection setting as this
> > > setting implies
> > > establishing connections in both directions. Also current
> implementation
> > > supports opening only
> > > client-to-server connections. Other types of connections like
> > > client-to-client or server-to-server
> > > will be implemented in separate tickets.
> > >
> > > [1] https://issues.apache.org/jira/browse/IGNITE-12438
> > > [2] https://issues.apache.org/jira/browse/IGNITE-13013
> > >
> > > --
> > > Sincerely yours,
> > > Ivan Bessonov
> > >
> >
>

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Dmitriy Pavlov <dp...@apache.org>.
Sorry for me being obnoxious here, but still. Mentioning ticket numbers
usually does not make a lot of sense for someone, who is outside of context
of popular/trendy tickets.

At the same time, usually contributors scan emails in the list using
subject. They decide what should go to archive and what should they read.
So, in future it may worth to replace all numbers with tickets' actual
topic/summary. People are still humans and they may be too lazy to open
tickets to decide if they need to read this topic or not.

пт, 22 мая 2020 г. в 02:48, Valentin Kulichenko <
valentin.kulichenko@gmail.com>:

> Ivan,
>
> Have you considered eliminating server to client connections altogether?
> Or, at the very least making the "client to server only" mode the default
> one?
>
> All the suggested names are confusing and not intuitive, and I doubt we
> will be able to find a good one. A server initiating a TCP connection with
> a client is confusing in the first place and creates a usability issue. We
> now want to solve it by introducing an additional configuration
> parameter, and therefore additional complexity. I don't think this is the
> right approach.
>
> What are the drawbacks of permanently switching to client-to-server
> connections? Is there any value provided by the server-to-client option?
>
> As for pair connections, I'm not sure I understand why there is a
> limitation. As far as I know, the idea behind this feature is that we
> maintain two connections between two nodes instead of one, so that every
> connection is used for communication in a single direction only. Why does
> it matter which node initiates the connection? Why can't one of the nodes
> (e.g., a client) initiate both connections, and then use them accordingly?
> Correct me if I'm wrong, but I don't see why we can't do this.
>
> -Val
>
> On Thu, May 21, 2020 at 1:58 PM Denis Magda <dm...@apache.org> wrote:
>
> > Ivan,
> >
> > Considering that the setting controls the way a communication SPI
> > connection is open I would add the new parameter to CommunicationSpi
> > interface naming it as follows:
> >
> > >
> > > CommunicationSpi.connectionInitiationMode
> > > {
> > >     BIDIRECTIONAL, //both clients and servers initiate a connection
> > > initiation procedure
> > >     CLIENTS_TO_SERVERS //servers cannot open a connection to clients,
> > only
> > > clients can do that
> > > }
> >
> >
> > The problem with the environment type approach is that private networks
> of
> > bare-metal environments usually impose restrictions similar to virtual
> > environments powered by Kubernetes. Thus, environmentType.VIRTUALIZED
> > doesn't cover all the cases and I'm struggling to come up with a
> universal
> > alternative.
> >
> > -
> > Denis
> >
> >
> > On Thu, May 21, 2020 at 5:38 AM Ivan Bessonov <be...@gmail.com>
> > wrote:
> >
> > > Hello Igniters,
> > >
> > > I'd like to discuss with you changes related to [1] and [2]. Both
> issues
> > > are mostly the same so
> > > let's discuss the core idea.
> > >
> > > *Motivation.*
> > >
> > > There are certain environments that don't allow Ignite server nodes to
> > open
> > > TCP connections to
> > > thick clients, e.g. K8S, AWS Lambda or Azure Functions. To operate in
> > such
> > > environments, the
> > > server needs a way to request a client to open an "inverse"
> communication
> > > connection to it.
> > >
> > > I've prepared a PR (still in progress) that introduces new mechanism of
> > > opening connection and
> > > related configuration.
> > >
> > > *Main idea*
> > >
> > > This mechanism is called "communication via discovery" or "inverse
> > > connection", it works as
> > > follows:
> > >  - server that needs to connect to "unreachable" thick client sends a
> > > specific Discovery message
> > >    (inverse communication request) to that client;
> > >  - client node upon receiving the request opens communication
> connection
> > to
> > > that server;
> > >  - server sees connection opened by client and proceeds with its task
> > (that
> > > required opening
> > >    connection to the client).
> > >
> > > Working name for new configuration parameter for this feature is
> > > environmentType, it is an
> > > enum with two values (again, working names): STANDALONE (default) and
> > > VIRTUALIZED.
> > > It is used as a hint to server to speed-up establishing of connections:
> > > when server sees a client
> > > with VIRTUALIZED environmentType it doesn't try to open connection to
> it
> > > and sends inverse
> > > communication request right away.
> > > If environmentType is STANDALONE then server tries to open a connection
> > in
> > > a regular way
> > > (iterating over all client addresses) and sends request only if all
> > > attempts failed.
> > >
> > > There is a concern about naming of the configuration as it catches only
> > one
> > > use-case: when we
> > > deal with some kind of virtualized environment like K8S.
> > > There are other options I've encountered in private discussion:
> > > - connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
> > > - networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
> > > - communicationViaDiscovery - ALWAYS / FALLBACK
> > > - isReachableFromAllNodes (true/false flag)
> > > - initiateConnectionsOnThisNode (true/false flag).
> > >
> > > *Limitations*
> > >
> > > The feature cannot be used along with pairedConnection setting as this
> > > setting implies
> > > establishing connections in both directions. Also current
> implementation
> > > supports opening only
> > > client-to-server connections. Other types of connections like
> > > client-to-client or server-to-server
> > > will be implemented in separate tickets.
> > >
> > > [1] https://issues.apache.org/jira/browse/IGNITE-12438
> > > [2] https://issues.apache.org/jira/browse/IGNITE-13013
> > >
> > > --
> > > Sincerely yours,
> > > Ivan Bessonov
> > >
> >
>

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Valentin Kulichenko <va...@gmail.com>.
Ivan,

Have you considered eliminating server to client connections altogether?
Or, at the very least making the "client to server only" mode the default
one?

All the suggested names are confusing and not intuitive, and I doubt we
will be able to find a good one. A server initiating a TCP connection with
a client is confusing in the first place and creates a usability issue. We
now want to solve it by introducing an additional configuration
parameter, and therefore additional complexity. I don't think this is the
right approach.

What are the drawbacks of permanently switching to client-to-server
connections? Is there any value provided by the server-to-client option?

As for pair connections, I'm not sure I understand why there is a
limitation. As far as I know, the idea behind this feature is that we
maintain two connections between two nodes instead of one, so that every
connection is used for communication in a single direction only. Why does
it matter which node initiates the connection? Why can't one of the nodes
(e.g., a client) initiate both connections, and then use them accordingly?
Correct me if I'm wrong, but I don't see why we can't do this.

-Val

On Thu, May 21, 2020 at 1:58 PM Denis Magda <dm...@apache.org> wrote:

> Ivan,
>
> Considering that the setting controls the way a communication SPI
> connection is open I would add the new parameter to CommunicationSpi
> interface naming it as follows:
>
> >
> > CommunicationSpi.connectionInitiationMode
> > {
> >     BIDIRECTIONAL, //both clients and servers initiate a connection
> > initiation procedure
> >     CLIENTS_TO_SERVERS //servers cannot open a connection to clients,
> only
> > clients can do that
> > }
>
>
> The problem with the environment type approach is that private networks of
> bare-metal environments usually impose restrictions similar to virtual
> environments powered by Kubernetes. Thus, environmentType.VIRTUALIZED
> doesn't cover all the cases and I'm struggling to come up with a universal
> alternative.
>
> -
> Denis
>
>
> On Thu, May 21, 2020 at 5:38 AM Ivan Bessonov <be...@gmail.com>
> wrote:
>
> > Hello Igniters,
> >
> > I'd like to discuss with you changes related to [1] and [2]. Both issues
> > are mostly the same so
> > let's discuss the core idea.
> >
> > *Motivation.*
> >
> > There are certain environments that don't allow Ignite server nodes to
> open
> > TCP connections to
> > thick clients, e.g. K8S, AWS Lambda or Azure Functions. To operate in
> such
> > environments, the
> > server needs a way to request a client to open an "inverse" communication
> > connection to it.
> >
> > I've prepared a PR (still in progress) that introduces new mechanism of
> > opening connection and
> > related configuration.
> >
> > *Main idea*
> >
> > This mechanism is called "communication via discovery" or "inverse
> > connection", it works as
> > follows:
> >  - server that needs to connect to "unreachable" thick client sends a
> > specific Discovery message
> >    (inverse communication request) to that client;
> >  - client node upon receiving the request opens communication connection
> to
> > that server;
> >  - server sees connection opened by client and proceeds with its task
> (that
> > required opening
> >    connection to the client).
> >
> > Working name for new configuration parameter for this feature is
> > environmentType, it is an
> > enum with two values (again, working names): STANDALONE (default) and
> > VIRTUALIZED.
> > It is used as a hint to server to speed-up establishing of connections:
> > when server sees a client
> > with VIRTUALIZED environmentType it doesn't try to open connection to it
> > and sends inverse
> > communication request right away.
> > If environmentType is STANDALONE then server tries to open a connection
> in
> > a regular way
> > (iterating over all client addresses) and sends request only if all
> > attempts failed.
> >
> > There is a concern about naming of the configuration as it catches only
> one
> > use-case: when we
> > deal with some kind of virtualized environment like K8S.
> > There are other options I've encountered in private discussion:
> > - connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
> > - networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
> > - communicationViaDiscovery - ALWAYS / FALLBACK
> > - isReachableFromAllNodes (true/false flag)
> > - initiateConnectionsOnThisNode (true/false flag).
> >
> > *Limitations*
> >
> > The feature cannot be used along with pairedConnection setting as this
> > setting implies
> > establishing connections in both directions. Also current implementation
> > supports opening only
> > client-to-server connections. Other types of connections like
> > client-to-client or server-to-server
> > will be implemented in separate tickets.
> >
> > [1] https://issues.apache.org/jira/browse/IGNITE-12438
> > [2] https://issues.apache.org/jira/browse/IGNITE-13013
> >
> > --
> > Sincerely yours,
> > Ivan Bessonov
> >
>

Re: [DISCUSSION] New Ignite settings for IGNITE-12438 and IGNITE-13013

Posted by Denis Magda <dm...@apache.org>.
Ivan,

Considering that the setting controls the way a communication SPI
connection is open I would add the new parameter to CommunicationSpi
interface naming it as follows:

>
> CommunicationSpi.connectionInitiationMode
> {
>     BIDIRECTIONAL, //both clients and servers initiate a connection
> initiation procedure
>     CLIENTS_TO_SERVERS //servers cannot open a connection to clients, only
> clients can do that
> }


The problem with the environment type approach is that private networks of
bare-metal environments usually impose restrictions similar to virtual
environments powered by Kubernetes. Thus, environmentType.VIRTUALIZED
doesn't cover all the cases and I'm struggling to come up with a universal
alternative.

-
Denis


On Thu, May 21, 2020 at 5:38 AM Ivan Bessonov <be...@gmail.com> wrote:

> Hello Igniters,
>
> I'd like to discuss with you changes related to [1] and [2]. Both issues
> are mostly the same so
> let's discuss the core idea.
>
> *Motivation.*
>
> There are certain environments that don't allow Ignite server nodes to open
> TCP connections to
> thick clients, e.g. K8S, AWS Lambda or Azure Functions. To operate in such
> environments, the
> server needs a way to request a client to open an "inverse" communication
> connection to it.
>
> I've prepared a PR (still in progress) that introduces new mechanism of
> opening connection and
> related configuration.
>
> *Main idea*
>
> This mechanism is called "communication via discovery" or "inverse
> connection", it works as
> follows:
>  - server that needs to connect to "unreachable" thick client sends a
> specific Discovery message
>    (inverse communication request) to that client;
>  - client node upon receiving the request opens communication connection to
> that server;
>  - server sees connection opened by client and proceeds with its task (that
> required opening
>    connection to the client).
>
> Working name for new configuration parameter for this feature is
> environmentType, it is an
> enum with two values (again, working names): STANDALONE (default) and
> VIRTUALIZED.
> It is used as a hint to server to speed-up establishing of connections:
> when server sees a client
> with VIRTUALIZED environmentType it doesn't try to open connection to it
> and sends inverse
> communication request right away.
> If environmentType is STANDALONE then server tries to open a connection in
> a regular way
> (iterating over all client addresses) and sends request only if all
> attempts failed.
>
> There is a concern about naming of the configuration as it catches only one
> use-case: when we
> deal with some kind of virtualized environment like K8S.
> There are other options I've encountered in private discussion:
> - connectionMode - ALWAYS_INITIATE / INITIATE_OR_ACCEPT
> - networkConnectivity - REACHABLE_ALWAYS / REACHABLE_ONE_WAY
> - communicationViaDiscovery - ALWAYS / FALLBACK
> - isReachableFromAllNodes (true/false flag)
> - initiateConnectionsOnThisNode (true/false flag).
>
> *Limitations*
>
> The feature cannot be used along with pairedConnection setting as this
> setting implies
> establishing connections in both directions. Also current implementation
> supports opening only
> client-to-server connections. Other types of connections like
> client-to-client or server-to-server
> will be implemented in separate tickets.
>
> [1] https://issues.apache.org/jira/browse/IGNITE-12438
> [2] https://issues.apache.org/jira/browse/IGNITE-13013
>
> --
> Sincerely yours,
> Ivan Bessonov
>