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/07/21 14:05:40 UTC

[5/6] ignite git commit: IGNITE-5786 .NET: Fix cache store session handling for cross-cache transactions

IGNITE-5786 .NET: Fix cache store session handling for cross-cache transactions

This closes #2331


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

Branch: refs/heads/master
Commit: c1a3b3744f89e27906621e62e9d73281791fcf30
Parents: 6de0571
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Jul 21 17:04:39 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Jul 21 17:04:39 2017 +0300

----------------------------------------------------------------------
 .../dotnet/PlatformDotNetCacheStore.java        | 31 +++++++
 .../Apache.Ignite.Core.Tests.csproj             |  5 ++
 .../Cache/Store/CacheStoreSessionTest.cs        | 90 +++++++++++++-------
 .../Store/CacheStoreSessionTestCodeConfig.cs    | 68 +++++++++++++++
 .../Store/CacheStoreSessionTestSharedFactory.cs | 48 +++++++++++
 .../Cache/Store/CacheStoreTest.cs               | 10 ++-
 .../cache-store-session-shared-factory.xml      | 76 +++++++++++++++++
 .../Config/Cache/Store/cache-store-session.xml  | 20 ++---
 .../Impl/Cache/Store/CacheStoreInternal.cs      | 14 ++-
 9 files changed, 320 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c1a3b374/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
index dd61a54..471eb01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
@@ -47,6 +47,7 @@ import javax.cache.integration.CacheLoaderException;
 import javax.cache.integration.CacheWriterException;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
 
 /**
@@ -90,6 +91,9 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
     /** Key used to distinguish session deployment.  */
     private static final Object KEY_SES = new Object();
 
+    /** Key to designate a set of stores that share current session.  */
+    private static final Object KEY_SES_STORES = new Object();
+
     /** */
     @CacheStoreSessionResource
     private CacheStoreSession ses;
@@ -337,6 +341,23 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
                     writer.writeLong(session());
                     writer.writeString(ses.cacheName());
                     writer.writeBoolean(commit);
+
+                    // When multiple stores (caches) participate in a single transaction,
+                    // they share a single session, but sessionEnd is called on each store.
+                    // Same thing happens on platform side: session is shared; each store must be notified,
+                    // then session should be closed.
+                    Collection<Long> stores = (Collection<Long>) ses.properties().get(KEY_SES_STORES);
+                    assert stores != null;
+
+                    stores.remove(ptr);
+                    boolean last = stores.isEmpty();
+
+                    writer.writeBoolean(last);
+
+                    if (last) {
+                        // Session object has been released on platform side, remove marker.
+                        ses.properties().remove(KEY_SES);
+                    }
                 }
             }, null);
         }
@@ -415,6 +436,16 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
             ses.properties().put(KEY_SES, sesPtr);
         }
 
+        // Keep track of all stores that use current session (cross-cache tx uses single session for all caches).
+        Collection<Long> stores = (Collection<Long>) ses.properties().get(KEY_SES_STORES);
+
+        if (stores == null) {
+            stores = new HashSet<>();
+            ses.properties().put(KEY_SES_STORES, stores);
+        }
+
+        stores.add(ptr);
+
         return sesPtr;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1a3b374/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 90b7970..e4f65bc 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
@@ -89,6 +89,8 @@
     <Compile Include="Cache\Query\Linq\CacheLinqTest.Misc.cs" />
     <Compile Include="Cache\Query\Linq\CacheLinqTest.Custom.cs" />
     <Compile Include="Cache\Query\Linq\CacheLinqTest.Contains.cs" />
+    <Compile Include="Cache\Store\CacheStoreSessionTestCodeConfig.cs" />
+    <Compile Include="Cache\Store\CacheStoreSessionTestSharedFactory.cs" />
     <Compile Include="Deployment\CacheGetFunc.cs" />
     <Compile Include="Deployment\GetAddressFunc.cs" />
     <Compile Include="Deployment\PeerAssemblyLoadingAllApisTest.cs" />
@@ -314,6 +316,9 @@
     <Content Include="Config\Cache\Affinity\affinity-function.xml">
       <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </Content>
+    <Content Include="Config\Cache\Store\cache-store-session-shared-factory.xml">
+      <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
+    </Content>
     <Content Include="Config\Cache\Store\cache-store-session.xml">
       <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </Content>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1a3b374/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs
index 315e285..818948c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTest.cs
@@ -28,16 +28,13 @@ namespace Apache.Ignite.Core.Tests.Cache.Store
     /// <summary>
     /// Tests for store session.
     /// </summary>
-    public sealed class CacheStoreSessionTest
+    public class CacheStoreSessionTest
     {
-        /** Grid name. */
-        private const string IgniteName = "grid";
-
         /** Cache 1 name. */
-        private const string Cache1 = "cache1";
+        protected const string Cache1 = "cache1";
 
         /** Cache 2 name. */
-        private const string Cache2 = "cache2";
+        protected const string Cache2 = "cache2";
 
         /** Operations. */
         private static ConcurrentBag<ICollection<Operation>> _dumps;
@@ -48,11 +45,26 @@ namespace Apache.Ignite.Core.Tests.Cache.Store
         [TestFixtureSetUp]
         public void BeforeTests()
         {
-            Ignition.Start(new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            Ignition.Start(GetIgniteConfiguration());
+        }
+
+        /// <summary>
+        /// Gets the ignite configuration.
+        /// </summary>
+        protected virtual IgniteConfiguration GetIgniteConfiguration()
+        {
+            return new IgniteConfiguration(TestUtils.GetTestConfiguration())
             {
-                IgniteInstanceName = IgniteName,
                 SpringConfigUrl = @"config\cache\store\cache-store-session.xml"
-            });
+            };
+        }
+
+        /// <summary>
+        /// Gets the store count.
+        /// </summary>
+        protected virtual int StoreCount
+        {
+            get { return 2; }
         }
 
         /// <summary>
@@ -61,21 +73,29 @@ namespace Apache.Ignite.Core.Tests.Cache.Store
         [TestFixtureTearDown]
         public void AfterTests()
         {
-            Ignition.StopAll(true);
+            try
+            {
+                TestUtils.AssertHandleRegistryHasItems(Ignition.GetIgnite(), 2, 1000);
+            }
+            finally 
+            {
+                Ignition.StopAll(true);
+            }
         }
         
         /// <summary>
         /// Test basic session API.
         /// </summary>
         [Test]
+        [Timeout(30000)]
         public void TestSession()
         {
             _dumps = new ConcurrentBag<ICollection<Operation>>();
 
-            var ignite = Ignition.GetIgnite(IgniteName);
+            var ignite = Ignition.GetIgnite();
 
-            var cache1 = Ignition.GetIgnite(IgniteName).GetCache<int, int>(Cache1);
-            var cache2 = Ignition.GetIgnite(IgniteName).GetCache<int, int>(Cache2);
+            var cache1 = ignite.GetCache<int, int>(Cache1);
+            var cache2 = ignite.GetCache<int, int>(Cache2);
 
             // 1. Test rollback.
             using (var tx = ignite.GetTransactions().TxStart())
@@ -86,11 +106,15 @@ namespace Apache.Ignite.Core.Tests.Cache.Store
                 tx.Rollback();
             }
 
-            Assert.AreEqual(1, _dumps.Count);
-            var ops = _dumps.First();
-            Assert.AreEqual(1, ops.Count);
+            // SessionEnd is called once per store instance.
+            Assert.AreEqual(StoreCount, _dumps.Count);
 
-            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.SesEnd && !op.Commit));
+            foreach (var ops in _dumps)
+            {
+                var op = ops.Single();
+                Assert.AreEqual(OperationType.SesEnd, op.Type);
+                Assert.IsFalse(op.Commit);
+            }
 
             _dumps = new ConcurrentBag<ICollection<Operation>>();
 
@@ -103,13 +127,17 @@ namespace Apache.Ignite.Core.Tests.Cache.Store
                 tx.Commit();
             }
 
-            Assert.AreEqual(1, _dumps.Count);
-            ops = _dumps.First();
-            Assert.AreEqual(3, ops.Count);
+            Assert.AreEqual(StoreCount, _dumps.Count);
 
