Thin client protocol message format

classic Classic list List threaded Threaded
26 messages Options
12
Reply | Threaded
Open this post in threaded view
|

Thin client protocol message format

Pavel Tupitsyn
Igniters,

Below is a proposed design for thin client protocol [1] [2] socket data
exchange format.

* Values are little-endian
* Every request and response message starts with 4-byte length (including
handshake)
* Ignite binary format is used for value serialization (via
GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary protocol has
to be implemented by clients anyway to work with cache values, so it makes
sense to use for all data exchange.


1) Socket connection is established on a port according
to ConnectorConfiguration.port

2) Handshake is performed.
 Request:
   int32 length = 8     // message length
   byte opCode = 1   // handshake command
   int16 verMajor
   int16 verMinor
   int16 verMaintenance
   byte clientCode = 2    // client type code (odbc, jdbc, platform)

 Response:
   uint32 length = 1
   byte success

3) Execute command. Request starts with a command code, then goes
command-specific data.
For example, IgniteCache<Integer, String>.put(1, "foo") will look like this:
 Request:
   int16 opCode    // OP_CACHE_GET
   int32 cacheId    // GridCacheUtils.cacheId
   byte flags          // skipStore, noRetry, etc
   binobject key

 Response:
   byte success
   binobject value

Where binobject corresponds to Ignite BinaryMarshaller format, which is
produced by BinaryWriter.writeObject method. Integer will be represented as
  byte typeCode = 3  // GridBinaryMarshaller.INT
  int32 value = 1

4) Goto (3)


Comments are welcome.

Pavel

[1]
http://apache-ignite-developers.2346864.n4.nabble.com/Support-for-Ignite-clients-in-any-language-thin-client-protocol-td20297.html
[2] https://issues.apache.org/jira/browse/IGNITE-5896
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Alexey Kuznetsov
Pavel,

How about data compression?
May be it make sense to add a byte with compression algorithm?
0 - none
1 - ZIP
2 - ....
....

On Tue, Aug 1, 2017 at 11:10 PM, Pavel Tupitsyn <[hidden email]>
wrote:

> Igniters,
>
> Below is a proposed design for thin client protocol [1] [2] socket data
> exchange format.
>
> * Values are little-endian
> * Every request and response message starts with 4-byte length (including
> handshake)
> * Ignite binary format is used for value serialization (via
> GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary protocol
> has
> to be implemented by clients anyway to work with cache values, so it makes
> sense to use for all data exchange.
>
>
> 1) Socket connection is established on a port according
> to ConnectorConfiguration.port
>
> 2) Handshake is performed.
>  Request:
>    int32 length = 8     // message length
>    byte opCode = 1   // handshake command
>    int16 verMajor
>    int16 verMinor
>    int16 verMaintenance
>    byte clientCode = 2    // client type code (odbc, jdbc, platform)
>
>  Response:
>    uint32 length = 1
>    byte success
>
> 3) Execute command. Request starts with a command code, then goes
> command-specific data.
> For example, IgniteCache<Integer, String>.put(1, "foo") will look like
> this:
>  Request:
>    int16 opCode    // OP_CACHE_GET
>    int32 cacheId    // GridCacheUtils.cacheId
>    byte flags          // skipStore, noRetry, etc
>    binobject key
>
>  Response:
>    byte success
>    binobject value
>
> Where binobject corresponds to Ignite BinaryMarshaller format, which is
> produced by BinaryWriter.writeObject method. Integer will be represented as
>   byte typeCode = 3  // GridBinaryMarshaller.INT
>   int32 value = 1
>
> 4) Goto (3)
>
>
> Comments are welcome.
>
> Pavel
>
> [1]
> http://apache-ignite-developers.2346864.n4.nabble.com/Support-for-Ignite-
> clients-in-any-language-thin-client-protocol-td20297.html
> [2] https://issues.apache.org/jira/browse/IGNITE-5896
>



--
Alexey Kuznetsov
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Nikolai Tikhonov
For supporting different cases, might be useful to add to handshake message
one byte for different flags. compression is a good example. Also it can be
some specific logic for different clients (C++, JS, Python), crypting and
etc.

On Tue, Aug 1, 2017 at 7:27 PM, Alexey Kuznetsov <[hidden email]>
wrote:

> Pavel,
>
> How about data compression?
> May be it make sense to add a byte with compression algorithm?
> 0 - none
> 1 - ZIP
> 2 - ....
> ....
>
> On Tue, Aug 1, 2017 at 11:10 PM, Pavel Tupitsyn <[hidden email]>
> wrote:
>
> > Igniters,
> >
> > Below is a proposed design for thin client protocol [1] [2] socket data
> > exchange format.
> >
> > * Values are little-endian
> > * Every request and response message starts with 4-byte length (including
> > handshake)
> > * Ignite binary format is used for value serialization (via
> > GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary protocol
> > has
> > to be implemented by clients anyway to work with cache values, so it
> makes
> > sense to use for all data exchange.
> >
> >
> > 1) Socket connection is established on a port according
> > to ConnectorConfiguration.port
> >
> > 2) Handshake is performed.
> >  Request:
> >    int32 length = 8     // message length
> >    byte opCode = 1   // handshake command
> >    int16 verMajor
> >    int16 verMinor
> >    int16 verMaintenance
> >    byte clientCode = 2    // client type code (odbc, jdbc, platform)
> >
> >  Response:
> >    uint32 length = 1
> >    byte success
> >
> > 3) Execute command. Request starts with a command code, then goes
> > command-specific data.
> > For example, IgniteCache<Integer, String>.put(1, "foo") will look like
> > this:
> >  Request:
> >    int16 opCode    // OP_CACHE_GET
> >    int32 cacheId    // GridCacheUtils.cacheId
> >    byte flags          // skipStore, noRetry, etc
> >    binobject key
> >
> >  Response:
> >    byte success
> >    binobject value
> >
> > Where binobject corresponds to Ignite BinaryMarshaller format, which is
> > produced by BinaryWriter.writeObject method. Integer will be represented
> as
> >   byte typeCode = 3  // GridBinaryMarshaller.INT
> >   int32 value = 1
> >
> > 4) Goto (3)
> >
> >
> > Comments are welcome.
> >
> > Pavel
> >
> > [1]
> > http://apache-ignite-developers.2346864.n4.nabble.
> com/Support-for-Ignite-
> > clients-in-any-language-thin-client-protocol-td20297.html
> > [2] https://issues.apache.org/jira/browse/IGNITE-5896
> >
>
>
>
> --
> Alexey Kuznetsov
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Pavel Tupitsyn
In reply to this post by Alexey Kuznetsov
Alexey, good idea. ODBC and JDBC could also benefit from this.

