You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2020/04/02 19:18:39 UTC

[GitHub] [ignite] isapego opened a new pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

isapego opened a new pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403951559
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
 ##########
 @@ -390,54 +397,62 @@ private static Socket createSocket(ClientChannelConfiguration cfg) throws IOExce
         return sock;
     }
 
-    /** Serialize String for thin client protocol. */
-    private static byte[] marshalString(String s) {
-        try (BinaryOutputStream out = new BinaryHeapOutputStream(s == null ? 1 : s.length() + 20);
-             BinaryRawWriterEx writer = new BinaryWriterExImpl(null, out, null, null)
-        ) {
-            writer.writeString(s);
-
-            return out.arrayCopy();
-        }
-    }
-
     /** Client handshake. */
-    private void handshake(String user, String pwd, Map<String, String> userAttrs)
+    private void handshake(ProtocolVersion ver, String user, String pwd, Map<String, String> userAttrs)
         throws ClientConnectionException, ClientAuthenticationException {
-        handshakeReq(user, pwd, userAttrs);
-        handshakeRes(user, pwd, userAttrs);
+        handshakeReq(ver, user, pwd, userAttrs);
+        handshakeRes(ver, user, pwd, userAttrs);
     }
 
     /** Send handshake request. */
-    private void handshakeReq(String user, String pwd, Map<String, String> userAttrs)
-        throws ClientConnectionException {
+    private void handshakeReq(ProtocolVersion proposedVer, String user, String pwd,
+        Map<String, String> userAttrs) throws ClientConnectionException {
         BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration(), null);
-        BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, new BinaryHeapOutputStream(32), null, null);
+        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, new BinaryHeapOutputStream(32), null, null)) {
 
 Review comment:
   Newline missed (codestyle)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403962584
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
 ##########
 @@ -446,39 +461,46 @@ private void handshakeRes(String user, String pwd, Map<String, String> userAttrs
 
         BinaryInputStream res = new BinaryHeapInputStream(dataInput.read(resSize));
 
-        try (BinaryReaderExImpl r = new BinaryReaderExImpl(null, res, null, true)) {
-            if (res.readBoolean()) { // Success flag.
-                if (ver.compareTo(V1_4_0) >= 0)
-                    srvNodeId = r.readUuid(); // Server node UUID.
-            }
-            else {
-                ProtocolVersion srvVer = new ProtocolVersion(res.readShort(), res.readShort(), res.readShort());
+        try (BinaryReaderExImpl reader = new BinaryReaderExImpl(null, res, null, true)) {
 
-                String err = r.readString();
+            boolean success = res.readBoolean();
 
 Review comment:
   Redundant newline (codestyle)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] ptupitsyn commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
ptupitsyn commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403118343
 
 

 ##########
 File path: modules/platforms/cpp/thin-client/src/impl/data_channel.h
 ##########
 @@ -59,10 +59,22 @@ namespace ignite
 
                 /** Version 1.2.0. */
                 static const ProtocolVersion VERSION_1_2_0;
+
+                /** Version 1.3.0. */
+                static const ProtocolVersion VERSION_1_3_0;
                 
                 /** Version 1.4.0. Added: Partition awareness support, IEP-23. */
                 static const ProtocolVersion VERSION_1_4_0;
 
+                /** Version 1.5.0. Transaction support. */
+                static const ProtocolVersion VERSION_1_5_0;
+
+                /** Version 1.6.0. Expiration Policy Configuration. */
+                static const ProtocolVersion VERSION_1_6_0;
+
+                /** Version 1.6.0. Features introduced. */
 
 Review comment:
   1.7.0

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r409859352
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientCache.java
 ##########
 @@ -520,9 +520,11 @@ private void writeCacheInfo(PayloadOutputChannel payloadCh) {
         TcpClientTransaction tx = transactions.tx();
 
         if (expiryPlc != null) {
-            if (payloadCh.clientChannel().serverVersion().compareTo(V1_6_0) < 0) {
-                throw new ClientProtocolError(String.format("Expire policies have not supported by the server " +
-                    "version %s, required version %s", payloadCh.clientChannel().serverVersion(), V1_6_0));
+            ProtocolContext protocolCtx = payloadCh.clientChannel().protocolCtx();
+
+            if (!protocolCtx.isFeatureSupported(ProtocolVersionFeature.EXPIRY_POLICY)) {
+                throw new ClientProtocolError(String.format("Expire policies are not supported by the server " +
+                    "version %s, required version %s", protocolCtx.version(), V1_6_0));
 
 Review comment:
   `V1_6_0` -> `ProtocolVersionFeature.EXPIRY_POLICY.verIntroduced()`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403962751
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
 ##########
 @@ -446,39 +461,46 @@ private void handshakeRes(String user, String pwd, Map<String, String> userAttrs
 
         BinaryInputStream res = new BinaryHeapInputStream(dataInput.read(resSize));
 
-        try (BinaryReaderExImpl r = new BinaryReaderExImpl(null, res, null, true)) {
-            if (res.readBoolean()) { // Success flag.
-                if (ver.compareTo(V1_4_0) >= 0)
-                    srvNodeId = r.readUuid(); // Server node UUID.
-            }
-            else {
-                ProtocolVersion srvVer = new ProtocolVersion(res.readShort(), res.readShort(), res.readShort());
+        try (BinaryReaderExImpl reader = new BinaryReaderExImpl(null, res, null, true)) {
 
-                String err = r.readString();
+            boolean success = res.readBoolean();
+            if (success) {
+                byte[] features = new byte[0];
+                if (ProtocolContext.isFeaturesSupported(proposedVer)) {
 
 Review comment:
   Newline missing, redundant braces (codestyle)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403960536
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
 ##########
 @@ -390,54 +397,62 @@ private static Socket createSocket(ClientChannelConfiguration cfg) throws IOExce
         return sock;
     }
 
-    /** Serialize String for thin client protocol. */
-    private static byte[] marshalString(String s) {
-        try (BinaryOutputStream out = new BinaryHeapOutputStream(s == null ? 1 : s.length() + 20);
-             BinaryRawWriterEx writer = new BinaryWriterExImpl(null, out, null, null)
-        ) {
-            writer.writeString(s);
-
-            return out.arrayCopy();
-        }
-    }
-
     /** Client handshake. */
-    private void handshake(String user, String pwd, Map<String, String> userAttrs)
+    private void handshake(ProtocolVersion ver, String user, String pwd, Map<String, String> userAttrs)
         throws ClientConnectionException, ClientAuthenticationException {
-        handshakeReq(user, pwd, userAttrs);
-        handshakeRes(user, pwd, userAttrs);
+        handshakeReq(ver, user, pwd, userAttrs);
+        handshakeRes(ver, user, pwd, userAttrs);
     }
 
     /** Send handshake request. */
-    private void handshakeReq(String user, String pwd, Map<String, String> userAttrs)
-        throws ClientConnectionException {
+    private void handshakeReq(ProtocolVersion proposedVer, String user, String pwd,
+        Map<String, String> userAttrs) throws ClientConnectionException {
         BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration(), null);
-        BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, new BinaryHeapOutputStream(32), null, null);
+        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, new BinaryHeapOutputStream(32), null, null)) {
+            ProtocolContext protocolContext = protocolContextFromVersion(proposedVer);
+
+            writer.writeInt(0); // reserve an integer for the request size
+            writer.writeByte((byte) ClientListenerRequest.HANDSHAKE);
+
+            writer.writeShort(proposedVer.major());
+            writer.writeShort(proposedVer.minor());
+            writer.writeShort(proposedVer.patch());
 
-        writer.writeInt(0); // reserve an integer for the request size
-        writer.writeByte((byte) ClientListenerRequest.HANDSHAKE);
+            writer.writeByte(ClientListenerNioListener.THIN_CLIENT);
 
-        writer.writeShort(ver.major());
-        writer.writeShort(ver.minor());
-        writer.writeShort(ver.patch());
+            if (protocolContext.isFeaturesSupported()) {
 
 Review comment:
   Already write it in the ticket comment: I think it's redundant to create a set of methods for each version, it does not add any readability, but quite the opposite brings complexity and confusion. Instead of just two lines of code (if version >= 1.7.0 then write supported features static array) there is protocol context created with underlying set, then some external method is invoked which dependent on version. Protocol context is thrown away when the method exits.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403929243
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java
 ##########
 @@ -139,9 +138,9 @@
      * Writes the cache configuration.
      * @param writer Writer.
      * @param cfg Configuration.
-     * @param ver Client version.
+     * @param protocolContext Client protocol context.
      */
-    static void write(BinaryRawWriterEx writer, CacheConfiguration cfg, ClientListenerProtocolVersion ver) {
+    static void write(BinaryRawWriterEx writer, CacheConfiguration cfg, ClientProtocolContext protocolContext) {
 
 Review comment:
   Abbreviation should be used for "context" (codestyle, in other classes too)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403935531
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientFeature.java
 ##########
 @@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client;
+
+import java.util.EnumSet;
+import org.apache.ignite.internal.ThinProtocolFeature;
+
+/**
+ * Defines supported features for thin client.
+ */
+public enum ClientFeature implements ThinProtocolFeature {
+    USER_ATTRIBUTES(0);
 
 Review comment:
   Comment is absent (codestyle)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403990148
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientProtocolContext.java
 ##########
 @@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.client;
+
+import java.util.EnumSet;
+import org.apache.ignite.internal.ThinProtocolFeature;
+import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
+
+import static org.apache.ignite.internal.processors.platform.client.ClientConnectionContext.*;
+
+/**
+ * Protocol context for JDBC protocol. Holds protocol version and supported features.
 
 Review comment:
   `JDBC` -> `thin client`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403986418
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/ThinProtocolFeature.java
 ##########
 @@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.EnumSet;
+
+/**
+ * The base feature class.
+ */
+public interface ThinProtocolFeature {
+    /**
+     * @return Feature ID.
+     */
+    int featureId();
+
+    /**
+     * @return Feature name.
+     */
+    String name();
+
+    /**
+     * @param features Feature set.
+     * @return Byte array representing all supported features by current node.
+     */
+    static <E extends Enum<E> & ThinProtocolFeature> byte[] featuresAsBytes(E[] features) {
 
 Review comment:
   The method is never used

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r409857251
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
 ##########
 @@ -366,8 +366,8 @@ void cacheConfiguration(ClientCacheConfiguration cfg, BinaryOutputStream out, Pr
                 });
             }
             else if (cfg.getExpiryPolicy() != null) {
-                throw new ClientProtocolError(String.format("Expire policies have not supported by the server " +
-                    "version %s, required version %s", ver, V1_6_0));
+                throw new ClientProtocolError(String.format("Expire policies are not supported by the server " +
+                    "version %s, required version %s", protocolCtx.version(), V1_6_0));
 
 Review comment:
   `V1_6_0` -> `ProtocolVersionFeature.EXPIRY_POLICY.verIntroduced()`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] ptupitsyn commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
ptupitsyn commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403113616
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
 ##########
 @@ -352,7 +351,7 @@ void cacheConfiguration(ClientCacheConfiguration cfg, BinaryOutputStream out, Pr
                 )
             );
 
-            if (ver.compareTo(V1_6_0) >= 0) {
+            if (protocolContext.isExpirationPolicySupported()) {
 
 Review comment:
   `ExpirationPolicy` -> `ExpiryPolicy`, I think that's the term we use in Ignite

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403988154
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientCache.java
 ##########
 @@ -520,9 +520,10 @@ private void writeCacheInfo(PayloadOutputChannel payloadCh) {
         TcpClientTransaction tx = transactions.tx();
 
         if (expiryPlc != null) {
-            if (payloadCh.clientChannel().serverVersion().compareTo(V1_6_0) < 0) {
-                throw new ClientProtocolError(String.format("Expire policies have not supported by the server " +
-                    "version %s, required version %s", payloadCh.clientChannel().serverVersion(), V1_6_0));
+            ProtocolContext protocolContext = payloadCh.clientChannel().protocolContext();
 
 Review comment:
   Newline missed (codestyle)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] ptupitsyn commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
ptupitsyn commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r409462084
 
 

 ##########
 File path: modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
 ##########
 @@ -229,12 +229,31 @@ string IQueryEntityInternal.GetAlias(string fieldName)
             return _aliasMap.TryGetValue(fieldName, out res) ? res : null;
         }
 
+        /// <summary>
+        /// Initializes a new instance of the <see cref="QueryEntity"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        internal QueryEntity(IBinaryRawReader reader)
+        {
+            Read(reader, ClientSocket.CurrentProtocolVersion);
+        }
+
         /// <summary>
         /// Initializes a new instance of the <see cref="QueryEntity"/> class.
         /// </summary>
         /// <param name="reader">The reader.</param>
         /// <param name="srvVer">Server version.</param>
         internal QueryEntity(IBinaryRawReader reader, ClientProtocolVersion srvVer)
+        {
+            Read(reader, srvVer);
+        }
+        
+        /// <summary>
+        /// Reads an instance of the <see cref="QueryEntity"/> class.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        /// <param name="srvVer">Server version.</param>
+        internal void Read(IBinaryRawReader reader, ClientProtocolVersion srvVer)
 
 Review comment:
   Ideally we should get rid of `ClientProtocolVersion` here and separate Thin and Thick logic completely. Thin client config logic should be in `ClientCacheConfigurationSerializer`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r409862650
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
 ##########
 @@ -446,39 +463,46 @@ private void handshakeRes(String user, String pwd, Map<String, String> userAttrs
 
         BinaryInputStream res = new BinaryHeapInputStream(dataInput.read(resSize));
 
-        try (BinaryReaderExImpl r = new BinaryReaderExImpl(null, res, null, true)) {
-            if (res.readBoolean()) { // Success flag.
-                if (ver.compareTo(V1_4_0) >= 0)
-                    srvNodeId = r.readUuid(); // Server node UUID.
-            }
-            else {
-                ProtocolVersion srvVer = new ProtocolVersion(res.readShort(), res.readShort(), res.readShort());
+        try (BinaryReaderExImpl reader = new BinaryReaderExImpl(null, res, null, true)) {
+            boolean success = res.readBoolean();
 
-                String err = r.readString();
+            if (success) {
+                byte[] features = new byte[0];
 
+                if (ProtocolContext.isFeatureSupported(proposedVer, ProtocolVersionFeature.BITMAP_FEATURES))
+                    features = reader.readByteArray();
+
+                protocolCtx = new ProtocolContext(proposedVer, ProtocolBitmaskFeature.enumSet(features));
+
+                if (protocolCtx.isFeatureSupported(ProtocolVersionFeature.PARTITION_AWARENESS)) {
+                    // Reading server UUID
+                    srvNodeId = reader.readUuid();
+                }
+            } else {
+                ProtocolVersion srvVer = new ProtocolVersion(res.readShort(), res.readShort(), res.readShort());
+
+                String err = reader.readString();
                 int errCode = ClientStatus.FAILED;
 
                 if (res.remaining() > 0)
-                    errCode = r.readInt();
+                    errCode = reader.readInt();
 
                 if (errCode == ClientStatus.AUTH_FAILED)
                     throw new ClientAuthenticationException(err);
-                else if (ver.equals(srvVer))
+                else if (proposedVer.equals(srvVer))
                     throw new ClientProtocolError(err);
                 else if (!supportedVers.contains(srvVer) ||
-                    (srvVer.compareTo(V1_1_0) < 0 && user != null && !user.isEmpty()))
+                    (srvVer.compareTo(V1_1_0) < 0 && !F.isEmpty(user)))
 
 Review comment:
   `srvVer.compareTo(V1_1_0) < 0` -> `!ProtocolContext.isFeatureSupported(srvVer, ProtocolVersionFeature.AUTHORIZATION)`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403961154
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
 ##########
 @@ -390,54 +397,62 @@ private static Socket createSocket(ClientChannelConfiguration cfg) throws IOExce
         return sock;
     }
 
-    /** Serialize String for thin client protocol. */
-    private static byte[] marshalString(String s) {
-        try (BinaryOutputStream out = new BinaryHeapOutputStream(s == null ? 1 : s.length() + 20);
-             BinaryRawWriterEx writer = new BinaryWriterExImpl(null, out, null, null)
-        ) {
-            writer.writeString(s);
-
-            return out.arrayCopy();
-        }
-    }
-
     /** Client handshake. */
-    private void handshake(String user, String pwd, Map<String, String> userAttrs)
+    private void handshake(ProtocolVersion ver, String user, String pwd, Map<String, String> userAttrs)
         throws ClientConnectionException, ClientAuthenticationException {
-        handshakeReq(user, pwd, userAttrs);
-        handshakeRes(user, pwd, userAttrs);
+        handshakeReq(ver, user, pwd, userAttrs);
+        handshakeRes(ver, user, pwd, userAttrs);
     }
 
     /** Send handshake request. */
-    private void handshakeReq(String user, String pwd, Map<String, String> userAttrs)
-        throws ClientConnectionException {
+    private void handshakeReq(ProtocolVersion proposedVer, String user, String pwd,
+        Map<String, String> userAttrs) throws ClientConnectionException {
         BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), new IgniteConfiguration(), null);
-        BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, new BinaryHeapOutputStream(32), null, null);
+        try (BinaryWriterExImpl writer = new BinaryWriterExImpl(ctx, new BinaryHeapOutputStream(32), null, null)) {
+            ProtocolContext protocolContext = protocolContextFromVersion(proposedVer);
+
+            writer.writeInt(0); // reserve an integer for the request size
+            writer.writeByte((byte) ClientListenerRequest.HANDSHAKE);
+
+            writer.writeShort(proposedVer.major());
+            writer.writeShort(proposedVer.minor());
+            writer.writeShort(proposedVer.patch());
 
-        writer.writeInt(0); // reserve an integer for the request size
-        writer.writeByte((byte) ClientListenerRequest.HANDSHAKE);
+            writer.writeByte(ClientListenerNioListener.THIN_CLIENT);
 
-        writer.writeShort(ver.major());
-        writer.writeShort(ver.minor());
-        writer.writeShort(ver.patch());
+            if (protocolContext.isFeaturesSupported()) {
+                byte[] features = ThinProtocolFeature.featuresAsBytes(protocolContext.features());
+                writer.writeByteArray(features);
+            }
+
+            if (protocolContext.isUserAttributesSupported())
+                writer.writeMap(userAttrs);
 
-        writer.writeByte(ClientListenerNioListener.THIN_CLIENT);
+            if (protocolContext.isAuthorizationSupported() && user != null && !user.isEmpty()) {
+                writer.writeString(user);
+                writer.writeString(pwd);
+            }
 
-        if (ver.compareTo(V1_7_0) >= 0)
-            writer.writeMap(userAttrs);
+            writer.out().writeInt(0, writer.out().position() - 4);// actual size
 
-        if (ver.compareTo(V1_1_0) >= 0 && user != null && !user.isEmpty()) {
-            writer.writeString(user);
-            writer.writeString(pwd);
+            write(writer.array(), writer.out().position());
         }
+    }
 
-        writer.out().writeInt(0, writer.out().position() - 4);// actual size
+    /**
+     * @param version Protocol version.
+     * @return Protocol context for a version.
+     */
+    private ProtocolContext protocolContextFromVersion(ProtocolVersion version) {
 
 Review comment:
   Abbreviation should be used for `version` (codestyle). But I think this method should ve removed at all (see the previous comment)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403933945
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
 ##########
 @@ -149,25 +157,37 @@ public ClientResourceRegistry resources() {
     }
 
     /**
-     * @return Currently used protocol version.
+     * @return Currently used protocol context.
      */
-    public ClientListenerProtocolVersion currentVersion() {
-        return currentVer;
+    public ClientProtocolContext currentProtocolContext() {
+        return currentProtocolContext;
     }
 
     /** {@inheritDoc} */
     @Override public void initializeFromHandshake(GridNioSession ses,
         ClientListenerProtocolVersion ver, BinaryReaderExImpl reader)
         throws IgniteCheckedException {
-        boolean hasMore;
+
+        EnumSet<ClientFeature> features = null;
+
+        if (ClientProtocolContext.isFeaturesSupported(ver)) {
+            byte [] cliFeatures = reader.readByteArray();
+
+            features = ClientFeature.enumSet(cliFeatures);
+
+            features.retainAll(ClientFeature.allFeaturesAsEnumSet());
 
 Review comment:
   It's redundant since the server will never get by 'enumSet' features which it doesn't know.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403940762
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/ProtocolVersion.java
 ##########
 @@ -31,6 +31,9 @@
     /** Protocol version: 1.4.0. Partition awareness. */
     public static final ProtocolVersion V1_4_0 = new ProtocolVersion((short)1, (short)4, (short)0);
 
+    /** Protocol version: 1.3.0. */
+    public static final ProtocolVersion V1_3_0 = new ProtocolVersion((short)1, (short)3, (short)0);
 
 Review comment:
   This version was introduced by mistake, we don't need it and new introduced method dependent on this version (which are never used) at all. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403928153
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
 ##########
 @@ -166,10 +165,10 @@ public static void writeDotNetConfiguration(BinaryRawWriterEx writer, PlatformDo
      * Reads cache configuration from a stream.
      *
      * @param in Stream.
-     * @param ver Client version.
+     * @param protocolContext Client protocol version.
      * @return Cache configuration.
      */
-    public static CacheConfiguration readCacheConfiguration(BinaryRawReaderEx in, ClientListenerProtocolVersion ver) {
+    public static CacheConfiguration readCacheConfiguration(BinaryRawReaderEx in, ClientProtocolContext protocolContext) {
 
 Review comment:
   As far as I understand, the protocol version was introduced to this method and almost all other methods in this class (except "query entity" methods) by mistake. It's effectively no-op for platform thick clients. I propose to remove redundant parameter from these methods instead of changing its type. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r403985871
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/ThinProtocolFeature.java
 ##########
 @@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.EnumSet;
+
+/**
+ * The base feature class.
+ */
+public interface ThinProtocolFeature {
+    /**
+     * @return Feature ID.
+     */
+    int featureId();
 
 Review comment:
   "public" modificator must be present on all interface methods (codestyle). But I think the whole interface should be removed (see my comment on ticket for details)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #7616: IGNITE-12853: Introduced features for the thin client protocol
URL: https://github.com/apache/ignite/pull/7616#discussion_r409864535
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientTransactions.java
 ##########
 @@ -88,13 +88,15 @@ private ClientTransaction txStart0(TransactionConcurrency concurrency, Transacti
         TcpClientTransaction tx0 = tx();
 
         if (tx0 != null)
-            throw new ClientException("A transaction has already started by the current thread.");
+            throw new ClientException("A transaction has already been started by the current thread.");
 
         tx0 = ch.service(ClientOperation.TX_START,
             req -> {
-                if (req.clientChannel().serverVersion().compareTo(V1_5_0) < 0) {
-                    throw new ClientProtocolError(String.format("Transactions have not supported by the server's " +
-                        "protocol version %s, required version %s", req.clientChannel().serverVersion(), V1_5_0));
+                ProtocolContext protocolCtx = req.clientChannel().protocolCtx();
+
+                if (!protocolCtx.isFeatureSupported(ProtocolVersionFeature.TRANSACTIONS)) {
+                    throw new ClientProtocolError(String.format("Transactions are not supported by the server's " +
+                        "protocol version %s, required version %s", protocolCtx.version(), V1_5_0));
 
 Review comment:
   `V1_5_0` -> `ProtocolVersionFeature.TRANSACTIONS.verIntroduced()`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services