You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dm...@apache.org on 2017/04/11 01:38:42 UTC

[42/50] [abbrv] ignite git commit: IGNITE-4935 .NET: IgniteConfiguration.EventStorageSpi

IGNITE-4935 .NET: IgniteConfiguration.EventStorageSpi

This closes #1761


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

Branch: refs/heads/ignite-1192
Commit: 4425b40f93f4b61e815dd7352022b5fcbe4765f0
Parents: 118baeb
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Apr 10 12:33:52 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Apr 10 12:34:12 2017 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       |  29 +++++-
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  66 +++++++++---
 .../IgniteConfigurationSerializerTest.cs        |  13 ++-
 .../IgniteConfigurationTest.cs                  |  15 ++-
 .../Apache.Ignite.Core.csproj                   |   3 +
 .../Events/IEventStorageSpi.cs                  |  33 ++++++
 .../Events/MemoryEventStorageSpi.cs             | 100 +++++++++++++++++++
 .../Events/NoopEventStorageSpi.cs               |  27 +++++
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  48 ++++++++-
 .../IgniteConfigurationSection.xsd              |  22 ++++
 .../Plugin/IPluginConfiguration.cs              |   4 +-
 11 files changed, 338 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/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 7fb25c8..f1ef625 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
@@ -67,9 +67,11 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.spi.eventstorage.EventStorageSpi;
+import org.apache.ignite.spi.eventstorage.NoopEventStorageSpi;
+import org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi;
 import org.apache.ignite.spi.swapspace.SwapSpaceSpi;
 import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
-import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpiMBean;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 
@@ -673,6 +675,18 @@ public class PlatformConfigurationUtils {
                 assert swapType == SWAP_TYP_NONE;
         }
 
+        switch (in.readByte()) {
+            case 1:
+                cfg.setEventStorageSpi(new NoopEventStorageSpi());
+                break;
+
+            case 2:
+                cfg.setEventStorageSpi(new MemoryEventStorageSpi()
+                        .setExpireCount(in.readLong())
+                        .setExpireAgeMs(in.readLong()));
+                break;
+        }
+
         readPluginConfiguration(cfg, in);
     }
 
@@ -1116,6 +1130,19 @@ public class PlatformConfigurationUtils {
             w.writeByte(SWAP_TYP_NONE);
         }
 
+        EventStorageSpi eventStorageSpi = cfg.getEventStorageSpi();
+
+        if (eventStorageSpi == null) {
+            w.writeByte((byte) 0);
+        } else if (eventStorageSpi instanceof NoopEventStorageSpi) {
+            w.writeByte((byte) 1);
+        } else if (eventStorageSpi instanceof MemoryEventStorageSpi) {
+            w.writeByte((byte) 2);
+
+            w.writeLong(((MemoryEventStorageSpi)eventStorageSpi).getExpireCount());
+            w.writeLong(((MemoryEventStorageSpi)eventStorageSpi).getExpireAgeMs());
+        }
+
         w.writeString(cfg.getIgniteHome());
 
         w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit());

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
index 994b334..329d307 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
@@ -50,15 +50,6 @@ namespace Apache.Ignite.Core.Tests
 
         /** */
         private IIgnite[] _grids;
-        
-        /** */
-        public static int IdGen;
-
-        [TestFixtureTearDown]
-        public void FixtureTearDown()
-        {
-            StopGrids();
-        }
 
         /// <summary>
         /// Executes before each test.
@@ -74,7 +65,7 @@ namespace Apache.Ignite.Core.Tests
         /// Executes after each test.
         /// </summary>
         [TearDown]
-        public virtual void TearDown()
+        public void TearDown()
         {
             try
             {
@@ -97,6 +88,15 @@ namespace Apache.Ignite.Core.Tests
         }
 
         /// <summary>
+        /// Fixture tear down.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            StopGrids();
+        }
+
+        /// <summary>
         /// Tests enable/disable of event types.
         /// </summary>
         [Test]