On Tue, Aug 1, 2017 at 7:27 PM, Alexey Kuznetsov <[hidden email]>
wrote:

> Pavel,
>
> How about data compression?
> May be it make sense to add a byte with compression algorithm?
> 0 - none
> 1 - ZIP
> 2 - ....
> ....
>
> On Tue, Aug 1, 2017 at 11:10 PM, Pavel Tupitsyn <[hidden email]>
> wrote:
>
> > Igniters,
> >
> > Below is a proposed design for thin client protocol [1] [2] socket data
> > exchange format.
> >
> > * Values are little-endian
> > * Every request and response message starts with 4-byte length (including
> > handshake)
> > * Ignite binary format is used for value serialization (via
> > GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary protocol
> > has
> > to be implemented by clients anyway to work with cache values, so it
> makes
> > sense to use for all data exchange.
> >
> >
> > 1) Socket connection is established on a port according
> > to ConnectorConfiguration.port
> >
> > 2) Handshake is performed.
> >  Request:
> >    int32 length = 8     // message length
> >    byte opCode = 1   // handshake command
> >    int16 verMajor
> >    int16 verMinor
> >    int16 verMaintenance
> >    byte clientCode = 2    // client type code (odbc, jdbc, platform)
> >
> >  Response:
> >    uint32 length = 1
> >    byte success
> >
> > 3) Execute command. Request starts with a command code, then goes
> > command-specific data.
> > For example, IgniteCache<Integer, String>.put(1, "foo") will look like
> > this:
> >  Request:
> >    int16 opCode    // OP_CACHE_GET
> >    int32 cacheId    // GridCacheUtils.cacheId
> >    byte flags          // skipStore, noRetry, etc
> >    binobject key
> >
> >  Response:
> >    byte success
> >    binobject value
> >
> > Where binobject corresponds to Ignite BinaryMarshaller format, which is
> > produced by BinaryWriter.writeObject method. Integer will be represented
> as
> >   byte typeCode = 3  // GridBinaryMarshaller.INT
> >   int32 value = 1
> >
> > 4) Goto (3)
> >
> >
> > Comments are welcome.
> >
> > Pavel
> >
> > [1]
> > http://apache-ignite-developers.2346864.n4.nabble.
> com/Support-for-Ignite-
> > clients-in-any-language-thin-client-protocol-td20297.html
> > [2] https://issues.apache.org/jira/browse/IGNITE-5896
> >
>
>
>
> --
> Alexey Kuznetsov
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

dsetrakyan
We should also leave 8 bytes of empty space for future changes.

⁣D.​

On Aug 1, 2017, 6:41 PM, at 6:41 PM, Pavel Tupitsyn <[hidden email]> wrote:

>Alexey, good idea. ODBC and JDBC could also benefit from this.
>
>On Tue, Aug 1, 2017 at 7:27 PM, Alexey Kuznetsov
><[hidden email]>
>wrote:
>
>> Pavel,
>>
>> How about data compression?
>> May be it make sense to add a byte with compression algorithm?
>> 0 - none
>> 1 - ZIP
>> 2 - ....
>> ....
>>
>> On Tue, Aug 1, 2017 at 11:10 PM, Pavel Tupitsyn
><[hidden email]>
>> wrote:
>>
>> > Igniters,
>> >
>> > Below is a proposed design for thin client protocol [1] [2] socket
>data
>> > exchange format.
>> >
>> > * Values are little-endian
>> > * Every request and response message starts with 4-byte length
>(including
>> > handshake)
>> > * Ignite binary format is used for value serialization (via
>> > GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary
>protocol
>> > has
>> > to be implemented by clients anyway to work with cache values, so
>it
>> makes
>> > sense to use for all data exchange.
>> >
>> >
>> > 1) Socket connection is established on a port according
>> > to ConnectorConfiguration.port
>> >
>> > 2) Handshake is performed.
>> >  Request:
>> >    int32 length = 8     // message length
>> >    byte opCode = 1   // handshake command
>> >    int16 verMajor
>> >    int16 verMinor
>> >    int16 verMaintenance
>> >    byte clientCode = 2    // client type code (odbc, jdbc,
>platform)
>> >
>> >  Response:
>> >    uint32 length = 1
>> >    byte success
>> >
>> > 3) Execute command. Request starts with a command code, then goes
>> > command-specific data.
>> > For example, IgniteCache<Integer, String>.put(1, "foo") will look
>like
>> > this:
>> >  Request:
>> >    int16 opCode    // OP_CACHE_GET
>> >    int32 cacheId    // GridCacheUtils.cacheId
>> >    byte flags          // skipStore, noRetry, etc
>> >    binobject key
>> >
>> >  Response:
>> >    byte success
>> >    binobject value
>> >
>> > Where binobject corresponds to Ignite BinaryMarshaller format,
>which is
>> > produced by BinaryWriter.writeObject method. Integer will be
>represented
>> as
>> >   byte typeCode = 3  // GridBinaryMarshaller.INT
>> >   int32 value = 1
>> >
>> > 4) Goto (3)
>> >
>> >
>> > Comments are welcome.
>> >
>> > Pavel
>> >
>> > [1]
>> > http://apache-ignite-developers.2346864.n4.nabble.
>> com/Support-for-Ignite-
>> > clients-in-any-language-thin-client-protocol-td20297.html
>> > [2] https://issues.apache.org/jira/browse/IGNITE-5896
>> >
>>
>>
>>
>> --
>> Alexey Kuznetsov
>>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Pavel Tupitsyn
Dmitry, we don't need any reserved bytes, because protocol is versioned.

