You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2017/01/24 12:26:17 UTC

ignite git commit: IGNITE-4457 Define cache plugin API in .NET

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.0 b0f848729 -> b9901f021


IGNITE-4457 Define cache plugin API in .NET

This closes #1375


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

Branch: refs/heads/ignite-2.0
Commit: b9901f02147319e811be21db1e3b9fd1970fa47d
Parents: b0f8487
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jan 24 15:26:05 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 24 15:26:05 2017 +0300

----------------------------------------------------------------------
 .../callback/PlatformCallbackGateway.java       |  68 ++++++
 .../platform/callback/PlatformCallbackOp.java   |  12 ++
 .../cache/PlatformCachePluginConfiguration.java |  58 +++++
 .../cache/PlatformCachePluginProvider.java      | 123 +++++++++++
 .../utils/PlatformConfigurationUtils.java       |  30 +++
 .../processors/plugin/CachePluginManager.java   |   8 +-
 .../Apache.Ignite.Core.Tests.csproj             |   3 +
 .../Cache/CacheConfigurationTest.cs             |  13 +-
 .../IgniteConfigurationSerializerTest.cs        |  11 +-
 .../Plugin/Cache/CachePlugin.cs                 | 127 +++++++++++
 .../Plugin/Cache/CachePluginConfiguration.cs    |  40 ++++
 .../Plugin/Cache/CachePluginTest.cs             | 215 +++++++++++++++++++
 .../Apache.Ignite.Core.csproj                   |   8 +
 .../Cache/Configuration/CacheConfiguration.cs   |  34 +++
 .../Communication/Tcp/TcpCommunicationSpi.cs    |   2 +-
 .../Discovery/Tcp/TcpDiscoverySpi.cs            |   2 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  18 +-
 .../IgniteConfigurationSection.xsd              |  21 ++
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |   2 +-
 .../Impl/Plugin/Cache/CachePluginContext.cs     |  83 +++++++
 .../Impl/Plugin/Cache/CachePluginProcessor.cs   |  77 +++++++
 .../Plugin/Cache/CachePluginProviderProxy.cs    |  76 +++++++
 .../Plugin/Cache/ICachePluginProviderProxy.cs   |  53 +++++
 .../Impl/Unmanaged/UnmanagedCallbackOp.cs       |   6 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |  58 +++++
 .../Cache/CachePluginProviderTypeAttribute.cs   |  51 +++++
 .../Plugin/Cache/ICachePluginConfiguration.cs   |  50 +++++
 .../Plugin/Cache/ICachePluginContext.cs         |  47 ++++
 .../Plugin/Cache/ICachePluginProvider.cs        |  52 +++++
 29 files changed, 1329 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