@@ -611,6 +611,36 @@ namespace Apache.Ignite.Core.Tests
         }
 
         /// <summary>
+        /// Tests the event store configuration.
+        /// </summary>
+        [Test]
+        public void TestConfiguration()
+        {
+            var cfg = _grid1.GetConfiguration().EventStorageSpi as MemoryEventStorageSpi;
+
+            Assert.IsNotNull(cfg);
+
+            Assert.AreEqual(MemoryEventStorageSpi.DefaultExpirationTimeout, cfg.ExpirationTimeout);
+            Assert.AreEqual(MemoryEventStorageSpi.DefaultMaxEventCount, cfg.MaxEventCount);
+
+            // Test user-defined event storage.
+            var igniteCfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                IgniteInstanceName = "grid4",
+                EventStorageSpi = new MyEventStorage()
+            };
+
+            var ex = Assert.Throws<IgniteException>(() => Ignition.Start(igniteCfg));
+            Assert.AreEqual("Failed to start Ignite.NET, check inner exception for details", ex.Message);
+
+            Assert.IsNotNull(ex.InnerException);
+            Assert.AreEqual("Unsupported IgniteConfiguration.EventStorageSpi: " +
+                            "'Apache.Ignite.Core.Tests.MyEventStorage'. Supported implementations: " +
+                            "'Apache.Ignite.Core.Events.NoopEventStorageSpi', " +
+                            "'Apache.Ignite.Core.Events.MemoryEventStorageSpi'.", ex.InnerException.Message);
+        }
+
+        /// <summary>
         /// Checks base event fields serialization.
         /// </summary>
         /// <param name="evt">The evt.</param>
@@ -666,7 +696,7 @@ namespace Apache.Ignite.Core.Tests
         /// <summary>
         /// Gets the Ignite configuration.
         /// </summary>
-        private static IgniteConfiguration Configuration(string springConfigUrl)
+        private static IgniteConfiguration GetConfiguration(string springConfigUrl)
         {
             return new IgniteConfiguration
             {
@@ -679,7 +709,8 @@ namespace Apache.Ignite.Core.Tests
                     {
                         new BinaryTypeConfiguration(typeof (RemoteEventBinarizableFilter))
                     }
-                }
+                },
+                EventStorageSpi = new MemoryEventStorageSpi()
             };
         }
 
@@ -761,9 +792,9 @@ namespace Apache.Ignite.Core.Tests
             if (_grid1 != null)
                 return;
 
-            _grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
-            _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
-            _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+            _grid1 = Ignition.Start(GetConfiguration("config\\compute\\compute-grid1.xml"));
+            _grid2 = Ignition.Start(GetConfiguration("config\\compute\\compute-grid2.xml"));
+            _grid3 = Ignition.Start(GetConfiguration("config\\compute\\compute-grid3.xml"));
 
             _grids = new[] {_grid1, _grid2, _grid3};
         }
@@ -1069,4 +1100,9 @@ namespace Apache.Ignite.Core.Tests
             throw new NotImplementedException();
         }
     }
+
+    public class MyEventStorage : IEventStorageSpi
+    {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/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 6015f80..1d39e69 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -137,6 +137,7 @@ namespace Apache.Ignite.Core.Tests
                             <pluginConfigurations>
                                 <iPluginConfiguration type='Apache.Ignite.Core.Tests.Plugin.TestIgnitePluginConfiguration, Apache.Ignite.Core.Tests' />
                             </pluginConfigurations>
+                            <eventStorageSpi type='MemoryEventStorageSpi' expirationTimeout='00:00:23.45' maxEventCount='129' />
                         </igniteConfig>";
 
             var cfg = IgniteConfiguration.FromXml(xml);
@@ -255,6 +256,11 @@ namespace Apache.Ignite.Core.Tests
 
             var cachePlugCfg = cacheCfg.PluginConfigurations.Cast<CachePluginConfiguration>().Single();
             Assert.AreEqual("baz", cachePlugCfg.TestProperty);
+
+            var eventStorage = cfg.EventStorageSpi as MemoryEventStorageSpi;
+            Assert.IsNotNull(eventStorage);
+            Assert.AreEqual(23.45, eventStorage.ExpirationTimeout.TotalSeconds);
+            Assert.AreEqual(129, eventStorage.MaxEventCount);
         }
 
         /// <summary>