On Tue, Aug 1, 2017 at 7:49 PM, <[hidden email]> wrote:

> We should also leave 8 bytes of empty space for future changes.
>
> ⁣D.​
>
> On Aug 1, 2017, 6:41 PM, at 6:41 PM, Pavel Tupitsyn <[hidden email]>
> wrote:
> >Alexey, good idea. ODBC and JDBC could also benefit from this.
> >
> >On Tue, Aug 1, 2017 at 7:27 PM, Alexey Kuznetsov
> ><[hidden email]>
> >wrote:
> >
> >> Pavel,
> >>
> >> How about data compression?
> >> May be it make sense to add a byte with compression algorithm?
> >> 0 - none
> >> 1 - ZIP
> >> 2 - ....
> >> ....
> >>
> >> On Tue, Aug 1, 2017 at 11:10 PM, Pavel Tupitsyn
> ><[hidden email]>
> >> wrote:
> >>
> >> > Igniters,
> >> >
> >> > Below is a proposed design for thin client protocol [1] [2] socket
> >data
> >> > exchange format.
> >> >
> >> > * Values are little-endian
> >> > * Every request and response message starts with 4-byte length
> >(including
> >> > handshake)
> >> > * Ignite binary format is used for value serialization (via
> >> > GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary
> >protocol
> >> > has
> >> > to be implemented by clients anyway to work with cache values, so
> >it
> >> makes
> >> > sense to use for all data exchange.
> >> >
> >> >
> >> > 1) Socket connection is established on a port according
> >> > to ConnectorConfiguration.port
> >> >
> >> > 2) Handshake is performed.
> >> >  Request:
> >> >    int32 length = 8     // message length
> >> >    byte opCode = 1   // handshake command
> >> >    int16 verMajor
> >> >    int16 verMinor
> >> >    int16 verMaintenance
> >> >    byte clientCode = 2    // client type code (odbc, jdbc,
> >platform)
> >> >
> >> >  Response:
> >> >    uint32 length = 1
> >> >    byte success
> >> >
> >> > 3) Execute command. Request starts with a command code, then goes
> >> > command-specific data.
> >> > For example, IgniteCache<Integer, String>.put(1, "foo") will look
> >like
> >> > this:
> >> >  Request:
> >> >    int16 opCode    // OP_CACHE_GET
> >> >    int32 cacheId    // GridCacheUtils.cacheId
> >> >    byte flags          // skipStore, noRetry, etc
> >> >    binobject key
> >> >
> >> >  Response:
> >> >    byte success
> >> >    binobject value
> >> >
> >> > Where binobject corresponds to Ignite BinaryMarshaller format,
> >which is
> >> > produced by BinaryWriter.writeObject method. Integer will be
> >represented
> >> as
> >> >   byte typeCode = 3  // GridBinaryMarshaller.INT
> >> >   int32 value = 1
> >> >
> >> > 4) Goto (3)
> >> >
> >> >
> >> > Comments are welcome.
> >> >
> >> > Pavel
> >> >
> >> > [1]
> >> > http://apache-ignite-developers.2346864.n4.nabble.
> >> com/Support-for-Ignite-
> >> > clients-in-any-language-thin-client-protocol-td20297.html
> >> > [2] https://issues.apache.org/jira/browse/IGNITE-5896
> >> >
> >>
> >>
> >>
> >> --
> >> Alexey Kuznetsov
> >>
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

dsetrakyan
Backward compatible?

⁣D.​

On Aug 1, 2017, 7:04 PM, at 7:04 PM, Pavel Tupitsyn <[hidden email]> wrote:

