You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2018/04/12 07:37:45 UTC

ignite git commit: IGNITE-8042: .NET thin client: authentication support. This closes #3790.

Repository: ignite
Updated Branches:
  refs/heads/master e333f306d -> 67023a88b


IGNITE-8042: .NET thin client: authentication support. This closes #3790.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/67023a88
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/67023a88
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/67023a88

Branch: refs/heads/master
Commit: 67023a88b8c9cf647196b6deb1759b4d01e33890
Parents: e333f30
Author: devozerov <vo...@gridgain.com>
Authored: Thu Apr 12 10:37:36 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Apr 12 10:37:36 2018 +0300

----------------------------------------------------------------------
 .../client/ClientAuthenticationException.java   |  16 +-
 .../internal/client/thin/TcpClientChannel.java  |   9 +-
 .../odbc/ClientListenerNioListener.java         |  19 +++
 .../client/ClientConnectionContext.java         |   8 +-
 .../platform/client/ClientStatus.java           |   5 +-
 .../Client/ClientConnectionTest.cs              | 161 +++++++++++++++++++
 .../Client/ClientStatusCode.cs                  |  12 +-
 .../Client/IgniteClientConfiguration.cs         |  13 ++
 .../IgniteClientConfigurationSection.xsd        |  10 ++
 .../Impl/Client/ClientProtocolVersion.cs        |  22 ++-
 .../Impl/Client/ClientSocket.cs                 |  85 +++++++++-
 11 files changed, 329 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