@@ -796,7 +802,12 @@ namespace Apache.Ignite.Core.Tests
                     ReadStripesNumber = 77,
                     BaseDirectory = "test"
                 },
-                PluginConfigurations = new[] {new TestIgnitePluginConfiguration() }
+                PluginConfigurations = new[] {new TestIgnitePluginConfiguration() },
+                EventStorageSpi = new MemoryEventStorageSpi
+                {
+                    ExpirationTimeout = TimeSpan.FromMilliseconds(12345),
+                    MaxEventCount = 257
+                }
             };
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 5063c21..f9f8a61 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -83,6 +83,7 @@ namespace Apache.Ignite.Core.Tests
             CheckDefaultValueAttributes(new AtomicConfiguration());
             CheckDefaultValueAttributes(new TransactionConfiguration());
             CheckDefaultValueAttributes(new FileSwapSpaceSpi());
+            CheckDefaultValueAttributes(new MemoryEventStorageSpi());
         }
 
         /// <summary>
@@ -198,6 +199,13 @@ namespace Apache.Ignite.Core.Tests
 
                 Assert.IsNotNull(resCfg.PluginConfigurations);
                 Assert.AreEqual(cfg.PluginConfigurations, resCfg.PluginConfigurations);
+
+                var eventCfg = cfg.EventStorageSpi as MemoryEventStorageSpi;
+                var resEventCfg = resCfg.EventStorageSpi as MemoryEventStorageSpi;
+                Assert.IsNotNull(eventCfg);
+                Assert.IsNotNull(resEventCfg);
+                Assert.AreEqual(eventCfg.ExpirationTimeout, resEventCfg.ExpirationTimeout);
+                Assert.AreEqual(eventCfg.MaxEventCount, resEventCfg.MaxEventCount);
             }
         }
 
@@ -546,7 +554,12 @@ namespace Apache.Ignite.Core.Tests
                         }
                     }
                 },
-                PluginConfigurations = new[] { new TestIgnitePluginConfiguration() }
+                PluginConfigurations = new[] { new TestIgnitePluginConfiguration() },
+                EventStorageSpi = new MemoryEventStorageSpi
+                {
+                    ExpirationTimeout = TimeSpan.FromSeconds(5),
+                    MaxEventCount = 10
+                }
             };
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/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 2a174eb..6553f1a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -93,6 +93,9 @@
   <ItemGroup>
     <Compile Include="Binary\BinaryArrayEqualityComparer.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
+    <Compile Include="Events\IEventStorageSpi.cs" />
+    <Compile Include="Events\MemoryEventStorageSpi.cs" />
+    <Compile Include="Events\NoopEventStorageSpi.cs" />
     <Compile Include="Impl\Binary\BinaryFieldEqualityComparer.cs" />
     <Compile Include="Binary\BinaryReflectiveSerializer.cs" />
     <Compile Include="Common\JavaException.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventStorageSpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventStorageSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventStorageSpi.cs