>Dmitry, we don't need any reserved bytes, because protocol is
>versioned.
>
>On Tue, Aug 1, 2017 at 7:49 PM, <[hidden email]> wrote:
>
>> We should also leave 8 bytes of empty space for future changes.
>>
>> ⁣D.​
>>
>> On Aug 1, 2017, 6:41 PM, at 6:41 PM, Pavel Tupitsyn
><[hidden email]>
>> wrote:
>> >Alexey, good idea. ODBC and JDBC could also benefit from this.
>> >
>> >On Tue, Aug 1, 2017 at 7:27 PM, Alexey Kuznetsov
>> ><[hidden email]>
>> >wrote:
>> >
>> >> Pavel,
>> >>
>> >> How about data compression?
>> >> May be it make sense to add a byte with compression algorithm?
>> >> 0 - none
>> >> 1 - ZIP
>> >> 2 - ....
>> >> ....
>> >>
>> >> On Tue, Aug 1, 2017 at 11:10 PM, Pavel Tupitsyn
>> ><[hidden email]>
>> >> wrote:
>> >>
>> >> > Igniters,
>> >> >
>> >> > Below is a proposed design for thin client protocol [1] [2]
>socket
>> >data
>> >> > exchange format.
>> >> >
>> >> > * Values are little-endian
>> >> > * Every request and response message starts with 4-byte length
>> >(including
>> >> > handshake)
>> >> > * Ignite binary format is used for value serialization (via
>> >> > GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary
>> >protocol
>> >> > has
>> >> > to be implemented by clients anyway to work with cache values,
>so
>> >it
>> >> makes
>> >> > sense to use for all data exchange.
>> >> >
>> >> >
>> >> > 1) Socket connection is established on a port according
>> >> > to ConnectorConfiguration.port
>> >> >
>> >> > 2) Handshake is performed.
>> >> >  Request:
>> >> >    int32 length = 8     // message length
>> >> >    byte opCode = 1   // handshake command
>> >> >    int16 verMajor
>> >> >    int16 verMinor
>> >> >    int16 verMaintenance
>> >> >    byte clientCode = 2    // client type code (odbc, jdbc,
>> >platform)
>> >> >
>> >> >  Response:
>> >> >    uint32 length = 1
>> >> >    byte success
>> >> >
>> >> > 3) Execute command. Request starts with a command code, then
>goes
>> >> > command-specific data.
>> >> > For example, IgniteCache<Integer, String>.put(1, "foo") will
>look
>> >like
>> >> > this:
>> >> >  Request:
>> >> >    int16 opCode    // OP_CACHE_GET
>> >> >    int32 cacheId    // GridCacheUtils.cacheId
>> >> >    byte flags          // skipStore, noRetry, etc
>> >> >    binobject key
>> >> >
>> >> >  Response:
>> >> >    byte success
>> >> >    binobject value
>> >> >
>> >> > Where binobject corresponds to Ignite BinaryMarshaller format,
>> >which is
>> >> > produced by BinaryWriter.writeObject method. Integer will be
>> >represented
>> >> as
>> >> >   byte typeCode = 3  // GridBinaryMarshaller.INT
>> >> >   int32 value = 1
>> >> >
>> >> > 4) Goto (3)
>> >> >
>> >> >
>> >> > Comments are welcome.
>> >> >
>> >> > Pavel
>> >> >
>> >> > [1]
>> >> > http://apache-ignite-developers.2346864.n4.nabble.
>> >> com/Support-for-Ignite-
>> >> > clients-in-any-language-thin-client-protocol-td20297.html
>> >> > [2] https://issues.apache.org/jira/browse/IGNITE-5896
>> >> >
>> >>
>> >>
>> >>
>> >> --
>> >> Alexey Kuznetsov
>> >>
>>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Pavel Tupitsyn
Yes, in my understanding, we should provide backwards compatibility via
versioning.

During handshake we pick RequestParser and RequestHandler (this is already
implemented, see SqlListenerNioListener.prepareContext).
Each protocol version would have separate implementations of these
interfaces.

New version of the protocol does not affect existing versions.

On Tue, Aug 1, 2017 at 8:09 PM, <[hidden email]> wrote:

> Backward compatible?
>
> ⁣D.​
>
> On Aug 1, 2017, 7:04 PM, at 7:04 PM, Pavel Tupitsyn <[hidden email]>
> wrote:
> >Dmitry, we don't need any reserved bytes, because protocol is
> >versioned.
> >
> >On Tue, Aug 1, 2017 at 7:49 PM, <[hidden email]> wrote:
> >
> >> We should also leave 8 bytes of empty space for future changes.
> >>
> >> ⁣D.​
> >>
> >> On Aug 1, 2017, 6:41 PM, at 6:41 PM, Pavel Tupitsyn
> ><[hidden email]>
> >> wrote:
> >> >Alexey, good idea. ODBC and JDBC could also benefit from this.
> >> >
> >> >On Tue, Aug 1, 2017 at 7:27 PM, Alexey Kuznetsov
> >> ><[hidden email]>
> >> >wrote:
> >> >
> >> >> Pavel,
> >> >>
> >> >> How about data compression?
> >> >> May be it make sense to add a byte with compression algorithm?
> >> >> 0 - none
> >> >> 1 - ZIP
> >> >> 2 - ....
> >> >> ....
> >> >>
> >> >> On Tue, Aug 1, 2017 at 11:10 PM, Pavel Tupitsyn
> >> ><[hidden email]>
> >> >> wrote:
> >> >>
> >> >> > Igniters,
> >> >> >
> >> >> > Below is a proposed design for thin client protocol [1] [2]
> >socket
> >> >data
> >> >> > exchange format.
> >> >> >
> >> >> > * Values are little-endian
> >> >> > * Every request and response message starts with 4-byte length
> >> >(including
> >> >> > handshake)
> >> >> > * Ignite binary format is used for value serialization (via
> >> >> > GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary
> >> >protocol
> >> >> > has
> >> >> > to be implemented by clients anyway to work with cache values,
> >so
> >> >it
> >> >> makes
> >> >> > sense to use for all data exchange.
> >> >> >
> >> >> >
> >> >> > 1) Socket connection is established on a port according
> >> >> > to ConnectorConfiguration.port
> >> >> >
> >> >> > 2) Handshake is performed.
> >> >> >  Request:
> >> >> >    int32 length = 8     // message length
> >> >> >    byte opCode = 1   // handshake command
> >> >> >    int16 verMajor
> >> >> >    int16 verMinor
> >> >> >    int16 verMaintenance
> >> >> >    byte clientCode = 2    // client type code (odbc, jdbc,
> >> >platform)
> >> >> >
> >> >> >  Response:
> >> >> >    uint32 length = 1
> >> >> >    byte success
> >> >> >
> >> >> > 3) Execute command. Request starts with a command code, then
> >goes
> >> >> > command-specific data.
> >> >> > For example, IgniteCache<Integer, String>.put(1, "foo") will
> >look
> >> >like
> >> >> > this:
> >> >> >  Request:
> >> >> >    int16 opCode    // OP_CACHE_GET
> >> >> >    int32 cacheId    // GridCacheUtils.cacheId
> >> >> >    byte flags          // skipStore, noRetry, etc
> >> >> >    binobject key
> >> >> >
> >> >> >  Response:
> >> >> >    byte success
> >> >> >    binobject value
> >> >> >
> >> >> > Where binobject corresponds to Ignite BinaryMarshaller format,
> >> >which is
> >> >> > produced by BinaryWriter.writeObject method. Integer will be
> >> >represented
> >> >> as
> >> >> >   byte typeCode = 3  // GridBinaryMarshaller.INT
> >> >> >   int32 value = 1
> >> >> >
> >> >> > 4) Goto (3)
> >> >> >
> >> >> >
> >> >> > Comments are welcome.
> >> >> >
> >> >> > Pavel
> >> >> >
> >> >> > [1]
> >> >> > http://apache-ignite-developers.2346864.n4.nabble.
> >> >> com/Support-for-Ignite-
> >> >> > clients-in-any-language-thin-client-protocol-td20297.html
> >> >> > [2] https://issues.apache.org/jira/browse/IGNITE-5896
> >> >> >
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> Alexey Kuznetsov
> >> >>
> >>
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

