http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
index 99ee2cd..800ed13 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
@@ -22,8 +22,8 @@ namespace Apache.Ignite.Core.Impl.Events
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.Linq;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Cluster;
-    using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Handle;
@@ -36,18 +36,19 @@ namespace Apache.Ignite.Core.Impl.Events
     /// <summary>
     /// Ignite events.
     /// </summary>
-    internal class Events : PlatformTarget, IEvents
+    internal sealed class Events : PlatformTarget, IEvents
     {
         /// <summary>
         /// Opcodes.
         /// </summary>
-        protected enum Op
+        private enum Op
         {
             RemoteQuery = 1,
             RemoteListen = 2,
             StopRemoteListen = 3,
             WaitForLocal = 4,
             LocalQuery = 5,
+            // ReSharper disable once UnusedMember.Local
             RecordLocal = 6,
             EnableLocal = 8,
             DisableLocal = 9,
@@ -58,54 +59,59 @@ namespace Apache.Ignite.Core.Impl.Events
         private readonly Dictionary<object, Dictionary<int, 
LocalHandledEventFilter>> _localFilters
             = new Dictionary<object, Dictionary<int, 
LocalHandledEventFilter>>();
 
-        /** Grid. */
-        protected readonly Ignite Ignite;
+        /** Cluster group. */
+        private readonly IClusterGroup _clusterGroup;
+        
+        /** Async instance. */
+        private readonly Lazy<Events> _asyncInstance;
 
         /// <summary>
-        /// Initializes a new instance of the <see cref="Events"/> class.
+        /// Initializes a new instance of the <see cref="Events" /> class.
         /// </summary>
         /// <param name="target">Target.</param>
         /// <param name="marsh">Marshaller.</param>
         /// <param name="clusterGroup">Cluster group.</param>
-        public Events(IUnmanagedTarget target, PortableMarshaller marsh, 
IClusterGroup clusterGroup)
+        public Events(IUnmanagedTarget target, PortableMarshaller marsh, 
IClusterGroup clusterGroup) 
             : base(target, marsh)
         {
             Debug.Assert(clusterGroup != null);
 
-            ClusterGroup = clusterGroup;
+            _clusterGroup = clusterGroup;
 
-            Ignite = (Ignite) clusterGroup.Ignite;
+            _asyncInstance = new Lazy<Events>(() => new Events(this));
         }
 
-        /** <inheritDoc /> */
-        public virtual IEvents WithAsync()
+        /// <summary>
+        /// Initializes a new async instance.
+        /// </summary>
+        /// <param name="events">The events.</param>
+        private Events(Events events) : 
base(UU.EventsWithAsync(events.Target), events.Marshaller)
         {
-            return new EventsAsync(UU.EventsWithAsync(Target), Marshaller, 
ClusterGroup);
+            _clusterGroup = events.ClusterGroup;
         }
 
         /** <inheritDoc /> */
-        public virtual bool IsAsync
+        public IClusterGroup ClusterGroup
         {
-            get { return false; }
+            get { return _clusterGroup; }
         }
 
-        /** <inheritDoc /> */
-        public virtual IFuture GetFuture()
+        /** */
+        private Ignite Ignite
         {
-            throw IgniteUtils.GetAsyncModeDisabledException();
+            get { return (Ignite) ClusterGroup.Ignite; }
         }
 
-        /** <inheritDoc /> */
-        public virtual IFuture<TResult> GetFuture<TResult>()
+        /// <summary>
+        /// Gets the asynchronous instance.
+        /// </summary>
+        private Events AsyncInstance
         {
-            throw IgniteUtils.GetAsyncModeDisabledException();
+            get { return _asyncInstance.Value; }
         }
 
         /** <inheritDoc /> */
-        public IClusterGroup ClusterGroup { get; private set; }
-
-        /** <inheritDoc /> */
-        public virtual ICollection<T> RemoteQuery<T>(IEventFilter<T> filter, 
TimeSpan? timeout = null, params int[] types)
+        public ICollection<T> RemoteQuery<T>(IEventFilter<T> filter, TimeSpan? 
timeout = null, params int[] types)
             where T : IEvent
         {
             IgniteArgumentCheck.NotNull(filter, "filter");
@@ -123,6 +129,16 @@ namespace Apache.Ignite.Core.Impl.Events
         }
 
         /** <inheritDoc /> */
+        public Task<ICollection<T>> RemoteQueryAsync<T>(IEventFilter<T> 
filter, TimeSpan? timeout = null, 
+            params int[] types) where T : IEvent
+        {
+            AsyncInstance.RemoteQuery(filter, timeout, types);
+
+            return GetFuture((futId, futTyp) => 
UU.TargetListenFutureForOperation(AsyncInstance.Target, futId, futTyp,
+                (int) Op.RemoteQuery), convertFunc: ReadEvents<T>).Task;
+        }
+
+        /** <inheritDoc /> */
         public ICollection<T> RemoteQuery<T>(IEventFilter<T> filter, TimeSpan? 
timeout = null, 
             IEnumerable<int> types = null) where T : IEvent
         {
@@ -130,7 +146,14 @@ namespace Apache.Ignite.Core.Impl.Events
         }
 
         /** <inheritDoc /> */
-        public virtual Guid? RemoteListen<T>(int bufSize = 1, TimeSpan? 
interval = null, bool autoUnsubscribe = true,
+        public Task<ICollection<T>> RemoteQueryAsync<T>(IEventFilter<T> 
filter, TimeSpan? timeout = null, 
+            IEnumerable<int> types = null) where T : IEvent
+        {
+            return RemoteQueryAsync(filter, timeout, TypesToArray(types));
+        }
+
+        /** <inheritDoc /> */
+        public Guid? RemoteListen<T>(int bufSize = 1, TimeSpan? interval = 
null, bool autoUnsubscribe = true,
             IEventFilter<T> localListener = null, IEventFilter<T> remoteFilter 
= null, params int[] types)
             where T : IEvent
         {
@@ -171,7 +194,7 @@ namespace Apache.Ignite.Core.Impl.Events
         }
 
         /** <inheritDoc /> */
-        public virtual void StopRemoteListen(Guid opId)
+        public void StopRemoteListen(Guid opId)
         {
             DoOutOp((int) Op.StopRemoteListen, writer =>
             {
@@ -186,13 +209,25 @@ namespace Apache.Ignite.Core.Impl.Events
         }
 
         /** <inheritDoc /> */
+        public Task<IEvent> WaitForLocalAsync(params int[] types)
+        {
+            return WaitForLocalAsync<IEvent>(null, types);
+        }
+
+        /** <inheritDoc /> */
         public IEvent WaitForLocal(IEnumerable<int> types)
         {
             return WaitForLocal(TypesToArray(types));
         }
 
         /** <inheritDoc /> */
-        public virtual T WaitForLocal<T>(IEventFilter<T> filter, params int[] 
types) where T : IEvent
+        public Task<IEvent> WaitForLocalAsync(IEnumerable<int> types)
+        {
+            return WaitForLocalAsync<IEvent>(null, TypesToArray(types));
+        }
+
+        /** <inheritDoc /> */
+        public T WaitForLocal<T>(IEventFilter<T> filter, params int[] types) 
where T : IEvent
         {
             long hnd = 0;
 
@@ -208,12 +243,46 @@ namespace Apache.Ignite.Core.Impl.Events
         }
 
         /** <inheritDoc /> */
+        public Task<T> WaitForLocalAsync<T>(IEventFilter<T> filter, params 
int[] types) where T : IEvent
+        {
+            long hnd = 0;
+
+            try
+            {
+                AsyncInstance.WaitForLocal0(filter, ref hnd, types);
+
+                var fut = GetFuture((futId, futTyp) => 
UU.TargetListenFutureForOperation(AsyncInstance.Target, futId,
+                    futTyp, (int) Op.WaitForLocal), convertFunc: reader => (T) 
EventReader.Read<IEvent>(reader));
+
+                if (filter != null)
+                {
+                    // Dispose handle as soon as future ends.
+                    fut.Task.ContinueWith(x => 
Ignite.HandleRegistry.Release(hnd));
+                }
+
+                return fut.Task;
+            }
+            catch (Exception)
+            {
+                Ignite.HandleRegistry.Release(hnd);
+                throw;
+            }
+
+        }
+
+        /** <inheritDoc /> */
         public T WaitForLocal<T>(IEventFilter<T> filter, IEnumerable<int> 
types) where T : IEvent
         {
             return WaitForLocal(filter, TypesToArray(types));
         }
 
         /** <inheritDoc /> */
+        public Task<T> WaitForLocalAsync<T>(IEventFilter<T> filter, 
IEnumerable<int> types) where T : IEvent
+        {
+            return WaitForLocalAsync(filter, TypesToArray(types));
+        }
+
+        /** <inheritDoc /> */
         public ICollection<IEvent> LocalQuery(params int[] types)
         {
             return DoOutInOp((int) Op.LocalQuery,
@@ -325,7 +394,7 @@ namespace Apache.Ignite.Core.Impl.Events
         /// <param name="types">Types of the events to wait for. 
         /// If not provided, all events will be passed to the filter.</param>
         /// <returns>Ignite event.</returns>
-        protected T WaitForLocal0<T>(IEventFilter<T> filter, ref long handle, 
params int[] types) where T : IEvent
+        private T WaitForLocal0<T>(IEventFilter<T> filter, ref long handle, 
params int[] types) where T : IEvent
         {
             if (filter != null)
                 handle = Ignite.HandleRegistry.Allocate(new LocalEventFilter
@@ -357,7 +426,7 @@ namespace Apache.Ignite.Core.Impl.Events
         /// <typeparam name="T">Event type.</typeparam>
         /// <param name="reader">Reader.</param>
         /// <returns>Resulting list or null.</returns>
-        private List<T> ReadEvents<T>(IPortableStream reader) where T : IEvent
+        private ICollection<T> ReadEvents<T>(IPortableStream reader) where T : 
IEvent
         {
             return ReadEvents<T>(Marshaller.StartUnmarshal(reader));
         }
@@ -368,7 +437,7 @@ namespace Apache.Ignite.Core.Impl.Events
         /// <typeparam name="T">Event type.</typeparam>
         /// <param name="portableReader">Reader.</param>
         /// <returns>Resulting list or null.</returns>
-        protected static List<T> ReadEvents<T>(PortableReaderImpl 
portableReader) where T : IEvent
+        private static ICollection<T> ReadEvents<T>(PortableReaderImpl 
portableReader) where T : IEvent
         {
             var count = portableReader.GetRawReader().ReadInt();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventsAsync.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventsAsync.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventsAsync.cs
deleted file mode 100644
index 461f2ae..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/EventsAsync.cs
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * 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.
- */
-
-namespace Apache.Ignite.Core.Impl.Events
-{
-    using System;
-    using System.Collections.Generic;
-    using System.Diagnostics.CodeAnalysis;
-    using System.Threading;
-    using Apache.Ignite.Core.Cluster;
-    using Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Events;
-    using Apache.Ignite.Core.Impl.Portable;
-    using Apache.Ignite.Core.Impl.Unmanaged;
-    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
-
-    /// <summary>
-    /// Async Ignite events.
-    /// </summary>
-    [SuppressMessage("Microsoft.Design", 
"CA1001:TypesThatOwnDisposableFieldsShouldBeDisposable")]
-    internal class EventsAsync : Events
-    {
-        /** */
-        private readonly ThreadLocal<int> _lastAsyncOp = new 
ThreadLocal<int>(() => OpNone);
-
-        /** */
-        private readonly ThreadLocal<IFuture> _curFut = new 
ThreadLocal<IFuture>();
-
-        /// <summary>
-        /// Initializes a new instance of the <see cref="Events"/> class.
-        /// </summary>
-        /// <param name="target">Target.</param>
-        /// <param name="marsh">Marshaller.</param>
-        /// <param name="clusterGroup">Cluster group.</param>
-        public EventsAsync(IUnmanagedTarget target, PortableMarshaller marsh, 
IClusterGroup clusterGroup)
-            : base(target, marsh, clusterGroup)
-        {
-            // No-op.
-        }
-
-        /** <inheritdoc /> */
-        public override ICollection<T> RemoteQuery<T>(IEventFilter<T> filter, 
TimeSpan? timeout = null,
-            params int[] types)
-        {
-            _lastAsyncOp.Value = (int) Op.RemoteQuery;
-
-            var result = base.RemoteQuery(filter, timeout, types);
-
-            // Result is a List<T> so we can't create proper converter later 
in GetFuture call from user.
-            // ReSharper disable once RedundantTypeArgumentsOfMethod 
(otherwise won't compile in VS2010 / TC)
-            _curFut.Value = GetFuture<List<T>>((futId, futTyp) => 
UU.TargetListenFutureForOperation(Target, futId, futTyp,
-                (int) Op.RemoteQuery), convertFunc: ReadEvents<T>);
-
-            return result;
-        }
-
-        /** <inheritdoc /> */
-        public override Guid? RemoteListen<T>(int bufSize = 1, TimeSpan? 
interval = null, bool autoUnsubscribe = true,
-            IEventFilter<T> localListener = null, IEventFilter<T> remoteFilter 
= null, params int[] types)
-        {
-            _lastAsyncOp.Value = (int) Op.RemoteListen;
-            _curFut.Value = null;
-
-            return base.RemoteListen(bufSize, interval, autoUnsubscribe, 
localListener, remoteFilter, types);
-        }
-
-        /** <inheritdoc /> */
-        public override void StopRemoteListen(Guid opId)
-        {
-            _lastAsyncOp.Value = (int) Op.StopRemoteListen;
-            _curFut.Value = null;
-
-            base.StopRemoteListen(opId);
-        }
-
-        /** <inheritdoc /> */
-        public override T WaitForLocal<T>(IEventFilter<T> filter, params int[] 
types)
-        {
-            _lastAsyncOp.Value = (int) Op.WaitForLocal;
-
-            long hnd = 0;
-
-            try
-            {
-                var result = WaitForLocal0(filter, ref hnd, types);
-
-                if (filter != null)
-                {
-                    // Dispose handle as soon as future ends.
-                    var fut = GetFuture<T>();
-
-                    _curFut.Value = fut;
-
-                    fut.Listen(() => Ignite.HandleRegistry.Release(hnd));
-                }
-                else
-                    _curFut.Value = null;
-
-                return result;
-            }
-            catch (Exception)
-            {
-                Ignite.HandleRegistry.Release(hnd);
-                throw;
-            }
-        }
-
-        /** <inheritdoc /> */
-        public override IEvents WithAsync()
-        {
-            return this;
-        }
-
-        /** <inheritdoc /> */
-        public override bool IsAsync
-        {
-            get { return true; }
-        }
-
-        /** <inheritdoc /> */
-        public override IFuture GetFuture()
-        {
-            return GetFuture<object>();
-        }
-
-        /** <inheritdoc /> */
-        public override IFuture<T> GetFuture<T>()
-        {
-            if (_curFut.Value != null)
-            {
-                var fut = _curFut.Value;
-                _curFut.Value = null;
-                return (IFuture<T>) fut;
-            }
-
-            Func<PortableReaderImpl, T> converter = null;
-
-            if (_lastAsyncOp.Value == (int) Op.WaitForLocal)
-                converter = reader => (T) EventReader.Read<IEvent>(reader);
-
-            return GetFuture((futId, futTyp) => 
UU.TargetListenFutureForOperation(Target, futId, futTyp, _lastAsyncOp.Value),
-                convertFunc: converter);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index 2ed7a75..30c6b66 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -353,9 +353,7 @@ namespace Apache.Ignite.Core.Impl
         /// </returns>
         public ICache<TK, TV> Cache<TK, TV>(IUnmanagedTarget nativeCache, bool 
keepPortable = false)
         {
-            var cacheImpl = new CacheImpl<TK, TV>(this, nativeCache, _marsh, 
false, keepPortable, false, false);
-
-            return new CacheProxyImpl<TK, TV>(cacheImpl);
+            return new CacheImpl<TK, TV>(this, nativeCache, _marsh, false, 
keepPortable, false, false);
         }
 
         /** <inheritdoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
index 7588945..532c19d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs
@@ -415,14 +415,5 @@ namespace Apache.Ignite.Core.Impl
 
             return res;
         }
-
-        /// <summary>
-        /// Gets the asynchronous mode disabled exception.
-        /// </summary>
-        /// <returns>Asynchronous mode disabled exception.</returns>
-        public static InvalidOperationException GetAsyncModeDisabledException()
-        {
-            return new InvalidOperationException("Asynchronous mode is 
disabled");
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs
index de94c74..df7d6ff 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/Messaging.cs
@@ -22,8 +22,8 @@ namespace Apache.Ignite.Core.Impl.Messaging
     using System.Collections.Generic;
     using System.Diagnostics;
     using System.Linq;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Cluster;
-    using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Collections;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Portable;
@@ -57,6 +57,15 @@ namespace Apache.Ignite.Core.Impl.Messaging
 
         /** Grid */
         private readonly Ignite _ignite;
+        
+        /** Async instance. */
+        private readonly Lazy<Messaging> _asyncInstance;
+
+        /** Async flag. */
+        private readonly bool _isAsync;
+
+        /** Cluster group. */
+        private readonly IClusterGroup _clusterGroup;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="Messaging" /> class.
@@ -69,15 +78,40 @@ namespace Apache.Ignite.Core.Impl.Messaging
         {
             Debug.Assert(prj != null);
 
-            ClusterGroup = prj;
+            _clusterGroup = prj;
 
             _ignite = (Ignite) prj.Ignite;
+
+            _asyncInstance = new Lazy<Messaging>(() => new Messaging(this));
+        }
+
+        /// <summary>
+        /// Initializes a new async instance.
+        /// </summary>
+        /// <param name="messaging">The messaging.</param>
+        private Messaging(Messaging messaging) : 
base(UU.MessagingWithASync(messaging.Target), messaging.Marshaller)
+        {
+            _isAsync = true;
+            _ignite = messaging._ignite;
+            _clusterGroup = messaging.ClusterGroup;
         }
 
         /** <inheritdoc /> */
-        public IClusterGroup ClusterGroup { get; private set; }
+        public IClusterGroup ClusterGroup
+        {
+            get { return _clusterGroup; }
+        }
+
+        /// <summary>
+        /// Gets the asynchronous instance.
+        /// </summary>
+        private Messaging AsyncInstance
+        {
+            get { return _asyncInstance.Value; }
+        }
 
         /** <inheritdoc /> */
+
         public void Send(object message, object topic = null)
         {
             IgniteArgumentCheck.NotNull(message, "message");
@@ -189,21 +223,22 @@ namespace Apache.Ignite.Core.Impl.Messaging
             {
                 Guid id = Guid.Empty;
 
-                DoOutInOp((int) Op.RemoteListen, writer =>
-                {
-                    writer.Write(filter0);
-                    writer.WriteLong(filterHnd);
-                    writer.Write(topic);
-                }, 
-                input =>
-                {
-                    var id0 = 
Marshaller.StartUnmarshal(input).GetRawReader().ReadGuid();
+                DoOutInOp((int) Op.RemoteListen,
+                    writer =>
+                    {
+                        writer.Write(filter0);
+                        writer.WriteLong(filterHnd);
+                        writer.Write(topic);
+                    },
+                    input =>
+                    {
+                        var id0 = 
Marshaller.StartUnmarshal(input).GetRawReader().ReadGuid();
 
-                    Debug.Assert(IsAsync || id0.HasValue);
+                        Debug.Assert(_isAsync || id0.HasValue);
 
-                    if (id0.HasValue)
-                        id = id0.Value;
-                });
+                        if (id0.HasValue)
+                            id = id0.Value;
+                    });
 
                 return id;
             }
@@ -216,36 +251,28 @@ namespace Apache.Ignite.Core.Impl.Messaging
         }
 
         /** <inheritdoc /> */
-        public void StopRemoteListen(Guid opId)
+        public Task<Guid> RemoteListenAsync<T>(IMessageListener<T> listener, 
object topic = null)
         {
-            DoOutOp((int) Op.StopRemoteListen, writer =>
-            {
-                writer.WriteGuid(opId);
-            });
-        }
+            AsyncInstance.RemoteListen(listener, topic);
 
-        /** <inheritdoc /> */
-        public virtual IMessaging WithAsync()
-        {
-            return new MessagingAsync(UU.MessagingWithASync(Target), 
Marshaller, ClusterGroup);
+            return AsyncInstance.GetTask<Guid>();
         }
 
         /** <inheritdoc /> */
-        public virtual bool IsAsync
+        public void StopRemoteListen(Guid opId)
         {
-            get { return false; }
+            DoOutOp((int) Op.StopRemoteListen, writer =>
+            {
+                writer.WriteGuid(opId);
+            });
         }
 
         /** <inheritdoc /> */
-        public virtual IFuture GetFuture()
+        public Task StopRemoteListenAsync(Guid opId)
         {
-            throw IgniteUtils.GetAsyncModeDisabledException();
-        }
+            AsyncInstance.StopRemoteListen(opId);
 
-        /** <inheritdoc /> */
-        public virtual IFuture<TResult> GetFuture<TResult>()
-        {
-            throw IgniteUtils.GetAsyncModeDisabledException();
+            return AsyncInstance.GetTask();
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessagingAsync.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessagingAsync.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessagingAsync.cs
deleted file mode 100644
index e899d4e..0000000
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Messaging/MessagingAsync.cs
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.
- */
-
-namespace Apache.Ignite.Core.Impl.Messaging
-{
-    using Apache.Ignite.Core.Cluster;
-    using Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Impl.Portable;
-    using Apache.Ignite.Core.Impl.Unmanaged;
-    using Apache.Ignite.Core.Messaging;
-    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
-
-    /// <summary>
-    /// Async messaging implementation.
-    /// </summary>
-    internal class MessagingAsync : Messaging
-    {
-        /// <summary>
-        /// Initializes a new instance of the <see cref="MessagingAsync" /> 
class.
-        /// </summary>
-        /// <param name="target">Target.</param>
-        /// <param name="marsh">Marshaller.</param>
-        /// <param name="prj">Cluster group.</param>
-        public MessagingAsync(IUnmanagedTarget target, PortableMarshaller 
marsh, 
-            IClusterGroup prj) : base(target, marsh, prj)
-        {
-            // No-op.
-        }
-
-        /** <inheritdoc /> */
-        public override IMessaging WithAsync()
-        {
-            return this;
-        }
-
-        /** <inheritdoc /> */
-        public override bool IsAsync
-        {
-            get { return true; }
-        }
-
-        /** <inheritdoc /> */
-        public override IFuture GetFuture()
-        {
-            return GetFuture<object>();
-        }
-
-        /** <inheritdoc /> */
-        public override IFuture<T> GetFuture<T>()
-        {
-            return GetFuture<T>((futId, futTyp) => 
UU.TargetListenFuture(Target, futId, futTyp));
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
index 6d0a324..49b57a5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
@@ -22,8 +22,7 @@ namespace Apache.Ignite.Core.Impl
     using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.IO;
-    using Apache.Ignite.Core.Cache;
-    using Apache.Ignite.Core.Common;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Memory;
     using Apache.Ignite.Core.Impl.Portable;
@@ -631,7 +630,7 @@ namespace Apache.Ignite.Core.Impl
         /// <param name="keepPortable">Keep portable flag, only applicable to 
object futures. False by default.</param>
         /// <param name="convertFunc">The function to read future result from 
stream.</param>
         /// <returns>Created future.</returns>
-        protected IFuture<T> GetFuture<T>(Action<long, int> listenAction, bool 
keepPortable = false,
+        protected Future<T> GetFuture<T>(Action<long, int> listenAction, bool 
keepPortable = false,
             Func<PortableReaderImpl, T> convertFunc = null)
         {
             var futType = FutureType.Object;
@@ -652,6 +651,22 @@ namespace Apache.Ignite.Core.Impl
             return fut;
         }
 
+        /// <summary>
+        /// Creates a task to listen for the last async op.
+        /// </summary>
+        protected Task GetTask()
+        {
+            return GetTask<object>();
+        }
+
+        /// <summary>
+        /// Creates a task to listen for the last async op.
+        /// </summary>
+        protected Task<T> GetTask<T>()
+        {
+            return GetFuture<T>((futId, futTyp) => 
UU.TargetListenFuture(Target, futId, futTyp)).Task;
+        }
+
         #endregion
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs
index ac65807f..fe1a146 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/Services.cs
@@ -22,8 +22,8 @@ namespace Apache.Ignite.Core.Impl.Services
     using System.Diagnostics;
     using System.Linq;
     using System.Reflection;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Cluster;
-    using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Portable;
     using Apache.Ignite.Core.Impl.Unmanaged;
@@ -33,7 +33,7 @@ namespace Apache.Ignite.Core.Impl.Services
     /// <summary>
     /// Services implementation.
     /// </summary>
-    internal class Services : PlatformTarget, IServices
+    internal sealed class Services : PlatformTarget, IServices
     {
         /** */
         private const int OpDeploy = 1;
@@ -54,10 +54,13 @@ namespace Apache.Ignite.Core.Impl.Services
         private readonly IClusterGroup _clusterGroup;
 
         /** Invoker portable flag. */
-        protected readonly bool KeepPortable;
+        private readonly bool _keepPortable;
 
         /** Server portable flag. */
-        protected readonly bool SrvKeepPortable;
+        private readonly bool _srvKeepPortable;
+
+        /** Async instance. */
+        private readonly Lazy<Services> _asyncInstance;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="Services" /> class.
@@ -74,74 +77,87 @@ namespace Apache.Ignite.Core.Impl.Services
             Debug.Assert(clusterGroup  != null);
 
             _clusterGroup = clusterGroup;
-            KeepPortable = keepPortable;
-            SrvKeepPortable = srvKeepPortable;
+            _keepPortable = keepPortable;
+            _srvKeepPortable = srvKeepPortable;
+
+            _asyncInstance = new Lazy<Services>(() => new Services(this));
         }
 
-        /** <inheritDoc /> */
-        public virtual IServices WithKeepPortable()
+        /// <summary>
+        /// Initializes a new async instance.
+        /// </summary>
+        /// <param name="services">The services.</param>
+        private Services(Services services) : 
base(UU.ServicesWithAsync(services.Target), services.Marshaller)
         {
-            if (KeepPortable)
-                return this;
-
-            return new Services(Target, Marshaller, _clusterGroup, true, 
SrvKeepPortable);
+            _clusterGroup = services.ClusterGroup;
+            _keepPortable = services._keepPortable;
+            _srvKeepPortable = services._srvKeepPortable;
         }
 
         /** <inheritDoc /> */
-        public virtual IServices WithServerKeepPortable()
+        public IServices WithKeepPortable()
         {
-            if (SrvKeepPortable)
+            if (_keepPortable)
                 return this;
 
-            return new Services(UU.ServicesWithServerKeepPortable(Target), 
Marshaller, _clusterGroup, KeepPortable, true);
+            return new Services(Target, Marshaller, _clusterGroup, true, 
_srvKeepPortable);
         }
 
         /** <inheritDoc /> */
-        public virtual IServices WithAsync()
+        public IServices WithServerKeepPortable()
         {
-            return new ServicesAsync(UU.ServicesWithAsync(Target), Marshaller, 
_clusterGroup, KeepPortable, SrvKeepPortable);
+            if (_srvKeepPortable)
+                return this;
+
+            return new Services(UU.ServicesWithServerKeepPortable(Target), 
Marshaller, _clusterGroup, _keepPortable, true);
         }
 
         /** <inheritDoc /> */
-        public virtual bool IsAsync
+        public IClusterGroup ClusterGroup
         {
-            get { return false; }
+            get { return _clusterGroup; }
         }
 
-        /** <inheritDoc /> */
-        public virtual IFuture GetFuture()
+        /// <summary>
+        /// Gets the asynchronous instance.
+        /// </summary>
+        private Services AsyncInstance
         {
-            throw new InvalidOperationException("Asynchronous mode is 
disabled");
+            get { return _asyncInstance.Value; }
         }
 
         /** <inheritDoc /> */
-        public virtual IFuture<TResult> GetFuture<TResult>()
+        public void DeployClusterSingleton(string name, IService service)
         {
-            throw new InvalidOperationException("Asynchronous mode is 
disabled");
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+            IgniteArgumentCheck.NotNull(service, "service");
+
+            DeployMultiple(name, service, 1, 1);
         }
 
         /** <inheritDoc /> */
-        public IClusterGroup ClusterGroup
+        public Task DeployClusterSingletonAsync(string name, IService service)
         {
-            get { return _clusterGroup; }
+            AsyncInstance.DeployClusterSingleton(name, service);
+
+            return AsyncInstance.GetTask();
         }
 
         /** <inheritDoc /> */
-        public void DeployClusterSingleton(string name, IService service)
+        public void DeployNodeSingleton(string name, IService service)
         {
             IgniteArgumentCheck.NotNullOrEmpty(name, "name");
             IgniteArgumentCheck.NotNull(service, "service");
 
-            DeployMultiple(name, service, 1, 1);
+            DeployMultiple(name, service, 0, 1);
         }
 
         /** <inheritDoc /> */
-        public void DeployNodeSingleton(string name, IService service)
+        public Task DeployNodeSingletonAsync(string name, IService service)
         {
-            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
-            IgniteArgumentCheck.NotNull(service, "service");
+            AsyncInstance.DeployNodeSingleton(name, service);
 
-            DeployMultiple(name, service, 0, 1);
+            return AsyncInstance.GetTask();
         }
 
         /** <inheritDoc /> */
@@ -163,6 +179,14 @@ namespace Apache.Ignite.Core.Impl.Services
         }
 
         /** <inheritDoc /> */
+        public Task DeployKeyAffinitySingletonAsync<TK>(string name, IService 
service, string cacheName, TK affinityKey)
+        {
+            AsyncInstance.DeployKeyAffinitySingleton(name, service, cacheName, 
affinityKey);
+
+            return AsyncInstance.GetTask();
+        }
+
+        /** <inheritDoc /> */
         public void DeployMultiple(string name, IService service, int 
totalCount, int maxPerNodeCount)
         {
             IgniteArgumentCheck.NotNullOrEmpty(name, "name");
@@ -178,6 +202,14 @@ namespace Apache.Ignite.Core.Impl.Services
         }
 
         /** <inheritDoc /> */
+        public Task DeployMultipleAsync(string name, IService service, int 
totalCount, int maxPerNodeCount)
+        {
+            AsyncInstance.DeployMultiple(name, service, totalCount, 
maxPerNodeCount);
+
+            return AsyncInstance.GetTask();
+        }
+
+        /** <inheritDoc /> */
         public void Deploy(ServiceConfiguration configuration)
         {
             IgniteArgumentCheck.NotNull(configuration, "configuration");
@@ -199,6 +231,14 @@ namespace Apache.Ignite.Core.Impl.Services
         }
 
         /** <inheritDoc /> */
+        public Task DeployAsync(ServiceConfiguration configuration)
+        {
+            AsyncInstance.Deploy(configuration);
+
+            return AsyncInstance.GetTask();
+        }
+
+        /** <inheritDoc /> */
         public void Cancel(string name)
         {
             IgniteArgumentCheck.NotNullOrEmpty(name, "name");
@@ -207,17 +247,33 @@ namespace Apache.Ignite.Core.Impl.Services
         }
 
         /** <inheritDoc /> */
+        public Task CancelAsync(string name)
+        {
+            AsyncInstance.Cancel(name);
+
+            return AsyncInstance.GetTask();
+        }
+
+        /** <inheritDoc /> */
         public void CancelAll()
         {
             UU.ServicesCancelAll(Target);
         }
 
         /** <inheritDoc /> */
+        public Task CancelAllAsync()
+        {
+            AsyncInstance.CancelAll();
+
+            return AsyncInstance.GetTask();
+        }
+
+        /** <inheritDoc /> */
         public ICollection<IServiceDescriptor> GetServiceDescriptors()
         {
             return DoInOp(OpDescriptors, stream =>
             {
-                var reader = Marshaller.StartUnmarshal(stream, KeepPortable);
+                var reader = Marshaller.StartUnmarshal(stream, _keepPortable);
 
                 var size = reader.ReadInt();
 
@@ -310,7 +366,7 @@ namespace Apache.Ignite.Core.Impl.Services
         {
             return DoOutInOp(OpInvokeMethod,
                 writer => ServiceProxySerializer.WriteProxyMethod(writer, 
method, args),
-                stream => ServiceProxySerializer.ReadInvocationResult(stream, 
Marshaller, KeepPortable), proxy.Target);
+                stream => ServiceProxySerializer.ReadInvocationResult(stream, 
Marshaller, _keepPortable), proxy.Target);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServicesAsync.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServicesAsync.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServicesAsync.cs
deleted file mode 100644
index 860de45..0000000
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Services/ServicesAsync.cs
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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.
- */
-
-namespace Apache.Ignite.Core.Impl.Services
-{
-    using Apache.Ignite.Core.Cluster;
-    using Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Impl.Portable;
-    using Apache.Ignite.Core.Impl.Unmanaged;
-    using Apache.Ignite.Core.Services;
-    using UU = Apache.Ignite.Core.Impl.Unmanaged.UnmanagedUtils;
-
-    /// <summary>
-    /// Async services implementation.
-    /// </summary>
-    internal class ServicesAsync : Services
-    {
-        /// <summary>
-        /// Initializes a new instance of the <see cref="ServicesAsync" /> 
class.
-        /// </summary>
-        /// <param name="target">Target.</param>
-        /// <param name="marsh">Marshaller.</param>
-        /// <param name="clusterGroup">Cluster group.</param>
-        /// <param name="keepPortable">Portable flag.</param>
-        /// <param name="srvKeepPortable">Server portable flag.</param>
-        public ServicesAsync(IUnmanagedTarget target, PortableMarshaller 
marsh, IClusterGroup clusterGroup,
-            bool keepPortable, bool srvKeepPortable)
-            : base(target, marsh, clusterGroup, keepPortable, srvKeepPortable)
-        {
-            // No-op
-        }
-
-        /** <inheritDoc /> */
-        public override bool IsAsync
-        {
-            get { return true; }
-        }
-
-        /** <inheritDoc /> */
-        public override IServices WithKeepPortable()
-        {
-            if (KeepPortable)
-                return this;
-
-            return new ServicesAsync(Target, Marshaller, ClusterGroup, true, 
SrvKeepPortable);
-        }
-
-        /** <inheritDoc /> */
-        public override IServices WithServerKeepPortable()
-        {
-            if (SrvKeepPortable)
-                return this;
-
-            return new ServicesAsync(Target, Marshaller, ClusterGroup, 
KeepPortable, true);
-        }
-
-        /** <inheritDoc /> */
-        public override IServices WithAsync()
-        {
-            return this;
-        }
-
-        /** <inheritDoc /> */
-        public override IFuture GetFuture()
-        {
-            return GetFuture<object>();
-        }
-
-        /** <inheritDoc /> */
-        public override IFuture<T> GetFuture<T>()
-        {
-            return GetFuture<T>((futId, futTyp) => 
UU.TargetListenFuture(Target, futId, futTyp));
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/AsyncTransaction.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/AsyncTransaction.cs
 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/AsyncTransaction.cs
deleted file mode 100644
index 82d1d55..0000000
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/AsyncTransaction.cs
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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.
- */
-
-namespace Apache.Ignite.Core.Impl.Transactions
-{
-    using System;
-    using System.Threading;
-    using Apache.Ignite.Core.Common;
-
-    /// <summary>
-    /// Grid async transaction facade.
-    /// </summary>
-    internal class AsyncTransaction : Transaction
-    {
-        /** */
-        private readonly ThreadLocal<IFuture> _curFut = new 
ThreadLocal<IFuture>();
-
-        /// <summary>
-        /// Initializes a new instance of the <see cref="AsyncTransaction"/> 
class.
-        /// </summary>
-        /// <param name="tx">The tx to wrap.</param>
-        public AsyncTransaction(TransactionImpl tx) : base(tx)
-        {
-            // No-op.
-        }
-
-        /** <inheritDoc /> */
-        public override bool IsAsync
-        {
-            get { return true; }
-        }
-
-        /** <inheritDoc /> */
-        public override IFuture<TResult> GetFuture<TResult>()
-        {
-            return GetFuture() as IFuture<TResult>;
-        }
-
-        /** <inheritDoc /> */
-        public override IFuture GetFuture()
-        {
-            var fut = _curFut.Value;
-
-            if (fut == null)
-                throw new InvalidOperationException("Asynchronous operation 
not started.");
-
-            _curFut.Value = null;
-
-            return fut;
-        }
-
-        /** <inheritDoc /> */
-        public override void Commit()
-        {
-            _curFut.Value = Tx.GetFutureOrError(() => Tx.CommitAsync());
-        }
-
-        /** <inheritDoc /> */
-        public override void Rollback()
-        {
-            _curFut.Value = Tx.GetFutureOrError(() => Tx.RollbackAsync());
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/Transaction.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/Transaction.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/Transaction.cs
index 35dad92..595300c 100644
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/Transaction.cs
+++ 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/Transaction.cs
@@ -19,16 +19,16 @@ namespace Apache.Ignite.Core.Impl.Transactions
 {
     using System;
     using System.Diagnostics.CodeAnalysis;
-    using Apache.Ignite.Core.Common;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Transactions;
 
     /// <summary>
     /// Ignite transaction facade.
     /// </summary>
-    internal class Transaction : ITransaction
+    internal sealed class Transaction : ITransaction
     {
         /** */
-        protected readonly TransactionImpl Tx;
+        private readonly TransactionImpl _tx;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="Transaction" /> class.
@@ -36,7 +36,7 @@ namespace Apache.Ignite.Core.Impl.Transactions
         /// <param name="tx">The tx to wrap.</param>
         public Transaction(TransactionImpl tx)
         {
-            Tx = tx;
+            _tx = tx;
         }
 
         /** <inheritDoc /> */
@@ -44,115 +44,103 @@ namespace Apache.Ignite.Core.Impl.Transactions
             Justification = "There is no finalizer.")]
         public void Dispose()
         {
-            Tx.Dispose();
-        }
-
-        /** <inheritDoc /> */
-        public ITransaction WithAsync()
-        {
-            return new AsyncTransaction(Tx);
-        }
-
-        /** <inheritDoc /> */
-        public virtual bool IsAsync
-        {
-            get { return false; }
-        }
-
-        /** <inheritDoc /> */
-        public virtual IFuture GetFuture()
-        {
-            throw IgniteUtils.GetAsyncModeDisabledException();
-        }
-        
-        /** <inheritDoc /> */
-        public virtual IFuture<TResult> GetFuture<TResult>()
-        {
-            throw IgniteUtils.GetAsyncModeDisabledException();
+            _tx.Dispose();
         }
 
         /** <inheritDoc /> */
         public Guid NodeId
         {
-            get { return Tx.NodeId; }
+            get { return _tx.NodeId; }
         }
 
         /** <inheritDoc /> */
         public long ThreadId
         {
-            get { return Tx.ThreadId; }
+            get { return _tx.ThreadId; }
         }
 
         /** <inheritDoc /> */
         public DateTime StartTime
         {
-            get { return Tx.StartTime; }
+            get { return _tx.StartTime; }
         }
 
         /** <inheritDoc /> */
         public TransactionIsolation Isolation
         {
-            get { return Tx.Isolation; }
+            get { return _tx.Isolation; }
         }
 
         /** <inheritDoc /> */
         public TransactionConcurrency Concurrency
         {
-            get { return Tx.Concurrency; }
+            get { return _tx.Concurrency; }
         }
 
         /** <inheritDoc /> */
         public TransactionState State
         {
-            get { return Tx.State; }
+            get { return _tx.State; }
         }
 
         /** <inheritDoc /> */
         public TimeSpan Timeout
         {
-            get { return Tx.Timeout; }
+            get { return _tx.Timeout; }
         }
 
         /** <inheritDoc /> */
         public bool IsRollbackOnly
         {
-            get { return Tx.IsRollbackOnly; }
+            get { return _tx.IsRollbackOnly; }
         }
 
         /** <inheritDoc /> */
         public bool SetRollbackonly()
         {
-            return Tx.SetRollbackOnly();
+            return _tx.SetRollbackOnly();
+        }
+
+        /** <inheritDoc /> */
+        public void Commit()
+        {
+            _tx.Commit();
+        }
+
+        /** <inheritDoc /> */
+        public Task CommitAsync()
+        {
+            return _tx.GetTask(() => _tx.CommitAsync());
         }
 
         /** <inheritDoc /> */
-        public virtual void Commit()
+        public void Rollback()
         {
-            Tx.Commit();
+            _tx.Rollback();
         }
 
         /** <inheritDoc /> */
-        public virtual void Rollback()
+        public Task RollbackAsync()
         {
-            Tx.Rollback();
+            return _tx.GetTask(() => _tx.RollbackAsync());
         }
 
         /** <inheritDoc /> */
         public void AddMeta<TV>(string name, TV val)
         {
-            Tx.AddMeta(name, val);
+            _tx.AddMeta(name, val);
         }
 
         /** <inheritDoc /> */
         public TV Meta<TV>(string name)
         {
-            return Tx.Meta<TV>(name);
+            return _tx.Meta<TV>(name);
         }
 
         /** <inheritDoc /> */
         public TV RemoveMeta<TV>(string name)
         {
-            return Tx.RemoveMeta<TV>(name);
+            return _tx.RemoveMeta<TV>(name);
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
index 3d1e57d..d32cd3d 100644
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
+++ 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
@@ -20,8 +20,7 @@ namespace Apache.Ignite.Core.Impl.Transactions
     using System;
     using System.Globalization;
     using System.Threading;
-    using Apache.Ignite.Core.Common;
-    using Apache.Ignite.Core.Impl.Common;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Transactions;
 
     /// <summary>
@@ -336,34 +335,26 @@ namespace Apache.Ignite.Core.Impl.Transactions
         /// <summary>
         /// Commits tx in async mode.
         /// </summary>
-        internal IFuture CommitAsync()
+        internal Task CommitAsync()
         {
             lock (this)
             {
                 ThrowIfClosed();
 
-                var fut = _txs.CommitAsync(this);
-
-                CloseWhenComplete(fut);
-
-                return fut;
+                return CloseWhenComplete(_txs.CommitAsync(this));
             }
         }
 
         /// <summary>
         /// Rolls tx back in async mode.
         /// </summary>
-        internal IFuture RollbackAsync()
+        internal Task RollbackAsync()
         {
             lock (this)
             {
                 ThrowIfClosed();
 
-                var fut = _txs.RollbackAsync(this);
-
-                CloseWhenComplete(fut);
-
-                return fut;
+                return CloseWhenComplete(_txs.RollbackAsync(this));
             }
         }
 
@@ -391,7 +382,7 @@ namespace Apache.Ignite.Core.Impl.Transactions
         /// <summary>
         /// Gets a value indicating whether this transaction is closed.
         /// </summary>
-        internal bool IsClosed
+        private bool IsClosed
         {
             get { return _state != null; }
         }
@@ -406,26 +397,26 @@ namespace Apache.Ignite.Core.Impl.Transactions
         }
 
         /// <summary>
-        /// Creates a future via provided factory if IsClosed is false; 
otherwise, return a future with an error.
+        /// Creates a task via provided factory if IsClosed is false; 
otherwise, return a task with an error.
         /// </summary>
-        internal IFuture GetFutureOrError(Func<IFuture> operationFactory)
+        internal Task GetTask(Func<Task> operationFactory)
         {
             lock (this)
             {
-                return IsClosed ? GetExceptionFuture() : operationFactory();
+                return IsClosed ? GetExceptionTask() : operationFactory();
             }
         }
 
         /// <summary>
-        /// Gets the future that throws an exception.
+        /// Gets the task that throws an exception.
         /// </summary>
-        private IFuture GetExceptionFuture()
+        private Task GetExceptionTask()
         {
-            var fut = new Future<object>();
-
-            fut.OnError(GetClosedException());
-
-            return fut;
+            var tcs = new TaskCompletionSource<object>();
+            
+            tcs.SetException(GetClosedException());
+            
+            return tcs.Task;
         }
 
         /// <summary>
@@ -449,11 +440,11 @@ namespace Apache.Ignite.Core.Impl.Transactions
         }
 
         /// <summary>
-        /// Closes this transaction upon future completion.
+        /// Closes this transaction upon task completion.
         /// </summary>
-        private void CloseWhenComplete(IFuture fut)
+        private Task CloseWhenComplete(Task task)
         {
-            fut.Listen(Close);
+            return task.ContinueWith(x => Close());
         }
 
         /** <inheritdoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs
 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs
index f2bc0f6..3305ba1 100644
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs
+++ 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs
@@ -18,7 +18,7 @@
 namespace Apache.Ignite.Core.Impl.Transactions
 {
     using System;
-    using Apache.Ignite.Core.Common;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Impl.Portable;
     using Apache.Ignite.Core.Impl.Unmanaged;
     using Apache.Ignite.Core.Portable;
@@ -177,17 +177,17 @@ namespace Apache.Ignite.Core.Impl.Transactions
         /// <summary>
         /// Commits tx in async mode.
         /// </summary>
-        internal IFuture CommitAsync(TransactionImpl tx)
+        internal Task CommitAsync(TransactionImpl tx)
         {
-            return GetFuture<object>((futId, futTyp) => 
UU.TransactionsCommitAsync(Target, tx.Id, futId));
+            return GetFuture<object>((futId, futTyp) => 
UU.TransactionsCommitAsync(Target, tx.Id, futId)).Task;
         }
 
         /// <summary>
         /// Rolls tx back in async mode.
         /// </summary>
-        internal IFuture RollbackAsync(TransactionImpl tx)
+        internal Task RollbackAsync(TransactionImpl tx)
         {
-            return GetFuture<object>((futId, futTyp) => 
UU.TransactionsRollbackAsync(Target, tx.Id, futId));
+            return GetFuture<object>((futId, futTyp) => 
UU.TransactionsRollbackAsync(Target, tx.Id, futId)).Task;
         }
  
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
index 9d986ec..54cfe28 100644
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -292,7 +292,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
                 IUnmanagedTarget cb0 = null;
 
                 if ((long) cb != 0)
-                    cb0 = new 
UnmanagedNonReleaseableTarget(_ctx.NativeContext, cb);
+                    cb0 = new UnmanagedNonReleaseableTarget(_ctx, cb);
 
                 using (PlatformMemoryStream stream = 
IgniteManager.Memory.Get(memPtr).GetStream())
                 {
@@ -618,7 +618,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
                                    
portableReceiver.Deserialize<StreamReceiverHolder>();
 
                     if (receiver != null)
-                        receiver.Receive(_ignite, new 
UnmanagedNonReleaseableTarget(_ctx.NativeContext, cache), stream,
+                        receiver.Receive(_ignite, new 
UnmanagedNonReleaseableTarget(_ctx, cache), stream,
                             keepPortable != 0);
                 }
             });

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedNonReleaseableTarget.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedNonReleaseableTarget.cs
 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedNonReleaseableTarget.cs
index fe2de77..d044c5f 100644
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedNonReleaseableTarget.cs
+++ 
b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedNonReleaseableTarget.cs
@@ -17,7 +17,6 @@
 
 namespace Apache.Ignite.Core.Impl.Unmanaged
 {
-    using System;
     using System.Diagnostics.CodeAnalysis;
 
     /// <summary>
@@ -26,7 +25,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
     internal unsafe class UnmanagedNonReleaseableTarget : IUnmanagedTarget
     {
         /** Context. */
-        private readonly void* _ctx;
+        private readonly UnmanagedContext _ctx;
 
         /** Target. */
         private readonly void* _target;
@@ -36,7 +35,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         /// </summary>
         /// <param name="ctx">Context.</param>
         /// <param name="target">Target.</param>
-        public UnmanagedNonReleaseableTarget(void* ctx, void* target)
+        public UnmanagedNonReleaseableTarget(UnmanagedContext ctx, void* 
target)
         {
             _ctx = ctx;
             _target = target;
@@ -45,7 +44,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         /** <inheritdoc /> */
         public void* Context
         {
-            get { return _ctx; }
+            get { return _ctx.NativeContext; }
         }
 
         /** <inheritdoc /> */
@@ -57,7 +56,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         /** <inheritdoc /> */
         public IUnmanagedTarget ChangeTarget(void* target)
         {
-            throw new NotSupportedException();
+            return new UnmanagedTarget(_ctx, target);
         }
 
         /** <inheritdoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Messaging/IMessaging.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Messaging/IMessaging.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Messaging/IMessaging.cs
index de2b0d9..53bf680 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Messaging/IMessaging.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Messaging/IMessaging.cs
@@ -19,8 +19,8 @@ namespace Apache.Ignite.Core.Messaging
 {
     using System;
     using System.Collections;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Cluster;
-    using Apache.Ignite.Core.Common;
 
     /// <summary>
     /// Provides functionality for topic-based message exchange among nodes 
defined by <see cref="IClusterGroup"/>.
@@ -29,7 +29,7 @@ namespace Apache.Ignite.Core.Messaging
     /// <para/>
     /// All members are thread-safe and may be used concurrently from multiple 
threads.
     /// </summary>
-    public interface IMessaging : IAsyncSupport<IMessaging>
+    public interface IMessaging
     {
         /// <summary>
         /// Gets the cluster group to which this instance belongs.
@@ -95,14 +95,34 @@ namespace Apache.Ignite.Core.Messaging
         /// <returns>
         /// Operation ID that can be passed to <see cref="StopRemoteListen"/> 
method to stop listening.
         /// </returns>
-        [AsyncSupported]
         Guid RemoteListen<T>(IMessageListener<T> listener, object topic = 
null);
 
         /// <summary>
+        /// Adds a message listener for a given topic to all nodes in the 
cluster group (possibly including
+        /// this node if it belongs to the cluster group as well). This means 
that any node within this cluster
+        /// group can send a message for a given topic and all nodes within 
the cluster group will receive
+        /// listener notifications.
+        /// </summary>
+        /// <param name="listener">
+        /// Predicate that is called on each received message. If predicate 
returns false,
+        /// then it will be unsubscribed from any further notifications.
+        /// </param>
+        /// <param name="topic">Topic to unsubscribe from.</param>
+        /// <returns>
+        /// Operation ID that can be passed to <see cref="StopRemoteListen"/> 
method to stop listening.
+        /// </returns>
+        Task<Guid> RemoteListenAsync<T>(IMessageListener<T> listener, object 
topic = null);
+
+        /// <summary>
         /// Unregisters all listeners identified with provided operation ID on 
all nodes in the cluster group.
         /// </summary>
         /// <param name="opId">Operation ID that was returned from <see 
cref="RemoteListen{T}"/> method.</param>
-        [AsyncSupported]
         void StopRemoteListen(Guid opId);
+
+        /// <summary>
+        /// Unregisters all listeners identified with provided operation ID on 
all nodes in the cluster group.
+        /// </summary>
+        /// <param name="opId">Operation ID that was returned from <see 
cref="RemoteListen{T}"/> method.</param>
+        Task StopRemoteListenAsync(Guid opId);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Services/IServices.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Services/IServices.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Services/IServices.cs
index ec1a044..da7394f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Services/IServices.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Services/IServices.cs
@@ -19,13 +19,13 @@ namespace Apache.Ignite.Core.Services
 {
     using System.Collections.Generic;
     using System.Diagnostics.CodeAnalysis;
+    using System.Threading.Tasks;
     using Apache.Ignite.Core.Cluster;
-    using Apache.Ignite.Core.Common;
 
     /// <summary>
     /// Defines functionality to deploy distributed services in the Ignite.
     /// </summary>
-    public interface IServices : IAsyncSupport<IServices>
+    public interface IServices
     {
         /// <summary>
         /// Gets the cluster group to which this instance belongs.
@@ -47,21 +47,43 @@ namespace Apache.Ignite.Core.Services
         /// </summary>
         /// <param name="name">Service name.</param>
         /// <param name="service">Service instance.</param>
-        [AsyncSupported]
         void DeployClusterSingleton(string name, IService service);
 
         /// <summary>
+        /// Deploys a cluster-wide singleton service. Ignite guarantees that 
there is always
+        /// one instance of the service in the cluster. In case if Ignite node 
on which the service
+        /// was deployed crashes or stops, Ignite will automatically redeploy 
it on another node.
+        /// However, if the node on which the service is deployed remains in 
topology, then the
+        /// service will always be deployed on that node only, regardless of 
topology changes.
+        /// <para />
+        /// Note that in case of topology changes, due to network delays, 
there may be a temporary situation
+        /// when a singleton service instance will be active on more than one 
node (e.g. crash detection delay).
+        /// </summary>
+        /// <param name="name">Service name.</param>
+        /// <param name="service">Service instance.</param>
+        Task DeployClusterSingletonAsync(string name, IService service);
+
+        /// <summary>
         /// Deploys a per-node singleton service. Ignite guarantees that there 
is always
         /// one instance of the service running on each node. Whenever new 
nodes are started
         /// within the underlying cluster group, Ignite will automatically 
deploy one instance of
-        /// the service on every new node.        
+        /// the service on every new node.
         /// </summary>
         /// <param name="name">Service name.</param>
         /// <param name="service">Service instance.</param>
-        [AsyncSupported]
         void DeployNodeSingleton(string name, IService service);
 
         /// <summary>
+        /// Deploys a per-node singleton service. Ignite guarantees that there 
is always
+        /// one instance of the service running on each node. Whenever new 
nodes are started
+        /// within the underlying cluster group, Ignite will automatically 
deploy one instance of
+        /// the service on every new node.
+        /// </summary>
+        /// <param name="name">Service name.</param>
+        /// <param name="service">Service instance.</param>
+        Task DeployNodeSingletonAsync(string name, IService service);
+
+        /// <summary>
         /// Deploys one instance of this service on the primary node for a 
given affinity key.
         /// Whenever topology changes and primary node assignment changes, 
Ignite will always
         /// make sure that the service is undeployed on the previous primary 
node and deployed
@@ -75,10 +97,25 @@ namespace Apache.Ignite.Core.Services
         /// <param name="cacheName">Name of the cache on which affinity for 
key should be calculated, null for
         /// default cache.</param>
         /// <param name="affinityKey">Affinity cache key.</param>
-        [AsyncSupported]
         void DeployKeyAffinitySingleton<TK>(string name, IService service, 
string cacheName, TK affinityKey);
 
         /// <summary>
+        /// Deploys one instance of this service on the primary node for a 
given affinity key.
+        /// Whenever topology changes and primary node assignment changes, 
Ignite will always
+        /// make sure that the service is undeployed on the previous primary 
node and deployed
+        /// on the new primary node.
+        /// <para />
+        /// Note that in case of topology changes, due to network delays, 
there may be a temporary situation
+        /// when a service instance will be active on more than one node (e.g. 
crash detection delay).
+        /// </summary>
+        /// <param name="name">Service name.</param>
+        /// <param name="service">Service instance.</param>
+        /// <param name="cacheName">Name of the cache on which affinity for 
key should be calculated, null for
+        /// default cache.</param>
+        /// <param name="affinityKey">Affinity cache key.</param>
+        Task DeployKeyAffinitySingletonAsync<TK>(string name, IService 
service, string cacheName, TK affinityKey);
+
+        /// <summary>
         /// Deploys multiple instances of the service on the grid. Ignite will 
deploy a
         /// maximum amount of services equal to <paramref name="totalCount" /> 
parameter making sure that
         /// there are no more than <paramref name="maxPerNodeCount" /> service 
instances running
@@ -90,18 +127,36 @@ namespace Apache.Ignite.Core.Services
         /// <param name="service">Service instance.</param>
         /// <param name="totalCount">Maximum number of deployed services in 
the grid, 0 for unlimited.</param>
         /// <param name="maxPerNodeCount">Maximum number of deployed services 
on each node, 0 for unlimited.</param>
-        [AsyncSupported]
         void DeployMultiple(string name, IService service, int totalCount, int 
maxPerNodeCount);
 
         /// <summary>
+        /// Deploys multiple instances of the service on the grid. Ignite will 
deploy a
+        /// maximum amount of services equal to <paramref name="totalCount" /> 
parameter making sure that
+        /// there are no more than <paramref name="maxPerNodeCount" /> service 
instances running
+        /// on each node. Whenever topology changes, Ignite will automatically 
rebalance
+        /// the deployed services within cluster to make sure that each node 
will end up with
+        /// about equal number of deployed instances whenever possible.
+        /// </summary>
+        /// <param name="name">Service name.</param>
+        /// <param name="service">Service instance.</param>
+        /// <param name="totalCount">Maximum number of deployed services in 
the grid, 0 for unlimited.</param>
+        /// <param name="maxPerNodeCount">Maximum number of deployed services 
on each node, 0 for unlimited.</param>
+        Task DeployMultipleAsync(string name, IService service, int 
totalCount, int maxPerNodeCount);
+
+        /// <summary>
         /// Deploys instances of the service in the Ignite according to 
provided configuration.
         /// </summary>
         /// <param name="configuration">Service configuration.</param>
-        [AsyncSupported]
         void Deploy(ServiceConfiguration configuration);
 
         /// <summary>
-        /// Cancels service deployment. If a service with specified name was 
deployed on the grid, 
+        /// Deploys instances of the service in the Ignite according to 
provided configuration.
+        /// </summary>
+        /// <param name="configuration">Service configuration.</param>
+        Task DeployAsync(ServiceConfiguration configuration);
+
+        /// <summary>
+        /// Cancels service deployment. If a service with specified name was 
deployed on the grid,
         /// then <see cref="IService.Cancel"/> method will be called on it.
         /// <para/>
         /// Note that Ignite cannot guarantee that the service exits from <see 
cref="IService.Execute"/>
@@ -109,19 +164,36 @@ namespace Apache.Ignite.Core.Services
         /// make sure that the service code properly reacts to cancellations.
         /// </summary>
         /// <param name="name">Name of the service to cancel.</param>
-        [AsyncSupported]
         void Cancel(string name);
 
         /// <summary>
+        /// Cancels service deployment. If a service with specified name was 
deployed on the grid,
+        /// then <see cref="IService.Cancel"/> method will be called on it.
+        /// <para/>
+        /// Note that Ignite cannot guarantee that the service exits from <see 
cref="IService.Execute"/>
+        /// method whenever <see cref="IService.Cancel"/> is called. It is up 
to the user to
+        /// make sure that the service code properly reacts to cancellations.
+        /// </summary>
+        /// <param name="name">Name of the service to cancel.</param>
+        Task CancelAsync(string name);
+
+        /// <summary>
         /// Cancels all deployed services.
         /// <para/>
-        /// Note that depending on user logic, it may still take extra time 
for a service to 
+        /// Note that depending on user logic, it may still take extra time 
for a service to
         /// finish execution, even after it was cancelled.
         /// </summary>
-        [AsyncSupported]
         void CancelAll();
 
         /// <summary>
+        /// Cancels all deployed services.
+        /// <para/>
+        /// Note that depending on user logic, it may still take extra time 
for a service to
+        /// finish execution, even after it was cancelled.
+        /// </summary>
+        Task CancelAllAsync();
+
+        /// <summary>
         /// Gets metadata about all deployed services.
         /// </summary>
         /// <returns>Metadata about all deployed services.</returns>
@@ -173,7 +245,7 @@ namespace Apache.Ignite.Core.Services
         /// </summary>
         /// <returns>Instance with portable mode enabled.</returns>
         IServices WithKeepPortable();
-        
+
         /// <summary>
         /// Returns an instance with server-side portable mode enabled.
         /// Service method arguments will be kept in portable form.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs
index e85d577..22a1ee1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs
@@ -19,7 +19,7 @@ namespace Apache.Ignite.Core.Transactions
 {
     using System;
     using System.Collections.Generic;
-    using Apache.Ignite.Core.Common;
+    using System.Threading.Tasks;
 
     /// <summary>
     /// Grid cache transaction. 
@@ -115,7 +115,7 @@ namespace Apache.Ignite.Core.Transactions
     ///     </code>
     /// </example>
     /// </summary>
-    public interface ITransaction : IDisposable, IAsyncSupport<ITransaction>
+    public interface ITransaction : IDisposable
     {
         /// <summary>
         /// ID of the node on which this transaction started.
@@ -196,16 +196,24 @@ namespace Apache.Ignite.Core.Transactions
         /// <summary>
         /// Commits this transaction.
         /// </summary>
-        [AsyncSupported]
         void Commit();
 
         /// <summary>
+        /// Commits this transaction.
+        /// </summary>
+        Task CommitAsync();
+
+        /// <summary>
         /// Rolls back this transaction.
         /// </summary>
-        [AsyncSupported]
         void Rollback();
 
         /// <summary>
+        /// Rolls back this transaction.
+        /// </summary>
+        Task RollbackAsync();
+
+        /// <summary>
         /// Adds a new metadata.
         /// </summary>
         /// <param name="name">Metadata name.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/cc1aa533/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/CrossPlatformExample.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/CrossPlatformExample.cs
 
b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/CrossPlatformExample.cs
index e23d615..270d147 100644
--- 
a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/CrossPlatformExample.cs
+++ 
b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Datagrid/CrossPlatformExample.cs
@@ -118,15 +118,13 @@ namespace Apache.Ignite.Examples.Datagrid
         /// Gets entry put by Java client. In order for entry to be in cache, 
Java client example
         /// must be run before this example.
         /// </summary>
-        /// <param name="Ignite">Ignite instance.</param>
+        /// <param name="ignite">Ignite instance.</param>
         private static void GetFromJava(IIgnite ignite)
         {
             var cache = ignite.GetOrCreateCache<int, 
IPortableObject>(CacheName)
-                .WithKeepPortable<int, IPortableObject>().WithAsync();
-
-            cache.Get(KeyJava);
+                .WithKeepPortable<int, IPortableObject>();
 
-            var orgPortable = 
cache.GetFuture<IPortableObject>().ToTask().Result;
+            var orgPortable = cache.GetAsync(KeyJava).Result;
 
             if (orgPortable == null)
             {
@@ -149,11 +147,9 @@ namespace Apache.Ignite.Examples.Datagrid
         private static void GetFromCpp(IIgnite ignite)
         {
             var cache = ignite.GetOrCreateCache<int, 
IPortableObject>(CacheName)
-                .WithKeepPortable<int, IPortableObject>().WithAsync();
-
-            cache.Get(KeyCpp);
+                .WithKeepPortable<int, IPortableObject>();
 
-            var orgPortable = cache.GetFuture<IPortableObject>().Get();
+            var orgPortable = cache.GetAsync(KeyCpp).Result;
 
             Console.WriteLine();
 

Reply via email to