new file mode 100644
index 0000000..f04c30c
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Events/IEventStorageSpi.cs
@@ -0,0 +1,33 @@
+\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.Events
+{
+    using System.Diagnostics.CodeAnalysis;
+
+    /// <summary>
+    /// Manages local event storage.
+    /// <para />
+    /// Only predefined implementations are supported: 
+    /// <see cref="NoopEventStorageSpi"/>, <see cref="MemoryEventStorageSpi"/>.
+    /// </summary>
+    [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")]
+    public interface IEventStorageSpi
+    {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/modules/platforms/dotnet/Apache.Ignite.Core/Events/MemoryEventStorageSpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Events/MemoryEventStorageSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Events/MemoryEventStorageSpi.cs
new file mode 100644
index 0000000..e00c155
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Events/MemoryEventStorageSpi.cs
@@ -0,0 +1,100 @@
+\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.Events
+{
+    using System;
+    using System.ComponentModel;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Binary;
+
+    /// <summary>
+    /// In-memory event storage.
+    /// </summary>
+    public class MemoryEventStorageSpi : IEventStorageSpi
+    {
+        /// <summary>
+        /// Default event count limit.
+        /// </summary>
+        public const long DefaultMaxEventCount = 10000;
+
+        /// <summary>
+        /// The default expiration timeout.
+        /// </summary>
+        public static readonly TimeSpan DefaultExpirationTimeout = TimeSpan.FromSeconds(-1);
+
+        /// <summary>
+        /// Gets or sets the expiration timeout for stored events.
+        /// Negative value means no expiration.
+        /// Defaults to -1 second.
+        /// </summary>
+        [DefaultValue(typeof(TimeSpan), "-0:0:1")]
+        public TimeSpan ExpirationTimeout { get; set; }
+
+        /// <summary>
+        /// Gets or sets the maximum event count to store. When this limit is reached, older events are removed.
+        /// Defaults to <see cref="DefaultMaxEventCount"/>.
+        /// </summary>
+        [DefaultValue(DefaultMaxEventCount)]
+        public long MaxEventCount { get; set; }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="MemoryEventStorageSpi"/> class.
+        /// </summary>
+        public MemoryEventStorageSpi()
+        {
+            ExpirationTimeout = DefaultExpirationTimeout;
+            MaxEventCount = DefaultMaxEventCount;
+        }
+
+        /// <summary>
+        /// Reads instance.
+        /// </summary>
+        internal static MemoryEventStorageSpi Read(IBinaryRawReader reader)
+        {
+            Debug.Assert(reader != null);
+
+            var eventCount = reader.ReadLong();
+            var timeout = reader.ReadLong();
+
+            return new MemoryEventStorageSpi
+            {
+                MaxEventCount = eventCount,
+                ExpirationTimeout = timeout < 0 || timeout > TimeSpan.MaxValue.TotalMilliseconds
+                    ? DefaultExpirationTimeout
+                    : TimeSpan.FromMilliseconds(timeout)
+            };
+        }
+
+        /// <summary>
+        /// Writes this instance.
+        /// </summary>
+        internal void Write(IBinaryRawWriter writer)
+        {
+            writer.WriteLong(MaxEventCount);
+
+            if (ExpirationTimeout == TimeSpan.MaxValue || ExpirationTimeout < TimeSpan.Zero)
+            {
+                writer.WriteLong(long.MaxValue);
+            }
+            else
+            {
+                writer.WriteLong((long) ExpirationTimeout.TotalMilliseconds);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/modules/platforms/dotnet/Apache.Ignite.Core/Events/NoopEventStorageSpi.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Events/NoopEventStorageSpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Events/NoopEventStorageSpi.cs
new file mode 100644
index 0000000..c69531f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Events/NoopEventStorageSpi.cs
@@ -0,0 +1,27 @@
+\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.Events
+{
+    /// <summary>
+    /// No-op event storage interface, does not store anything.
+    /// </summary>
+    public class NoopEventStorageSpi : IEventStorageSpi
+    {
+        // No-op.
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/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 28842e9..20e2be8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -30,6 +30,7 @@
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cluster;
+    using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Communication;
     using Apache.Ignite.Core.Communication.Tcp;
     using Apache.Ignite.Core.DataStructures.Configuration;
@@ -348,7 +349,33 @@
             // Swap space
             SwapSpaceSerializer.Write(writer, SwapSpaceSpi);
 
-            // Plugins
+            // Event storage
+            if (EventStorageSpi == null)
+            {
+                writer.WriteByte(0);
+            }
+            else if (EventStorageSpi is NoopEventStorageSpi)
+            {
+                writer.WriteByte(1);
+            }
+            else
+            {
+                var memEventStorage = EventStorageSpi as MemoryEventStorageSpi;
+
+                if (memEventStorage == null)
+                {
+                    throw new IgniteException(string.Format(
+                        "Unsupported IgniteConfiguration.EventStorageSpi: '{0}'. " +
+                        "Supported implementations: '{1}', '{2}'.",
+                        EventStorageSpi.GetType(), typeof(NoopEventStorageSpi), typeof(MemoryEventStorageSpi)));
+                }
+
+                writer.WriteByte(2);
+
+                memEventStorage.Write(writer);
+            }
+
+            // Plugins (should be last)
             if (PluginConfigurations != null)
             {
                 var pos = writer.Stream.Position;
@@ -484,6 +511,17 @@
 
             // Swap
             SwapSpaceSpi = SwapSpaceSerializer.Read(r);
+
+            // Event storage
+            switch (r.ReadByte())
+            {
+                case 1: EventStorageSpi = new NoopEventStorageSpi();
+                    break;
+
+                case 2:
+                    EventStorageSpi = MemoryEventStorageSpi.Read(r);
+                    break;
+            }
         }
 
         /// <summary>
@@ -923,5 +961,13 @@
         /// </summary>
         [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")]
         public ICollection<IPluginConfiguration> PluginConfigurations { get; set; }
+
+        /// <summary>
+        /// Gets or sets the event storage interface.
+        /// <para />
+        /// Only predefined implementations are supported: 
+        /// <see cref="NoopEventStorageSpi"/>, <see cref="MemoryEventStorageSpi"/>.
+        /// </summary>
+        public IEventStorageSpi EventStorageSpi { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/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 375d128..3691aa0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -1208,6 +1208,28 @@
                         </xs:attribute>
                     </xs:complexType>
                 </xs:element>
+                <xs:element name="eventStorageSpi" minOccurs="0">
+                    <xs:annotation>
+                        <xs:documentation>Event storage space SPI.</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:attribute name="expirationTimeout" type="xs:string">
+                            <xs:annotation>
+                                <xs:documentation>Expiration timeout for stored events. Negative for unlimited.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                        <xs:attribute name="maxEventCount" type="xs:long">
+                            <xs:annotation>
+                                <xs:documentation>Maximum event count to store.</xs:documentation>
+                            </xs:annotation>
+                        </xs:attribute>
+                    </xs:complexType>
+                </xs:element>
                 <xs:element name="pluginConfigurations" minOccurs="0">
                     <xs:annotation>
                         <xs:documentation>Plugin configurations.</xs:documentation>

http://git-wip-us.apache.org/repos/asf/ignite/blob/4425b40f/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/IPluginConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/IPluginConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/IPluginConfiguration.cs
index 4d594f4..47c6599 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/IPluginConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Plugin/IPluginConfiguration.cs
@@ -17,11 +17,10 @@
 
 namespace Apache.Ignite.Core.Plugin
 {
-    using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Binary;
 
     /// <summary>
-    /// Plugin configuration marker interface.
+    /// Plugin configuration interface.
     /// <para />
     /// Implementations should be linked to corresponding <see cref="IPluginProvider{TConfig}"/>
     /// via <see cref="PluginProviderTypeAttribute"/>.
@@ -41,7 +40,6 @@ namespace Apache.Ignite.Core.Plugin
     /// </code>
     /// </example>
     /// </summary>
-    [SuppressMessage("Microsoft.Design", "CA1040:AvoidEmptyInterfaces")]
     public interface IPluginConfiguration
     {
         /// <summary>