dsetrakyan
How about allowing 8 bytes of extra space for any user-specific additional
data? For example, I can see that some implementations of this protocol
will need to pass some extra parameters to the server. Of course, in that
case the server side protocol handler should have a hook to react to these
extra parameters.

Makes sense?

D.

On Tue, Aug 1, 2017 at 7:17 PM, Pavel Tupitsyn <[hidden email]> wrote:

> Yes, in my understanding, we should provide backwards compatibility via
> versioning.
>
> During handshake we pick RequestParser and RequestHandler (this is already
> implemented, see SqlListenerNioListener.prepareContext).
> Each protocol version would have separate implementations of these
> interfaces.
>
> New version of the protocol does not affect existing versions.
>
> On Tue, Aug 1, 2017 at 8:09 PM, <[hidden email]> wrote:
>
> > Backward compatible?
> >
> > ⁣D.​
> >
> > On Aug 1, 2017, 7:04 PM, at 7:04 PM, Pavel Tupitsyn <
> [hidden email]>
> > wrote:
> > >Dmitry, we don't need any reserved bytes, because protocol is
> > >versioned.
> > >
> > >On Tue, Aug 1, 2017 at 7:49 PM, <[hidden email]> wrote:
> > >
> > >> We should also leave 8 bytes of empty space for future changes.
> > >>
> > >> ⁣D.​
> > >>
> > >> On Aug 1, 2017, 6:41 PM, at 6:41 PM, Pavel Tupitsyn
> > ><[hidden email]>
> > >> wrote:
> > >> >Alexey, good idea. ODBC and JDBC could also benefit from this.
> > >> >
> > >> >On Tue, Aug 1, 2017 at 7:27 PM, Alexey Kuznetsov
> > >> ><[hidden email]>
> > >> >wrote:
> > >> >
> > >> >> Pavel,
> > >> >>
> > >> >> How about data compression?
> > >> >> May be it make sense to add a byte with compression algorithm?
> > >> >> 0 - none
> > >> >> 1 - ZIP
> > >> >> 2 - ....
> > >> >> ....
> > >> >>
> > >> >> On Tue, Aug 1, 2017 at 11:10 PM, Pavel Tupitsyn
> > >> ><[hidden email]>
> > >> >> wrote:
> > >> >>
> > >> >> > Igniters,
> > >> >> >
> > >> >> > Below is a proposed design for thin client protocol [1] [2]
> > >socket
> > >> >data
> > >> >> > exchange format.
> > >> >> >
> > >> >> > * Values are little-endian
> > >> >> > * Every request and response message starts with 4-byte length
> > >> >(including
> > >> >> > handshake)
> > >> >> > * Ignite binary format is used for value serialization (via
> > >> >> > GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary
> > >> >protocol
> > >> >> > has
> > >> >> > to be implemented by clients anyway to work with cache values,
> > >so
> > >> >it
> > >> >> makes
> > >> >> > sense to use for all data exchange.
> > >> >> >
> > >> >> >
> > >> >> > 1) Socket connection is established on a port according
> > >> >> > to ConnectorConfiguration.port
> > >> >> >
> > >> >> > 2) Handshake is performed.
> > >> >> >  Request:
> > >> >> >    int32 length = 8     // message length
> > >> >> >    byte opCode = 1   // handshake command
> > >> >> >    int16 verMajor
> > >> >> >    int16 verMinor
> > >> >> >    int16 verMaintenance
> > >> >> >    byte clientCode = 2    // client type code (odbc, jdbc,
> > >> >platform)
> > >> >> >
> > >> >> >  Response:
> > >> >> >    uint32 length = 1
> > >> >> >    byte success
> > >> >> >
> > >> >> > 3) Execute command. Request starts with a command code, then
> > >goes
> > >> >> > command-specific data.
> > >> >> > For example, IgniteCache<Integer, String>.put(1, "foo") will
> > >look
> > >> >like
> > >> >> > this:
> > >> >> >  Request:
> > >> >> >    int16 opCode    // OP_CACHE_GET
> > >> >> >    int32 cacheId    // GridCacheUtils.cacheId
> > >> >> >    byte flags          // skipStore, noRetry, etc
> > >> >> >    binobject key
> > >> >> >
> > >> >> >  Response:
> > >> >> >    byte success
> > >> >> >    binobject value
> > >> >> >
> > >> >> > Where binobject corresponds to Ignite BinaryMarshaller format,
> > >> >which is
> > >> >> > produced by BinaryWriter.writeObject method. Integer will be
> > >> >represented
> > >> >> as
> > >> >> >   byte typeCode = 3  // GridBinaryMarshaller.INT
> > >> >> >   int32 value = 1
> > >> >> >
> > >> >> > 4) Goto (3)
> > >> >> >
> > >> >> >
> > >> >> > Comments are welcome.
> > >> >> >
> > >> >> > Pavel
> > >> >> >
> > >> >> > [1]
> > >> >> > http://apache-ignite-developers.2346864.n4.nabble.
> > >> >> com/Support-for-Ignite-
> > >> >> > clients-in-any-language-thin-client-protocol-td20297.html
> > >> >> > [2] https://issues.apache.org/jira/browse/IGNITE-5896
> > >> >> >
> > >> >>
> > >> >>
> > >> >>
> > >> >> --
> > >> >> Alexey Kuznetsov
> > >> >>
> > >>
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Vladimir Ozerov
Dima,