-            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Write && Cache1.Equals(op.CacheName) && 1.Equals(op.Key) && 1.Equals(op.Value)));
-            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Write && Cache2.Equals(op.CacheName) && 2.Equals(op.Key) && 2.Equals(op.Value)));
-            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.SesEnd && op.Commit));
+            foreach (var ops in _dumps)
+            {
+                Assert.AreEqual(2 + StoreCount, ops.Count);
+                Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Write
+                                                   && Cache1 == op.CacheName && 1 == op.Key && 1 == op.Value));
+                Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Write
+                                                   && Cache2 == op.CacheName && 2 == op.Key && 2 == op.Value));
+                Assert.AreEqual(StoreCount, ops.Count(op => op.Type == OperationType.SesEnd && op.Commit));
+            }
 
             _dumps = new ConcurrentBag<ICollection<Operation>>();
 
@@ -122,13 +150,17 @@ namespace Apache.Ignite.Core.Tests.Cache.Store
                 tx.Commit();
             }
 
-            Assert.AreEqual(1, _dumps.Count);
-            ops = _dumps.First();
-            Assert.AreEqual(3, ops.Count);
+            Assert.AreEqual(StoreCount, _dumps.Count);
+            foreach (var ops in _dumps)
+            {
+                Assert.AreEqual(2 + StoreCount, ops.Count);
 
-            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Delete && Cache1.Equals(op.CacheName) && 1.Equals(op.Key)));
-            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Delete && Cache2.Equals(op.CacheName) && 2.Equals(op.Key)));
-            Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.SesEnd && op.Commit));
+                Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Delete
+                                                   && Cache1 == op.CacheName && 1 == op.Key));
+                Assert.AreEqual(1, ops.Count(op => op.Type == OperationType.Delete
+                                                   && Cache2 == op.CacheName && 2 == op.Key));
+                Assert.AreEqual(StoreCount, ops.Count(op => op.Type == OperationType.SesEnd && op.Commit));
+            }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1a3b374/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTestCodeConfig.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTestCodeConfig.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTestCodeConfig.cs
new file mode 100644
index 0000000..0b5f474
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTestCodeConfig.cs
@@ -0,0 +1,68 @@
+/*
+ * 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.Cache.Store
+{
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Store;
+    using Apache.Ignite.Core.Common;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests store session with programmatic configuration (uses different store factory on Java side).
+    /// </summary>
+    [TestFixture]
+    public class CacheStoreSessionTestCodeConfig : CacheStoreSessionTest
+    {
+        /** <inheritdoc /> */
+        protected override IgniteConfiguration GetIgniteConfiguration()
+        {
+            return new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                CacheConfiguration = new[]
+                {
+                    new CacheConfiguration(Cache1)
+                    {
+                        AtomicityMode = CacheAtomicityMode.Transactional,
+                        ReadThrough = true,
+                        WriteThrough = true,
+                        CacheStoreFactory = new StoreFactory()
+                    },
+                    new CacheConfiguration(Cache2)
+                    {
+                        AtomicityMode = CacheAtomicityMode.Transactional,
+                        ReadThrough = true,
+                        WriteThrough = true,
+                        CacheStoreFactory = new StoreFactory()
+                    }
+                }
+            };
+        }
+
+        /// <summary>
+        /// Store factory.
+        /// </summary>
+        private class StoreFactory : IFactory<ICacheStore>
+        {
+            /** <inheritdoc /> */
+            public ICacheStore CreateInstance()
+            {
+                return new Store();
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1a3b374/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTestSharedFactory.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTestSharedFactory.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTestSharedFactory.cs
new file mode 100644
index 0000000..2af5915
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreSessionTestSharedFactory.cs
@@ -0,0 +1,48 @@
+/*
+ * 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.Cache.Store
+{
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Session test with shared PlatformDotNetCacheStoreFactory,
+    /// which causes the same store insance to be used for both caches.
+    /// </summary>
+    [TestFixture]
+    public class CacheStoreSessionTestSharedFactory : CacheStoreSessionTest
+    {
+        /** <inheritdoc /> */
+        protected override IgniteConfiguration GetIgniteConfiguration()
+        {
+            return new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                SpringConfigUrl = @"config\cache\store\cache-store-session-shared-factory.xml"
+            };
+        }
+
+        /** <inheritdoc /> */
+        protected override int StoreCount
+        {
+            get
+            {
+                // Shared PlatformDotNetCacheStoreFactory results in a single store instance.
+                return 1;
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1a3b374/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs
index e05f4bd..d3e4ab6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Store/CacheStoreTest.cs
@@ -64,7 +64,15 @@ namespace Apache.Ignite.Core.Tests.Cache.Store
         [TestFixtureTearDown]
         public void AfterTests()
         {
-            Ignition.StopAll(true);
+            try
+            {
+                // 3 stores are expected in HandleRegistry.
+                TestUtils.AssertHandleRegistryHasItems(Ignition.GetIgnite(), 3, 1000);
+            }
+            finally
+            {
+                Ignition.StopAll(true);
+            }
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1a3b374/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session-shared-factory.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session-shared-factory.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session-shared-factory.xml
new file mode 100644
index 0000000..05515c4
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session-shared-factory.xml
@@ -0,0 +1,76 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="storeFactory" class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+        <property name="typeName" value="Apache.Ignite.Core.Tests.Cache.Store.CacheStoreSessionTest+Store, Apache.Ignite.Core.Tests"/>
+    </bean>
+  
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache1"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="readThrough" value="true"/>
+
+                    <property name="cacheStoreFactory" ref="storeFactory" />
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="cacheMode" value="LOCAL"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeThrough" value="true"/>
+                    <property name="readThrough" value="true"/>
+
+                    <property name="cacheStoreFactory" ref="storeFactory" />
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+                <property name="socketTimeout" value="300" />
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1a3b374/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session.xml
index 3cc9efa..14dc78e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/Cache/Store/cache-store-session.xml
@@ -25,18 +25,10 @@
         http://www.springframework.org/schema/beans/spring-beans.xsd
         http://www.springframework.org/schema/util
         http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="storeFactory" class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
-        <property name="typeName" value="Apache.Ignite.Core.Tests.Cache.Store.CacheStoreSessionTest+Store, Apache.Ignite.Core.Tests"/>
-    </bean>
-  
     <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <property name="localHost" value="127.0.0.1"/>
         <property name="connectorConfiguration"><null/></property>
 
-        <property name="includeEventTypes">
-            <util:constant static-field="org.apache.ignite.events.EventType.EVTS_CACHE"/>
-        </property>
-      
         <property name="cacheConfiguration">
             <list>
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
@@ -46,7 +38,11 @@
                     <property name="writeThrough" value="true"/>
                     <property name="readThrough" value="true"/>
 
-                    <property name="cacheStoreFactory" ref="storeFactory" />
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+                            <property name="typeName" value="Apache.Ignite.Core.Tests.Cache.Store.CacheStoreSessionTest+Store, Apache.Ignite.Core.Tests"/>
+                        </bean>
+                    </property>
                 </bean>
 
                 <bean class="org.apache.ignite.configuration.CacheConfiguration">
@@ -56,7 +52,11 @@
                     <property name="writeThrough" value="true"/>
                     <property name="readThrough" value="true"/>
 
-                    <property name="cacheStoreFactory" ref="storeFactory" />
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactory">
+                            <property name="typeName" value="Apache.Ignite.Core.Tests.Cache.Store.CacheStoreSessionTest+Store, Apache.Ignite.Core.Tests"/>
+                        </bean>
+                    </property>
                 </bean>
             </list>
         </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1a3b374/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs
index f147579..df4c1ae 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStoreInternal.cs
@@ -111,6 +111,8 @@ namespace Apache.Ignite.Core.Impl.Cache.Store
 
             CacheStoreSession ses = grid.HandleRegistry.Get<CacheStoreSession>(sesId, true);
 
+            // Session cache name may change in cross-cache transaction.
+            // Single session is used for all stores in cross-cache transactions.
             ses.CacheName = rawReader.ReadString();
 
             _sesProxy.SetSession(ses);
@@ -223,11 +225,19 @@ namespace Apache.Ignite.Core.Impl.Cache.Store
                         break;
 
                     case OpSesEnd:
-                        grid.HandleRegistry.Release(sesId);
+                    {
+                        var commit = rawReader.ReadBoolean();
+                        var last = rawReader.ReadBoolean();
 
-                        _store.SessionEnd(rawReader.ReadBoolean());
+                        if (last)
+                        {
+                            grid.HandleRegistry.Release(sesId);
+                        }
+
+                        _store.SessionEnd(commit);
 
                         break;
+                    }
 
                     default:
                         throw new IgniteException("Invalid operation type: " + opType);