index 0c24db8..526690a 100644
--- a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
+++ b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
@@ -24,22 +24,10 @@ public class ClientAuthenticationException extends ClientException {
     /** Serial version uid. */
     private static final long serialVersionUID = 0L;
 
-    /** Message. */
-    private static final String MSG = "Invalid user name or password";
-
     /**
      * Default constructor.
      */
-    public ClientAuthenticationException() {
-        super(MSG);
-    }
-
-    /**
-     * Constructs a new exception with the specified cause.
-     *
-     * @param cause the cause.
-     */
-    public ClientAuthenticationException(Throwable cause) {
-        super(MSG, cause);
+    public ClientAuthenticationException(String msg) {
+        super(msg);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
index 8e8294f..10dc865 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
@@ -272,8 +272,13 @@ class TcpClientChannel implements ClientChannel {
             try (BinaryReaderExImpl r = new BinaryReaderExImpl(null, res, null, true)) {
                 String err = r.readString();
 
-                if (err != null && err.toUpperCase().matches(".*USER.*INCORRECT.*"))
-                    throw new ClientAuthenticationException();
+                int errCode = ClientStatus.FAILED;
+
+                if (res.remaining() > 0)
+                    errCode = r.readInt();
+
+                if (errCode == ClientStatus.AUTH_FAILED)
+                    throw new ClientAuthenticationException(err);
                 else if (ver.equals(srvVer))
                     throw new ClientProtocolError(err);
                 else if (!supportedVers.contains(srvVer) ||

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
index 53b14d7..407c1a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
@@ -26,9 +26,11 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
+import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext;
 import org.apache.ignite.internal.processors.odbc.odbc.OdbcConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter;
 import org.apache.ignite.internal.util.nio.GridNioSession;
@@ -228,6 +230,18 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<byte
 
             connCtx.handler().writeHandshake(writer);
         }
+        catch (IgniteAccessControlException authEx) {
+            writer.writeBoolean(false);
+
+            writer.writeShort((short)0);
+            writer.writeShort((short)0);
+            writer.writeShort((short)0);
+
+            writer.doWriteString(authEx.getMessage());
+
+            if (ver.compareTo(ClientConnectionContext.VER_1_1_0) >= 0)
+                writer.writeInt(ClientStatus.AUTH_FAILED);
+        }
         catch (IgniteCheckedException e) {
             U.warn(log, "Error during handshake [rmtAddr=" + ses.remoteAddress() + ", msg=" + e.getMessage() + ']');
 
@@ -239,10 +253,15 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<byte
                 currVer = connCtx.currentVersion();
 
             writer.writeBoolean(false);
+
             writer.writeShort(currVer.major());
             writer.writeShort(currVer.minor());
             writer.writeShort(currVer.maintenance());
+
             writer.doWriteString(e.getMessage());
+
+            if (ver.compareTo(ClientConnectionContext.VER_1_1_0) >= 0)
+                writer.writeInt(ClientStatus.FAILED);
         }
 
         ses.send(writer.array());

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
index 061aab3..056ea83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
@@ -44,10 +44,10 @@ import static org.apache.ignite.plugin.security.SecuritySubjectType.REMOTE_CLIEN
  */
 public class ClientConnectionContext implements ClientListenerConnectionContext {
     /** Version 1.0.0. */
-    private static final ClientListenerProtocolVersion VER_1_0_0 = ClientListenerProtocolVersion.create(1, 0, 0);
+    public static final ClientListenerProtocolVersion VER_1_0_0 = ClientListenerProtocolVersion.create(1, 0, 0);
 
     /** Version 1.1.0. */
-    private static final ClientListenerProtocolVersion VER_1_1_0 = ClientListenerProtocolVersion.create(1, 1, 0);
+    public static final ClientListenerProtocolVersion VER_1_1_0 = ClientListenerProtocolVersion.create(1, 1, 0);
 
     /** Supported versions. */
     private static final Collection<ClientListenerProtocolVersion> SUPPORTED_VERS = Arrays.asList(VER_1_1_0, VER_1_0_0);
@@ -144,12 +144,12 @@ public class ClientConnectionContext implements ClientListenerConnectionContext
             authCtx = thirdPartyAuthentication(user, pwd).authorizationContext();
         else if (kernalCtx.authentication().enabled()) {
             if (user == null || user.length() == 0)
-                throw new IgniteCheckedException("Unauthenticated sessions are prohibited.");
+                throw new IgniteAccessControlException("Unauthenticated sessions are prohibited.");
 
             authCtx = kernalCtx.authentication().authenticate(user, pwd);
 
             if (authCtx == null)
-                throw new IgniteCheckedException("Unknown authentication error.");
+                throw new IgniteAccessControlException("Unknown authentication error.");
         }
 
         handler = new ClientRequestHandler(this, authCtx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
index b8dfb1f..e63812c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
@@ -49,6 +49,9 @@ public final class ClientStatus {
     /** Resource does not exist. */
     public static final int RESOURCE_DOES_NOT_EXIST = 1011;
 
-    /** Resource does not exist. */
+    /** Authorization failure. */
     public static final int SECURITY_VIOLATION = 1012;
+
+    /** Authentication failed. */
+    public static final int AUTH_FAILED = 2000;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
index 9da9a03..2ea17a8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
@@ -19,13 +19,17 @@ namespace Apache.Ignite.Core.Tests.Client
 {
     using System;
     using System.Collections.Generic;
+    using System.IO;
     using System.Linq;
     using System.Net;
     using System.Net.Sockets;
     using System.Text.RegularExpressions;
     using System.Threading;
     using System.Threading.Tasks;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.Core.Client;
+    using Apache.Ignite.Core.Client.Cache;
     using Apache.Ignite.Core.Configuration;
     using NUnit.Framework;
 
@@ -34,6 +38,18 @@ namespace Apache.Ignite.Core.Tests.Client
     /// </summary>
     public class ClientConnectionTest
     {
+        /** Temp dir for WAL. */
+        private readonly string _tempDir = TestUtils.GetTempDirectoryName();
+
+        /// <summary>
+        /// Sets up the test.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            TestUtils.ClearWorkDir();
+        }
+
         /// <summary>
         /// Fixture tear down.
         /// </summary>
@@ -41,6 +57,13 @@ namespace Apache.Ignite.Core.Tests.Client
         public void TearDown()
         {
             Ignition.StopAll(true);
+
+            if (Directory.Exists(_tempDir))
+            {
+                Directory.Delete(_tempDir, true);
+            }
+
+            TestUtils.ClearWorkDir();
         }
 
         /// <summary>
@@ -55,6 +78,107 @@ namespace Apache.Ignite.Core.Tests.Client
         }
 
         /// <summary>
+        /// Tests that empty username or password are not allowed.
+        /// </summary>
+        [Test]
+        public void TestAuthenticationEmptyCredentials()
+        {
+            using (Ignition.Start(SecureServerConfig()))
+            {
+                var cliCfg = SecureClientConfig();
+
+                cliCfg.Password = null;
+                var ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Password cannot be null"));
+
+                cliCfg.Password = "";
+                ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Password cannot be empty"));
+
+                cliCfg.Password = "ignite";
+
+                cliCfg.Username = null;
+                ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be null"));
+
+                cliCfg.Username = "";
+                ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be empty"));
+            }
+        }
+
+        /// <summary>
+        /// Test invalid username or password.
+        /// </summary>
+        [Test]
+        public void TestAuthenticationInvalidCredentials()
+        {
+            using (Ignition.Start(SecureServerConfig()))
+            {
+                var cliCfg = SecureClientConfig();
+
+                cliCfg.Username = "invalid";
+
+                var ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.True(ex.StatusCode == ClientStatusCode.AuthenticationFailed);
+
+                cliCfg.Username = "ignite";
+                cliCfg.Password = "invalid";
+
+                ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.True(ex.StatusCode == ClientStatusCode.AuthenticationFailed);
+            }
+        }
+
+        /// <summary>
+        /// Test authentication.
+        /// </summary>
+        [Test]
+        public void TestAuthentication()
+        {
+            using (var srv = Ignition.Start(SecureServerConfig()))
+            {
+                srv.GetCluster().SetActive(true);
+
+                using (var cli = Ignition.StartClient(SecureClientConfig()))
+                {
+                    CacheClientConfiguration ccfg = new CacheClientConfiguration()
+                    {
+                        Name = "TestCache",
+                        QueryEntities = new[]
+                        {
+                            new QueryEntity
+                            {
+                                KeyType = typeof(string),
+                                ValueType = typeof(string),
+                            },
+                        },
+                    };
+
+                    ICacheClient<string, string> cache = cli.GetOrCreateCache<string, string>(ccfg);
+
+                    cache.Put("key1", "val1");
+
+                    cache.Query(new SqlFieldsQuery("CREATE USER \"my_User\" WITH PASSWORD 'my_Password'")).GetAll();
+                }
+
+                var cliCfg = SecureClientConfig();
+
+                cliCfg.Username = "my_User";
+                cliCfg.Password = "my_Password";
+
+                using (var cli = Ignition.StartClient(cliCfg))
+                {
+                    ICacheClient<string, string> cache = cli.GetCache<string, string>("TestCache");
+
+                    string val = cache.Get("key1");
+
+                    Assert.True(val == "val1");
+                }
+            }
+        }
+
+        /// <summary>
         /// Tests that multiple clients can connect to one server.
         /// </summary>
         [Test]
@@ -374,5 +498,42 @@ namespace Apache.Ignite.Core.Tests.Client
             
             throw new Exception("SocketException not found.", origEx);
         }
+
+        /// <summary>
+        /// Create server configuration with enabled authentication.
+        /// </summary>
+        /// <returns>Server configuration.</returns>
+        private IgniteConfiguration SecureServerConfig()
+        {
+            return new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                AuthenticationEnabled = true,
+                DataStorageConfiguration = new DataStorageConfiguration()
+                {
+                    StoragePath = Path.Combine(_tempDir, "Store"),
+                    WalPath = Path.Combine(_tempDir, "WalStore"),
+                    WalArchivePath = Path.Combine(_tempDir, "WalArchive"),
+                    DefaultDataRegionConfiguration = new DataRegionConfiguration()
+                    {
+                        Name = "default",
+                        PersistenceEnabled = true
+                    }
+                }
+            };
+        }
+
+        /// <summary>
+        /// Create client configuration with enabled authentication.
+        /// </summary>
+        /// <returns>Client configuration.</returns>
+        private static IgniteClientConfiguration SecureClientConfig()
+        {
+            return new IgniteClientConfiguration()
+            {
+                Host = "localhost",
+                Username = "ignite",
+                Password = "ignite"
+            };
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs
index 3f5ee8e..3bdd9e1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs
@@ -52,6 +52,16 @@ namespace Apache.Ignite.Core.Client
         /// <summary>
         /// The too many cursors (see <see cref="ClientConnectorConfiguration.MaxOpenCursorsPerConnection"/>).
         /// </summary>
-        TooManyCursors = 1010
+        TooManyCursors = 1010,
+
+        /// <summary>
+        /// Authorization failure.
+        /// </summary>
+        SecurityViolation = 1012,
+
+        /// <summary>
+        /// Authentication failed.
+        /// </summary>
+        AuthenticationFailed = 2000
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
index 8730f39..3252495 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
@@ -90,6 +90,9 @@ namespace Apache.Ignite.Core.Client
 
             BinaryProcessor = cfg.BinaryProcessor;
             SslStreamFactory = cfg.SslStreamFactory;
+
+            Username = cfg.Username;
+            Password = cfg.Password;
         }
 
         /// <summary>
@@ -146,6 +149,16 @@ namespace Apache.Ignite.Core.Client
         public ISslStreamFactory SslStreamFactory { get; set; }
 
         /// <summary>
+        /// Username to be used to connect to secured cluster.
+        /// </summary>
+        public string Username { get; set; }
+
+        /// <summary>
+        /// Password to be used to connect to secured cluster.
+        /// </summary>
+        public string Password { get; set; }
+
+        /// <summary>
         /// Gets or sets custom binary processor. Internal property for tests.
         /// </summary>
         internal IBinaryProcessor BinaryProcessor { get; set; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
index 569ee6f..7e6caff 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
@@ -237,6 +237,16 @@
                     <xs:documentation>Socket operation timeout. Zero or negative for infinite timeout.</xs:documentation>
                 </xs:annotation>
             </xs:attribute>
+            <xs:attribute name="username" type="xs:string">
+                <xs:annotation>
+                    <xs:documentation>Username to be used to connect to secured cluster.</xs:documentation>
+                </xs:annotation>
+            </xs:attribute>
+            <xs:attribute name="password" type="xs:string">
+                <xs:annotation>
+                    <xs:documentation>Password to be used to connect to secured cluster.</xs:documentation>
+                </xs:annotation>
+            </xs:attribute>
         </xs:complexType>
     </xs:element>
 </xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs
index bfdf5a3..4fe5c71 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs
@@ -22,7 +22,7 @@ namespace Apache.Ignite.Core.Impl.Client
     /// <summary>
     /// Client protocol version.
     /// </summary>
-    internal struct ClientProtocolVersion : IEquatable<ClientProtocolVersion>
+    internal struct ClientProtocolVersion : IEquatable<ClientProtocolVersion>, IComparable<ClientProtocolVersion>
     {
         /** */
         private readonly short _major;
@@ -68,6 +68,26 @@ namespace Apache.Ignite.Core.Impl.Client
         }
 
         /// <summary>
+        /// Compare this version to other version.
+        /// </summary>
+        /// <param name="other"></param>
+        /// <returns></returns>
+        public int CompareTo(ClientProtocolVersion other)
+        {
+            int res = Major - other.Major;
+
+            if (res == 0)
+            {
+                res = Minor - other.Minor;
+
+                if (res == 0)
+                    res = Maintenance - other.Maintenance;
+            }
+
+            return res;
+        }
+
+        /// <summary>
         /// Returns a value indicating whether specified instance equals to current.
         /// </summary>
         public bool Equals(ClientProtocolVersion other)

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
index fca5dab..27d8f0b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
@@ -28,6 +28,7 @@ namespace Apache.Ignite.Core.Impl.Client
     using System.Net.Sockets;
     using System.Threading;
     using System.Threading.Tasks;
+    using System.Xml.Schema;
     using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary;
@@ -38,8 +39,14 @@ namespace Apache.Ignite.Core.Impl.Client
     /// </summary>
     internal sealed class ClientSocket : IDisposable
     {
+        /** Version 1.0.0. */
+        private static readonly ClientProtocolVersion Ver100 = new ClientProtocolVersion(1, 0, 0);
+
+        /** Version 1.1.0. */
+        private static readonly ClientProtocolVersion Ver110 = new ClientProtocolVersion(1, 1, 0);
+
         /** Current version. */
-        private static readonly ClientProtocolVersion CurrentProtocolVersion = new ClientProtocolVersion(1, 0, 0);
+        private static readonly ClientProtocolVersion CurrentProtocolVersion = Ver110;
 
         /** Handshake opcode. */
         private const byte OpHandshake = 1;
@@ -98,7 +105,9 @@ namespace Apache.Ignite.Core.Impl.Client
             _socket = Connect(clientConfiguration);
             _stream = GetSocketStream(_socket, clientConfiguration);
 
-            Handshake(version ?? CurrentProtocolVersion);
+            Validate(clientConfiguration);
+
+            Handshake(clientConfiguration, version ?? CurrentProtocolVersion);
 
             // Check periodically if any request has timed out.
             if (_timeout > TimeSpan.Zero)
@@ -112,6 +121,31 @@ namespace Apache.Ignite.Core.Impl.Client
         }
 
         /// <summary>
+        /// Validate configuration.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        private void Validate(IgniteClientConfiguration cfg)
+        {
+            if (cfg.Username != null)
+            {
+                if (cfg.Username.Length == 0)
+                    throw new IgniteClientException("IgniteClientConfiguration.Username cannot be empty.");
+
+                if (cfg.Password == null)
+                    throw new IgniteClientException("IgniteClientConfiguration.Password cannot be null when Username is set.");
+            }
+
+            if (cfg.Password != null)
+            {
+                if (cfg.Password.Length == 0)
+                    throw new IgniteClientException("IgniteClientConfiguration.Password cannot be empty.");
+
+                if (cfg.Username == null)
+                    throw new IgniteClientException("IgniteClientConfiguration.Username cannot be null when Password is set.");
+            }
+        }
+
+        /// <summary>
         /// Performs a send-receive operation.
         /// </summary>
         public T DoOutInOp<T>(ClientOp opId, Action<IBinaryStream> writeAction,
@@ -226,8 +260,10 @@ namespace Apache.Ignite.Core.Impl.Client
         /// <summary>
         /// Performs client protocol handshake.
         /// </summary>
-        private void Handshake(ClientProtocolVersion version)
+        private void Handshake(IgniteClientConfiguration clientConfiguration, ClientProtocolVersion version)
         {
+            bool auth = version.CompareTo(Ver110) >= 0 && clientConfiguration.Username != null;
+
             // Send request.
             int messageLen;
             var buf = WriteMessage(stream =>
@@ -242,10 +278,19 @@ namespace Apache.Ignite.Core.Impl.Client
 
                 // Client type: platform.
                 stream.WriteByte(ClientType);
-            }, 12, out messageLen);
 
-            Debug.Assert(messageLen == 12);
+                // Authentication data.
+                if (auth)
+                {
+                    var writer = BinaryUtils.Marshaller.StartMarshal(stream);
+
+                    writer.WriteString(clientConfiguration.Username);
+                    writer.WriteString(clientConfiguration.Password);
 
+                    BinaryUtils.Marshaller.FinishMarshal(writer);
+                }
+            }, 12, out messageLen);
+            
             _stream.Write(buf, 0, messageLen);
 
             // Decode response.
@@ -253,6 +298,7 @@ namespace Apache.Ignite.Core.Impl.Client
 
             using (var stream = new BinaryHeapStream(res))
             {
+                // Read input.
                 var success = stream.ReadBool();
 
                 if (success)
@@ -265,9 +311,32 @@ namespace Apache.Ignite.Core.Impl.Client
 
                 var errMsg = BinaryUtils.Marshaller.Unmarshal<string>(stream);
 
-                throw new IgniteClientException(string.Format(
-                    "Client handshake failed: '{0}'. Client version: {1}. Server version: {2}",
-                    errMsg, version, serverVersion));
+                ClientStatusCode errCode = ClientStatusCode.Fail;
+
+                if (stream.Remaining > 0)
+                {
+                    errCode = (ClientStatusCode) stream.ReadInt();
+                }
+
+                // Authentication error is handled immediately.
+                if (errCode == ClientStatusCode.AuthenticationFailed)
+                {
+                    throw new IgniteClientException(errMsg, null, ClientStatusCode.AuthenticationFailed);
+                }
+
+                // Re-try if possible.
+                bool retry = serverVersion.CompareTo(version) < 0 && serverVersion.Equals(Ver100);
+
+                if (retry)
+                {
+                    Handshake(clientConfiguration, serverVersion);
+                }
+                else
+                {
+                    throw new IgniteClientException(string.Format(
+                        "Client handshake failed: '{0}'. Client version: {1}. Server version: {2}",
+                        errMsg, version, serverVersion), null, errCode);
+                }
             }
         }