There is no need for these extra bytes. Protocol version would be enough.
Extensibility points wil be designed separately. We already have experience
with it in scope of .NET/CPP platforms, it is solved by special command
types where user can hook his own code.

On Wed, Aug 2, 2017 at 3:56 AM, Dmitriy Setrakyan <[hidden email]>
wrote:

> How about allowing 8 bytes of extra space for any user-specific additional
> data? For example, I can see that some implementations of this protocol
> will need to pass some extra parameters to the server. Of course, in that
> case the server side protocol handler should have a hook to react to these
> extra parameters.
>
> Makes sense?
>
> D.
>
> On Tue, Aug 1, 2017 at 7:17 PM, Pavel Tupitsyn <[hidden email]>
> wrote:
>
> > Yes, in my understanding, we should provide backwards compatibility via
> > versioning.
> >
> > During handshake we pick RequestParser and RequestHandler (this is
> already
> > implemented, see SqlListenerNioListener.prepareContext).
> > Each protocol version would have separate implementations of these
> > interfaces.
> >
> > New version of the protocol does not affect existing versions.
> >
> > On Tue, Aug 1, 2017 at 8:09 PM, <[hidden email]> wrote:
> >
> > > Backward compatible?
> > >
> > > ⁣D.​
> > >
> > > On Aug 1, 2017, 7:04 PM, at 7:04 PM, Pavel Tupitsyn <
> > [hidden email]>
> > > wrote:
> > > >Dmitry, we don't need any reserved bytes, because protocol is
> > > >versioned.
> > > >
> > > >On Tue, Aug 1, 2017 at 7:49 PM, <[hidden email]> wrote:
> > > >
> > > >> We should also leave 8 bytes of empty space for future changes.
> > > >>
> > > >> ⁣D.​
> > > >>
> > > >> On Aug 1, 2017, 6:41 PM, at 6:41 PM, Pavel Tupitsyn
> > > ><[hidden email]>
> > > >> wrote:
> > > >> >Alexey, good idea. ODBC and JDBC could also benefit from this.
> > > >> >
> > > >> >On Tue, Aug 1, 2017 at 7:27 PM, Alexey Kuznetsov
> > > >> ><[hidden email]>
> > > >> >wrote:
> > > >> >
> > > >> >> Pavel,
> > > >> >>
> > > >> >> How about data compression?
> > > >> >> May be it make sense to add a byte with compression algorithm?
> > > >> >> 0 - none
> > > >> >> 1 - ZIP
> > > >> >> 2 - ....
> > > >> >> ....
> > > >> >>
> > > >> >> On Tue, Aug 1, 2017 at 11:10 PM, Pavel Tupitsyn
> > > >> ><[hidden email]>
> > > >> >> wrote:
> > > >> >>
> > > >> >> > Igniters,
> > > >> >> >
> > > >> >> > Below is a proposed design for thin client protocol [1] [2]
> > > >socket
> > > >> >data
> > > >> >> > exchange format.
> > > >> >> >
> > > >> >> > * Values are little-endian
> > > >> >> > * Every request and response message starts with 4-byte length
> > > >> >(including
> > > >> >> > handshake)
> > > >> >> > * Ignite binary format is used for value serialization (via
> > > >> >> > GridBinaryMarshaller/BinaryWriter/BinaryReader). Ignite binary
> > > >> >protocol
> > > >> >> > has
> > > >> >> > to be implemented by clients anyway to work with cache values,
> > > >so
> > > >> >it
> > > >> >> makes
> > > >> >> > sense to use for all data exchange.
> > > >> >> >
> > > >> >> >
> > > >> >> > 1) Socket connection is established on a port according
> > > >> >> > to ConnectorConfiguration.port
> > > >> >> >
> > > >> >> > 2) Handshake is performed.
> > > >> >> >  Request:
> > > >> >> >    int32 length = 8     // message length
> > > >> >> >    byte opCode = 1   // handshake command
> > > >> >> >    int16 verMajor
> > > >> >> >    int16 verMinor
> > > >> >> >    int16 verMaintenance
> > > >> >> >    byte clientCode = 2    // client type code (odbc, jdbc,
> > > >> >platform)
> > > >> >> >
> > > >> >> >  Response:
> > > >> >> >    uint32 length = 1
> > > >> >> >    byte success
> > > >> >> >
> > > >> >> > 3) Execute command. Request starts with a command code, then
> > > >goes
> > > >> >> > command-specific data.
> > > >> >> > For example, IgniteCache<Integer, String>.put(1, "foo") will
> > > >look
> > > >> >like
> > > >> >> > this:
> > > >> >> >  Request:
> > > >> >> >    int16 opCode    // OP_CACHE_GET
> > > >> >> >    int32 cacheId    // GridCacheUtils.cacheId
> > > >> >> >    byte flags          // skipStore, noRetry, etc
> > > >> >> >    binobject key
> > > >> >> >
> > > >> >> >  Response:
> > > >> >> >    byte success
> > > >> >> >    binobject value
> > > >> >> >
> > > >> >> > Where binobject corresponds to Ignite BinaryMarshaller format,
> > > >> >which is
> > > >> >> > produced by BinaryWriter.writeObject method. Integer will be
> > > >> >represented
> > > >> >> as
> > > >> >> >   byte typeCode = 3  // GridBinaryMarshaller.INT
> > > >> >> >   int32 value = 1
> > > >> >> >
> > > >> >> > 4) Goto (3)
> > > >> >> >
> > > >> >> >
> > > >> >> > Comments are welcome.
> > > >> >> >
> > > >> >> > Pavel
> > > >> >> >
> > > >> >> > [1]
> > > >> >> > http://apache-ignite-developers.2346864.n4.nabble.
> > > >> >> com/Support-for-Ignite-
> > > >> >> > clients-in-any-language-thin-client-protocol-td20297.html
> > > >> >> > [2] https://issues.apache.org/jira/browse/IGNITE-5896
> > > >> >> >
> > > >> >>
> > > >> >>
> > > >> >>
> > > >> >> --
> > > >> >> Alexey Kuznetsov
> > > >> >>
> > > >>
> > >
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