index 9842068..fc311da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
@@ -1155,6 +1155,74 @@ public class PlatformCallbackGateway {
     }
 
     /**
+     * Create cache plugin.
+     *
+     * @param memPtr Memory pointer.
+     * @return Pointer.
+     */
+    public long cachePluginCreate(long memPtr) {
+        enter();
+
+        try {
+            return PlatformCallbackUtils.inLongOutLong(envPtr, PlatformCallbackOp.CachePluginCreate, memPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify cache plugin on ignite start.
+     *
+     * @param objPtr Object pointer.
+     */
+    public void cachePluginIgniteStart(long objPtr) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.inLongOutLong(envPtr, PlatformCallbackOp.CachePluginIgniteStart, objPtr);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Notify cache plugin on ignite start.
+     *
+     * @param objPtr Object pointer.
+     */
+    public void cachePluginIgniteStop(long objPtr, boolean cancel) {
+        enter();
+
+        try {
+            PlatformCallbackUtils.inLongLongLongObjectOutLong(envPtr, PlatformCallbackOp.CachePluginIgniteStop, objPtr,
+                    cancel ? 1 : 0, 0, null);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
+     * Destroy cache plugin.
+     *
+     * @param objPtr Object pointer.
+     */
+    public void cachePluginDestroy(long objPtr, boolean cancel) {
+        if (!lock.enterBusy())
+            return;  // no need to destroy plugins on grid stop
+
+        try {
+            PlatformCallbackUtils.inLongLongLongObjectOutLong(envPtr, PlatformCallbackOp.CachePluginDestroy,
+                    objPtr, cancel ? 1 : 0, 0, null);
+        }
+        finally {
+            leave();
+        }
+    }
+
+    /**
      * Enter gateway.
      */
     protected void enter() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackOp.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackOp.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackOp.java
index b0bd3a5..500a4f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackOp.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackOp.java
@@ -209,4 +209,16 @@ class PlatformCallbackOp {
 
     /** */
     public static final int PluginProcessorIgniteStop = 63;
+
+    /** */
+    public static final int CachePluginCreate = 64;
+
+    /** */
+    public static final int CachePluginDestroy = 65;
+
+    /** */
+    public static final int CachePluginIgniteStart = 66;
+
+    /** */
+    public static final int CachePluginIgniteStop = 67;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/plugin/cache/PlatformCachePluginConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/plugin/cache/PlatformCachePluginConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/plugin/cache/PlatformCachePluginConfiguration.java
new file mode 100644
index 0000000..d055f00
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/plugin/cache/PlatformCachePluginConfiguration.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.plugin.cache;
+
+import org.apache.ignite.plugin.CachePluginConfiguration;
+import org.apache.ignite.plugin.CachePluginContext;
+import org.apache.ignite.plugin.CachePluginProvider;
+
+/**
+ * Platform cache plugin configuration.
+ */
+public class PlatformCachePluginConfiguration implements CachePluginConfiguration {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Native configuration object. */
+    private final Object nativeCfg;
+
+    /**
+     * Ctor.
+     *
+     * @param nativeCfg Native configuration object.
+     */
+    public PlatformCachePluginConfiguration(Object nativeCfg) {
+        assert nativeCfg != null;
+
+        this.nativeCfg = nativeCfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CachePluginProvider createProvider(CachePluginContext ctx) {
+        return new PlatformCachePluginProvider(ctx, nativeCfg);
+    }
+
+    /**
+     * Gets the native configuration object.
+     *
+     * @return Native configuration object.
+     */
+    public Object nativeCfg() {
+        return nativeCfg;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/plugin/cache/PlatformCachePluginProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/plugin/cache/PlatformCachePluginProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/plugin/cache/PlatformCachePluginProvider.java
new file mode 100644
index 0000000..d23bd8b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/plugin/cache/PlatformCachePluginProvider.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.plugin.cache;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.utils.PlatformConfigurationUtils;
+import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
+import org.apache.ignite.plugin.CachePluginConfiguration;
+import org.apache.ignite.plugin.CachePluginContext;
+import org.apache.ignite.plugin.CachePluginProvider;
+import org.jetbrains.annotations.Nullable;
+
+import javax.cache.Cache;
+
+/**
+ * Platform cache plugin provider.
+ */
+class PlatformCachePluginProvider implements CachePluginProvider {
+    /** Context. */
+    private final CachePluginContext ctx;
+
+    /** Native config. */
+    private final Object nativeCfg;
+
+    /** Pointer to native plugin. */
+    protected long ptr;
+
+    /**
+     * Ctor.
+     *
+     * @param ctx Context.
+     */
+    PlatformCachePluginProvider(CachePluginContext ctx, Object nativeCfg) {
+        assert ctx != null;
+        assert nativeCfg != null;
+
+        this.ctx = ctx;
+        this.nativeCfg = nativeCfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteCheckedException {
+        PlatformContext platformCtx = PlatformUtils.platformContext(ctx.grid());
+
+        try (PlatformMemory mem = platformCtx.memory().allocate()) {
+            PlatformOutputStream out = mem.output();
+
+            BinaryRawWriterEx writer = platformCtx.writer(out);
+
+            writer.writeObjectDetached(nativeCfg);
+
+            PlatformConfigurationUtils.writeIgniteConfiguration(writer, ctx.igniteConfiguration());
+            PlatformConfigurationUtils.writeCacheConfiguration(writer, ctx.igniteCacheConfiguration());
+
+            out.synchronize();
+
+            ptr = platformCtx.gateway().cachePluginCreate(mem.pointer());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) {
+        PlatformContext platformCtx = PlatformUtils.platformContext(ctx.grid());
+
+        platformCtx.gateway().cachePluginDestroy(ptr, cancel);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIgniteStart() throws IgniteCheckedException {
+        PlatformContext platformCtx = PlatformUtils.platformContext(ctx.grid());
+
+        platformCtx.gateway().cachePluginIgniteStart(ptr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIgniteStop(boolean cancel) {
+        PlatformContext platformCtx = PlatformUtils.platformContext(ctx.grid());
+
+        platformCtx.gateway().cachePluginIgniteStop(ptr, cancel);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void validate() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void validateRemote(CacheConfiguration locCfg, CachePluginConfiguration locPluginCcfg,
+        CacheConfiguration rmtCfg, ClusterNode rmtNode) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object unwrapCacheEntry(Cache.Entry entry, Class cls) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object createComponent(Class cls) {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index c0fde97..ea8f361 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -47,7 +47,9 @@ import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.binary.*;
 import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction;
 import org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory;
+import org.apache.ignite.internal.processors.platform.plugin.cache.PlatformCachePluginConfiguration;
 import org.apache.ignite.platform.dotnet.*;
+import org.apache.ignite.plugin.CachePluginConfiguration;
 import org.apache.ignite.spi.communication.CommunicationSpi;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiMBean;
@@ -198,6 +200,17 @@ public class PlatformConfigurationUtils {
         ccfg.setAffinity(readAffinityFunction(in));
         ccfg.setExpiryPolicyFactory(readExpiryPolicyFactory(in));
 
+        int pluginCnt = in.readInt();
+
+        if (pluginCnt > 0) {
+            CachePluginConfiguration[] plugins = new CachePluginConfiguration[pluginCnt];
+
+            for (int i = 0; i < pluginCnt; i++)
+                plugins[i] = new PlatformCachePluginConfiguration(in.readObjectDetached());
+
+            ccfg.setPluginConfigurations(plugins);
+        }
+
         return ccfg;
     }
 
@@ -829,6 +842,23 @@ public class PlatformConfigurationUtils {
         writeEvictionPolicy(writer, ccfg.getEvictionPolicy());
         writeAffinityFunction(writer, ccfg.getAffinity());
         writeExpiryPolicyFactory(writer, ccfg.getExpiryPolicyFactory());
+
+        CachePluginConfiguration[] plugins = ccfg.getPluginConfigurations();
+        if (plugins != null) {
+            int cnt = 0;
+
+            for (CachePluginConfiguration cfg : plugins) {
+                if (cfg instanceof PlatformCachePluginConfiguration)
+                    cnt++;
+            }
+
+            writer.writeInt(cnt);
+
+            for (CachePluginConfiguration cfg : plugins) {
+                if (cfg instanceof PlatformCachePluginConfiguration)
+                    writer.writeObject(((PlatformCachePluginConfiguration)cfg).nativeCfg());
+            }
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/CachePluginManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/CachePluginManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/CachePluginManager.java
index d0efc0a..fa33d3a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/CachePluginManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/CachePluginManager.java
@@ -86,8 +86,8 @@ public class CachePluginManager extends GridCacheManagerAdapter {
 
     /** {@inheritDoc} */
     @Override protected void onKernalStop0(boolean cancel) {
-        for (ListIterator<CachePluginProvider> iter = providersList.listIterator(); iter.hasPrevious();)
-            iter.previous().onIgniteStop(cancel);
+        for (int i = providersList.size() - 1; i >= 0; i--)
+            providersList.get(i).onIgniteStop(cancel);
     }
 
     /** {@inheritDoc} */
@@ -98,8 +98,8 @@ public class CachePluginManager extends GridCacheManagerAdapter {
 
     /** {@inheritDoc} */
     @Override protected void stop0(boolean cancel) {
-        for (ListIterator<CachePluginProvider> iter = providersList.listIterator(); iter.hasPrevious();)
-            iter.previous().stop(cancel);
+        for (int i = providersList.size() - 1; i >= 0; i--)
+            providersList.get(i).stop(cancel);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 785bb58..9430114 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -94,6 +94,9 @@
     <Compile Include="Plugin\TestIgnitePlugin.cs" />
     <Compile Include="Plugin\TestIgnitePluginConfiguration.cs" />
     <Compile Include="Plugin\TestIgnitePluginProvider.cs" />
+    <Compile Include="Plugin\Cache\CachePlugin.cs" />
+    <Compile Include="Plugin\Cache\CachePluginConfiguration.cs" />
+    <Compile Include="Plugin\Cache\CachePluginTest.cs" />
     <Compile Include="TestAppConfig.cs" />
     <Compile Include="Binary\BinaryBuilderSelfTestFullFooter.cs" />
     <Compile Include="Binary\BinaryCompactFooterInteropTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index fb8725c..118e115 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -30,6 +30,7 @@ namespace Apache.Ignite.Core.Tests.Cache
     using Apache.Ignite.Core.Cache.Store;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Cache.Affinity;
+    using Apache.Ignite.Core.Tests.Plugin.Cache;
     using NUnit.Framework;
 
     /// <summary>
@@ -263,6 +264,13 @@ namespace Apache.Ignite.Core.Tests.Cache
             AssertConfigsAreEqual(x.NearConfiguration, y.NearConfiguration);
             AssertConfigsAreEqual(x.EvictionPolicy, y.EvictionPolicy);
             AssertConfigsAreEqual(x.AffinityFunction, y.AffinityFunction);
+
+            if (x.PluginConfigurations != null)
+            {
+                Assert.IsNotNull(y.PluginConfigurations);
+                Assert.AreEqual(x.PluginConfigurations.Select(p => p.GetType()),
+                    y.PluginConfigurations.Select(p => p.GetType()));
+            }
         }
 
         /// <summary>
@@ -529,7 +537,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                         Fields = new[]
                         {
                             new QueryField("length", typeof(int)), 
-                            new QueryField("name", typeof(string)) {IsKeyField = true}, 
+                            new QueryField("name", typeof(string)) {IsKeyField = true},
                             new QueryField("location", typeof(string)),
                         },
                         Aliases = new [] {new QueryAlias("length", "len") },
@@ -566,7 +574,8 @@ namespace Apache.Ignite.Core.Tests.Cache
                     ExcludeNeighbors = true
                 },
                 ExpiryPolicyFactory = new ExpiryFactory(),
-                EnableStatistics = true
+                EnableStatistics = true,
+                PluginConfigurations = new[] { new CachePluginConfiguration() }
             };
         }
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 3a0a4b5..95c0289 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -52,6 +52,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.SwapSpace.File;
     using Apache.Ignite.Core.Tests.Binary;
     using Apache.Ignite.Core.Tests.Plugin;
+    using Apache.Ignite.Core.Tests.Plugin.Cache;
     using Apache.Ignite.Core.Transactions;
     using Apache.Ignite.NLog;
     using NUnit.Framework;
@@ -116,6 +117,7 @@ namespace Apache.Ignite.Core.Tests
                                     </nearConfiguration>
                                     <affinityFunction type='RendezvousAffinityFunction' partitions='99' excludeNeighbors='true' />
                                     <expiryPolicyFactory type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+MyPolicyFactory, Apache.Ignite.Core.Tests' />
+                                    <pluginConfigurations><iCachePluginConfiguration type='Apache.Ignite.Core.Tests.Plugin.Cache.CachePluginConfiguration, Apache.Ignite.Core.Tests' testProperty='baz' /></pluginConfigurations>
                                 </cacheConfiguration>
                                 <cacheConfiguration name='secondCache' />
                             </cacheConfiguration>
@@ -242,6 +244,9 @@ namespace Apache.Ignite.Core.Tests
             var plugins = cfg.PluginConfigurations;
             Assert.IsNotNull(plugins);
             Assert.IsNotNull(plugins.Cast<TestIgnitePluginConfiguration>().SingleOrDefault());
+
+            var cachePlugCfg = cacheCfg.PluginConfigurations.Cast<CachePluginConfiguration>().Single();
+            Assert.AreEqual("baz", cachePlugCfg.TestProperty);
         }
 
         /// <summary>
@@ -675,7 +680,11 @@ namespace Apache.Ignite.Core.Tests
                             Partitions = 48
                         },
                         ExpiryPolicyFactory = new MyPolicyFactory(),
-                        EnableStatistics = true
+                        EnableStatistics = true,
+                        PluginConfigurations = new[]
+                        {
+                            new CachePluginConfiguration()
+                        }
                     }
                 },
                 ClientMode = true,

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePlugin.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePlugin.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePlugin.cs
new file mode 100644
index 0000000..dcc53ca
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePlugin.cs
@@ -0,0 +1,127 @@
+\ufeff/*
+ * 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.Tests.Plugin.Cache
+{
+    using System.Collections.Concurrent;
+    using System.Collections.Generic;
+    using System.IO;
+    using Apache.Ignite.Core.Plugin.Cache;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Test cache plugin.
+    /// </summary>
+    public class CachePlugin : ICachePluginProvider<CachePluginConfiguration>
+    {
+        /** */
+        private static readonly ConcurrentDictionary<CachePlugin, object> Instances = 
+            new ConcurrentDictionary<CachePlugin, object>();
+
+        /// <summary>
+        /// Gets the instances.
+        /// </summary>
+        public static IEnumerable<CachePlugin> GetInstances()
+        {
+            return Instances.Keys;
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CachePlugin"/> class.
+        /// </summary>
+        public CachePlugin()
+        {
+            Assert.IsTrue(Instances.TryAdd(this, null));
+        }
+
+        /** <inheritdoc /> */
+        public void Start(ICachePluginContext<CachePluginConfiguration> cachePluginContext)
+        {
+            Context = cachePluginContext;
+            Started = true;
+
+            Assert.IsNotNull(Context);
+            Assert.IsNotNull(Context.IgniteConfiguration);
+            Assert.IsNotNull(Context.CachePluginConfiguration);
+            Assert.IsNotNull(Context.CacheConfiguration);
+            Assert.IsNotNull(Context.Ignite);
+
+            // Check that Ignite is operational.
+            Assert.GreaterOrEqual(2, Context.Ignite.GetCluster().GetNodes().Count);
+
+            Throw();
+        }
+
+        /** <inheritdoc /> */
+        public void Stop(bool cancel)
+        {
+            Stopped = cancel;
+
+            object unused;
+            Assert.IsTrue(Instances.TryRemove(this, out unused));
+        }
+
+        /** <inheritdoc /> */
+        public void OnIgniteStart()
+        {
+            // Check that Ignite is operational.
+            Assert.GreaterOrEqual(2, Context.Ignite.GetCluster().GetNodes().Count);
+
+            IgniteStarted = true;
+        }
+
+        /** <inheritdoc /> */
+        public void OnIgniteStop(bool cancel)
+        {
+            IgniteStopped = cancel;
+        }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether this <see cref="CachePlugin"/> is started.
+        /// </summary>
+        public bool Started { get; private set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether this <see cref="CachePlugin"/> is started.
+        /// </summary>
+        public bool IgniteStarted { get; private set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether this <see cref="CachePlugin"/> is stopped.
+        /// </summary>
+        public bool? Stopped { get; private set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether this <see cref="CachePlugin"/> is stopped.
+        /// </summary>
+        public bool? IgniteStopped { get; private set; }
+
+        /// <summary>
+        /// Gets the context.
+        /// </summary>
+        public ICachePluginContext<CachePluginConfiguration> Context { get; private set; }
+
+        /// <summary>
+        /// Throws an error when configured.
+        /// </summary>
+        private void Throw()
+        {
+            if (Context.CachePluginConfiguration.ThrowError)
+                throw new IOException("Failure in cache plugin provider");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePluginConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePluginConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePluginConfiguration.cs
new file mode 100644
index 0000000..4627aa0
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePluginConfiguration.cs
@@ -0,0 +1,40 @@
+\ufeff/*
+ * 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.Tests.Plugin.Cache
+{
+    using System;
+    using Apache.Ignite.Core.Plugin.Cache;
+
+    /// <summary>
+    /// Cache plugin config.
+    /// </summary>
+    [Serializable]
+    [CachePluginProviderType(typeof(CachePlugin))]
+    public class CachePluginConfiguration : ICachePluginConfiguration
+    {
+        /// <summary>
+        /// Gets or sets the test property.
+        /// </summary>
+        public string TestProperty { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether the plugin should throw an error.
+        /// </summary>
+        public bool ThrowError { get; set; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePluginTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePluginTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePluginTest.cs
new file mode 100644
index 0000000..583d314
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Plugin/Cache/CachePluginTest.cs
@@ -0,0 +1,215 @@
+\ufeff/*
+ * 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.Tests.Plugin.Cache
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Cache;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Plugin.Cache;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for cache plugins.
+    /// </summary>
+    public class CachePluginTest
+    {
+        /** */
+        private const string CacheName = "staticCache";
+
+        /** */
+        private const string DynCacheName = "dynamicCache";
+
+        /** */
+        private IIgnite _grid1;
+
+        /** */
+        private IIgnite _grid2;
+
+        /** */
+        private readonly List<CachePlugin> _plugins = new List<CachePlugin>();
+
+        /// <summary>
+        /// Fixture set up.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void FixtureSetUp()
+        {
+            _plugins.Clear();
+
+            _grid1 = Ignition.Start(GetConfig("grid1"));
+            _grid2 = Ignition.Start(GetConfig("grid2"));
+        }
+
+        /// <summary>
+        /// Fixture tear down.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            // One plugin is expected in registry.
+            TestUtils.AssertHandleRegistryHasItems(10, 1, _grid1, _grid2);
+
+            Ignition.StopAll(true);
+
+            // Check IgniteStop callbacks.
+            foreach (var plugin in _plugins)
+            {
+                Assert.AreEqual(true, plugin.IgniteStopped);
+            }
+        }
+
+        /// <summary>
+        /// Tests with static cache.
+        /// </summary>
+        [Test]
+        public void TestStaticCache()
+        {
+            foreach (var ignite in new[] {_grid1, _grid2})
+            {
+                var plugin = CheckCachePlugin(ignite, CacheName, "foo");
+
+                _plugins.Add(plugin);
+            }
+        }
+
+        /// <summary>
+        /// Tests with dynamic cache.
+        /// </summary>
+        [Test]
+        public void TestDynamicCache()
+        {
+            var cacheConfig = new CacheConfiguration(DynCacheName)
+            {
+                PluginConfigurations = new[] {new CachePluginConfiguration {TestProperty = "bar"}}
+            };
+
+            _grid1.CreateCache<int, int>(cacheConfig);
+
+            var plugins = new List<CachePlugin>();
+
+            foreach (var ignite in new[] { _grid1, _grid2 })
+            {
+                var plugin = CheckCachePlugin(ignite, DynCacheName, "bar");
+
+                plugins.Add(plugin);
+            }
+
+            // Destroy cache to remove plugin from handle registry.
+            _grid1.DestroyCache(DynCacheName);
+
+            foreach (var plugin in plugins)
+            {
+                Assert.AreEqual(true, plugin.Stopped);
+                Assert.AreEqual(true, plugin.IgniteStopped);  // This is weird, but correct from Java logic POV.
+            }
+        }
+
+        /// <summary>
+        /// Non-serializable plugin config results in a meaningful exception.
+        /// </summary>
+        [Test]
+        public void TestNonSerializablePlugin()
+        {
+            var ex = Assert.Throws<InvalidOperationException>(() => _grid1.CreateCache<int, int>(new CacheConfiguration
+            {
+                PluginConfigurations = new[] {new NonSerializableCachePluginConfig()}
+            }));
+
+            Assert.AreEqual("Invalid cache configuration: ICachePluginConfiguration should be Serializable.", 
+                ex.Message);
+        }
+
+        /// <summary>
+        /// Errors in plugin configuration result in meaningful exception.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-4474 Ignite.createCache hangs on exception in CachePluginConfiguration.createProvider")]
+        public void TestErrorInPlugin()
+        {
+            // Throws exception.
+            var cacheEx = Assert.Throws<CacheException>(() => _grid1.CreateCache<int, int>(new CacheConfiguration
+            {
+                PluginConfigurations = new[] { new ThrowCachePluginConfig()  }
+            }));
+
+            Assert.AreEqual("hi!", cacheEx.Message);
+        }
+
+        /// <summary>
+        /// Checks the cache plugin.
+        /// </summary>
+        private static CachePlugin CheckCachePlugin(IIgnite ignite, string cacheName, string propValue)
+        {
+            // Check config.
+            var plugCfg = ignite.GetCache<int, int>(cacheName).GetConfiguration()
+                .PluginConfigurations.Cast<CachePluginConfiguration>().Single();
+            Assert.AreEqual(propValue, plugCfg.TestProperty);
+
+            // Check started plugin.
+            var plugin = CachePlugin.GetInstances().Single(x => x.Context.Ignite == ignite &&
+                                                                x.Context.CacheConfiguration.Name == cacheName);
+            Assert.IsTrue(plugin.Started);
+            Assert.IsTrue(plugin.IgniteStarted);
+            Assert.IsNull(plugin.Stopped);
+            Assert.IsNull(plugin.IgniteStopped);
+
+            var ctx = plugin.Context;
+            Assert.AreEqual(ignite.Name, ctx.IgniteConfiguration.GridName);
+            Assert.AreEqual(cacheName, ctx.CacheConfiguration.Name);
+            Assert.AreEqual(propValue, ctx.CachePluginConfiguration.TestProperty);
+
+            return plugin;
+        }
+
+        /// <summary>
+        /// Gets the configuration.
+        /// </summary>
+        private static IgniteConfiguration GetConfig(string name)
+        {
+            return new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                GridName = name,
+                CacheConfiguration = new[]
+                {
+                    new CacheConfiguration(CacheName)
+                    {
+                        PluginConfigurations = new[]
+                        {
+                            new CachePluginConfiguration {TestProperty = "foo"}
+                        }
+                    }
+                }
+            };
+        }
+
+        [CachePluginProviderType(typeof(CachePlugin))]
+        private class NonSerializableCachePluginConfig : ICachePluginConfiguration
+        {
+            // No-op.
+        }
+
+        [Serializable]
+        [CachePluginProviderType(typeof(string))]
+        private class ThrowCachePluginConfig : ICachePluginConfiguration
+        {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 5c0ac94..93970c0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -420,10 +420,14 @@
     <Compile Include="Impl\Binary\Structure\BinaryStructureJumpTable.cs" />
     <Compile Include="Impl\Binary\Structure\BinaryStructureUpdate.cs" />
     <Compile Include="Impl\Binary\TypeResolver.cs" />
+    <Compile Include="Impl\Plugin\Cache\CachePluginProcessor.cs" />
+    <Compile Include="Impl\Plugin\Cache\CachePluginProviderProxy.cs" />
+    <Compile Include="Impl\Plugin\Cache\ICachePluginProviderProxy.cs" />
     <Compile Include="Impl\Plugin\IPluginProviderProxy.cs" />
     <Compile Include="Impl\Plugin\PluginContext.cs" />
     <Compile Include="Impl\Plugin\PluginProcessor.cs" />
     <Compile Include="Impl\Plugin\PluginProviderProxy.cs" />
+    <Compile Include="Impl\Plugin\Cache\CachePluginContext.cs" />
     <Compile Include="Impl\Resource\IResourceInjector.cs" />
     <Compile Include="Impl\Resource\ResourceFieldInjector.cs" />
     <Compile Include="Impl\Resource\ResourceMethodInjector.cs" />
@@ -480,11 +484,15 @@
     <Compile Include="Binary\BinaryObjectException.cs" />
     <Compile Include="Binary\BinaryTypeConfiguration.cs" />
     <Compile Include="Binary\BinaryTypeNames.cs" />
+    <Compile Include="Plugin\Cache\CachePluginProviderTypeAttribute.cs" />
     <Compile Include="Plugin\IPluginConfiguration.cs" />
     <Compile Include="Plugin\IPluginContext.cs" />
     <Compile Include="Plugin\IPluginProvider.cs" />
     <Compile Include="Plugin\PluginNotFoundException.cs" />
     <Compile Include="Plugin\PluginProviderTypeAttribute.cs" />
+    <Compile Include="Plugin\Cache\ICachePluginConfiguration.cs" />
+    <Compile Include="Plugin\Cache\ICachePluginContext.cs" />
+    <Compile Include="Plugin\Cache\ICachePluginProvider.cs" />
     <Compile Include="Properties\AssemblyInfo.cs" />
     <Compile Include="Resource\InstanceResourceAttribute.cs" />
     <Compile Include="Resource\Package-Info.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
index 2795111..0e270dc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/CacheConfiguration.cs
@@ -39,6 +39,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
     using Apache.Ignite.Core.Impl.Cache.Affinity;
     using Apache.Ignite.Core.Impl.Cache.Expiry;
     using Apache.Ignite.Core.Log;
+    using Apache.Ignite.Core.Plugin.Cache;
 
     /// <summary>
     /// Defines grid cache configuration.
@@ -283,6 +284,11 @@ namespace Apache.Ignite.Core.Cache.Configuration
             EvictionPolicy = EvictionPolicyBase.Read(reader);
             AffinityFunction = AffinityFunctionSerializer.Read(reader);
             ExpiryPolicyFactory = ExpiryPolicySerializer.ReadPolicyFactory(reader);
+
+            count = reader.ReadInt();
+            PluginConfigurations = count == 0
+                ? null
+                : Enumerable.Range(0, count).Select(x => reader.ReadObject<ICachePluginConfiguration>()).ToList();
         }
 
         /// <summary>
@@ -358,6 +364,28 @@ namespace Apache.Ignite.Core.Cache.Configuration
             EvictionPolicyBase.Write(writer, EvictionPolicy);
             AffinityFunctionSerializer.Write(writer, AffinityFunction);
             ExpiryPolicySerializer.WritePolicyFactory(writer, ExpiryPolicyFactory);
+
+            if (PluginConfigurations != null)
+            {
+                writer.WriteInt(PluginConfigurations.Count);
+
+                foreach (var cachePlugin in PluginConfigurations)
+                {
+                    if (cachePlugin == null)
+                        throw new InvalidOperationException("Invalid cache configuration: " +
+                                                            "ICachePluginConfiguration can't be null.");
+
+                    if (!cachePlugin.GetType().IsSerializable)
+                        throw new InvalidOperationException("Invalid cache configuration: " +
+                                                            "ICachePluginConfiguration should be Serializable.");
+
+                    writer.WriteObject(cachePlugin);
+                }
+            }
+            else
+            {
+                writer.WriteInt(0);
+            }
         }
 
         /// <summary>
@@ -712,5 +740,11 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// These statistics can be retrieved via <see cref="ICache{TK,TV}.GetMetrics()"/>.
         /// </summary>
         public bool EnableStatistics { get; set; }
+
+        /// <summary>
+        /// Gets or sets the plugin configurations.
+        /// </summary>
+        [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")]
+        public ICollection<ICachePluginConfiguration> PluginConfigurations { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs
index afd3b57..d272906 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Communication/Tcp/TcpCommunicationSpi.cs
@@ -101,7 +101,7 @@ namespace Apache.Ignite.Core.Communication.Tcp
         /// Initializes a new instance of the <see cref="TcpCommunicationSpi"/> class.
         /// </summary>
         /// <param name="reader">The reader.</param>
-        internal TcpCommunicationSpi(BinaryReader reader)
+        internal TcpCommunicationSpi(IBinaryRawReader reader)
         {
             AckSendThreshold = reader.ReadInt();
             ConnectTimeout = reader.ReadLongAsTimespan();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoverySpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoverySpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoverySpi.cs
index e03be3c..a99f7b0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoverySpi.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoverySpi.cs
@@ -122,7 +122,7 @@ namespace Apache.Ignite.Core.Discovery.Tcp
         /// Initializes a new instance of the <see cref="TcpDiscoverySpi"/> class.
         /// </summary>
         /// <param name="reader">The reader.</param>
-        internal TcpDiscoverySpi(BinaryReader reader)
+        internal TcpDiscoverySpi(IBinaryRawReader reader)
         {
             IpFinder = reader.ReadBoolean() ? TcpDiscoveryIpFinderBase.ReadInstance(reader) : null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 036b05f..0676608 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -45,7 +45,6 @@
     using Apache.Ignite.Core.Plugin;
     using Apache.Ignite.Core.SwapSpace;
     using Apache.Ignite.Core.Transactions;
-    using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader;
     using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter;
 
     /// <summary>
@@ -175,11 +174,16 @@
         }
 
         /// <summary>
-        /// Initializes a new instance of the <see cref="IgniteConfiguration"/> class from a reader.
+        /// Initializes a new instance of the <see cref="IgniteConfiguration" /> class from a reader.
         /// </summary>
         /// <param name="binaryReader">The binary reader.</param>
-        internal IgniteConfiguration(BinaryReader binaryReader)
+        /// <param name="baseConfig">The base configuration.</param>
+        internal IgniteConfiguration(IBinaryRawReader binaryReader, IgniteConfiguration baseConfig)
         {
+            Debug.Assert(binaryReader != null);
+            Debug.Assert(baseConfig != null);
+
+            CopyLocalProperties(baseConfig);
             Read(binaryReader);
         }
 
@@ -353,7 +357,7 @@
         /// Reads data from specified reader into current instance.
         /// </summary>
         /// <param name="r">The binary reader.</param>
-        private void ReadCore(BinaryReader r)
+        private void ReadCore(IBinaryRawReader r)
         {
             // Simple properties
             _clientMode = r.ReadBooleanNullable();
@@ -447,12 +451,10 @@
         /// Reads data from specified reader into current instance.
         /// </summary>
         /// <param name="binaryReader">The binary reader.</param>
-        private void Read(BinaryReader binaryReader)
+        private void Read(IBinaryRawReader binaryReader)
         {
             ReadCore(binaryReader);
 
-            CopyLocalProperties(binaryReader.Marshaller.Ignite.Configuration);
-
             // Misc
             IgniteHome = binaryReader.ReadString();
 
@@ -480,7 +482,7 @@
             {
                 BinaryConfiguration = new BinaryConfiguration(cfg.BinaryConfiguration);
             }
-            
+
             JvmClasspath = cfg.JvmClasspath;
             JvmOptions = cfg.JvmOptions;
             Assemblies = cfg.Assemblies;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index 7ba02c1..2962951 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -532,6 +532,27 @@
                                                 </xs:attribute>
                                             </xs:complexType>
                                         </xs:element>
+                                        <xs:element name="pluginConfigurations" minOccurs="0">
+                                            <xs:annotation>
+                                                <xs:documentation>Cache plugin configurations.</xs:documentation>
+                                            </xs:annotation>
+                                            <xs:complexType>
+                                                <xs:sequence>
+                                                    <xs:element name="iCachePluginConfiguration" maxOccurs="unbounded">
+                                                        <xs:annotation>
+                                                            <xs:documentation>Cache plugin configuration.</xs:documentation>
+                                                        </xs:annotation>
+                                                        <xs:complexType>
+                                                            <xs:attribute name="type" type="xs:string" use="required">
+                                                                <xs:annotation>
+                                                                    <xs:documentation>Assembly-qualified type name.</xs:documentation>
+                                                                </xs:annotation>
+                                                            </xs:attribute>
+                                                        </xs:complexType>
+                                                    </xs:element>
+                                                </xs:sequence>
+                                            </xs:complexType>
+                                        </xs:element>
                                     </xs:all>
                                     <xs:attribute name="name" type="xs:string">
                                         <xs:annotation>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/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 651fb83..f79822d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -654,7 +654,7 @@ namespace Apache.Ignite.Core.Impl
 
                 stream.SynchronizeInput();
 
-                return new IgniteConfiguration(_marsh.StartUnmarshal(stream));
+                return new IgniteConfiguration(_marsh.StartUnmarshal(stream), _cfg);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginContext.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginContext.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginContext.cs
new file mode 100644
index 0000000..e52f01a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginContext.cs
@@ -0,0 +1,83 @@
+\ufeff/*
+ * 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.Plugin.Cache
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Plugin.Cache;
+
+    /// <summary>
+    /// Cache plugin context.
+    /// </summary>
+    internal class CachePluginContext<T> : ICachePluginContext<T> where T : ICachePluginConfiguration
+    {
+        /** */
+        private readonly IgniteConfiguration _igniteConfiguration;
+        
+        /** */
+        private readonly CacheConfiguration _cacheConfiguration;
+        
+        /** */
+        private readonly T _cachePluginConfiguration;
+
+        /** */
+        private readonly IIgnite _ignite;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CachePluginContext{T}"/> class.
+        /// </summary>
+        public CachePluginContext(IgniteConfiguration igniteConfiguration, CacheConfiguration cacheConfiguration, 
+            T cachePluginConfiguration, IIgnite ignite)
+        {
+            Debug.Assert(igniteConfiguration != null);
+            Debug.Assert(cacheConfiguration != null);
+            Debug.Assert(cachePluginConfiguration != null);
+            Debug.Assert(ignite != null);
+
+            _igniteConfiguration = igniteConfiguration;
+            _cacheConfiguration = cacheConfiguration;
+            _cachePluginConfiguration = cachePluginConfiguration;
+            _ignite = ignite;
+        }
+
+        /** <inheritdoc /> */
+        public IgniteConfiguration IgniteConfiguration
+        {
+            get { return _igniteConfiguration; }
+        }
+
+        /** <inheritdoc /> */
+        public CacheConfiguration CacheConfiguration
+        {
+            get { return _cacheConfiguration; }
+        }
+
+        /** <inheritdoc /> */
+        public T CachePluginConfiguration
+        {
+            get { return _cachePluginConfiguration; }
+        }
+
+        /** <inheritdoc /> */
+        public IIgnite Ignite
+        {
+            get { return _ignite; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginProcessor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginProcessor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginProcessor.cs
new file mode 100644
index 0000000..2d88d73
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginProcessor.cs
@@ -0,0 +1,77 @@
+\ufeff/*
+ * 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.Plugin.Cache
+{
+    using System;
+    using System.Diagnostics;
+    using System.Linq;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Plugin;
+    using Apache.Ignite.Core.Plugin.Cache;
+
+    /// <summary>
+    /// Cache plugin processor.
+    /// </summary>
+    internal static class CachePluginProcessor
+    {
+        /// <summary>
+        /// Creates the provider proxy.
+        /// </summary>
+        public static ICachePluginProviderProxy CreateProviderProxy(ICachePluginConfiguration pluginConfiguration)
+        {
+            Debug.Assert(pluginConfiguration != null);
+
+            var cfgType = pluginConfiguration.GetType();
+
+            var attributes = cfgType.GetCustomAttributes(true).OfType<CachePluginProviderTypeAttribute>().ToArray();
+
+            if (attributes.Length == 0)
+            {
+                throw new IgniteException(string.Format("{0} of type {1} has no {2}",
+                    typeof(IPluginConfiguration), cfgType, typeof(CachePluginProviderTypeAttribute)));
+
+            }
+
+            if (attributes.Length > 1)
+            {
+                throw new IgniteException(string.Format("{0} of type {1} has more than one {2}",
+                    typeof(IPluginConfiguration), cfgType, typeof(CachePluginProviderTypeAttribute)));
+            }
+
+            var providerType = attributes[0].CachePluginProviderType;
+
+            var iface = providerType.GetInterfaces()
+                .SingleOrDefault(i => i.IsGenericType &&
+                                      i.GetGenericTypeDefinition() == typeof(ICachePluginProvider<>) &&
+                                      i.GetGenericArguments()[0] == cfgType);
+
+            if (iface == null)
+            {
+                throw new IgniteException(string.Format("{0} does not implement {1}",
+                    providerType, typeof(ICachePluginProvider<>).MakeGenericType(cfgType)));
+            }
+
+            var pluginProvider = Activator.CreateInstance(providerType);
+
+            var providerProxyType = typeof(CachePluginProviderProxy<>).MakeGenericType(cfgType);
+
+            return (ICachePluginProviderProxy)Activator.CreateInstance(
+                providerProxyType, pluginConfiguration, pluginProvider);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginProviderProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginProviderProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginProviderProxy.cs
new file mode 100644
index 0000000..7628989
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/CachePluginProviderProxy.cs
@@ -0,0 +1,76 @@
+\ufeff/*
+ * 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.Plugin.Cache
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Plugin.Cache;
+
+    /// <summary>
+    /// Wraps user-defined generic <see cref="ICachePluginProvider{TConfig}"/>.
+    /// </summary>
+    internal class CachePluginProviderProxy<T> : ICachePluginProviderProxy where T : ICachePluginConfiguration
+    {
+        /** */
+        private readonly T _cachePluginConfiguration;
+
+        /** */
+        private readonly ICachePluginProvider<T> _pluginProvider;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="CachePluginProviderProxy{T}" /> class.
+        /// </summary>
+        /// <param name="cachePluginConfiguration">The cache plugin configuration.</param>
+        /// <param name="pluginProvider">The plugin provider.</param>
+        public CachePluginProviderProxy(T cachePluginConfiguration, ICachePluginProvider<T> pluginProvider)
+        {
+            Debug.Assert(cachePluginConfiguration != null);
+            Debug.Assert(pluginProvider != null);
+
+            _cachePluginConfiguration = cachePluginConfiguration;
+            _pluginProvider = pluginProvider;
+        }
+
+        /** <inheritdoc /> */
+        public void Start(IgniteConfiguration igniteConfiguration, CacheConfiguration cacheConfiguration,
+            IIgnite ignite)
+        {
+            _pluginProvider.Start(new CachePluginContext<T>(igniteConfiguration, 
+                cacheConfiguration, _cachePluginConfiguration, ignite));
+        }
+
+        /** <inheritdoc /> */
+        public void Stop(bool cancel)
+        {
+            _pluginProvider.Stop(cancel);
+        }
+
+        /** <inheritdoc /> */
+        public void OnIgniteStart()
+        {
+            _pluginProvider.OnIgniteStart();
+        }
+
+        /** <inheritdoc /> */
+        public void OnIgniteStop(bool cancel)
+        {
+            _pluginProvider.OnIgniteStop(cancel);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/ICachePluginProviderProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/ICachePluginProviderProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/ICachePluginProviderProxy.cs
new file mode 100644
index 0000000..c1f9506
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Plugin/Cache/ICachePluginProviderProxy.cs
@@ -0,0 +1,53 @@
+\ufeff/*
+ * 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.Plugin.Cache
+{
+    using System;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Plugin.Cache;
+
+    /// <summary>
+    /// Wraps user-defined generic <see cref="ICachePluginProvider{TConfig}"/>.
+    /// </summary>
+    internal interface ICachePluginProviderProxy
+    {
+        /// <summary>
+        /// Starts the plugin provider.
+        /// </summary>
+        void Start(IgniteConfiguration igniteConfiguration, CacheConfiguration cacheConfiguration, IIgnite ignite);
+
+        /// <summary>
+        /// Stops the plugin provider.
+        /// </summary>
+        /// <param name="cancel">if set to <c>true</c>, all ongoing operations should be canceled.</param>
+        void Stop(bool cancel);
+
+        /// <summary>
+        /// Called when Ignite has been started and is fully functional.
+        /// <para />
+        /// Use <see cref="IIgnite.Stopping"/> and <see cref="IIgnite.Stopped"/> to track shutdown process.
+        /// </summary>
+        void OnIgniteStart();
+
+        /// <summary>
+        /// Callback to notify that Ignite is about to stop.
+        /// </summary>
+        /// <param name="cancel">if set to <c>true</c>, all ongoing operations should be canceled.</param>
+        void OnIgniteStop(bool cancel);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackOp.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackOp.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackOp.cs
index da61319..27d1124 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackOp.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbackOp.cs
@@ -83,6 +83,10 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         ComputeTaskLocalJobResult = 60,
         ComputeJobExecuteLocal = 61,
         PluginProcessorStop = 62,
-        PluginProcessorIgniteStop = 63
+        PluginProcessorIgniteStop = 63,
+        CachePluginCreate = 64,
+        CachePluginDestroy = 65,
+        CachePluginIgniteStart = 66,
+        CachePluginIgniteStop = 67
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/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 a42342b..53dd47a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -26,6 +26,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
     using System.Runtime.InteropServices;
     using System.Threading;
     using Apache.Ignite.Core.Cache.Affinity;
+    using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary;
@@ -42,10 +43,12 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
     using Apache.Ignite.Core.Impl.Log;
     using Apache.Ignite.Core.Impl.Memory;
     using Apache.Ignite.Core.Impl.Messaging;
+    using Apache.Ignite.Core.Impl.Plugin.Cache;
     using Apache.Ignite.Core.Impl.Resource;
     using Apache.Ignite.Core.Impl.Services;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Log;
+    using Apache.Ignite.Core.Plugin.Cache;
     using Apache.Ignite.Core.Services;
     using UU = UnmanagedUtils;
 
@@ -239,6 +242,10 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             AddHandler(UnmanagedCallbackOp.ComputeJobExecuteLocal, ComputeJobExecuteLocal);
             AddHandler(UnmanagedCallbackOp.PluginProcessorStop, PluginProcessorStop);
             AddHandler(UnmanagedCallbackOp.PluginProcessorIgniteStop, PluginProcessorIgniteStop);
+            AddHandler(UnmanagedCallbackOp.CachePluginCreate, CachePluginCreate);
+            AddHandler(UnmanagedCallbackOp.CachePluginDestroy, CachePluginDestroy);
+            AddHandler(UnmanagedCallbackOp.CachePluginIgniteStart, CachePluginIgniteStart);
+            AddHandler(UnmanagedCallbackOp.CachePluginIgniteStop, CachePluginIgniteStop);
         }
 
         /// <summary>
@@ -1228,6 +1235,57 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         #endregion
 
+        #region PLUGINS
+
+        private long CachePluginCreate(long objPtr)
+        {
+            using (var stream = IgniteManager.Memory.Get(objPtr).GetStream())
+            {
+                var reader = BinaryUtils.Marshaller.StartUnmarshal(stream);
+
+                var cachePluginCfg = reader.ReadObject<ICachePluginConfiguration>();
+                var providerProxy = CachePluginProcessor.CreateProviderProxy(cachePluginCfg);
+
+                var igniteCfg = new IgniteConfiguration(reader, _ignite.Configuration);
+                var cacheCfg = new CacheConfiguration(reader);
+
+                providerProxy.Start(igniteCfg, cacheCfg, _ignite);
+
+                return _handleRegistry.Allocate(providerProxy);
+            }
+        }
+
+        private long CachePluginDestroy(long objPtr, long cancel, long unused, void* arg)
+        {
+            var pluginProvider = _handleRegistry.Get<ICachePluginProviderProxy>(objPtr, true);
+
+            pluginProvider.Stop(cancel != 0);
+
+            _ignite.HandleRegistry.Release(objPtr);
+
+            return 0;
+        }
+
+        private long CachePluginIgniteStart(long objPtr)
+        {
+            var pluginProvider = _handleRegistry.Get<ICachePluginProviderProxy>(objPtr, true);
+
+            pluginProvider.OnIgniteStart();
+
+            return 0;
+        }
+
+        private long CachePluginIgniteStop(long objPtr, long cancel, long unused, void* arg)
+        {
+            var pluginProvider = _handleRegistry.Get<ICachePluginProviderProxy>(objPtr, true);
+
+            pluginProvider.OnIgniteStop(cancel != 0);
+
+            return 0;
+        }
+
+        #endregion
+
         #region HELPERS
 
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/CachePluginProviderTypeAttribute.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/CachePluginProviderTypeAttribute.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/CachePluginProviderTypeAttribute.cs
new file mode 100644
index 0000000..690cf8b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/CachePluginProviderTypeAttribute.cs
@@ -0,0 +1,51 @@
+\ufeff/*
+ * 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.Plugin.Cache
+{
+    using System;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// When applied to <see cref="ICachePluginConfiguration"/>, defines the type of
+    /// <see cref="ICachePluginProvider{T}"/> to instantiate.
+    /// </summary>
+    public class CachePluginProviderTypeAttribute : Attribute
+    {
+        /** */
+        private readonly Type _cachePluginProviderType;
+
+        /// <summary>
+        /// Initializes new instance of <see cref="CachePluginProviderTypeAttribute"/> class.
+        /// </summary>
+        /// <param name="cachePluginProviderType">Type of the plugin provider.</param>
+        public CachePluginProviderTypeAttribute(Type cachePluginProviderType)
+        {
+            IgniteArgumentCheck.NotNull(cachePluginProviderType, "pluginProviderType");
+
+            _cachePluginProviderType = cachePluginProviderType;
+        }
+
+        /// <summary>
+        /// Gets the plugin provider type.
+        /// </summary>
+        public Type CachePluginProviderType
+        {
+            get { return _cachePluginProviderType; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginConfiguration.cs
new file mode 100644
index 0000000..5ea3b51
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginConfiguration.cs
@@ -0,0 +1,50 @@
+\ufeff/*
+ * 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.Plugin.Cache
+{
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Cache.Configuration;
+
+    /// <summary>
+    /// Cache plugin configuration marker interface. Starting point to extend <see cref="CacheConfiguration"/>
+    /// and extend existing cache functionality.
+    /// <para />
+    /// Implementations should be linked to corresponding <see cref="ICachePluginProvider{TConfig}"/>
+    /// via <see cref="CachePluginProviderTypeAttribute"/>.
+    /// <example>
+    /// Example plugin implementation:
+    /// <code>
+    /// [CachePluginProviderType(typeof(MyCachePluginProvider))]
+    /// class MyCachePluginConfig : ICachePluginConfiguration
+    /// {
+    ///     int CustomProperty { get; set; }
+    /// }
+    /// 
+    /// class MyCachePluginProvider : ICachePluginProvider&lt;MyCachePluginConfig&gt;
+    /// {
+    ///     ...
+    /// }
+    /// </code>
+    /// </example>
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")]
+    public interface ICachePluginConfiguration
+    {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginContext.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginContext.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginContext.cs
new file mode 100644
index 0000000..399db1e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginContext.cs
@@ -0,0 +1,47 @@
+/*
+ * 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.Plugin.Cache
+{
+    using Apache.Ignite.Core.Cache.Configuration;
+
+    /// <summary>
+    /// Cache plugin context.
+    /// </summary>
+    public interface ICachePluginContext<out T> where T : ICachePluginConfiguration
+    {
+        /// <summary>
+        /// Gets the Ignite configuration.
+        /// </summary>
+        IgniteConfiguration IgniteConfiguration { get; }
+
+        /// <summary>
+        /// Gets the Ignite cache configuration.
+        /// </summary>
+        CacheConfiguration CacheConfiguration { get; }
+
+        /// <summary>
+        /// Gets the cache plugin configuration.
+        /// </summary>
+        T CachePluginConfiguration { get; }
+
+        /// <summary>
+        /// Gets the Ignite.
+        /// </summary>
+        IIgnite Ignite { get; }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b9901f02/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginProvider.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginProvider.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginProvider.cs
new file mode 100644
index 0000000..73439cd
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/Cache/ICachePluginProvider.cs
@@ -0,0 +1,52 @@
+/*
+ * 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.Plugin.Cache
+{
+    using System;
+
+    /// <summary>
+    /// Cache plugin provider.
+    /// <para />
+    /// Implementations will be passed to remote nodes and should be marked with <see cref="SerializableAttribute"/>.
+    /// </summary>
+    public interface ICachePluginProvider<in TConfig> where TConfig : ICachePluginConfiguration
+    {
+        /// <summary>
+        /// Starts the plugin provider.
+        /// </summary>
+        void Start(ICachePluginContext<TConfig> cachePluginContext);
+
+        /// <summary>
+        /// Stops the plugin provider.
+        /// </summary>
+        /// <param name="cancel">if set to <c>true</c>, all ongoing operations should be canceled.</param>
+        void Stop(bool cancel);
+
+        /// <summary>
+        /// Called when Ignite has been started and is fully functional.
+        /// <para />
+        /// </summary>
+        void OnIgniteStart();
+
+        /// <summary>
+        /// Callback to notify that Ignite is about to stop.
+        /// </summary>
+        /// <param name="cancel">if set to <c>true</c>, all ongoing operations should be canceled.</param>
+        void OnIgniteStop(bool cancel);
+    }
+}
\ No newline at end of file