yzhdanov
Here are my observations.

1. Let's create wiki page where we will keep the protocol definition and
reflect all the changes.

2. I would put op_code to the first place. This will make possible to
eliminate length field for many messages. Look at your handshake request -
it is always of fixed length. Why do we need length then? Variable length
operations - puts, putalls, getalls, etc will definitely need length field
(or keys count and length of each key separately in each binary object -
let's discuss it later).

3. I would also add build date and revision hash to handshake. Same as we
do for Ignite.

4. I would like to have explicit protocol version for client to make
possible for newer clients to work with older servers. Moreover, I think
there may be some third party protocol implementations on other platforms
which may not be driven by Ignite community and their versioning may be
different. So, explicit protocol version is really handy here.

Thanks!

--Yakov
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Vladimir Ozerov
Yakov,

Yes, explicit protocol versioning already used in ODBC/JDBC. Looks like we
should continue this practice in this protocol as well.

On Wed, Aug 2, 2017 at 10:44 AM, Yakov Zhdanov <[hidden email]> wrote:

> Here are my observations.
>
> 1. Let's create wiki page where we will keep the protocol definition and
> reflect all the changes.
>
> 2. I would put op_code to the first place. This will make possible to
> eliminate length field for many messages. Look at your handshake request -
> it is always of fixed length. Why do we need length then? Variable length
> operations - puts, putalls, getalls, etc will definitely need length field
> (or keys count and length of each key separately in each binary object -
> let's discuss it later).
>
> 3. I would also add build date and revision hash to handshake. Same as we
> do for Ignite.
>
> 4. I would like to have explicit protocol version for client to make
> possible for newer clients to work with older servers. Moreover, I think
> there may be some third party protocol implementations on other platforms
> which may not be driven by Ignite community and their versioning may be
> different. So, explicit protocol version is really handy here.
>
> Thanks!
>
> --Yakov
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Alexey Goncharuk
Do I understand correctly that this is not a multiplexed protocol? Are we
ok to have a separate connection for each thread? I would also add a
requestId field to allow multiple concurrent requests at a time.

2017-08-02 10:50 GMT+03:00 Vladimir Ozerov <[hidden email]>:

> Yakov,
>
> Yes, explicit protocol versioning already used in ODBC/JDBC. Looks like we
> should continue this practice in this protocol as well.
>
> On Wed, Aug 2, 2017 at 10:44 AM, Yakov Zhdanov <[hidden email]>
> wrote:
>
> > Here are my observations.
> >
> > 1. Let's create wiki page where we will keep the protocol definition and
> > reflect all the changes.
> >
> > 2. I would put op_code to the first place. This will make possible to
> > eliminate length field for many messages. Look at your handshake request
> -
> > it is always of fixed length. Why do we need length then? Variable length
> > operations - puts, putalls, getalls, etc will definitely need length
> field
> > (or keys count and length of each key separately in each binary object -
> > let's discuss it later).
> >
> > 3. I would also add build date and revision hash to handshake. Same as we
> > do for Ignite.
> >
> > 4. I would like to have explicit protocol version for client to make
> > possible for newer clients to work with older servers. Moreover, I think
> > there may be some third party protocol implementations on other platforms
> > which may not be driven by Ignite community and their versioning may be
> > different. So, explicit protocol version is really handy here.
> >
> > Thanks!
> >
> > --Yakov
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Vladimir Ozerov
I think that in the first iteration it would be enough to have
connection-per-thread approach. But in future we definitely would like to
multiplex threads over a single connection and to support async operations.
That said, we definitely need request ID. Let's add it right now to avoid
compatibility issues in future.

On Wed, Aug 2, 2017 at 10:59 AM, Alexey Goncharuk <
[hidden email]> wrote:

> Do I understand correctly that this is not a multiplexed protocol? Are we
> ok to have a separate connection for each thread? I would also add a
> requestId field to allow multiple concurrent requests at a time.
>
> 2017-08-02 10:50 GMT+03:00 Vladimir Ozerov <[hidden email]>:
>
> > Yakov,
> >
> > Yes, explicit protocol versioning already used in ODBC/JDBC. Looks like
> we
> > should continue this practice in this protocol as well.
> >
> > On Wed, Aug 2, 2017 at 10:44 AM, Yakov Zhdanov <[hidden email]>
> > wrote:
> >
> > > Here are my observations.
> > >
> > > 1. Let's create wiki page where we will keep the protocol definition
> and
> > > reflect all the changes.
> > >
> > > 2. I would put op_code to the first place. This will make possible to
> > > eliminate length field for many messages. Look at your handshake
> request
> > -
> > > it is always of fixed length. Why do we need length then? Variable
> length
> > > operations - puts, putalls, getalls, etc will definitely need length
> > field
> > > (or keys count and length of each key separately in each binary object
> -
> > > let's discuss it later).
> > >
> > > 3. I would also add build date and revision hash to handshake. Same as
> we
> > > do for Ignite.
> > >
> > > 4. I would like to have explicit protocol version for client to make
> > > possible for newer clients to work with older servers. Moreover, I
> think
> > > there may be some third party protocol implementations on other
> platforms
> > > which may not be driven by Ignite community and their versioning may be
> > > different. So, explicit protocol version is really handy here.
> > >
> > > Thanks!
> > >
> > > --Yakov
> > >
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

yzhdanov
In reply to this post by Alexey Goncharuk
Agree with Alex. I think our implementations should share single connection
over threads in the process.

--Yakov
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Pavel Tupitsyn
Yakov,

> 2. put op_code to the first place. This will make possible to eliminate
length field for many messages
This will require extensive refactoring of existing socket pipeline for
very little benefit: almost all messages are of variable length.

> 3. build date and revision hash to handshake
Please clarify, build version of what? Client or Ignite? Does it come in or
out?


Vladimir, Alexey,
Yes, we should definitely add multiplexing in future, but for now let's
make it work in a simple way.

As for requestId, do I understand correctly that client includes some ID in
the request,
and Ignite just copies it to the corresponding response message?

Pavel

On Wed, Aug 2, 2017 at 11:37 AM, Yakov Zhdanov <[hidden email]> wrote:

> Agree with Alex. I think our implementations should share single connection
> over threads in the process.
>
> --Yakov
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Vladimir Ozerov
Pavel,

I do not see how it can effect something existing, as nothing exists yet
:-) Let's put current SQL aside, we will merge them into protocol later.
The main point of length is to move request parsing and deserialization to
separate thread. Without it we will have to pre-process all requests in a
single thread, what can become a bottleneck. What Yakov suggests, is
similar to our communication architecture: we take request length, copy
request bytes and then push them to another thread for processing. Makes
sense to me.

As far as request ID, yes, this is just and unique ID to correlate
request(s) and response(s).

On Wed, Aug 2, 2017 at 11:49 AM, Pavel Tupitsyn <[hidden email]>
wrote:

> Yakov,
>
> > 2. put op_code to the first place. This will make possible to eliminate
> length field for many messages
> This will require extensive refactoring of existing socket pipeline for
> very little benefit: almost all messages are of variable length.
>
> > 3. build date and revision hash to handshake
> Please clarify, build version of what? Client or Ignite? Does it come in or
> out?
>
>
> Vladimir, Alexey,
> Yes, we should definitely add multiplexing in future, but for now let's
> make it work in a simple way.
>
> As for requestId, do I understand correctly that client includes some ID in
> the request,
> and Ignite just copies it to the corresponding response message?
>
> Pavel
>
> On Wed, Aug 2, 2017 at 11:37 AM, Yakov Zhdanov <[hidden email]>
> wrote:
>
> > Agree with Alex. I think our implementations should share single
> connection
> > over threads in the process.
> >
> > --Yakov
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

yzhdanov
In reply to this post by Pavel Tupitsyn
Pavel,

2. Disagree here. In your sample responses contain only success flag which
is byte and 4 bytes length. We can easily avoid that just sending
OP_CODE_SUCCESS or OP_CODE_FAILURE.
3. In and out where applicable.

--Yakov
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

Alexey Kuznetsov
Pavel,

I remember one thing that is very useful for thing client:
  Response should contains Node ID that send response to thin client.

This can be used for debugging needs.

On Wed, Aug 2, 2017 at 4:10 PM, Yakov Zhdanov <[hidden email]> wrote:

> Pavel,
>
> 2. Disagree here. In your sample responses contain only success flag which
> is byte and 4 bytes length. We can easily avoid that just sending
> OP_CODE_SUCCESS or OP_CODE_FAILURE.
> 3. In and out where applicable.
>
> --Yakov
>



--
Alexey Kuznetsov
Reply | Threaded
Open this post in threaded view
|

Re: Thin client protocol message format

yzhdanov
>  Response should contains Node ID that send response to thin client.

This should be set up in handshake and used for all requests made over that
connection.

--Yakov

2017-08-02 14:12 GMT+03:00 Alexey Kuznetsov <[hidden email]>:

> Pavel,
>
> I remember one thing that is very useful for thing client:
>   Response should contains Node ID that send response to thin client.
>
> This can be used for debugging needs.
>
> On Wed, Aug 2, 2017 at 4:10 PM, Yakov Zhdanov <[hidden email]> wrote:
>
> > Pavel,
> >
> > 2. Disagree here. In your sample responses contain only success flag
> which
> > is byte and 4 bytes length. We can easily avoid that just sending
> > OP_CODE_SUCCESS or OP_CODE_FAILURE.
> > 3. In and out where applicable.
> >
> > --Yakov
> >
>
>
>
> --
> Alexey Kuznetsov
>
12