You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/02 10:57:26 UTC

[01/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-876-2 21b6a033e -> c74ddc421


IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: b97441f994bb34b23c54848f6520f506b4094590
Parents: 896b426
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu May 14 19:12:43 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu May 14 19:12:43 2015 -0700

----------------------------------------------------------------------
 .../store/jdbc/CacheJdbcPersonStore.java        | 137 +++----------
 .../store/jdbc/CacheJdbcStoreExample.java       |  14 ++
 .../cache/store/CacheStoreSessionListener.java  |  27 +++
 .../jdbc/CacheStoreSessionJdbcListener.java     | 100 +++++++++
 .../configuration/CacheConfiguration.java       |  26 +++
 .../configuration/IgniteConfiguration.java      |  31 ++-
 .../cache/GridCacheSharedContext.java           |   4 -
 .../cache/store/CacheOsStoreManager.java        |   1 -
 .../store/GridCacheStoreManagerAdapter.java     | 134 +++++++++---
 .../cache/transactions/IgniteTxAdapter.java     |  24 ++-
 .../transactions/IgniteTxLocalAdapter.java      |  57 ++++--
 ...cheStoreSessionListenerAbstractSelfTest.java | 204 +++++++++++++++++++
 .../CacheStoreSessionJdbcListenerSelfTest.java  | 145 +++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 .../CacheStoreSessionHibernateListener.java     | 106 ++++++++++
 ...heStoreSessionHibernateListenerSelfTest.java | 150 ++++++++++++++
 modules/spring/pom.xml                          |  14 ++
 .../spring/CacheStoreSessionSpringListener.java | 125 ++++++++++++
 ...CacheStoreSessionSpringListenerSelfTest.java | 179 ++++++++++++++++
 .../testsuites/IgniteSpringTestSuite.java       |   3 +
 20 files changed, 1307 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
index 791f861..856512b 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
@@ -19,26 +19,21 @@ package org.apache.ignite.examples.datagrid.store.jdbc;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
 import org.apache.ignite.examples.datagrid.store.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.jetbrains.annotations.*;
 
 import javax.cache.*;
 import javax.cache.integration.*;
 import java.sql.*;
-import java.util.*;
 
 /**
  * Example of {@link CacheStore} implementation that uses JDBC
  * transaction with cache transactions and maps {@link Long} to {@link Person}.
- *
  */
 public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
-    /** Transaction metadata attribute name. */
-    private static final String ATTR_NAME = "SIMPLE_STORE_CONNECTION";
-
-    /** Auto-injected store session. */
+    /** Store session. */
     @CacheStoreSessionResource
     private CacheStoreSession ses;
 
@@ -58,11 +53,12 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
      * @throws IgniteException If failed.
      */
     private void prepareDb() throws IgniteException {
-        try (Connection conn = openConnection(false); Statement st = conn.createStatement()) {
+        try (
+            Connection conn = DriverManager.getConnection("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
+            Statement st = conn.createStatement()
+        ) {
             st.execute("create table if not exists PERSONS (id number unique, firstName varchar(255), " +
                 "lastName varchar(255))");
-
-            conn.commit();
         }
         catch (SQLException e) {
             throw new IgniteException("Failed to create database table.", e);
@@ -70,32 +66,11 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
     }
 
     /** {@inheritDoc} */
-    @Override public void sessionEnd(boolean commit) {
-        Map<String, Connection> props = ses.properties();
-
-        try (Connection conn = props.remove(ATTR_NAME)) {
-            if (conn != null) {
-                if (commit)
-                    conn.commit();
-                else
-                    conn.rollback();
-            }
-
-            System.out.println(">>> Transaction ended [commit=" + commit + ']');
-        }
-        catch (SQLException e) {
-            throw new CacheWriterException("Failed to end transaction: " + ses.transaction(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override public Person load(Long key) {
         System.out.println(">>> Loading key: " + key);
 
-        Connection conn = null;
-
         try {
-            conn = connection();
+            Connection conn = connection();
 
             try (PreparedStatement st = conn.prepareStatement("select * from PERSONS where id=?")) {
                 st.setString(1, key.toString());
@@ -109,9 +84,6 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
         catch (SQLException e) {
             throw new CacheLoaderException("Failed to load object: " + key, e);
         }
-        finally {
-            end(conn);
-        }
 
         return null;
     }
@@ -124,10 +96,8 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
 
         System.out.println(">>> Putting [key=" + key + ", val=" + val +  ']');
 
-        Connection conn = null;
-
         try {
-            conn = connection();
+            Connection conn = connection();
 
             int updated;
 
@@ -157,19 +127,14 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
         catch (SQLException e) {
             throw new CacheLoaderException("Failed to put object [key=" + key + ", val=" + val + ']', e);
         }
-        finally {
-            end(conn);
-        }
     }
 
     /** {@inheritDoc} */
     @Override public void delete(Object key) {
         System.out.println(">>> Removing key: " + key);
 
-        Connection conn = null;
-
         try {
-            conn = connection();
+            Connection conn = connection();
 
             try (PreparedStatement st = conn.prepareStatement("delete from PERSONS where id=?")) {
                 st.setLong(1, (Long)key);
@@ -180,9 +145,6 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
         catch (SQLException e) {
             throw new CacheWriterException("Failed to remove object: " + key, e);
         }
-        finally {
-            end(conn);
-        }
     }
 
     /** {@inheritDoc} */
@@ -192,22 +154,23 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
 
         final int entryCnt = (Integer)args[0];
 
-        try (Connection conn = connection()) {
-            try (PreparedStatement st = conn.prepareStatement("select * from PERSONS")) {
-                try (ResultSet rs = st.executeQuery()) {
-                    int cnt = 0;
+        Connection conn = connection();
 
-                    while (cnt < entryCnt && rs.next()) {
-                        Person person = new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
+        try (
+            PreparedStatement st = conn.prepareStatement("select * from PERSONS");
+            ResultSet rs = st.executeQuery()
+        ) {
+            int cnt = 0;
 
-                        clo.apply(person.getId(), person);
+            while (cnt < entryCnt && rs.next()) {
+                Person person = new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
 
-                        cnt++;
-                    }
+                clo.apply(person.getId(), person);
 
-                    System.out.println(">>> Loaded " + cnt + " values into cache.");
-                }
+                cnt++;
             }
+
+            System.out.println(">>> Loaded " + cnt + " values into cache.");
         }
         catch (SQLException e) {
             throw new CacheLoaderException("Failed to load values from cache store.", e);
@@ -215,61 +178,11 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
     }
 
     /**
-     * @return Connection.
-     * @throws SQLException In case of error.
-     */
-    private Connection connection() throws SQLException  {
-        // If there is an ongoing transaction,
-        // we must reuse the same connection.
-        if (ses.isWithinTransaction()) {
-            Map<Object, Object> props = ses.properties();
-
-            Connection conn = (Connection)props.get(ATTR_NAME);
-
-            if (conn == null) {
-                conn = openConnection(false);
-
-                // Store connection in session properties, so it can be accessed
-                // for other operations on the same transaction.
-                props.put(ATTR_NAME, conn);
-            }
-
-            return conn;
-        }
-        // Transaction can be null in case of simple load or put operation.
-        else
-            return openConnection(true);
-    }
-
-    /**
-     * Closes allocated resources depending on transaction status.
-     *
-     * @param conn Allocated connection.
-     */
-    private void end(@Nullable Connection conn) {
-        if (!ses.isWithinTransaction() && conn != null) {
-            // Close connection right away if there is no transaction.
-            try {
-                conn.close();
-            }
-            catch (SQLException ignored) {
-                // No-op.
-            }
-        }
-    }
-
-    /**
-     * Gets connection from a pool.
+     * Gets JDBC connection attached to current session.
      *
-     * @param autocommit {@code true} If connection should use autocommit mode.
-     * @return Pooled connection.
-     * @throws SQLException In case of error.
+     * @return Connection.
      */
-    private Connection openConnection(boolean autocommit) throws SQLException {
-        Connection conn = DriverManager.getConnection("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
-
-        conn.setAutoCommit(autocommit);
-
-        return conn;
+    private Connection connection() {
+        return ses.<String, Connection>properties().get(CacheStoreSessionJdbcListener.JDBC_CONN_KEY);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
index 1cb73c9..82e1079 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
@@ -18,10 +18,13 @@
 package org.apache.ignite.examples.datagrid.store.jdbc;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.examples.*;
 import org.apache.ignite.examples.datagrid.store.*;
 import org.apache.ignite.transactions.*;
+import org.h2.jdbcx.*;
 
 import javax.cache.configuration.*;
 import java.util.*;
@@ -71,6 +74,17 @@ public class CacheJdbcStoreExample {
             // Configure JDBC store.
             cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(CacheJdbcPersonStore.class));
 
+            // Configure JDBC session listener.
+            cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
+                @Override public CacheStoreSessionListener create() {
+                    CacheStoreSessionJdbcListener lsnr = new CacheStoreSessionJdbcListener();
+
+                    lsnr.setDataSource(JdbcConnectionPool.create("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1", "", ""));
+
+                    return lsnr;
+                }
+            });
+
             cacheCfg.setReadThrough(true);
             cacheCfg.setWriteThrough(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
new file mode 100644
index 0000000..e57714b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
@@ -0,0 +1,27 @@
+/*
+ * 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.cache.store;
+
+/**
+ * TODO
+ */
+public interface CacheStoreSessionListener {
+    public void onSessionStart(CacheStoreSession ses);
+
+    public void onSessionEnd(CacheStoreSession ses, boolean commit);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
new file mode 100644
index 0000000..9622063
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
@@ -0,0 +1,100 @@
+/*
+ * 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.cache.store.jdbc;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
+
+import javax.cache.integration.*;
+import javax.sql.*;
+import java.sql.*;
+import java.util.*;
+
+/**
+ * TODO
+ */
+public class CacheStoreSessionJdbcListener implements CacheStoreSessionListener {
+    /** Session key for JDBC connection. */
+    public static final String JDBC_CONN_KEY = "__jdbc_conn_";
+
+    /** Data source. */
+    private DataSource dataSrc;
+
+    /** Store session. */
+    @CacheStoreSessionResource
+    private CacheStoreSession ses;
+
+    /**
+     * Sets data source.
+     *
+     * @param dataSrc Data source.
+     */
+    public void setDataSource(DataSource dataSrc) {
+        A.notNull(dataSrc, "dataSrc");
+
+        this.dataSrc = dataSrc;
+    }
+
+    /**
+     * Gets data source.
+     *
+     * @return Data source.
+     */
+    public DataSource getDataSource() {
+        return dataSrc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionStart(CacheStoreSession ses) {
+        Map<String, Connection> props = ses.properties();
+
+        if (!props.containsKey(JDBC_CONN_KEY)) {
+            try {
+                Connection conn = dataSrc.getConnection();
+
+                conn.setAutoCommit(false);
+
+                props.put(JDBC_CONN_KEY, conn);
+            }
+            catch (SQLException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+        Connection conn = ses.<String, Connection>properties().remove(JDBC_CONN_KEY);
+
+        if (conn != null) {
+            try {
+                if (commit)
+                    conn.commit();
+                else
+                    conn.rollback();
+            }
+            catch (SQLException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+            finally {
+                U.closeQuiet(conn);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index df6b2ee..33a5711c 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -316,6 +316,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Cache topology validator. */
     private TopologyValidator topValidator;
 
+    /** Cache store session listeners. */
+    private Factory<? extends CacheStoreSessionListener>[] storeSesLsnrs;
+
     /** Empty constructor (all values are initialized to their defaults). */
     public CacheConfiguration() {
         /* No-op. */
@@ -389,6 +392,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         sqlOnheapRowCacheSize = cc.getSqlOnheapRowCacheSize();
         startSize = cc.getStartSize();
         storeFactory = cc.getCacheStoreFactory();
+        storeSesLsnrs = cc.getCacheStoreSessionListenerFactories();
         swapEnabled = cc.isSwapEnabled();
         tmLookupClsName = cc.getTransactionManagerLookupClassName();
         topValidator = cc.getTopologyValidator();
@@ -1734,6 +1738,28 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         return this;
     }
 
+    /**
+     * Gets cache store session listener factories.
+     *
+     * @return Cache store session listener factories.
+     */
+    public Factory<? extends CacheStoreSessionListener>[] getCacheStoreSessionListenerFactories() {
+        return storeSesLsnrs;
+    }
+
+    /**
+     * Cache store session listener factories.
+     *
+     * @param storeSesLsnrs Cache store session listener factories.
+     * @return {@code this} for chaining.
+     */
+    public CacheConfiguration setCacheStoreSessionListenerFactories(
+        Factory<? extends CacheStoreSessionListener>... storeSesLsnrs) {
+        this.storeSesLsnrs = storeSesLsnrs;
+
+        return this;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index ebe2b8e..96ac7e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.configuration;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.events.*;
@@ -52,6 +53,7 @@ import org.apache.ignite.spi.loadbalancing.roundrobin.*;
 import org.apache.ignite.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 
+import javax.cache.configuration.*;
 import javax.cache.event.*;
 import javax.cache.expiry.*;
 import javax.cache.integration.*;
@@ -334,9 +336,6 @@ public class IgniteConfiguration {
     /** Cache configurations. */
     private CacheConfiguration[] cacheCfg;
 
-    /** Client cache configurations. */
-    private NearCacheConfiguration[] nearCacheCfg;
-
     /** Client mode flag. */
     private Boolean clientMode;
 
@@ -398,6 +397,9 @@ public class IgniteConfiguration {
     /** User's class loader. */
     private ClassLoader classLdr;
 
+    /** Cache store session listeners. */
+    private Factory<CacheStoreSessionListener>[] storeSesLsnrs;
+
     /**
      * Creates valid grid configuration with all default values.
      */
@@ -478,6 +480,7 @@ public class IgniteConfiguration {
         segResolvers = cfg.getSegmentationResolvers();
         sndRetryCnt = cfg.getNetworkSendRetryCount();
         sndRetryDelay = cfg.getNetworkSendRetryDelay();
+        storeSesLsnrs = cfg.getCacheStoreSessionListenerFactories();
         svcCfgs = cfg.getServiceConfiguration();
         sysPoolSize = cfg.getSystemThreadPoolSize();
         timeSrvPortBase = cfg.getTimeServerPortBase();
@@ -2242,6 +2245,28 @@ public class IgniteConfiguration {
         return classLdr;
     }
 
+    /**
+     * Gets cache store session listener factories.
+     *
+     * @return Cache store session listener factories.
+     */
+    public Factory<CacheStoreSessionListener>[] getCacheStoreSessionListenerFactories() {
+        return storeSesLsnrs;
+    }
+
+    /**
+     * Cache store session listener factories.
+     *
+     * @param storeSesLsnrs Cache store session listener factories.
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setCacheStoreSessionListenerFactories(
+        Factory<CacheStoreSessionListener>... storeSesLsnrs) {
+        this.storeSesLsnrs = storeSesLsnrs;
+
+        return this;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgniteConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 294c2b0..dacd1aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -441,10 +441,6 @@ public class GridCacheSharedContext<K, V> {
                 if (activeCacheCtx.cacheId() != cacheCtx.cacheId())
                     return false;
             }
-
-            // Check that caches have the same store.
-            if (activeCacheCtx.store().store() != cacheCtx.store().store())
-                return false;
         }
 
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
index 5fde622..02fe679 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.store;
 
-import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index f9a966c..a9ea2c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -35,6 +35,7 @@ import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
+import javax.cache.configuration.*;
 import javax.cache.integration.*;
 import java.util.*;
 
@@ -59,11 +60,17 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     private ThreadLocal<SessionData> sesHolder;
 
     /** */
+    private ThreadLocalSession locSes;
+
+    /** */
     private boolean locStore;
 
     /** */
     private boolean writeThrough;
 
+    /** */
+    private CacheStoreSessionListener[] sesLsnrs;
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void initialize(@Nullable CacheStore cfgStore, Map sesHolders) throws IgniteCheckedException {
@@ -84,19 +91,43 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             sesHolder0 = ((Map<CacheStore, ThreadLocal>)sesHolders).get(cfgStore);
 
             if (sesHolder0 == null) {
-                ThreadLocalSession locSes = new ThreadLocalSession();
+                sesHolder0 = new ThreadLocal<>();
 
-                if (ctx.resource().injectStoreSession(cfgStore, locSes)) {
-                    sesHolder0 = locSes.sesHolder;
+                locSes = new ThreadLocalSession(sesHolder0);
 
+                if (ctx.resource().injectStoreSession(cfgStore, locSes))
                     sesHolders.put(cfgStore, sesHolder0);
-                }
             }
+            else
+                locSes = new ThreadLocalSession(sesHolder0);
         }
 
         sesHolder = sesHolder0;
 
         locStore = U.hasAnnotation(cfgStore, CacheLocalStore.class);
+
+        sesLsnrs = createSessionListeners(cfg.getCacheStoreSessionListenerFactories());
+
+        if (sesLsnrs == null)
+            sesLsnrs = createSessionListeners(ctx.config().getCacheStoreSessionListenerFactories());
+    }
+
+    /**
+     * Creates session listeners.
+     *
+     * @param factories Factories.
+     * @return Listeners.
+     */
+    private CacheStoreSessionListener[] createSessionListeners(Factory<CacheStoreSessionListener>[] factories) {
+        if (factories == null)
+            return null;
+
+        CacheStoreSessionListener[] lsnrs = new CacheStoreSessionListener[factories.length];
+
+        for (int i = 0; i < factories.length; i++)
+            lsnrs[i] = factories[i].create();
+
+        return lsnrs;
     }
 
     /** {@inheritDoc} */
@@ -215,14 +246,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             Object val = null;
 
             try {
                 val = singleThreadGate.load(storeKey);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -234,7 +265,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheLoaderException(e));
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -349,7 +380,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 IgniteBiInClosure<Object, Object> c = new CI2<Object, Object>() {
@@ -380,7 +411,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 else
                     singleThreadGate.loadAll(keys0, c);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -392,7 +423,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheLoaderException(e));
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -408,7 +439,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(null);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 store.loadCache(new IgniteBiInClosure<Object, Object>() {
@@ -431,7 +462,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                     }
                 }, args);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (CacheLoaderException e) {
                 throw new IgniteCheckedException(e);
@@ -440,7 +471,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheLoaderException(e));
             }
             finally {
-                sessionEnd0(null, thewEx);
+                sessionEnd0(null, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -473,12 +504,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 store.write(new CacheEntryImpl<>(key, locStore ? F.t(val, ver) : val));
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -490,7 +521,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheWriterException(e));
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -522,12 +553,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
                 sessionInit0(tx);
 
-                boolean thewEx = true;
+                boolean threwEx = true;
 
                 try {
                     store.writeAll(entries);
 
-                    thewEx = false;
+                    threwEx = false;
                 }
                 catch (ClassCastException e) {
                     handleClassCastException(e);
@@ -548,7 +579,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                     throw new IgniteCheckedException(e);
                 }
                 finally {
-                    sessionEnd0(tx, thewEx);
+                    sessionEnd0(tx, threwEx);
                 }
 
                 if (log.isDebugEnabled())
@@ -576,12 +607,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 store.delete(key);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -593,7 +624,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheWriterException(e));
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -625,12 +656,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 store.deleteAll(keys0);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -645,7 +676,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(e);
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -675,6 +706,11 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         sessionInit0(tx);
 
         try {
+            if (sesLsnrs != null) {
+                for (CacheStoreSessionListener lsnr : sesLsnrs)
+                    lsnr.onSessionEnd(locSes, commit);
+            }
+
             store.sessionEnd(commit);
         }
         finally {
@@ -715,9 +751,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
      * @param tx Current transaction.
      */
     private void sessionInit0(@Nullable IgniteInternalTx tx) {
-        if (sesHolder == null)
-            return;
-
+        assert sesHolder != null;
         assert sesHolder.get() == null;
 
         SessionData ses;
@@ -738,6 +772,15 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             ses = new SessionData(null, cctx.name());
 
         sesHolder.set(ses);
+
+        if (!ses.started()) {
+            if (sesLsnrs != null) {
+                for (CacheStoreSessionListener lsnr : sesLsnrs)
+                    lsnr.onSessionStart(locSes);
+            }
+
+            ses.onStarted();
+        }
     }
 
     /**
@@ -745,8 +788,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
      */
     private void sessionEnd0(@Nullable IgniteInternalTx tx, boolean threwEx) throws IgniteCheckedException {
         try {
-            if (tx == null)
-                store.sessionEnd(threwEx);
+            if (tx == null) {
+                if (sesLsnrs != null) {
+                    for (CacheStoreSessionListener lsnr : sesLsnrs)
+                        lsnr.onSessionEnd(locSes, !threwEx);
+                }
+
+                store.sessionEnd(!threwEx);
+            }
         }
         catch (Exception e) {
             if (!threwEx)
@@ -788,6 +837,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         @GridToStringInclude
         private Map<Object, Object> props;
 
+        /** */
+        private boolean started;
+
         /**
          * @param tx Current transaction.
          * @param cacheName Cache name.
@@ -828,6 +880,19 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             this.cacheName = cacheName;
         }
 
+        /**
+         */
+        private void onStarted() {
+            started = true;
+        }
+
+        /**
+         * @return If session is started.
+         */
+        private boolean started() {
+            return started;
+        }
+
         /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(SessionData.class, this, "tx", CU.txString(tx));
@@ -839,7 +904,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
      */
     private static class ThreadLocalSession implements CacheStoreSession {
         /** */
-        private final ThreadLocal<SessionData> sesHolder = new ThreadLocal<>();
+        private final ThreadLocal<SessionData> sesHolder;
+
+        /**
+         * @param sesHolder Session holder.
+         */
+        private ThreadLocalSession(ThreadLocal<SessionData> sesHolder) {
+            this.sesHolder = sesHolder;
+        }
 
         /** {@inheritDoc} */
         @Nullable @Override public Transaction transaction() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 64cc77f..f513b59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -437,7 +437,12 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @Override public boolean storeUsed() {
-        return storeEnabled() && store() != null;
+        if (!storeEnabled())
+            return false;
+
+        Collection<CacheStoreManager> stores = stores();
+
+        return stores != null && !stores.isEmpty();
     }
 
     /**
@@ -445,13 +450,20 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
      *
      * @return Store manager.
      */
-    protected CacheStoreManager store() {
-        if (!activeCacheIds().isEmpty()) {
-            int cacheId = F.first(activeCacheIds());
+    protected Collection<CacheStoreManager> stores() {
+        Collection<Integer> cacheIds = activeCacheIds();
 
-            CacheStoreManager store = cctx.cacheContext(cacheId).store();
+        if (!cacheIds.isEmpty()) {
+            Collection<CacheStoreManager> stores = new ArrayList<>(cacheIds.size());
+
+            for (int cacheId : cacheIds) {
+                CacheStoreManager store = cctx.cacheContext(cacheId).store();
+
+                if (store.configured())
+                    stores.add(store);
+            }
 
-            return store.configured() ? store : null;
+            return stores;
         }
 
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 7e9095c..40bb36e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -496,17 +496,24 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
      */
     @SuppressWarnings({"CatchGenericClass"})
     protected void batchStoreCommit(Iterable<IgniteTxEntry> writeEntries) throws IgniteCheckedException {
-        CacheStoreManager store = store();
+        if (!storeEnabled() || (internal() && !groupLock()))
+            return;
 
-        if (store != null && store.isWriteThrough() && storeEnabled() &&
-            (!internal() || groupLock()) && (near() || store.isWriteToStoreFromDht())) {
+        Collection<CacheStoreManager> stores = stores();
+
+        if (stores == null || stores.isEmpty())
+            return;
+
+        boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
+
+        if (near() || isWriteToStoreFromDht) {
             try {
                 if (writeEntries != null) {
                     Map<Object, IgniteBiTuple<Object, GridCacheVersion>> putMap = null;
                     List<Object> rmvCol = null;
                     CacheStoreManager writeStore = null;
 
-                    boolean skipNear = near() && store.isWriteToStoreFromDht();
+                    boolean skipNear = near() && isWriteToStoreFromDht;
 
                     for (IgniteTxEntry e : writeEntries) {
                         if ((skipNear && e.cached().isNear()) || e.skipStore())
@@ -560,12 +567,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                 val = cacheCtx.toCacheObject(cacheCtx.unwrapTemporary(interceptorVal));
                             }
 
-                            if (putMap == null)
-                                putMap = new LinkedHashMap<>(writeMap().size(), 1.0f);
+                            if (writeStore == null)
+                                writeStore = cacheCtx.store();
 
-                            putMap.put(CU.value(key, cacheCtx, false), F.t(CU.value(val, cacheCtx, false), ver));
+                            if (writeStore.isWriteThrough()) {
+                                if (putMap == null)
+                                    putMap = new LinkedHashMap<>(writeMap().size(), 1.0f);
 
-                            writeStore = cacheCtx.store();
+                                putMap.put(CU.value(key, cacheCtx, false), F.t(CU.value(val, cacheCtx, false), ver));
+                            }
                         }
                         else if (op == DELETE) {
                             // Batch-process all puts if needed.
@@ -597,12 +607,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     continue;
                             }
 
-                            if (rmvCol == null)
-                                rmvCol = new ArrayList<>();
+                            if (writeStore == null)
+                                writeStore = cacheCtx.store();
 
-                            rmvCol.add(key.value(cacheCtx.cacheObjectContext(), false));
+                            if (writeStore.isWriteThrough()) {
+                                if (rmvCol == null)
+                                    rmvCol = new ArrayList<>();
 
-                            writeStore = cacheCtx.store();
+                                rmvCol.add(key.value(cacheCtx.cacheObjectContext(), false));
+                            }
                         }
                         else if (log.isDebugEnabled())
                             log.debug("Ignoring NOOP entry for batch store commit: " + e);
@@ -626,7 +639,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 }
 
                 // Commit while locks are held.
-                store.sessionEnd(this, true);
+                for (CacheStoreManager store : stores)
+                    store.sessionEnd(this, true);
             }
             catch (IgniteCheckedException ex) {
                 commitError(ex);
@@ -994,11 +1008,12 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
             }
         }
         else {
-            CacheStoreManager store = store();
+            Collection<CacheStoreManager> stores = stores();
 
-            if (store != null && (!internal() || groupLock())) {
+            if (stores != null && !stores.isEmpty() && (!internal() || groupLock())) {
                 try {
-                    store.sessionEnd(this, true);
+                    for (CacheStoreManager store : stores)
+                        store.sessionEnd(this, true);
                 }
                 catch (IgniteCheckedException e) {
                     commitError(e);
@@ -1099,11 +1114,13 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
                 cctx.tm().rollbackTx(this);
 
-                CacheStoreManager store = store();
+                Collection<CacheStoreManager> stores = stores();
 
-                if (store != null && (near() || store.isWriteToStoreFromDht())) {
-                    if (!internal() || groupLock())
-                        store.sessionEnd(this, false);
+                if (stores != null && !stores.isEmpty() && (near() || F.first(stores).isWriteToStoreFromDht())) {
+                    if (!internal() || groupLock()) {
+                        for (CacheStoreManager store : stores)
+                            store.sessionEnd(this, false);
+                    }
                 }
             }
             catch (Error | IgniteCheckedException | RuntimeException e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
new file mode 100644
index 0000000..5a01c2d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
@@ -0,0 +1,204 @@
+/*
+ * 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.cache.store;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.configuration.*;
+import java.io.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Tests for store session listeners.
+ */
+public abstract class CacheStoreSessionListenerAbstractSelfTest extends GridCommonAbstractTest implements Serializable {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    protected static final AtomicInteger loadCacheCnt = new AtomicInteger();
+
+    /** */
+    protected static final AtomicInteger loadCnt = new AtomicInteger();
+
+    /** */
+    protected static final AtomicInteger writeCnt = new AtomicInteger();
+
+    /** */
+    protected static final AtomicInteger deleteCnt = new AtomicInteger();
+
+    /** */
+    protected static final AtomicInteger reuseCnt = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        loadCacheCnt.set(0);
+        loadCnt.set(0);
+        writeCnt.set(0);
+        deleteCnt.set(0);
+        reuseCnt.set(0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicCache() throws Exception {
+        CacheConfiguration<Integer, Integer> cfg = cacheConfiguration(null, CacheAtomicityMode.ATOMIC);
+
+        try (IgniteCache<Integer, Integer> cache = ignite(0).createCache(cfg)) {
+            cache.loadCache(null);
+            cache.get(1);
+            cache.put(1, 1);
+            cache.remove(1);
+        }
+
+        assertEquals(3, loadCacheCnt.get());
+        assertEquals(1, loadCnt.get());
+        assertEquals(1, writeCnt.get());
+        assertEquals(1, deleteCnt.get());
+        assertEquals(0, reuseCnt.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTransactionalCache() throws Exception {
+        CacheConfiguration<Integer, Integer> cfg = cacheConfiguration(null, CacheAtomicityMode.TRANSACTIONAL);
+
+        try (IgniteCache<Integer, Integer> cache = ignite(0).createCache(cfg)) {
+            cache.loadCache(null);
+            cache.get(1);
+            cache.put(1, 1);
+            cache.remove(1);
+        }
+
+        assertEquals(3, loadCacheCnt.get());
+        assertEquals(1, loadCnt.get());
+        assertEquals(1, writeCnt.get());
+        assertEquals(1, deleteCnt.get());
+        assertEquals(0, reuseCnt.get());
+
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitTransaction() throws Exception {
+        CacheConfiguration<Integer, Integer> cfg = cacheConfiguration(null, CacheAtomicityMode.TRANSACTIONAL);
+
+        try (IgniteCache<Integer, Integer> cache = ignite(0).createCache(cfg)) {
+            try (Transaction tx = ignite(0).transactions().txStart()) {
+                cache.put(1, 1);
+                cache.put(2, 2);
+                cache.remove(3);
+                cache.remove(4);
+
+                tx.commit();
+            }
+        }
+
+        assertEquals(2, writeCnt.get());
+        assertEquals(2, deleteCnt.get());
+        assertEquals(3, reuseCnt.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheTransaction() throws Exception {
+        CacheConfiguration<Integer, Integer> cfg1 = cacheConfiguration("cache1", CacheAtomicityMode.TRANSACTIONAL);
+        CacheConfiguration<Integer, Integer> cfg2 = cacheConfiguration("cache2", CacheAtomicityMode.TRANSACTIONAL);
+
+        try (
+            IgniteCache<Integer, Integer> cache1 = ignite(0).createCache(cfg1);
+            IgniteCache<Integer, Integer> cache2 = ignite(0).createCache(cfg2)
+        ) {
+            try (Transaction tx = ignite(0).transactions().txStart()) {
+                cache1.put(1, 1);
+                cache2.put(2, 2);
+                cache1.remove(3);
+                cache2.remove(4);
+
+                tx.commit();
+            }
+        }
+
+        assertEquals(2, writeCnt.get());
+        assertEquals(2, deleteCnt.get());
+        assertEquals(3, reuseCnt.get());
+    }
+
+    /**
+     * @param name Cache name.
+     * @param atomicity Atomicity mode.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfiguration(String name, CacheAtomicityMode atomicity) {
+        CacheConfiguration<Integer, Integer> cfg = new CacheConfiguration<>();
+
+        cfg.setName(name);
+        cfg.setAtomicityMode(atomicity);
+        cfg.setCacheStoreFactory(storeFactory());
+        cfg.setCacheStoreSessionListenerFactories(sessionListenerFactory());
+        cfg.setReadThrough(true);
+        cfg.setWriteThrough(true);
+        cfg.setLoadPreviousValue(true);
+
+        return cfg;
+    }
+
+    /**
+     * @return Store factory.
+     */
+    protected abstract Factory<? extends CacheStore<Integer, Integer>> storeFactory();
+
+    /**
+     * @return Session listener factory.
+     */
+    protected abstract Factory<CacheStoreSessionListener> sessionListenerFactory();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
new file mode 100644
index 0000000..9020e0d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
@@ -0,0 +1,145 @@
+/*
+ * 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.cache.store.jdbc;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.h2.jdbcx.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.sql.*;
+import java.util.*;
+
+/**
+ * Tests for {@link CacheStoreSessionJdbcListener}.
+ */
+public class CacheStoreSessionJdbcListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
+        return new Factory<CacheStore<Integer, Integer>>() {
+            @Override public CacheStore<Integer, Integer> create() {
+                return new Store();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
+        return new Factory<CacheStoreSessionListener>() {
+            @Override public CacheStoreSessionListener create() {
+                CacheStoreSessionJdbcListener lsnr = new CacheStoreSessionJdbcListener();
+
+                lsnr.setDataSource(JdbcConnectionPool.create("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1", "", ""));
+
+                return lsnr;
+            }
+        };
+    }
+
+    /**
+     */
+    private static class Store extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private static String SES_CONN_KEY = "ses_conn";
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
+            loadCacheCnt.incrementAndGet();
+
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            loadCnt.incrementAndGet();
+
+            checkConnection();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            writeCnt.incrementAndGet();
+
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            deleteCnt.incrementAndGet();
+
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            assertNull(connection());
+        }
+
+        /**
+         */
+        private void checkConnection() {
+            Connection conn = connection();
+
+            assertNotNull(conn);
+
+            try {
+                assertFalse(conn.isClosed());
+                assertFalse(conn.getAutoCommit());
+            }
+            catch (SQLException e) {
+                throw new RuntimeException(e);
+            }
+
+            verifySameInstance(conn);
+        }
+
+        /**
+         * @param conn Connection.
+         */
+        private void verifySameInstance(Connection conn) {
+            Map<String, Connection> props = ses.properties();
+
+            Connection sesConn = props.get(SES_CONN_KEY);
+
+            if (sesConn == null)
+                props.put(SES_CONN_KEY, conn);
+            else {
+                assertSame(conn, sesConn);
+
+                reuseCnt.incrementAndGet();
+            }
+        }
+
+        /**
+         * @return Connection.
+         */
+        private Connection connection() {
+            return ses.<String, Connection>properties().get(CacheStoreSessionJdbcListener.JDBC_CONN_KEY);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index aaf7e5b..afb67f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.*;
+import org.apache.ignite.cache.store.jdbc.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
@@ -130,6 +131,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(CacheOffheapMapEntrySelfTest.class);
 
+        suite.addTestSuite(CacheStoreSessionJdbcListenerSelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
new file mode 100644
index 0000000..eff5e6c
--- /dev/null
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
@@ -0,0 +1,106 @@
+/*
+ * 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.cache.store.hibernate;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
+import org.hibernate.*;
+
+import javax.cache.integration.*;
+import java.util.*;
+
+/**
+ * TODO
+ */
+public class CacheStoreSessionHibernateListener implements CacheStoreSessionListener {
+    /** Session key for JDBC connection. */
+    public static final String HIBERNATE_SES_KEY = "__hibernate_ses_";
+
+    /** Hibernate session factory. */
+    private SessionFactory sesFactory;
+
+    /** Store session. */
+    @CacheStoreSessionResource
+    private CacheStoreSession ses;
+
+    /**
+     * Sets Hibernate session factory.
+     *
+     * @param sesFactory Session factory.
+     */
+    public void setSessionFactory(SessionFactory sesFactory) {
+        A.notNull(sesFactory, "sesFactory");
+
+        this.sesFactory = sesFactory;
+    }
+
+    /**
+     * Gets Hibernate session factory.
+     *
+     * @return Session factory.
+     */
+    public SessionFactory getSessionFactory() {
+        return sesFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionStart(CacheStoreSession ses) {
+        Map<String, Session> props = ses.properties();
+
+        if (!props.containsKey(HIBERNATE_SES_KEY)) {
+            try {
+                Session hibSes = sesFactory.openSession();
+
+                props.put(HIBERNATE_SES_KEY, hibSes);
+
+                if (ses.isWithinTransaction())
+                    hibSes.beginTransaction();
+            }
+            catch (HibernateException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+        Session hibSes = ses.<String, Session>properties().remove(HIBERNATE_SES_KEY);
+
+        if (hibSes != null) {
+            try {
+                Transaction tx = hibSes.getTransaction();
+
+                if (commit) {
+                    hibSes.flush();
+
+                    if (tx != null)
+                        tx.commit();
+                }
+                else if (tx != null)
+                    tx.rollback();
+            }
+            catch (HibernateException e) {
+                throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e);
+            }
+            finally {
+                hibSes.close();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
new file mode 100644
index 0000000..85b0b95
--- /dev/null
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
@@ -0,0 +1,150 @@
+/*
+ * 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.cache.store.hibernate;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.hibernate.*;
+import org.hibernate.cfg.Configuration;
+import org.hibernate.service.*;
+
+import javax.cache.Cache;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.util.*;
+
+/**
+ * Tests for {@link CacheStoreSessionJdbcListener}.
+ */
+public class CacheStoreSessionHibernateListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
+        return new Factory<CacheStore<Integer, Integer>>() {
+            @Override public CacheStore<Integer, Integer> create() {
+                return new Store();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
+        return new Factory<CacheStoreSessionListener>() {
+            @Override public CacheStoreSessionListener create() {
+                CacheStoreSessionHibernateListener lsnr = new CacheStoreSessionHibernateListener();
+
+                Configuration cfg = new Configuration().
+                    setProperty("hibernate.dialect", "org.hibernate.dialect.H2Dialect").
+                    setProperty("hibernate.connection.datasource", "jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
+
+                lsnr.setSessionFactory(cfg.buildSessionFactory(new ServiceRegistryBuilder().buildServiceRegistry()));
+
+                return lsnr;
+            }
+        };
+    }
+
+    /**
+     */
+    private static class Store extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private static String SES_CONN_KEY = "ses_conn";
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
+            loadCacheCnt.incrementAndGet();
+
+            checkSession();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            loadCnt.incrementAndGet();
+
+            checkSession();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            writeCnt.incrementAndGet();
+
+            checkSession();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            deleteCnt.incrementAndGet();
+
+            checkSession();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            assertNull(session());
+        }
+
+        /**
+         */
+        private void checkSession() {
+            Session hibSes = session();
+
+            assertNotNull(hibSes);
+
+            assertTrue(hibSes.isOpen());
+
+            if (ses.isWithinTransaction())
+                assertNotNull(hibSes.getTransaction());
+            else
+                assertNull(hibSes.getTransaction());
+
+            verifySameInstance(hibSes);
+        }
+
+        /**
+         * @param hibSes Session.
+         */
+        private void verifySameInstance(Session hibSes) {
+            Map<String, Session> props = ses.properties();
+
+            Session sesConn = props.get(SES_CONN_KEY);
+
+            if (sesConn == null)
+                props.put(SES_CONN_KEY, hibSes);
+            else {
+                assertSame(hibSes, sesConn);
+
+                reuseCnt.incrementAndGet();
+            }
+        }
+
+        /**
+         * @return Connection.
+         */
+        private Session session() {
+            return ses.<String, Session>properties().get(CacheStoreSessionHibernateListener.HIBERNATE_SES_KEY);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 8494ad0..58f4356 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -103,6 +103,20 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-jdbc</artifactId>
+            <version>${spring.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>com.h2database</groupId>
+            <artifactId>h2</artifactId>
+            <version>1.3.175</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
new file mode 100644
index 0000000..a2cf622
--- /dev/null
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
@@ -0,0 +1,125 @@
+/*
+ * 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.cache.store.spring;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.transactions.*;
+import org.springframework.transaction.*;
+import org.springframework.transaction.support.*;
+
+import javax.cache.integration.*;
+
+/**
+ * TODO
+ */
+public class CacheStoreSessionSpringListener implements CacheStoreSessionListener {
+    /** Session key for transaction status. */
+    public static final String TX_STATUS_KEY = "__spring_tx_status_";
+
+    /** Transaction manager. */
+    private PlatformTransactionManager txMgr;
+
+    /**
+     * Sets transaction manager.
+     *
+     * @param txMgr Transaction manager.
+     */
+    public void setTransactionManager(PlatformTransactionManager txMgr) {
+        A.notNull(txMgr, "txMgr");
+
+        this.txMgr = txMgr;
+    }
+
+    /**
+     * Gets transaction manager.
+     *
+     * @return Transaction manager.
+     */
+    public PlatformTransactionManager getTransactionManager() {
+        return txMgr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionStart(CacheStoreSession ses) {
+        if (ses.isWithinTransaction()) {
+            try {
+                ses.properties().put(TX_STATUS_KEY, txMgr.getTransaction(definition(ses.transaction())));
+            }
+            catch (TransactionException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+        if (ses.isWithinTransaction()) {
+            TransactionStatus tx = ses.<String, TransactionStatus>properties().remove(TX_STATUS_KEY);
+
+            if (tx != null) {
+                try {
+                    if (commit)
+                        txMgr.commit(tx);
+                    else
+                        txMgr.rollback(tx);
+                }
+                catch (TransactionException e) {
+                    throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e);
+                }
+            }
+        }
+    }
+
+    /**
+     * Gets DB transaction isolation level based on ongoing cache transaction isolation.
+     *
+     * @return DB transaction isolation.
+     */
+    private TransactionDefinition definition(Transaction tx) {
+        assert tx != null;
+
+        DefaultTransactionDefinition def = new DefaultTransactionDefinition();
+
+        def.setIsolationLevel(isolationLevel(tx.isolation()));
+
+        return def;
+    }
+
+    /**
+     * Gets DB transaction isolation level based on ongoing cache transaction isolation.
+     *
+     * @param isolation Cache transaction isolation.
+     * @return DB transaction isolation.
+     */
+    private int isolationLevel(TransactionIsolation isolation) {
+        switch (isolation) {
+            case READ_COMMITTED:
+                return TransactionDefinition.ISOLATION_READ_COMMITTED;
+
+            case REPEATABLE_READ:
+                return TransactionDefinition.ISOLATION_REPEATABLE_READ;
+
+            case SERIALIZABLE:
+                return TransactionDefinition.ISOLATION_SERIALIZABLE;
+
+            default:
+                throw new IllegalStateException(); // Will never happen.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
new file mode 100644
index 0000000..a7ca317
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
@@ -0,0 +1,179 @@
+/*
+ * 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.cache.store.spring;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.springframework.jdbc.core.*;
+import org.springframework.jdbc.datasource.*;
+import org.springframework.transaction.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import javax.sql.*;
+import java.sql.*;
+import java.util.*;
+
+/**
+ * Tests for {@link CacheStoreSessionJdbcListener}.
+ */
+public class CacheStoreSessionSpringListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
+    /** */
+    private static final DataSource DATA_SRC = new DriverManagerDataSource("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
+
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
+        return new Factory<CacheStore<Integer, Integer>>() {
+            @Override public CacheStore<Integer, Integer> create() {
+                return new Store(new JdbcTemplate(DATA_SRC));
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
+        return new Factory<CacheStoreSessionListener>() {
+            @Override public CacheStoreSessionListener create() {
+                CacheStoreSessionSpringListener lsnr = new CacheStoreSessionSpringListener();
+
+                lsnr.setTransactionManager(new DataSourceTransactionManager(DATA_SRC));
+
+                return lsnr;
+            }
+        };
+    }
+
+    /**
+     */
+    private static class Store extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private static String SES_CONN_KEY = "ses_conn";
+
+        /** */
+        private final JdbcTemplate jdbc;
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /**
+         * @param jdbc JDBC template.
+         */
+        private Store(JdbcTemplate jdbc) {
+            this.jdbc = jdbc;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
+            loadCacheCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            loadCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            writeCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            deleteCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            assertNull(transaction());
+        }
+
+        /**
+         */
+        private void checkTransaction() {
+            TransactionStatus tx = transaction();
+
+            if (ses.isWithinTransaction()) {
+                assertNotNull(tx);
+                assertFalse(tx.isCompleted());
+            }
+            else
+                assertNull(tx);
+        }
+
+        /**
+         * @return Transaction status.
+         */
+        private TransactionStatus transaction() {
+            return ses.<String, TransactionStatus>properties().get(CacheStoreSessionSpringListener.TX_STATUS_KEY);
+        }
+
+        /**
+         */
+        private void checkConnection() {
+            Connection conn = DataSourceUtils.getConnection(jdbc.getDataSource());
+
+            assertNotNull(conn);
+
+            try {
+                assertFalse(conn.isClosed());
+                assertEquals(!ses.isWithinTransaction(), conn.getAutoCommit());
+            }
+            catch (SQLException e) {
+                throw new RuntimeException(e);
+            }
+
+            verifySameInstance(conn);
+        }
+
+        /**
+         * @param conn Connection.
+         */
+        private void verifySameInstance(Connection conn) {
+            Map<String, Connection> props = ses.properties();
+
+            Connection sesConn = props.get(SES_CONN_KEY);
+
+            if (sesConn == null)
+                props.put(SES_CONN_KEY, conn);
+            else {
+                assertSame(conn, sesConn);
+
+                reuseCnt.incrementAndGet();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b97441f9/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
index 8251c18..0b7e471 100644
--- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
+++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.*;
+import org.apache.ignite.cache.store.spring.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.p2p.*;
 import org.apache.ignite.spring.*;
@@ -47,6 +48,8 @@ public class IgniteSpringTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(IgniteStartFromStreamConfigurationTest.class));
 
+        suite.addTestSuite(CacheStoreSessionSpringListenerSelfTest.class);
+
         return suite;
     }
 }


[23/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
new file mode 100644
index 0000000..81736cd
--- /dev/null
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
@@ -0,0 +1,233 @@
+/*
+ * 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.cache.store.spring;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.transactions.*;
+import org.springframework.jdbc.core.*;
+import org.springframework.jdbc.datasource.*;
+import org.springframework.transaction.*;
+import org.springframework.transaction.support.*;
+
+import javax.cache.integration.*;
+import javax.sql.*;
+
+/**
+ * Cache store session listener based on Spring transaction management.
+ * <p>
+ * This listener starts a new DB transaction for each session and commits
+ * or rolls it back when session ends. If there is no ongoing
+ * cache transaction, this listener is no-op.
+ * <p>
+ * Store implementation can use any Spring APIs like {@link JdbcTemplate}
+ * and others. The listener will guarantee that if there is an
+ * ongoing cache transaction, all store operations within this
+ * transaction will be automatically enlisted in the same database
+ * transaction.
+ * <p>
+ * {@link CacheSpringStoreSessionListener} requires that either
+ * {@link #setTransactionManager(PlatformTransactionManager) transaction manager}
+ * or {@link #setDataSource(DataSource) data source} is configured. If non of them is
+ * provided, exception is thrown. Is both are provided, data source will be
+ * ignored.
+ * <p>
+ * If there is a transaction, a {@link TransactionStatus} object will be saved
+ * as a store session {@link CacheStoreSession#attachment() attachment}. It
+ * can be used to acquire current DB transaction status.
+ */
+public class CacheSpringStoreSessionListener implements CacheStoreSessionListener, LifecycleAware {
+    /** Transaction manager. */
+    private PlatformTransactionManager txMgr;
+
+    /** Data source. */
+    private DataSource dataSrc;
+
+    /** Propagation behavior. */
+    private int propagation = TransactionDefinition.PROPAGATION_REQUIRED;
+
+    /** Logger. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /**
+     * Sets transaction manager.
+     * <p>
+     * Either transaction manager or data source is required.
+     * If none is provided, exception will be thrown on startup.
+     *
+     * @param txMgr Transaction manager.
+     */
+    public void setTransactionManager(PlatformTransactionManager txMgr) {
+        this.txMgr = txMgr;
+    }
+
+    /**
+     * Gets transaction manager.
+     *
+     * @return Transaction manager.
+     */
+    public PlatformTransactionManager getTransactionManager() {
+        return txMgr;
+    }
+
+    /**
+     * Sets data source.
+     * <p>
+     * Either transaction manager or data source is required.
+     * If none is provided, exception will be thrown on startup.
+     *
+     * @param dataSrc Data source.
+     */
+    public void setDataSource(DataSource dataSrc) {
+        this.dataSrc = dataSrc;
+    }
+
+    /**
+     * Gets data source.
+     *
+     * @return Data source.
+     */
+    public DataSource getDataSource() {
+        return dataSrc;
+    }
+
+    /**
+     * Sets propagation behavior.
+     * <p>
+     * This parameter is optional.
+     *
+     * @param propagation Propagation behavior.
+     */
+    public void setPropagationBehavior(int propagation) {
+        this.propagation = propagation;
+    }
+
+    /**
+     * Gets propagation behavior.
+     *
+     * @return Propagation behavior.
+     */
+    public int getPropagationBehavior() {
+        return propagation;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteException {
+        if (txMgr == null && dataSrc == null)
+            throw new IgniteException("Either transaction manager or data source is required by " +
+                getClass().getSimpleName() + '.');
+
+        if (dataSrc != null) {
+            if (txMgr == null)
+                txMgr = new DataSourceTransactionManager(dataSrc);
+            else
+                U.warn(log, "Data source configured in " + getClass().getSimpleName() +
+                    " will be ignored (transaction manager is already set).");
+        }
+
+        assert txMgr != null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionStart(CacheStoreSession ses) {
+        if (ses.isWithinTransaction()) {
+            try {
+                TransactionDefinition def = definition(ses.transaction(), ses.cacheName());
+
+                ses.attach(txMgr.getTransaction(def));
+            }
+            catch (TransactionException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+        if (ses.isWithinTransaction()) {
+            TransactionStatus tx = ses.attachment();
+
+            if (tx != null) {
+                ses.attach(null);
+
+                try {
+                    if (commit)
+                        txMgr.commit(tx);
+                    else
+                        txMgr.rollback(tx);
+                }
+                catch (TransactionException e) {
+                    throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e);
+                }
+            }
+        }
+    }
+
+    /**
+     * Gets DB transaction isolation level based on ongoing cache transaction isolation.
+     *
+     * @return DB transaction isolation.
+     */
+    private TransactionDefinition definition(Transaction tx, String cacheName) {
+        assert tx != null;
+
+        DefaultTransactionDefinition def = new DefaultTransactionDefinition();
+
+        def.setName("Ignite Tx [cache=" + (cacheName != null ? cacheName : "<default>") + ", id=" + tx.xid() + ']');
+        def.setIsolationLevel(isolationLevel(tx.isolation()));
+        def.setPropagationBehavior(propagation);
+
+        long timeoutSec = (tx.timeout() + 500) / 1000;
+
+        if (timeoutSec > 0 && timeoutSec < Integer.MAX_VALUE)
+            def.setTimeout((int)timeoutSec);
+
+        return def;
+    }
+
+    /**
+     * Gets DB transaction isolation level based on ongoing cache transaction isolation.
+     *
+     * @param isolation Cache transaction isolation.
+     * @return DB transaction isolation.
+     */
+    private int isolationLevel(TransactionIsolation isolation) {
+        switch (isolation) {
+            case READ_COMMITTED:
+                return TransactionDefinition.ISOLATION_READ_COMMITTED;
+
+            case REPEATABLE_READ:
+                return TransactionDefinition.ISOLATION_REPEATABLE_READ;
+
+            case SERIALIZABLE:
+                return TransactionDefinition.ISOLATION_SERIALIZABLE;
+
+            default:
+                throw new IllegalStateException(); // Will never happen.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
deleted file mode 100644
index e5201ba..0000000
--- a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.store.spring;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.store.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lifecycle.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.transactions.*;
-import org.springframework.jdbc.core.*;
-import org.springframework.jdbc.datasource.*;
-import org.springframework.transaction.*;
-import org.springframework.transaction.support.*;
-
-import javax.cache.integration.*;
-import javax.sql.*;
-
-/**
- * Cache store session listener based on Spring transaction management.
- * <p>
- * This listener starts a new DB transaction for each session and commits
- * or rolls it back when session ends. If there is no ongoing
- * cache transaction, this listener is no-op.
- * <p>
- * Store implementation can use any Spring APIs like {@link JdbcTemplate}
- * and others. The listener will guarantee that if there is an
- * ongoing cache transaction, all store operations within this
- * transaction will be automatically enlisted in the same database
- * transaction.
- * <p>
- * {@link CacheStoreSessionSpringListener} requires that either
- * {@link #setTransactionManager(PlatformTransactionManager) transaction manager}
- * or {@link #setDataSource(DataSource) data source} is configured. If non of them is
- * provided, exception is thrown. Is both are provided, data source will be
- * ignored.
- * <p>
- * If there is a transaction, a {@link TransactionStatus} object will be stored
- * in store session {@link CacheStoreSession#properties() properties} and can be
- * accessed at any moment by {@link #TX_STATUS_KEY} key. This can be used to
- * acquire current DB transaction status.
- */
-public class CacheStoreSessionSpringListener implements CacheStoreSessionListener, LifecycleAware {
-    /** Session key for transaction status. */
-    public static final String TX_STATUS_KEY = "__spring_tx_status_";
-
-    /** Transaction manager. */
-    private PlatformTransactionManager txMgr;
-
-    /** Data source. */
-    private DataSource dataSrc;
-
-    /** Propagation behavior. */
-    private int propagation = TransactionDefinition.PROPAGATION_REQUIRED;
-
-    /** Logger. */
-    @LoggerResource
-    private IgniteLogger log;
-
-    /**
-     * Sets transaction manager.
-     * <p>
-     * Either transaction manager or data source is required.
-     * If none is provided, exception will be thrown on startup.
-     *
-     * @param txMgr Transaction manager.
-     */
-    public void setTransactionManager(PlatformTransactionManager txMgr) {
-        this.txMgr = txMgr;
-    }
-
-    /**
-     * Gets transaction manager.
-     *
-     * @return Transaction manager.
-     */
-    public PlatformTransactionManager getTransactionManager() {
-        return txMgr;
-    }
-
-    /**
-     * Sets data source.
-     * <p>
-     * Either transaction manager or data source is required.
-     * If none is provided, exception will be thrown on startup.
-     *
-     * @param dataSrc Data source.
-     */
-    public void setDataSource(DataSource dataSrc) {
-        this.dataSrc = dataSrc;
-    }
-
-    /**
-     * Gets data source.
-     *
-     * @return Data source.
-     */
-    public DataSource getDataSource() {
-        return dataSrc;
-    }
-
-    /**
-     * Sets propagation behavior.
-     * <p>
-     * This parameter is optional.
-     *
-     * @param propagation Propagation behavior.
-     */
-    public void setPropagationBehavior(int propagation) {
-        this.propagation = propagation;
-    }
-
-    /**
-     * Gets propagation behavior.
-     *
-     * @return Propagation behavior.
-     */
-    public int getPropagationBehavior() {
-        return propagation;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void start() throws IgniteException {
-        if (txMgr == null && dataSrc == null)
-            throw new IgniteException("Either transaction manager or data source is required by " +
-                getClass().getSimpleName() + '.');
-
-        if (dataSrc != null) {
-            if (txMgr == null)
-                txMgr = new DataSourceTransactionManager(dataSrc);
-            else
-                U.warn(log, "Data source configured in " + getClass().getSimpleName() +
-                    " will be ignored (transaction manager is already set).");
-        }
-
-        assert txMgr != null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stop() throws IgniteException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionStart(CacheStoreSession ses) {
-        if (ses.isWithinTransaction()) {
-            try {
-                TransactionDefinition def = definition(ses.transaction(), ses.cacheName());
-
-                ses.properties().put(TX_STATUS_KEY, txMgr.getTransaction(def));
-            }
-            catch (TransactionException e) {
-                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
-        if (ses.isWithinTransaction()) {
-            TransactionStatus tx = ses.<String, TransactionStatus>properties().remove(TX_STATUS_KEY);
-
-            if (tx != null) {
-                try {
-                    if (commit)
-                        txMgr.commit(tx);
-                    else
-                        txMgr.rollback(tx);
-                }
-                catch (TransactionException e) {
-                    throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e);
-                }
-            }
-        }
-    }
-
-    /**
-     * Gets DB transaction isolation level based on ongoing cache transaction isolation.
-     *
-     * @return DB transaction isolation.
-     */
-    private TransactionDefinition definition(Transaction tx, String cacheName) {
-        assert tx != null;
-
-        DefaultTransactionDefinition def = new DefaultTransactionDefinition();
-
-        def.setName("Ignite Tx [cache=" + (cacheName != null ? cacheName : "<default>") + ", id=" + tx.xid() + ']');
-        def.setIsolationLevel(isolationLevel(tx.isolation()));
-        def.setPropagationBehavior(propagation);
-
-        long timeoutSec = (tx.timeout() + 500) / 1000;
-
-        if (timeoutSec > 0 && timeoutSec < Integer.MAX_VALUE)
-            def.setTimeout((int)timeoutSec);
-
-        return def;
-    }
-
-    /**
-     * Gets DB transaction isolation level based on ongoing cache transaction isolation.
-     *
-     * @param isolation Cache transaction isolation.
-     * @return DB transaction isolation.
-     */
-    private int isolationLevel(TransactionIsolation isolation) {
-        switch (isolation) {
-            case READ_COMMITTED:
-                return TransactionDefinition.ISOLATION_READ_COMMITTED;
-
-            case REPEATABLE_READ:
-                return TransactionDefinition.ISOLATION_REPEATABLE_READ;
-
-            case SERIALIZABLE:
-                return TransactionDefinition.ISOLATION_SERIALIZABLE;
-
-            default:
-                throw new IllegalStateException(); // Will never happen.
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListenerSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListenerSelfTest.java
new file mode 100644
index 0000000..74f5c69
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListenerSelfTest.java
@@ -0,0 +1,197 @@
+/*
+ * 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.cache.store.spring;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.springframework.jdbc.core.*;
+import org.springframework.jdbc.datasource.*;
+import org.springframework.transaction.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import javax.sql.*;
+import java.sql.*;
+import java.util.*;
+
+/**
+ * Tests for {@link CacheJdbcStoreSessionListener}.
+ */
+public class CacheSpringStoreSessionListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
+    /** */
+    private static final DataSource DATA_SRC = new DriverManagerDataSource(URL);
+
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
+        return new Factory<CacheStore<Integer, Integer>>() {
+            @Override public CacheStore<Integer, Integer> create() {
+                return new Store(new JdbcTemplate(DATA_SRC));
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
+        return new Factory<CacheStoreSessionListener>() {
+            @Override public CacheStoreSessionListener create() {
+                CacheSpringStoreSessionListener lsnr = new CacheSpringStoreSessionListener();
+
+                lsnr.setDataSource(DATA_SRC);
+
+                return lsnr;
+            }
+        };
+    }
+
+    /**
+     */
+    private static class Store extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private static String SES_CONN_KEY = "ses_conn";
+
+        /** */
+        private final JdbcTemplate jdbc;
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /**
+         * @param jdbc JDBC template.
+         */
+        private Store(JdbcTemplate jdbc) {
+            this.jdbc = jdbc;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
+            loadCacheCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            loadCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            writeCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+
+            if (write.get()) {
+                String table;
+
+                switch (ses.cacheName()) {
+                    case "cache1":
+                        table = "Table1";
+
+                        break;
+
+                    case "cache2":
+                        if (fail.get())
+                            throw new CacheWriterException("Expected failure.");
+
+                        table = "Table2";
+
+                        break;
+
+                    default:
+                        throw new CacheWriterException("Wring cache: " + ses.cacheName());
+                }
+
+                jdbc.update("INSERT INTO " + table + " (key, value) VALUES (?, ?)",
+                    entry.getKey(), entry.getValue());
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            deleteCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            assertNull(ses.attachment());
+        }
+
+        /**
+         */
+        private void checkTransaction() {
+            TransactionStatus tx = ses.attachment();
+
+            if (ses.isWithinTransaction()) {
+                assertNotNull(tx);
+                assertFalse(tx.isCompleted());
+            }
+            else
+                assertNull(tx);
+        }
+
+        /**
+         */
+        private void checkConnection() {
+            Connection conn = DataSourceUtils.getConnection(jdbc.getDataSource());
+
+            assertNotNull(conn);
+
+            try {
+                assertFalse(conn.isClosed());
+                assertEquals(!ses.isWithinTransaction(), conn.getAutoCommit());
+            }
+            catch (SQLException e) {
+                throw new RuntimeException(e);
+            }
+
+            verifySameInstance(conn);
+        }
+
+        /**
+         * @param conn Connection.
+         */
+        private void verifySameInstance(Connection conn) {
+            Map<String, Connection> props = ses.properties();
+
+            Connection sesConn = props.get(SES_CONN_KEY);
+
+            if (sesConn == null)
+                props.put(SES_CONN_KEY, conn);
+            else {
+                assertSame(conn, sesConn);
+
+                reuseCnt.incrementAndGet();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
deleted file mode 100644
index 83ed249..0000000
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.store.spring;
-
-import org.apache.ignite.cache.store.*;
-import org.apache.ignite.cache.store.jdbc.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.springframework.jdbc.core.*;
-import org.springframework.jdbc.datasource.*;
-import org.springframework.transaction.*;
-
-import javax.cache.*;
-import javax.cache.configuration.*;
-import javax.cache.integration.*;
-import javax.sql.*;
-import java.sql.*;
-import java.util.*;
-
-/**
- * Tests for {@link CacheStoreSessionJdbcListener}.
- */
-public class CacheStoreSessionSpringListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
-    /** */
-    private static final DataSource DATA_SRC = new DriverManagerDataSource(URL);
-
-    /** {@inheritDoc} */
-    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
-        return new Factory<CacheStore<Integer, Integer>>() {
-            @Override public CacheStore<Integer, Integer> create() {
-                return new Store(new JdbcTemplate(DATA_SRC));
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
-        return new Factory<CacheStoreSessionListener>() {
-            @Override public CacheStoreSessionListener create() {
-                CacheStoreSessionSpringListener lsnr = new CacheStoreSessionSpringListener();
-
-                lsnr.setDataSource(DATA_SRC);
-
-                return lsnr;
-            }
-        };
-    }
-
-    /**
-     */
-    private static class Store extends CacheStoreAdapter<Integer, Integer> {
-        /** */
-        private static String SES_CONN_KEY = "ses_conn";
-
-        /** */
-        private final JdbcTemplate jdbc;
-
-        /** */
-        @CacheStoreSessionResource
-        private CacheStoreSession ses;
-
-        /**
-         * @param jdbc JDBC template.
-         */
-        private Store(JdbcTemplate jdbc) {
-            this.jdbc = jdbc;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
-            loadCacheCnt.incrementAndGet();
-
-            checkTransaction();
-            checkConnection();
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer load(Integer key) throws CacheLoaderException {
-            loadCnt.incrementAndGet();
-
-            checkTransaction();
-            checkConnection();
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
-            throws CacheWriterException {
-            writeCnt.incrementAndGet();
-
-            checkTransaction();
-            checkConnection();
-
-            if (write.get()) {
-                String table;
-
-                switch (ses.cacheName()) {
-                    case "cache1":
-                        table = "Table1";
-
-                        break;
-
-                    case "cache2":
-                        if (fail.get())
-                            throw new CacheWriterException("Expected failure.");
-
-                        table = "Table2";
-
-                        break;
-
-                    default:
-                        throw new CacheWriterException("Wring cache: " + ses.cacheName());
-                }
-
-                jdbc.update("INSERT INTO " + table + " (key, value) VALUES (?, ?)",
-                    entry.getKey(), entry.getValue());
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void delete(Object key) throws CacheWriterException {
-            deleteCnt.incrementAndGet();
-
-            checkTransaction();
-            checkConnection();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void sessionEnd(boolean commit) {
-            assertNull(transaction());
-        }
-
-        /**
-         */
-        private void checkTransaction() {
-            TransactionStatus tx = transaction();
-
-            if (ses.isWithinTransaction()) {
-                assertNotNull(tx);
-                assertFalse(tx.isCompleted());
-            }
-            else
-                assertNull(tx);
-        }
-
-        /**
-         * @return Transaction status.
-         */
-        private TransactionStatus transaction() {
-            return ses.<String, TransactionStatus>properties().get(CacheStoreSessionSpringListener.TX_STATUS_KEY);
-        }
-
-        /**
-         */
-        private void checkConnection() {
-            Connection conn = DataSourceUtils.getConnection(jdbc.getDataSource());
-
-            assertNotNull(conn);
-
-            try {
-                assertFalse(conn.isClosed());
-                assertEquals(!ses.isWithinTransaction(), conn.getAutoCommit());
-            }
-            catch (SQLException e) {
-                throw new RuntimeException(e);
-            }
-
-            verifySameInstance(conn);
-        }
-
-        /**
-         * @param conn Connection.
-         */
-        private void verifySameInstance(Connection conn) {
-            Map<String, Connection> props = ses.properties();
-
-            Connection sesConn = props.get(SES_CONN_KEY);
-
-            if (sesConn == null)
-                props.put(SES_CONN_KEY, conn);
-            else {
-                assertSame(conn, sesConn);
-
-                reuseCnt.incrementAndGet();
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
index 0b7e471..12dd494 100644
--- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
+++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
@@ -48,7 +48,7 @@ public class IgniteSpringTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(IgniteStartFromStreamConfigurationTest.class));
 
-        suite.addTestSuite(CacheStoreSessionSpringListenerSelfTest.class);
+        suite.addTestSuite(CacheSpringStoreSessionListenerSelfTest.class);
 
         return suite;
     }


[08/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: b308321d880f540fbda759be9265161a8fd02689
Parents: 463883d
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon May 18 18:22:29 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon May 18 18:22:29 2015 -0700

----------------------------------------------------------------------
 .../store/GridCacheStoreManagerAdapter.java     | 44 +++++++++++++++++---
 1 file changed, 39 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b308321d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index b608bb6..0eaa0cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -69,7 +69,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     private boolean writeThrough;
 
     /** */
-    private CacheStoreSessionListener[] sesLsnrs;
+    private Collection<CacheStoreSessionListener> sesLsnrs;
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
@@ -77,6 +77,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         GridKernalContext ctx = igniteContext();
         CacheConfiguration cfg = cacheConfiguration();
 
+        if (cfgStore != null && !cfg.isWriteThrough() && !cfg.isReadThrough()) {
+            U.quietAndWarn(log,
+                "Persistence store is configured, but both read-through and write-through are disabled. This " +
+                "configuration makes sense if the store implements loadCache method only. If this is the " +
+                "case, ignore this warning. Otherwise, fix the configuration for cache: " + cfg.getName(),
+                "Persistence store is configured, but both read-through and write-through are disabled.");
+        }
+
         writeThrough = cfg.isWriteThrough();
 
         this.cfgStore = cfgStore;
@@ -118,14 +126,25 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
      * @param factories Factories.
      * @return Listeners.
      */
-    private CacheStoreSessionListener[] createSessionListeners(Factory<CacheStoreSessionListener>[] factories) {
+    private Collection<CacheStoreSessionListener> createSessionListeners(Factory<CacheStoreSessionListener>[] factories)
+        throws IgniteCheckedException {
         if (factories == null)
             return null;
 
-        CacheStoreSessionListener[] lsnrs = new CacheStoreSessionListener[factories.length];
+        Collection<CacheStoreSessionListener> lsnrs = new ArrayList<>(factories.length);
+
+        for (Factory<CacheStoreSessionListener> factory : factories) {
+            CacheStoreSessionListener lsnr = factory.create();
 
-        for (int i = 0; i < factories.length; i++)
-            lsnrs[i] = factories[i].create();
+            if (lsnr != null) {
+                cctx.kernalContext().resource().injectGeneric(lsnr);
+
+                if (lsnr instanceof LifecycleAware)
+                    ((LifecycleAware)lsnr).start();
+
+                lsnrs.add(lsnr);
+            }
+        }
 
         return lsnrs;
     }
@@ -195,6 +214,21 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 U.error(log(), "Failed to stop cache store.", e);
             }
         }
+
+        if (sesLsnrs != null) {
+            for (CacheStoreSessionListener lsnr : sesLsnrs) {
+                if (lsnr instanceof LifecycleAware)
+                    ((LifecycleAware)lsnr).stop();
+
+                try {
+                    cctx.kernalContext().resource().cleanupGeneric(lsnr);
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to remove injected resources from store session listener (ignoring): " +
+                        lsnr, e);
+                }
+            }
+        }
     }
 
     /** {@inheritDoc} */


[11/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: 2c28381ea9156f04e916e3fc616a928ef9686882
Parents: 9acbbca
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed May 20 16:56:28 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed May 20 16:56:28 2015 -0700

----------------------------------------------------------------------
 .../cache/store/GridCacheStoreManagerAdapter.java | 18 ++++++++++--------
 1 file changed, 10 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c28381e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 0eaa0cb..a4a4212 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -77,14 +77,6 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         GridKernalContext ctx = igniteContext();
         CacheConfiguration cfg = cacheConfiguration();
 
-        if (cfgStore != null && !cfg.isWriteThrough() && !cfg.isReadThrough()) {
-            U.quietAndWarn(log,
-                "Persistence store is configured, but both read-through and write-through are disabled. This " +
-                "configuration makes sense if the store implements loadCache method only. If this is the " +
-                "case, ignore this warning. Otherwise, fix the configuration for cache: " + cfg.getName(),
-                "Persistence store is configured, but both read-through and write-through are disabled.");
-        }
-
         writeThrough = cfg.isWriteThrough();
 
         this.cfgStore = cfgStore;
@@ -198,6 +190,16 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException("Failed to start cache store: " + e, e);
             }
         }
+
+        CacheConfiguration cfg = cctx.config();
+
+        if (cfgStore != null && !cfg.isWriteThrough() && !cfg.isReadThrough()) {
+            U.quietAndWarn(log,
+                "Persistence store is configured, but both read-through and write-through are disabled. This " +
+                "configuration makes sense if the store implements loadCache method only. If this is the " +
+                "case, ignore this warning. Otherwise, fix the configuration for cache: " + cfg.getName(),
+                "Persistence store is configured, but both read-through and write-through are disabled.");
+        }
     }
 
     /** {@inheritDoc} */


[18/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: 4a55d29fb278a93d4fef1cfe600607b11bda2d0f
Parents: b37d004
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu May 21 19:52:58 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu May 21 19:52:58 2015 -0700

----------------------------------------------------------------------
 .../transactions/IgniteTxLocalAdapter.java      | 44 +++++++++++---------
 1 file changed, 24 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a55d29f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index fdaef47..3e5a63c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -661,21 +661,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         }
     }
 
-    /**
-     * @param stores Store managers.
-     * @return If {@code isWriteToStoreFromDht} value same for all stores.
-     */
-    private boolean isWriteToStoreFromDhtValid(Collection<CacheStoreManager> stores) {
-        boolean exp = F.first(stores).isWriteToStoreFromDht();
-
-        for (CacheStoreManager store : stores) {
-            if (store.isWriteToStoreFromDht() != exp)
-                return false;
-        }
-
-        return true;
-    }
-
     /** {@inheritDoc} */
     @SuppressWarnings({"CatchGenericClass"})
     @Override public void userCommit() throws IgniteCheckedException {
@@ -1101,13 +1086,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 if (!internal()) {
                     Collection<CacheStoreManager> stores = stores();
 
-                    assert isWriteToStoreFromDhtValid(stores) :
-                        "isWriteToStoreFromDht can't be different within one transaction";
+                    if (stores != null && !stores.isEmpty()) {
+                        assert isWriteToStoreFromDhtValid(stores) :
+                            "isWriteToStoreFromDht can't be different within one transaction";
 
-                    boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
+                        boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
 
-                    if (stores != null && !stores.isEmpty() && (near() || isWriteToStoreFromDht))
-                        sessionEnd(stores, false);
+                        if (stores != null && !stores.isEmpty() && (near() || isWriteToStoreFromDht))
+                            sessionEnd(stores, false);
+                    }
                 }
             }
             catch (Error | IgniteCheckedException | RuntimeException e) {
@@ -3326,6 +3313,23 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     }
 
     /**
+     * @param stores Store managers.
+     * @return If {@code isWriteToStoreFromDht} value same for all stores.
+     */
+    private boolean isWriteToStoreFromDhtValid(Collection<CacheStoreManager> stores) {
+        if (stores != null && !stores.isEmpty()) {
+            boolean exp = F.first(stores).isWriteToStoreFromDht();
+
+            for (CacheStoreManager store : stores) {
+                if (store.isWriteToStoreFromDht() != exp)
+                    return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
      * Post-lock closure alias.
      *
      * @param <T> Return type.


[33/39] incubator-ignite git commit: # IGNITE-891 - Minor

Posted by sb...@apache.org.
# IGNITE-891 - Minor


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

Branch: refs/heads/ignite-876-2
Commit: 58918a8a9375a1bf0dc4c66e11ae02a0ec8fda2f
Parents: cbbd897
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon Jun 1 16:41:47 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon Jun 1 16:41:47 2015 -0700

----------------------------------------------------------------------
 .../datagrid/store/hibernate/CacheHibernateStoreExample.java     | 1 +
 .../examples/datagrid/store/spring/CacheSpringStoreExample.java  | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/58918a8a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
index ac19072..25760f0 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
@@ -77,6 +77,7 @@ public class CacheHibernateStoreExample {
             // Configure Hibernate store.
             cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(CacheHibernatePersonStore.class));
 
+            // Configure Hibernate session listener.
             cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
                 @Override public CacheStoreSessionListener create() {
                     CacheHibernateStoreSessionListener lsnr = new CacheHibernateStoreSessionListener();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/58918a8a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
index 9be6672..aa624bc 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
@@ -70,10 +70,10 @@ public class CacheSpringStoreExample {
             // Set atomicity as transaction, since we are showing transactions in example.
             cacheCfg.setAtomicityMode(TRANSACTIONAL);
 
-            // Configure JDBC store.
+            // Configure Spring store.
             cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(CacheSpringPersonStore.class));
 
-            // Configure JDBC session listener.
+            // Configure Spring session listener.
             cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
                 @Override public CacheStoreSessionListener create() {
                     CacheJdbcStoreSessionListener lsnr = new CacheJdbcStoreSessionListener();


[05/39] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891


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

Branch: refs/heads/ignite-876-2
Commit: d13b8f9bbc198b7e4670d2181d77d4b43519c294
Parents: 16f045f ba7fddb
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon May 18 17:18:14 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon May 18 17:18:14 2015 -0700

----------------------------------------------------------------------
 .../internal/interop/InteropIgnition.java       |  65 ++++++-
 .../internal/interop/InteropProcessor.java      |  13 +-
 .../managers/communication/GridIoManager.java   |   6 +-
 .../GridLifecycleAwareMessageFilter.java        |   5 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  12 +-
 .../near/IgniteCacheNearOnlyTxTest.java         | 190 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |   1 +
 7 files changed, 280 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[03/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: 79258ba1e5b6e009ffe3d8bc504f81955ffb5c21
Parents: 0c52898
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri May 15 16:08:59 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri May 15 16:08:59 2015 -0700

----------------------------------------------------------------------
 .../ignite/cache/store/CacheStoreSessionListener.java  | 13 ++++++++++++-
 .../store/jdbc/CacheStoreSessionJdbcListener.java      |  2 +-
 .../hibernate/CacheStoreSessionHibernateListener.java  |  2 +-
 .../store/spring/CacheStoreSessionSpringListener.java  |  2 +-
 4 files changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/79258ba1/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
index e57714b..cba66c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
@@ -18,10 +18,21 @@
 package org.apache.ignite.cache.store;
 
 /**
- * TODO
+ * Store session listener.
  */
 public interface CacheStoreSessionListener {
+    /**
+     * On session start callback.
+     *
+     * @param ses Current session.
+     */
     public void onSessionStart(CacheStoreSession ses);
 
+    /**
+     * On session end callback.
+     *
+     * @param ses Current session.
+     * @param commit {@code True} if transaction should commit, {@code false} for rollback.
+     */
     public void onSessionEnd(CacheStoreSession ses, boolean commit);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/79258ba1/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
index 9622063..87ed5c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
@@ -27,7 +27,7 @@ import java.sql.*;
 import java.util.*;
 
 /**
- * TODO
+ * Cache store session listener based on JDBC connection.
  */
 public class CacheStoreSessionJdbcListener implements CacheStoreSessionListener {
     /** Session key for JDBC connection. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/79258ba1/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
index eff5e6c..e1fd4b2 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
@@ -26,7 +26,7 @@ import javax.cache.integration.*;
 import java.util.*;
 
 /**
- * TODO
+ * Cache store session listener based on Hibernate session.
  */
 public class CacheStoreSessionHibernateListener implements CacheStoreSessionListener {
     /** Session key for JDBC connection. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/79258ba1/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
index a2cf622..e0caad5 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
@@ -26,7 +26,7 @@ import org.springframework.transaction.support.*;
 import javax.cache.integration.*;
 
 /**
- * TODO
+ * Cache store session listener based on Spring cache manager.
  */
 public class CacheStoreSessionSpringListener implements CacheStoreSessionListener {
     /** Session key for transaction status. */


[27/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: e6cc139efa4bb33a334521b6ad0e463ff5b390e8
Parents: 9f88b05
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Sun May 24 23:36:21 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Sun May 24 23:36:21 2015 -0700

----------------------------------------------------------------------
 ...heStoreSessionListenerLifecycleSelfTest.java | 395 +++++++++++++++++++
 1 file changed, 395 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e6cc139e/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifecycleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifecycleSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifecycleSelfTest.java
new file mode 100644
index 0000000..814c8a5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifecycleSelfTest.java
@@ -0,0 +1,395 @@
+/*
+ * 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.cache.store;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * Store session listeners test.
+ */
+public class CacheStoreSessionListenerLifecycleSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final Queue<String> evts = new ConcurrentLinkedDeque<>();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCacheStoreSessionListenerFactories(
+            new SessionListenerFactory("Shared 1"),
+            new SessionListenerFactory("Shared 2")
+        );
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        evts.clear();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoCaches() throws Exception {
+        try {
+            startGrid();
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList("Shared 1 START", "Shared 2 START", "Shared 1 STOP", "Shared 2 STOP"),
+            evts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoOverride() throws Exception {
+        try {
+            Ignite ignite = startGrid();
+
+            for (int i = 0; i < 2; i++) {
+                CacheConfiguration<Integer, Integer> cacheCfg = cacheConfiguration("cache-" + i);
+
+                cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+                ignite.createCache(cacheCfg);
+            }
+
+            ignite.cache("cache-0").put(1, 1);
+            ignite.cache("cache-1").put(1, 1);
+
+            try (Transaction tx = ignite.transactions().txStart()) {
+                ignite.cache("cache-0").put(2, 2);
+                ignite.cache("cache-0").put(3, 3);
+                ignite.cache("cache-1").put(2, 2);
+                ignite.cache("cache-1").put(3, 3);
+
+                tx.commit();
+            }
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList(
+            "Shared 1 START",
+            "Shared 2 START",
+
+            // Put to cache-0.
+            "Shared 1 SESSION START cache-0",
+            "Shared 2 SESSION START cache-0",
+            "Shared 1 SESSION END cache-0",
+            "Shared 2 SESSION END cache-0",
+
+            // Put to cache-1.
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            // Transaction.
+            "Shared 1 SESSION START cache-0",
+            "Shared 2 SESSION START cache-0",
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "Shared 1 SESSION END cache-0",
+            "Shared 2 SESSION END cache-0",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            "Shared 1 STOP",
+            "Shared 2 STOP"
+        ), evts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartialOverride() throws Exception {
+        try {
+            Ignite ignite = startGrid();
+
+            for (int i = 0; i < 2; i++) {
+                String name = "cache-" + i;
+
+                CacheConfiguration cacheCfg = cacheConfiguration(name);
+
+                cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+                if (i == 0) {
+                    cacheCfg.setCacheStoreSessionListenerFactories(
+                        new SessionListenerFactory(name + " 1"),
+                        new SessionListenerFactory(name + " 2")
+                    );
+                }
+
+                ignite.createCache(cacheCfg);
+            }
+
+            ignite.cache("cache-0").put(1, 1);
+            ignite.cache("cache-1").put(1, 1);
+
+            try (Transaction tx = ignite.transactions().txStart()) {
+                ignite.cache("cache-0").put(2, 2);
+                ignite.cache("cache-0").put(3, 3);
+                ignite.cache("cache-1").put(2, 2);
+                ignite.cache("cache-1").put(3, 3);
+
+                tx.commit();
+            }
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList(
+            "Shared 1 START",
+            "Shared 2 START",
+            "cache-0 1 START",
+            "cache-0 2 START",
+
+            // Put to cache-0.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+
+            // Put to cache-1.
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            // Transaction.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            "cache-0 1 STOP",
+            "cache-0 2 STOP",
+            "Shared 1 STOP",
+            "Shared 2 STOP"
+        ), evts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOverride() throws Exception {
+        try {
+            Ignite ignite = startGrid();
+
+            for (int i = 0; i < 2; i++) {
+                String name = "cache-" + i;
+
+                CacheConfiguration cacheCfg = cacheConfiguration(name);
+
+                cacheCfg.setCacheStoreSessionListenerFactories(new SessionListenerFactory(name + " 1"), new SessionListenerFactory(name + " 2"));
+
+                ignite.createCache(cacheCfg);
+            }
+
+            ignite.cache("cache-0").put(1, 1);
+            ignite.cache("cache-1").put(1, 1);
+
+            try (Transaction tx = ignite.transactions().txStart()) {
+                ignite.cache("cache-0").put(2, 2);
+                ignite.cache("cache-0").put(3, 3);
+                ignite.cache("cache-1").put(2, 2);
+                ignite.cache("cache-1").put(3, 3);
+
+                tx.commit();
+            }
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList(
+            "Shared 1 START",
+            "Shared 2 START",
+            "cache-0 1 START",
+            "cache-0 2 START",
+            "cache-1 1 START",
+            "cache-1 2 START",
+
+            // Put to cache-0.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+
+            // Put to cache-1.
+            "cache-1 1 SESSION START cache-1",
+            "cache-1 2 SESSION START cache-1",
+            "cache-1 1 SESSION END cache-1",
+            "cache-1 2 SESSION END cache-1",
+
+            // Transaction.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "cache-1 1 SESSION START cache-1",
+            "cache-1 2 SESSION START cache-1",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+            "cache-1 1 SESSION END cache-1",
+            "cache-1 2 SESSION END cache-1",
+
+            "cache-0 1 STOP",
+            "cache-0 2 STOP",
+            "cache-1 1 STOP",
+            "cache-1 2 STOP",
+            "Shared 1 STOP",
+            "Shared 2 STOP"
+        ), evts);
+    }
+
+    /**
+     * @param name Cache name.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfiguration(String name) {
+        CacheConfiguration<Integer, Integer> cacheCfg = new CacheConfiguration<>(name);
+
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+        cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(Store.class));
+        cacheCfg.setWriteThrough(true);
+
+        return cacheCfg;
+    }
+
+    /**
+     */
+    private static class SessionListener implements CacheStoreSessionListener, LifecycleAware {
+        /** */
+        private final String name;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * @param name Name.
+         */
+        private SessionListener(String name) {
+            this.name = name;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void start() throws IgniteException {
+            assertNotNull(ignite);
+
+            evts.add(name + " START");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void stop() throws IgniteException {
+            assertNotNull(ignite);
+
+            evts.add(name + " STOP");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionStart(CacheStoreSession ses) {
+            assertNotNull(ignite);
+
+            evts.add(name + " SESSION START " + ses.cacheName());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+            assertNotNull(ignite);
+
+            evts.add(name + " SESSION END " + ses.cacheName());
+        }
+    }
+
+    /**
+     */
+    private static class SessionListenerFactory implements Factory<CacheStoreSessionListener> {
+        /** */
+        private String name;
+
+        /**
+         * @param name Name.
+         */
+        private SessionListenerFactory(String name) {
+            this.name = name;
+        }
+
+        @Override public CacheStoreSessionListener create() {
+            return new SessionListener(name);
+        }
+    }
+
+    /**
+     */
+    public static class Store extends CacheStoreAdapter<Integer, Integer> {
+        public Store() {
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            // No-op.
+        }
+    }
+}


[29/39] incubator-ignite git commit: Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Posted by sb...@apache.org.
Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891


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

Branch: refs/heads/ignite-876-2
Commit: 084b6893c24667febb86defe9a3e8362a1f46b1e
Parents: f0b980c e16cb03
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed May 27 15:24:20 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed May 27 15:24:20 2015 -0700

----------------------------------------------------------------------
 dev-tools/gradle/wrapper/gradle-wrapper.jar     | Bin 51017 -> 0 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |  18 +-
 dev-tools/gradlew                               | 163 +++++++---
 dev-tools/slurp.sh                              |  14 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      | 310 +++++++++++++------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../processors/cache/IgniteCacheProxy.java      |   3 +-
 .../GridDistributedTxRemoteAdapter.java         |  10 +-
 .../processors/query/GridQueryIndexing.java     |  16 +
 .../processors/query/GridQueryProcessor.java    |  52 ++--
 .../core/src/main/resources/ignite.properties   |   2 +-
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 .../query/h2/GridH2IndexingGeoSelfTest.java     |  20 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  17 +
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  18 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   4 +-
 .../query/h2/sql/GridSqlOperationType.java      |   2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   2 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   2 +-
 .../local/IgniteCacheLocalQuerySelfTest.java    |   6 +
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 .../licenses/jcraft-revised-bsd.txt             |  28 --
 modules/visor-console/pom.xml                   |   2 +-
 .../visor/commands/node/VisorNodeCommand.scala  |   2 +-
 .../commands/tasks/VisorTasksCommand.scala      |   2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  63 +++-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |  13 +-
 53 files changed, 554 insertions(+), 273 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/084b6893/modules/spring/pom.xml
----------------------------------------------------------------------


[10/39] incubator-ignite git commit: Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Posted by sb...@apache.org.
Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891


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

Branch: refs/heads/ignite-876-2
Commit: 9acbbcaaa73c239710bb5ac750f584cccc105c27
Parents: b308321 4b4158f
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed May 20 16:35:41 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed May 20 16:35:41 2015 -0700

----------------------------------------------------------------------
 LICENSE                                         | 238 ++++++++++++++
 LICENSE.txt                                     | 238 --------------
 NOTICE                                          |  12 +
 NOTICE.txt                                      |  12 -
 assembly/release-base.xml                       |   4 +-
 dev-tools/build.gradle                          |  35 +-
 dev-tools/gradle/wrapper/gradle-wrapper.jar     | Bin 0 -> 51017 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |   6 +
 dev-tools/gradlew                               | 164 ++++++++++
 dev-tools/src/main/groovy/jiraslurp.groovy      | 326 ++++++++++++++-----
 modules/core/src/main/java/META-INF/LICENSE     | 238 ++++++++++++++
 modules/core/src/main/java/META-INF/NOTICE      |  12 +
 .../internal/GridEventConsumeHandler.java       |  26 ++
 .../apache/ignite/internal/IgniteKernal.java    |  26 +-
 .../interop/InteropAwareEventFilter.java        |  37 +++
 .../interop/InteropLocalEventListener.java      |  28 ++
 .../eventstorage/GridEventStorageManager.java   |  24 +-
 .../processors/cache/GridCacheIoManager.java    |   6 +-
 .../GridCachePartitionExchangeManager.java      |   4 +-
 .../distributed/GridDistributedTxMapping.java   |   5 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   6 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   1 +
 .../dht/atomic/GridDhtAtomicCache.java          |   4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  42 ++-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  36 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   4 +-
 .../distributed/near/GridNearCacheEntry.java    |   2 +-
 .../cache/distributed/near/GridNearTxLocal.java |   5 +-
 .../near/GridNearTxPrepareFutureAdapter.java    |   9 +-
 .../near/GridNearTxPrepareResponse.java         |  28 +-
 .../transactions/IgniteTxLocalAdapter.java      |   4 +-
 .../cache/transactions/IgniteTxManager.java     |  24 ++
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java | 145 +++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 parent/pom.xml                                  |   5 +-
 pom.xml                                         |  41 ++-
 37 files changed, 1403 insertions(+), 398 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9acbbcaa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------


[15/39] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-876-2
Commit: dd847bd6ae23122fb9e182f8aff7d63de625e034
Parents: 68811e7 4929c7d
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu May 21 11:01:49 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu May 21 11:01:49 2015 -0700

----------------------------------------------------------------------
 dev-tools/build.gradle                          |   2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      | 213 ++++++++++++++-----
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |  72 +++++++
 .../IgniteCacheEvictionSelfTestSuite.java       |   1 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   4 +-
 36 files changed, 265 insertions(+), 89 deletions(-)
----------------------------------------------------------------------



[31/39] incubator-ignite git commit: Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Posted by sb...@apache.org.
Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java


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

Branch: refs/heads/ignite-876-2
Commit: cbbd897784d49539cb2b99fd59e773954f20bded
Parents: 0d9f1a7 97d0b04
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon Jun 1 10:34:55 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon Jun 1 10:34:55 2015 -0700

----------------------------------------------------------------------
 dev-tools/src/main/groovy/jiraslurp.groovy      |    3 +
 examples/pom.xml                                |    2 +-
 modules/aop/pom.xml                             |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/clients/pom.xml                         |    2 +-
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 modules/core/pom.xml                            |    2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    8 +-
 .../java/org/apache/ignite/IgniteServices.java  |    5 +-
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |  187 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   27 +-
 .../configuration/IgniteConfiguration.java      |   10 +-
 .../org/apache/ignite/igfs/IgfsUserContext.java |  119 +
 .../igfs/secondary/IgfsSecondaryFileSystem.java |    7 +
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   27 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   20 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../internal/igfs/common/IgfsMarshaller.java    |   35 +-
 .../igfs/common/IgfsPathControlRequest.java     |   22 +
 .../internal/managers/GridManagerAdapter.java   |   59 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   48 +
 .../discovery/GridDiscoveryManager.java         |  214 +-
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../processors/cache/CacheMetricsImpl.java      |  367 +-
 .../cache/CacheMetricsMXBeanImpl.java           |  100 +
 .../processors/cache/CacheMetricsSnapshot.java  |  380 +-
 .../cache/DynamicCacheChangeBatch.java          |   20 +-
 .../cache/DynamicCacheDescriptor.java           |    2 +
 .../processors/cache/GridCacheAdapter.java      |   29 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |    6 +-
 .../processors/cache/GridCacheEntryEx.java      |    6 +
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMapEntry.java     |    5 +
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   71 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |   74 +-
 .../processors/cache/GridCacheProxyImpl.java    |   12 +
 .../cache/GridCacheSharedContext.java           |    1 +
 .../processors/cache/GridCacheSwapManager.java  |  118 +-
 .../processors/cache/GridCacheUtils.java        |  234 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedCacheEntry.java  |    7 -
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |    8 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    6 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   22 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   78 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  313 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   28 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 +-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   83 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |   12 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   15 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   54 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  836 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../datastreamer/DataStreamerImpl.java          |    2 +
 .../internal/processors/hadoop/HadoopJob.java   |    2 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |    8 +-
 .../internal/processors/igfs/IgfsImpl.java      |    8 +-
 .../processors/igfs/IgfsIpcHandler.java         |  184 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |    9 +-
 .../internal/processors/igfs/IgfsServer.java    |    4 +-
 .../internal/processors/igfs/IgfsUtils.java     |   16 +
 .../service/GridServiceProcessor.java           |  121 +-
 .../ignite/internal/util/GridJavaProcess.java   |   30 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |    2 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |   80 +
 .../apache/ignite/plugin/PluginProvider.java    |   26 +-
 .../org/apache/ignite/services/Service.java     |    5 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   37 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   47 -
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |   40 +
 .../spi/discovery/DiscoverySpiListener.java     |    5 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1478 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4766 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  170 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5799 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   10 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   24 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   67 +
 .../messages/TcpDiscoveryClientPingRequest.java |   56 +
 .../TcpDiscoveryClientPingResponse.java         |   67 +
 .../TcpDiscoveryCustomEventMessage.java         |   41 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   28 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   43 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |    2 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    6 +
 .../tcp/messages/TcpDiscoveryPingResponse.java  |   15 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    8 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 .../ignite/internal/GridAffinitySelfTest.java   |    1 +
 .../internal/GridDiscoveryEventSelfTest.java    |    7 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    7 +-
 .../internal/GridProjectionAbstractTest.java    |   16 +
 .../GridProjectionForCachesSelfTest.java        |   11 +-
 .../internal/GridReleaseTypeSelfTest.java       |   77 +-
 .../apache/ignite/internal/GridSelfTest.java    |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   62 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |  122 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   46 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    1 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    3 +
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../processors/cache/GridCacheTestEntryEx.java  |    4 +
 .../GridCacheVariableTopologySelfTest.java      |   12 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |    2 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   29 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   13 +-
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   20 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   47 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |   25 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java |  412 ++
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   93 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../DataStreamerMultiThreadedSelfTest.java      |    2 -
 .../igfs/IgfsClientCacheSelfTest.java           |    3 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |  251 +
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    8 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1171 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../testframework/GridSpiTestContext.java       |   25 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/common/GridCommonAbstractTest.java   |   59 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    4 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |    1 +
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    3 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 modules/hadoop/pom.xml                          |    2 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  165 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  107 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   32 +-
 .../internal/processors/hadoop/HadoopUtils.java |   10 +-
 .../hadoop/SecondaryFileSystemProvider.java     |   53 +-
 .../hadoop/fs/HadoopDistributedFileSystem.java  |   91 -
 .../hadoop/fs/HadoopFileSystemsUtils.java       |   17 -
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |  204 +
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |    6 +
 .../hadoop/igfs/HadoopIgfsInProc.java           |  170 +-
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |    2 +-
 .../hadoop/igfs/HadoopIgfsOutProc.java          |   33 +-
 .../hadoop/igfs/HadoopIgfsWrapper.java          |   19 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |    4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   56 +-
 ...oopSecondaryFileSystemConfigurationTest.java |    4 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   63 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |    2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |    2 +
 .../hadoop/HadoopFileSystemsTest.java           |   23 +-
 .../collections/HadoopSkipListSelfTest.java     |    4 +-
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    4 +
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   16 +
 modules/jcl/pom.xml                             |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/mesos/README.txt                        |   28 +
 modules/mesos/licenses/apache-2.0.txt           |  202 +
 modules/mesos/licenses/jetty-epl-license.txt    |   69 +
 modules/mesos/pom.xml                           |  101 +
 .../apache/ignite/mesos/ClusterProperties.java  |  519 ++
 .../apache/ignite/mesos/IgniteFramework.java    |  119 +
 .../apache/ignite/mesos/IgniteScheduler.java    |  361 ++
 .../org/apache/ignite/mesos/IgniteTask.java     |   86 +
 .../org/apache/ignite/mesos/package-info.java   |   22 +
 .../ignite/mesos/resource/IgniteProvider.java   |  234 +
 .../ignite/mesos/resource/JettyServer.java      |   61 +
 .../ignite/mesos/resource/ResourceHandler.java  |  142 +
 .../ignite/mesos/resource/ResourceProvider.java |  120 +
 .../ignite/mesos/resource/package-info.java     |   22 +
 .../main/resources/ignite-default-config.xml    |   35 +
 .../org/apache/ignite/IgniteMesosTestSuite.java |   38 +
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |  464 ++
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    2 +-
 modules/schema-import/pom.xml                   |    2 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web/pom.xml                             |    2 +-
 modules/yardstick/pom.xml                       |    2 +-
 parent/pom.xml                                  |    4 +
 pom.xml                                         |    3 +-
 scripts/git-format-patch.sh                     |   14 +-
 scripts/git-patch-prop.sh                       |    2 +-
 327 files changed, 23033 insertions(+), 11102 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbbd8977/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbbd8977/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbbd8977/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbbd8977/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbbd8977/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 7096da5,32d6acb..3bd2a45
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@@ -1795,53 -1617,23 +1620,74 @@@ public class GridCacheUtils 
      }
  
      /**
+      * @param node Node.
+      * @return {@code True} if given node is client node (has flag {@link IgniteConfiguration#isClientMode()} set).
+      */
+     public static boolean clientNode(ClusterNode node) {
+         Boolean clientModeAttr = node.attribute(IgniteNodeAttributes.ATTR_CLIENT_MODE);
+ 
+         assert clientModeAttr != null : node;
+ 
+         return clientModeAttr != null && clientModeAttr;
+     }
+ 
+     /**
+      * @param node Node.
+      * @param filter Node filter.
+      * @return {@code True} if node is not client node and pass given filter.
+      */
+     public static boolean affinityNode(ClusterNode node, IgnitePredicate<ClusterNode> filter) {
+         return !clientNode(node) && filter.apply(node);
+     }
++
++    /**
 +     * Creates and starts store session listeners.
 +     *
 +     * @param ctx Kernal context.
 +     * @param factories Factories.
 +     * @return Listeners.
 +     * @throws IgniteCheckedException In case of error.
 +     */
 +    public static Collection<CacheStoreSessionListener> startStoreSessionListeners(GridKernalContext ctx,
 +        Factory<CacheStoreSessionListener>[] factories) throws IgniteCheckedException {
 +        if (factories == null)
 +            return null;
 +
 +        Collection<CacheStoreSessionListener> lsnrs = new ArrayList<>(factories.length);
 +
 +        for (Factory<CacheStoreSessionListener> factory : factories) {
 +            CacheStoreSessionListener lsnr = factory.create();
 +
 +            if (lsnr != null) {
 +                ctx.resource().injectGeneric(lsnr);
 +
 +                if (lsnr instanceof LifecycleAware)
 +                    ((LifecycleAware)lsnr).start();
 +
 +                lsnrs.add(lsnr);
 +            }
 +        }
 +
 +        return lsnrs;
 +    }
 +
 +    /**
 +     * Stops store session listeners.
 +     *
 +     * @param ctx Kernal context.
 +     * @param sesLsnrs Session listeners.
 +     * @throws IgniteCheckedException In case of error.
 +     */
 +    public static void stopStoreSessionListeners(GridKernalContext ctx, Collection<CacheStoreSessionListener> sesLsnrs)
 +        throws IgniteCheckedException {
 +        if (sesLsnrs == null)
 +            return;
 +
 +        for (CacheStoreSessionListener lsnr : sesLsnrs) {
 +            if (lsnr instanceof LifecycleAware)
 +                ((LifecycleAware)lsnr).stop();
 +
 +            ctx.resource().cleanupGeneric(lsnr);
 +        }
 +    }
  }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbbd8977/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbbd8977/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbbd8977/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cbbd8977/modules/spring/pom.xml
----------------------------------------------------------------------


[04/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: 16f045f8d5e8ca4950bc5b0ec55a83db2b7164d1
Parents: 79258ba
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon May 18 17:18:07 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon May 18 17:18:07 2015 -0700

----------------------------------------------------------------------
 .../cache/store/CacheStoreManager.java          |   2 +-
 .../store/GridCacheStoreManagerAdapter.java     |  25 +++-
 .../transactions/IgniteTxLocalAdapter.java      |  59 ++++++---
 ...cheStoreSessionListenerAbstractSelfTest.java | 111 ++++++++++++++++
 .../CacheStoreSessionJdbcListenerSelfTest.java  |  39 +++++-
 .../IgniteCrossCacheTxStoreSelfTest.java        | 131 +++++++++++++++----
 ...heStoreSessionHibernateListenerSelfTest.java |   6 +-
 ...CacheStoreSessionSpringListenerSelfTest.java |  27 +++-
 8 files changed, 337 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16f045f8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
index d9f50ac..327b879 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
@@ -160,7 +160,7 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
      * @param commit Commit.
      * @throws IgniteCheckedException If failed.
      */
-    public void sessionEnd(IgniteInternalTx tx, boolean commit) throws IgniteCheckedException;
+    public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException;
 
     /**
      * End session initiated by write-behind store.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16f045f8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index a9ea2c0..aeca58f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -637,8 +637,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys)
-        throws IgniteCheckedException {
+    @Override public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) throws IgniteCheckedException {
         if (F.isEmpty(keys))
             return true;
 
@@ -700,7 +699,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void sessionEnd(IgniteInternalTx tx, boolean commit) throws IgniteCheckedException {
+    @Override public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException {
         assert store != null;
 
         sessionInit0(tx);
@@ -711,10 +710,11 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                     lsnr.onSessionEnd(locSes, commit);
             }
 
-            store.sessionEnd(commit);
+            if (!sesHolder.get().storeEnded(store))
+                store.sessionEnd(commit);
         }
         finally {
-            if (sesHolder != null) {
+            if (last && sesHolder != null) {
                 sesHolder.set(null);
 
                 tx.removeMeta(SES_ATTR);
@@ -752,7 +752,6 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
      */
     private void sessionInit0(@Nullable IgniteInternalTx tx) {
         assert sesHolder != null;
-        assert sesHolder.get() == null;
 
         SessionData ses;
 
@@ -794,7 +793,8 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                         lsnr.onSessionEnd(locSes, !threwEx);
                 }
 
-                store.sessionEnd(!threwEx);
+                if (!sesHolder.get().storeEnded(store))
+                    store.sessionEnd(!threwEx);
             }
         }
         catch (Exception e) {
@@ -840,6 +840,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         /** */
         private boolean started;
 
+        /** */
+        private final Set<CacheStore> endedStores = new GridSetWrapper<>(new IdentityHashMap<CacheStore, Object>());
+
         /**
          * @param tx Current transaction.
          * @param cacheName Cache name.
@@ -893,6 +896,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             return started;
         }
 
+        /**
+         * @param store Cache store.
+         * @return Whether session already ended on this store instance.
+         */
+        private boolean storeEnded(CacheStore store) {
+            return !endedStores.add(store);
+        }
+
         /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(SessionData.class, this, "tx", CU.txString(tx));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16f045f8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index fa64e12..854448d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -531,11 +531,13 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                             }
 
                             // Batch-process puts if cache ID has changed.
-                            if (writeStore != null && writeStore != cacheCtx.store() && putMap != null && !putMap.isEmpty()) {
-                                writeStore.putAll(this, putMap);
+                            if (writeStore != null && writeStore != cacheCtx.store()) {
+                                if (putMap != null && !putMap.isEmpty()) {
+                                    writeStore.putAll(this, putMap);
 
-                                // Reset.
-                                putMap.clear();
+                                    // Reset.
+                                    putMap.clear();
+                                }
 
                                 writeStore = null;
                             }
@@ -574,11 +576,13 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                 writeStore = null;
                             }
 
-                            if (writeStore != null && writeStore != cacheCtx.store() && rmvCol != null && !rmvCol.isEmpty()) {
-                                writeStore.removeAll(this, rmvCol);
+                            if (writeStore != null && writeStore != cacheCtx.store()) {
+                                if (rmvCol != null && !rmvCol.isEmpty()) {
+                                    writeStore.removeAll(this, rmvCol);
 
-                                // Reset.
-                                rmvCol.clear();
+                                    // Reset.
+                                    rmvCol.clear();
+                                }
 
                                 writeStore = null;
                             }
@@ -623,8 +627,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 }
 
                 // Commit while locks are held.
-                for (CacheStoreManager store : stores)
-                    store.sessionEnd(this, true);
+                sessionEnd(stores, true);
             }
             catch (IgniteCheckedException ex) {
                 commitError(ex);
@@ -649,6 +652,10 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
                 throw new IgniteCheckedException("Failed to commit transaction to database: " + this, ex);
             }
+            finally {
+                if (isRollbackOnly())
+                    sessionEnd(stores, false);
+            }
         }
     }
 
@@ -984,13 +991,12 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 cctx.tm().resetContext();
             }
         }
-        else {
+        else if (!internal()) {
             Collection<CacheStoreManager> stores = stores();
 
-            if (stores != null && !stores.isEmpty() && !internal()) {
+            if (stores != null && !stores.isEmpty()) {
                 try {
-                    for (CacheStoreManager store : stores)
-                        store.sessionEnd(this, true);
+                    sessionEnd(stores, true);
                 }
                 catch (IgniteCheckedException e) {
                     commitError(e);
@@ -1091,13 +1097,11 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
                 cctx.tm().rollbackTx(this);
 
-                Collection<CacheStoreManager> stores = stores();
+                if (!internal()) {
+                    Collection<CacheStoreManager> stores = stores();
 
-                if (stores != null && !stores.isEmpty() && (near() || F.first(stores).isWriteToStoreFromDht())) {
-                    if (!internal()) {
-                        for (CacheStoreManager store : stores)
-                            store.sessionEnd(this, false);
-                    }
+                    if (stores != null && !stores.isEmpty() && (near() || F.first(stores).isWriteToStoreFromDht()))
+                        sessionEnd(stores, false);
                 }
             }
             catch (Error | IgniteCheckedException | RuntimeException e) {
@@ -1109,6 +1113,21 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     }
 
     /**
+     * @param stores Store managers.
+     * @param commit Commit flag.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sessionEnd(Collection<CacheStoreManager> stores, boolean commit) throws IgniteCheckedException {
+        Iterator<CacheStoreManager> it = stores.iterator();
+
+        while (it.hasNext()) {
+            CacheStoreManager store = it.next();
+
+            store.sessionEnd(this, commit, !it.hasNext());
+        }
+    }
+
+    /**
      * Checks if there is a cached or swapped value for
      * {@link #getAllAsync(GridCacheContext, Collection, GridCacheEntryEx, boolean, boolean, boolean, boolean)} method.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16f045f8/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
index 5a01c2d..5df8f68 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.cache.store;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -27,7 +28,9 @@ import org.apache.ignite.testframework.junits.common.*;
 import org.apache.ignite.transactions.*;
 
 import javax.cache.configuration.*;
+import javax.cache.integration.*;
 import java.io.*;
+import java.sql.*;
 import java.util.concurrent.atomic.*;
 
 /**
@@ -38,6 +41,9 @@ public abstract class CacheStoreSessionListenerAbstractSelfTest extends GridComm
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** */
+    protected static final String URL = "jdbc:h2:mem:example;DB_CLOSE_DELAY=-1";
+
+    /** */
     protected static final AtomicInteger loadCacheCnt = new AtomicInteger();
 
     /** */
@@ -52,6 +58,12 @@ public abstract class CacheStoreSessionListenerAbstractSelfTest extends GridComm
     /** */
     protected static final AtomicInteger reuseCnt = new AtomicInteger();
 
+    /** */
+    protected static final AtomicBoolean write = new AtomicBoolean();
+
+    /** */
+    protected static final AtomicBoolean fail = new AtomicBoolean();
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -77,11 +89,22 @@ public abstract class CacheStoreSessionListenerAbstractSelfTest extends GridComm
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            conn.createStatement().executeUpdate("DROP TABLE IF EXISTS Table1");
+            conn.createStatement().executeUpdate("DROP TABLE IF EXISTS Table2");
+
+            conn.createStatement().executeUpdate("CREATE TABLE Table1 (key INT, value INT)");
+            conn.createStatement().executeUpdate("CREATE TABLE Table2 (key INT, value INT)");
+        }
+
         loadCacheCnt.set(0);
         loadCnt.set(0);
         writeCnt.set(0);
         deleteCnt.set(0);
         reuseCnt.set(0);
+
+        write.set(false);
+        fail.set(false);
     }
 
     /**
@@ -174,6 +197,94 @@ public abstract class CacheStoreSessionListenerAbstractSelfTest extends GridComm
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testCommit() throws Exception {
+        write.set(true);
+
+        CacheConfiguration<Integer, Integer> cfg1 = cacheConfiguration("cache1", CacheAtomicityMode.TRANSACTIONAL);
+        CacheConfiguration<Integer, Integer> cfg2 = cacheConfiguration("cache2", CacheAtomicityMode.TRANSACTIONAL);
+
+        try (
+            IgniteCache<Integer, Integer> cache1 = ignite(0).createCache(cfg1);
+            IgniteCache<Integer, Integer> cache2 = ignite(0).createCache(cfg2)
+        ) {
+            try (Transaction tx = ignite(0).transactions().txStart()) {
+                cache1.put(1, 1);
+                cache2.put(2, 2);
+
+                tx.commit();
+            }
+        }
+
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            checkTable(conn, 1, false);
+            checkTable(conn, 2, false);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRollback() throws Exception {
+        write.set(true);
+        fail.set(true);
+
+        CacheConfiguration<Integer, Integer> cfg1 = cacheConfiguration("cache1", CacheAtomicityMode.TRANSACTIONAL);
+        CacheConfiguration<Integer, Integer> cfg2 = cacheConfiguration("cache2", CacheAtomicityMode.TRANSACTIONAL);
+
+        try (
+            IgniteCache<Integer, Integer> cache1 = ignite(0).createCache(cfg1);
+            IgniteCache<Integer, Integer> cache2 = ignite(0).createCache(cfg2)
+        ) {
+            try (Transaction tx = ignite(0).transactions().txStart()) {
+                cache1.put(1, 1);
+                cache2.put(2, 2);
+
+                tx.commit();
+
+                assert false : "Exception was not thrown.";
+            }
+            catch (IgniteException e) {
+                CacheWriterException we = X.cause(e, CacheWriterException.class);
+
+                assertNotNull(we);
+
+                assertEquals("Expected failure.", we.getMessage());
+            }
+        }
+
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            checkTable(conn, 1, true);
+            checkTable(conn, 2, true);
+        }
+    }
+
+    /**
+     * @param conn Connection.
+     * @param idx Table index.
+     * @param empty If table expected to be empty.
+     * @throws Exception In case of error.
+     */
+    private void checkTable(Connection conn, int idx, boolean empty) throws Exception {
+        ResultSet rs = conn.createStatement().executeQuery("SELECT key, value FROM Table" + idx);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            int key = rs.getInt(1);
+            int val = rs.getInt(2);
+
+            assertEquals(idx, key);
+            assertEquals(idx, val);
+
+            cnt++;
+        }
+
+        assertEquals(empty ? 0 : 1, cnt);
+    }
+
+    /**
      * @param name Cache name.
      * @param atomicity Atomicity mode.
      * @return Cache configuration.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16f045f8/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
index 9020e0d..e4dac88 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
@@ -47,7 +47,7 @@ public class CacheStoreSessionJdbcListenerSelfTest extends CacheStoreSessionList
             @Override public CacheStoreSessionListener create() {
                 CacheStoreSessionJdbcListener lsnr = new CacheStoreSessionJdbcListener();
 
-                lsnr.setDataSource(JdbcConnectionPool.create("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1", "", ""));
+                lsnr.setDataSource(JdbcConnectionPool.create(URL, "", ""));
 
                 return lsnr;
             }
@@ -86,6 +86,43 @@ public class CacheStoreSessionJdbcListenerSelfTest extends CacheStoreSessionList
             writeCnt.incrementAndGet();
 
             checkConnection();
+
+            if (write.get()) {
+                Connection conn = connection();
+
+                try {
+                    String table;
+
+                    switch (ses.cacheName()) {
+                        case "cache1":
+                            table = "Table1";
+
+                            break;
+
+                        case "cache2":
+                            if (fail.get())
+                                throw new CacheWriterException("Expected failure.");
+
+                            table = "Table2";
+
+                            break;
+
+                        default:
+                            throw new CacheWriterException("Wring cache: " + ses.cacheName());
+                    }
+
+                    PreparedStatement stmt = conn.prepareStatement(
+                        "INSERT INTO " + table + " (key, value) VALUES (?, ?)");
+
+                    stmt.setInt(1, entry.getKey());
+                    stmt.setInt(2, entry.getValue());
+
+                    stmt.executeUpdate();
+                }
+                catch (SQLException e) {
+                    throw new CacheWriterException(e);
+                }
+            }
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16f045f8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
index cb32b13..f72ea47 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
@@ -101,19 +101,28 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
         grid(0).cache("cacheA").removeAll();
         grid(0).cache("cacheB").removeAll();
         grid(0).cache("cacheC").removeAll();
+
+        for (CacheStore store : firstStores.values())
+            ((TestStore)store).clear();
+
+        for (CacheStore store : secondStores.values())
+            ((TestStore)store).clear();
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testWriteThrough() throws Exception {
+    public void testSameStore() throws Exception {
         IgniteEx grid = grid(0);
 
         TestStore firstStore = (TestStore)firstStores.get(grid.name());
+        TestStore secondStore = (TestStore)secondStores.get(grid.name());
 
         assertNotNull(firstStore);
+        assertNotNull(secondStore);
 
-        Collection<String> evts = firstStore.events();
+        Collection<String> firstStoreEvts = firstStore.events();
+        Collection<String> secondStoreEvts = secondStore.events();
 
         try (Transaction tx = grid.transactions().txStart()) {
             IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
@@ -138,58 +147,122 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
         }
 
         assertEqualsCollections(F.asList(
-                "writeAll cacheA 2",
-                "writeAll cacheB 2",
-                "deleteAll cacheA 2",
-                "deleteAll cacheB 2",
-                "write cacheA",
-                "delete cacheA",
-                "write cacheB",
-                "sessionEnd true"
-            ),
-            evts);
+            "writeAll cacheA 2",
+            "writeAll cacheB 2",
+            "deleteAll cacheA 2",
+            "deleteAll cacheB 2",
+            "write cacheA",
+            "delete cacheA",
+            "write cacheB",
+            "sessionEnd true"
+        ),
+        firstStoreEvts);
+
+        assertEquals(0, secondStoreEvts.size());
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testIncompatibleCaches1() throws Exception {
+    public void testDifferentStores() throws Exception {
         IgniteEx grid = grid(0);
 
-        try (Transaction ignored = grid.transactions().txStart()) {
+        TestStore firstStore = (TestStore)firstStores.get(grid.name());
+        TestStore secondStore = (TestStore)secondStores.get(grid.name());
+
+        assertNotNull(firstStore);
+        assertNotNull(secondStore);
+
+        Collection<String> firstStoreEvts = firstStore.events();
+        Collection<String> secondStoreEvts = secondStore.events();
+
+        try (Transaction tx = grid.transactions().txStart()) {
             IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
             IgniteCache<Object, Object> cacheC = grid.cache("cacheC");
 
-            cacheA.put("1", "2");
+            cacheA.put("1", "1");
+            cacheA.put("2", "2");
+            cacheC.put("1", "1");
+            cacheC.put("2", "2");
+
+            cacheA.remove("3");
+            cacheA.remove("4");
+            cacheC.remove("3");
+            cacheC.remove("4");
+
+            cacheA.put("5", "5");
+            cacheA.remove("6");
 
-            cacheC.put("1", "2");
+            cacheC.put("7", "7");
 
-            fail("Must not allow to enlist caches with different stores to one transaction");
-        }
-        catch (CacheException e) {
-            assertTrue(e.getMessage().contains("Failed to enlist new cache to existing transaction"));
+            tx.commit();
         }
+
+        assertEqualsCollections(F.asList(
+            "writeAll cacheA 2",
+            "deleteAll cacheA 2",
+            "write cacheA",
+            "delete cacheA",
+            "sessionEnd true"
+        ),
+        firstStoreEvts);
+
+        assertEqualsCollections(F.asList(
+            "writeAll cacheC 2",
+            "deleteAll cacheC 2",
+            "write cacheC",
+            "sessionEnd true"
+        ),
+        secondStoreEvts);
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testIncompatibleCaches2() throws Exception {
+    public void testNonPersistentCache() throws Exception {
         IgniteEx grid = grid(0);
 
-        try (Transaction ignored = grid.transactions().txStart()) {
+        TestStore firstStore = (TestStore)firstStores.get(grid.name());
+        TestStore secondStore = (TestStore)secondStores.get(grid.name());
+
+        assertNotNull(firstStore);
+        assertNotNull(secondStore);
+
+        Collection<String> firstStoreEvts = firstStore.events();
+        Collection<String> secondStoreEvts = secondStore.events();
+
+        try (Transaction tx = grid.transactions().txStart()) {
             IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
-            IgniteCache<Object, Object> cacheC = grid.cache("cacheD");
+            IgniteCache<Object, Object> cacheD = grid.cache("cacheD");
+
+            cacheA.put("1", "1");
+            cacheA.put("2", "2");
+            cacheD.put("1", "1");
+            cacheD.put("2", "2");
 
-            cacheA.put("1", "2");
+            cacheA.remove("3");
+            cacheA.remove("4");
+            cacheD.remove("3");
+            cacheD.remove("4");
+
+            cacheA.put("5", "5");
+            cacheA.remove("6");
 
-            cacheC.put("1", "2");
+            cacheD.put("7", "7");
 
-            fail("Must not allow to enlist caches with different stores to one transaction");
-        }
-        catch (CacheException e) {
-            assertTrue(e.getMessage().contains("Failed to enlist new cache to existing transaction"));
+            tx.commit();
         }
+
+        assertEqualsCollections(F.asList(
+            "writeAll cacheA 2",
+            "deleteAll cacheA 2",
+            "write cacheA",
+            "delete cacheA",
+            "sessionEnd true"
+        ),
+        firstStoreEvts);
+
+        assertEquals(0, secondStoreEvts.size());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16f045f8/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
index 85b0b95..d631393 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
@@ -23,7 +23,6 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.hibernate.*;
 import org.hibernate.cfg.Configuration;
-import org.hibernate.service.*;
 
 import javax.cache.Cache;
 import javax.cache.configuration.*;
@@ -50,10 +49,9 @@ public class CacheStoreSessionHibernateListenerSelfTest extends CacheStoreSessio
                 CacheStoreSessionHibernateListener lsnr = new CacheStoreSessionHibernateListener();
 
                 Configuration cfg = new Configuration().
-                    setProperty("hibernate.dialect", "org.hibernate.dialect.H2Dialect").
-                    setProperty("hibernate.connection.datasource", "jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
+                    setProperty("hibernate.connection.url", URL);
 
-                lsnr.setSessionFactory(cfg.buildSessionFactory(new ServiceRegistryBuilder().buildServiceRegistry()));
+                lsnr.setSessionFactory(cfg.buildSessionFactory());
 
                 return lsnr;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16f045f8/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
index a7ca317..79d5b5e 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
@@ -37,7 +37,7 @@ import java.util.*;
  */
 public class CacheStoreSessionSpringListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
     /** */
-    private static final DataSource DATA_SRC = new DriverManagerDataSource("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
+    private static final DataSource DATA_SRC = new DriverManagerDataSource(URL);
 
     /** {@inheritDoc} */
     @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
@@ -106,6 +106,31 @@ public class CacheStoreSessionSpringListenerSelfTest extends CacheStoreSessionLi
 
             checkTransaction();
             checkConnection();
+
+            if (write.get()) {
+                String table;
+
+                switch (ses.cacheName()) {
+                    case "cache1":
+                        table = "Table1";
+
+                        break;
+
+                    case "cache2":
+                        if (fail.get())
+                            throw new CacheWriterException("Expected failure.");
+
+                        table = "Table2";
+
+                        break;
+
+                    default:
+                        throw new CacheWriterException("Wring cache: " + ses.cacheName());
+                }
+
+                jdbc.update("INSERT INTO " + table + " (key, value) VALUES (?, ?)",
+                    entry.getKey(), entry.getValue());
+            }
         }
 
         /** {@inheritDoc} */


[21/39] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-876-2
Commit: 9e10c037e7afbc987d6599b8fb5bc5939de6f562
Parents: dd847bd 050f429
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri May 22 12:40:33 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri May 22 12:40:33 2015 -0700

----------------------------------------------------------------------
 .gitignore                                      |   3 +-
 dev-tools/slurp.sh                              |  74 +++++++++++
 dev-tools/src/main/groovy/jiraslurp.groovy      | 132 +++++++++++--------
 .../ClientAbstractConnectivitySelfTest.java     |  14 ++
 .../internal/interop/InteropException.java      |  71 ++++++++++
 .../interop/InteropNoCallbackException.java     |  50 +++++++
 .../processors/cache/GridCacheAdapter.java      |  16 ++-
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../processors/hadoop/v2/HadoopV2Context.java   |  10 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../config/benchmark-client-mode.properties     |  89 +++++++++++++
 11 files changed, 396 insertions(+), 71 deletions(-)
----------------------------------------------------------------------



[20/39] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891


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

Branch: refs/heads/ignite-876-2
Commit: 7ab0304c85589ca195c294e985e23d4560cbd7dd
Parents: e47f85a d3cb5c4
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri May 22 00:02:21 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri May 22 00:02:21 2015 -0700

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheAdapter.java | 16 ++++++++++++----
 1 file changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[12/39] incubator-ignite git commit: # IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
# IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: d4bcd19b6b68355e37fa446319030112b0e52d65
Parents: 2c28381
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed May 20 17:35:00 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed May 20 17:35:00 2015 -0700

----------------------------------------------------------------------
 .../store/GridCacheStoreManagerAdapter.java     | 68 ++++++++++----------
 1 file changed, 34 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d4bcd19b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index a4a4212..79ac86d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -105,40 +105,6 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         sesHolder = sesHolder0;
 
         locStore = U.hasAnnotation(cfgStore, CacheLocalStore.class);
-
-        sesLsnrs = createSessionListeners(cfg.getCacheStoreSessionListenerFactories());
-
-        if (sesLsnrs == null)
-            sesLsnrs = createSessionListeners(ctx.config().getCacheStoreSessionListenerFactories());
-    }
-
-    /**
-     * Creates session listeners.
-     *
-     * @param factories Factories.
-     * @return Listeners.
-     */
-    private Collection<CacheStoreSessionListener> createSessionListeners(Factory<CacheStoreSessionListener>[] factories)
-        throws IgniteCheckedException {
-        if (factories == null)
-            return null;
-
-        Collection<CacheStoreSessionListener> lsnrs = new ArrayList<>(factories.length);
-
-        for (Factory<CacheStoreSessionListener> factory : factories) {
-            CacheStoreSessionListener lsnr = factory.create();
-
-            if (lsnr != null) {
-                cctx.kernalContext().resource().injectGeneric(lsnr);
-
-                if (lsnr instanceof LifecycleAware)
-                    ((LifecycleAware)lsnr).start();
-
-                lsnrs.add(lsnr);
-            }
-        }
-
-        return lsnrs;
     }
 
     /** {@inheritDoc} */
@@ -200,6 +166,40 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 "case, ignore this warning. Otherwise, fix the configuration for cache: " + cfg.getName(),
                 "Persistence store is configured, but both read-through and write-through are disabled.");
         }
+
+        sesLsnrs = createSessionListeners(cfg.getCacheStoreSessionListenerFactories());
+
+        if (sesLsnrs == null)
+            sesLsnrs = createSessionListeners(cctx.kernalContext().config().getCacheStoreSessionListenerFactories());
+    }
+
+    /**
+     * Creates session listeners.
+     *
+     * @param factories Factories.
+     * @return Listeners.
+     */
+    private Collection<CacheStoreSessionListener> createSessionListeners(Factory<CacheStoreSessionListener>[] factories)
+        throws IgniteCheckedException {
+        if (factories == null)
+            return null;
+
+        Collection<CacheStoreSessionListener> lsnrs = new ArrayList<>(factories.length);
+
+        for (Factory<CacheStoreSessionListener> factory : factories) {
+            CacheStoreSessionListener lsnr = factory.create();
+
+            if (lsnr != null) {
+                cctx.kernalContext().resource().injectGeneric(lsnr);
+
+                if (lsnr instanceof LifecycleAware)
+                    ((LifecycleAware)lsnr).start();
+
+                lsnrs.add(lsnr);
+            }
+        }
+
+        return lsnrs;
     }
 
     /** {@inheritDoc} */


[09/39] incubator-ignite git commit: Minor

Posted by sb...@apache.org.
Minor


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

Branch: refs/heads/ignite-876-2
Commit: 3e35b9a34abac727a683d4db0454ec438044956d
Parents: f9a4dd7
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed May 20 00:10:00 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed May 20 00:10:00 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/configuration/IgniteConfiguration.java | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3e35b9a3/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index ebe2b8e..59844d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -334,9 +334,6 @@ public class IgniteConfiguration {
     /** Cache configurations. */
     private CacheConfiguration[] cacheCfg;
 
-    /** Client cache configurations. */
-    private NearCacheConfiguration[] nearCacheCfg;
-
     /** Client mode flag. */
     private Boolean clientMode;
 


[25/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: ada1b2a7c4d82722cc5721bad50042af7216bfdc
Parents: 990bf9e
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Sun May 24 20:42:53 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Sun May 24 20:42:53 2015 -0700

----------------------------------------------------------------------
 .../hibernate/CacheHibernatePersonStore.java    |  27 +-
 .../store/jdbc/CacheJdbcPersonStore.java        |  69 ++--
 .../store/jdbc/CacheJdbcStoreExample.java       |   3 +-
 .../store/spring/CacheSpringPersonStore.java    | 128 ++++++
 .../store/spring/CacheSpringStoreExample.java   | 143 +++++++
 .../datagrid/store/spring/package-info.java     |  22 ++
 .../processors/cache/GridCacheProcessor.java    |   4 +-
 .../processors/cache/GridCacheUtils.java        |  25 +-
 .../store/GridCacheStoreManagerAdapter.java     |  61 ++-
 ...heStoreSessionListenerLifeCycleSelfTest.java | 395 +++++++++++++++++++
 .../IgniteCrossCacheTxStoreSelfTest.java        |  24 --
 .../junits/common/GridCommonAbstractTest.java   |  24 ++
 .../spring/CacheSpringStoreSessionListener.java |   2 +-
 13 files changed, 810 insertions(+), 117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
index 557ec6f..80a9f22 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
@@ -54,13 +54,7 @@ public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
         Long key = entry.getKey();
         Person val = entry.getValue();
 
-        System.out.println(">>> Store put [key=" + key + ", val=" + val + ']');
-
-        if (val == null) {
-            delete(key);
-
-            return;
-        }
+        System.out.println(">>> Store write [key=" + key + ", val=" + val + ']');
 
         Session hibSes = ses.attachment();
 
@@ -75,13 +69,14 @@ public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
     /** {@inheritDoc} */
     @SuppressWarnings({"JpaQueryApiInspection"})
     @Override public void delete(Object key) {
-        System.out.println(">>> Store remove [key=" + key + ']');
+        System.out.println(">>> Store delete [key=" + key + ']');
 
         Session hibSes = ses.attachment();
 
         try {
-            hibSes.createQuery("delete " + Person.class.getSimpleName() + " where key = :key")
-                .setParameter("key", key).setFlushMode(FlushMode.ALWAYS).executeUpdate();
+            hibSes.createQuery("delete " + Person.class.getSimpleName() + " where key = :key").
+                setParameter("key", key).
+                executeUpdate();
         }
         catch (HibernateException e) {
             throw new CacheWriterException("Failed to remove value from cache store [key=" + key + ']', e);
@@ -100,13 +95,13 @@ public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
         try {
             int cnt = 0;
 
-            List res = hibSes.createCriteria(Person.class).list();
-
-            if (res != null) {
-                Iterator iter = res.iterator();
+            List list = hibSes.createCriteria(Person.class).
+                setMaxResults(entryCnt).
+                list();
 
-                while (cnt < entryCnt && iter.hasNext()) {
-                    Person person = (Person)iter.next();
+            if (list != null) {
+                for (Object obj : list) {
+                    Person person = (Person)obj;
 
                     clo.apply(person.getId(), person);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
index 6eb0386..ed14a99 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
@@ -22,9 +22,11 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.examples.datagrid.store.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
+import org.h2.jdbcx.*;
 
 import javax.cache.*;
 import javax.cache.integration.*;
+import javax.sql.*;
 import java.sql.*;
 
 /**
@@ -32,6 +34,10 @@ import java.sql.*;
  * transaction with cache transactions and maps {@link Long} to {@link Person}.
  */
 public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
+    /** Data source. */
+    public static final DataSource DATA_SRC =
+        JdbcConnectionPool.create("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1", "", "");
+
     /** Store session. */
     @CacheStoreSessionResource
     private CacheStoreSession ses;
@@ -52,12 +58,10 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
      * @throws IgniteException If failed.
      */
     private void prepareDb() throws IgniteException {
-        try (
-            Connection conn = DriverManager.getConnection("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
-            Statement st = conn.createStatement()
-        ) {
-            st.execute("create table if not exists PERSONS (id number unique, firstName varchar(255), " +
-                "lastName varchar(255))");
+        try (Connection conn = DATA_SRC.getConnection()) {
+            conn.createStatement().execute(
+                "create table if not exists PERSONS (" +
+                "id number unique, firstName varchar(255), lastName varchar(255))");
         }
         catch (SQLException e) {
             throw new IgniteException("Failed to create database table.", e);
@@ -66,34 +70,28 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
 
     /** {@inheritDoc} */
     @Override public Person load(Long key) {
-        System.out.println(">>> Loading key: " + key);
+        System.out.println(">>> Store load [key=" + key + ']');
 
-        try {
-            Connection conn = ses.attachment();
+        Connection conn = ses.attachment();
 
-            try (PreparedStatement st = conn.prepareStatement("select * from PERSONS where id=?")) {
-                st.setString(1, key.toString());
+        try (PreparedStatement st = conn.prepareStatement("select * from PERSONS where id = ?")) {
+            st.setString(1, key.toString());
 
-                ResultSet rs = st.executeQuery();
+            ResultSet rs = st.executeQuery();
 
-                if (rs.next())
-                    return new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
-            }
+            return rs.next() ? new Person(rs.getLong(1), rs.getString(2), rs.getString(3)) : null;
         }
         catch (SQLException e) {
-            throw new CacheLoaderException("Failed to load object: " + key, e);
+            throw new CacheLoaderException("Failed to load object [key=" + key + ']', e);
         }
-
-        return null;
     }
 
     /** {@inheritDoc} */
     @Override public void write(Cache.Entry<? extends Long, ? extends Person> entry) {
         Long key = entry.getKey();
-
         Person val = entry.getValue();
 
-        System.out.println(">>> Putting [key=" + key + ", val=" + val +  ']');
+        System.out.println(">>> Store write [key=" + key + ", val=" + val + ']');
 
         try {
             Connection conn = ses.attachment();
@@ -103,7 +101,7 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
             // Try update first. If it does not work, then try insert.
             // Some databases would allow these to be done in one 'upsert' operation.
             try (PreparedStatement st = conn.prepareStatement(
-                "update PERSONS set firstName=?, lastName=? where id=?")) {
+                "update PERSONS set firstName = ?, lastName = ? where id = ?")) {
                 st.setString(1, val.getFirstName());
                 st.setString(2, val.getLastName());
                 st.setLong(3, val.getId());
@@ -114,7 +112,7 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
             // If update failed, try to insert.
             if (updated == 0) {
                 try (PreparedStatement st = conn.prepareStatement(
-                    "insert into PERSONS (id, firstName, lastName) values(?, ?, ?)")) {
+                    "insert into PERSONS (id, firstName, lastName) values (?, ?, ?)")) {
                     st.setLong(1, val.getId());
                     st.setString(2, val.getFirstName());
                     st.setString(3, val.getLastName());
@@ -124,25 +122,23 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
             }
         }
         catch (SQLException e) {
-            throw new CacheLoaderException("Failed to put object [key=" + key + ", val=" + val + ']', e);
+            throw new CacheWriterException("Failed to write object [key=" + key + ", val=" + val + ']', e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void delete(Object key) {
-        System.out.println(">>> Removing key: " + key);
+        System.out.println(">>> Store delete [key=" + key + ']');
 
-        try {
-            Connection conn = ses.attachment();
+        Connection conn = ses.attachment();
 
-            try (PreparedStatement st = conn.prepareStatement("delete from PERSONS where id=?")) {
-                st.setLong(1, (Long)key);
+        try (PreparedStatement st = conn.prepareStatement("delete from PERSONS where id=?")) {
+            st.setLong(1, (Long)key);
 
-                st.executeUpdate();
-            }
+            st.executeUpdate();
         }
         catch (SQLException e) {
-            throw new CacheWriterException("Failed to remove object: " + key, e);
+            throw new CacheWriterException("Failed to delete object [key=" + key + ']', e);
         }
     }
 
@@ -155,13 +151,14 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
 
         Connection conn = ses.attachment();
 
-        try (
-            PreparedStatement st = conn.prepareStatement("select * from PERSONS");
-            ResultSet rs = st.executeQuery()
-        ) {
+        try (PreparedStatement stmt = conn.prepareStatement("select * from PERSONS limit ?")) {
+            stmt.setInt(1, entryCnt);
+
+            ResultSet rs = stmt.executeQuery();
+
             int cnt = 0;
 
-            while (cnt < entryCnt && rs.next()) {
+            while (rs.next()) {
                 Person person = new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
 
                 clo.apply(person.getId(), person);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
index 74e262c..637d6dc 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
@@ -24,7 +24,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.examples.*;
 import org.apache.ignite.examples.datagrid.store.*;
 import org.apache.ignite.transactions.*;
-import org.h2.jdbcx.*;
 
 import javax.cache.configuration.*;
 import java.util.*;
@@ -79,7 +78,7 @@ public class CacheJdbcStoreExample {
                 @Override public CacheStoreSessionListener create() {
                     CacheJdbcStoreSessionListener lsnr = new CacheJdbcStoreSessionListener();
 
-                    lsnr.setDataSource(JdbcConnectionPool.create("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1", "", ""));
+                    lsnr.setDataSource(CacheJdbcPersonStore.DATA_SRC);
 
                     return lsnr;
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java
new file mode 100644
index 0000000..50149ba
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java
@@ -0,0 +1,128 @@
+/*
+ * 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.examples.datagrid.store.spring;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.examples.datagrid.store.*;
+import org.apache.ignite.lang.*;
+import org.springframework.dao.*;
+import org.springframework.jdbc.core.*;
+import org.springframework.jdbc.datasource.*;
+
+import javax.cache.*;
+import javax.cache.integration.*;
+import javax.sql.*;
+import java.sql.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Example of {@link CacheStore} implementation that uses JDBC
+ * transaction with cache transactions and maps {@link Long} to {@link Person}.
+ */
+public class CacheSpringPersonStore extends CacheStoreAdapter<Long, Person> {
+    /** Data source. */
+    public static final DataSource DATA_SRC = new DriverManagerDataSource("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
+
+    /** Spring JDBC template. */
+    private JdbcTemplate jdbcTemplate;
+
+    /**
+     * Constructor.
+     *
+     * @throws IgniteException If failed.
+     */
+    public CacheSpringPersonStore() throws IgniteException {
+        jdbcTemplate = new JdbcTemplate(DATA_SRC);
+
+        prepareDb();
+    }
+
+    /**
+     * Prepares database for example execution. This method will create a
+     * table called "PERSONS" so it can be used by store implementation.
+     *
+     * @throws IgniteException If failed.
+     */
+    private void prepareDb() throws IgniteException {
+        jdbcTemplate.update(
+            "create table if not exists PERSONS (" +
+            "id number unique, firstName varchar(255), lastName varchar(255))");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Person load(Long key) {
+        System.out.println(">>> Store load [key=" + key + ']');
+
+        try {
+            return jdbcTemplate.queryForObject("select * from PERSONS where id = ?", new RowMapper<Person>() {
+                @Override public Person mapRow(ResultSet rs, int rowNum) throws SQLException {
+                    return new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
+                }
+            }, key);
+        }
+        catch (EmptyResultDataAccessException ignored) {
+            return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(Cache.Entry<? extends Long, ? extends Person> entry) {
+        Long key = entry.getKey();
+        Person val = entry.getValue();
+
+        System.out.println(">>> Store write [key=" + key + ", val=" + val + ']');
+
+        int updated = jdbcTemplate.update("update PERSONS set firstName = ?, lastName = ? where id = ?",
+            val.getFirstName(), val.getLastName(), val.getId());
+
+        if (updated == 0) {
+            jdbcTemplate.update("insert into PERSONS (id, firstName, lastName) values (?, ?, ?)",
+                val.getId(), val.getFirstName(), val.getLastName());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void delete(Object key) {
+        System.out.println(">>> Store delete [key=" + key + ']');
+
+        jdbcTemplate.update("delete from PERSONS where id = ?", key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadCache(final IgniteBiInClosure<Long, Person> clo, Object... args) {
+        if (args == null || args.length == 0 || args[0] == null)
+            throw new CacheLoaderException("Expected entry count parameter is not provided.");
+
+        int entryCnt = (Integer)args[0];
+
+        final AtomicInteger cnt = new AtomicInteger();
+
+        jdbcTemplate.query("select * from PERSONS limit ?", new RowCallbackHandler() {
+            @Override public void processRow(ResultSet rs) throws SQLException {
+                Person person = new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
+
+                clo.apply(person.getId(), person);
+
+                cnt.incrementAndGet();
+            }
+        }, entryCnt);
+
+        System.out.println(">>> Loaded " + cnt + " values into cache.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
new file mode 100644
index 0000000..9be6672
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
@@ -0,0 +1,143 @@
+/*
+ * 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.examples.datagrid.store.spring;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.examples.*;
+import org.apache.ignite.examples.datagrid.store.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.configuration.*;
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * Demonstrates usage of cache with underlying persistent store configured.
+ * <p>
+ * This example uses {@link CacheSpringPersonStore} as a persistent store.
+ * <p>
+ * Remote nodes can be started with {@link ExampleNodeStartup} in another JVM which will
+ * start node with {@code examples/config/example-ignite.xml} configuration.
+ */
+public class CacheSpringStoreExample {
+    /** Cache name. */
+    private static final String CACHE_NAME = CacheSpringStoreExample.class.getSimpleName();
+
+    /** Heap size required to run this example. */
+    public static final int MIN_MEMORY = 1024 * 1024 * 1024;
+
+    /** Number of entries to load. */
+    private static final int ENTRY_COUNT = 100_000;
+
+    /** Global person ID to use across entire example. */
+    private static final Long id = Math.abs(UUID.randomUUID().getLeastSignificantBits());
+
+    /**
+     * Executes example.
+     *
+     * @param args Command line arguments, none required.
+     * @throws IgniteException If example execution failed.
+     */
+    public static void main(String[] args) throws IgniteException {
+        ExamplesUtils.checkMinMemory(MIN_MEMORY);
+
+        // To start ignite with desired configuration uncomment the appropriate line.
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println();
+            System.out.println(">>> Cache store example started.");
+
+            CacheConfiguration<Long, Person> cacheCfg = new CacheConfiguration<>(CACHE_NAME);
+
+            // Set atomicity as transaction, since we are showing transactions in example.
+            cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+            // Configure JDBC store.
+            cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(CacheSpringPersonStore.class));
+
+            // Configure JDBC session listener.
+            cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
+                @Override public CacheStoreSessionListener create() {
+                    CacheJdbcStoreSessionListener lsnr = new CacheJdbcStoreSessionListener();
+
+                    lsnr.setDataSource(CacheSpringPersonStore.DATA_SRC);
+
+                    return lsnr;
+                }
+            });
+
+            cacheCfg.setReadThrough(true);
+            cacheCfg.setWriteThrough(true);
+
+            try (IgniteCache<Long, Person> cache = ignite.getOrCreateCache(cacheCfg)) {
+                // Make initial cache loading from persistent store. This is a
+                // distributed operation and will call CacheStore.loadCache(...)
+                // method on all nodes in topology.
+                loadCache(cache);
+
+                // Start transaction and execute several cache operations with
+                // read/write-through to persistent store.
+                executeTransaction(cache);
+            }
+        }
+    }
+
+    /**
+     * Makes initial cache loading.
+     *
+     * @param cache Cache to load.
+     */
+    private static void loadCache(IgniteCache<Long, Person> cache) {
+        long start = System.currentTimeMillis();
+
+        // Start loading cache from persistent store on all caching nodes.
+        cache.loadCache(null, ENTRY_COUNT);
+
+        long end = System.currentTimeMillis();
+
+        System.out.println(">>> Loaded " + cache.size() + " keys with backups in " + (end - start) + "ms.");
+    }
+
+    /**
+     * Executes transaction with read/write-through to persistent store.
+     *
+     * @param cache Cache to execute transaction on.
+     */
+    private static void executeTransaction(IgniteCache<Long, Person> cache) {
+        try (Transaction tx = Ignition.ignite().transactions().txStart()) {
+            Person val = cache.get(id);
+
+            System.out.println("Read value: " + val);
+
+            val = cache.getAndPut(id, new Person(id, "Isaac", "Newton"));
+
+            System.out.println("Overwrote old value: " + val);
+
+            val = cache.get(id);
+
+            System.out.println("Read value: " + val);
+
+            tx.commit();
+        }
+
+        System.out.println("Read value after commit: " + cache.get(id));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java
new file mode 100644
index 0000000..211239f
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains Spring-based cache store implementation.
+ */
+package org.apache.ignite.examples.datagrid.store.spring;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 5b57817..4457f98 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -567,7 +567,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheConfiguration[] cfgs = ctx.config().getCacheConfiguration();
 
-        sharedCtx = createSharedContext(ctx, CU.createStoreSessionListeners(ctx,
+        sharedCtx = createSharedContext(ctx, CU.startStoreSessionListeners(ctx,
             ctx.config().getCacheStoreSessionListenerFactories()));
 
         ctx.performance().add("Disable serializable transactions (set 'txSerializableEnabled' to false)",
@@ -813,6 +813,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             mgr.stop(cancel);
         }
 
+        CU.stopStoreSessionListeners(ctx, sharedCtx.storeSessionListeners());
+
         sharedCtx.cleanup();
 
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 6968fcb..7096da5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1795,13 +1795,14 @@ public class GridCacheUtils {
     }
 
     /**
-     * Creates store session listeners.
+     * Creates and starts store session listeners.
      *
      * @param ctx Kernal context.
      * @param factories Factories.
      * @return Listeners.
+     * @throws IgniteCheckedException In case of error.
      */
-    public static Collection<CacheStoreSessionListener> createStoreSessionListeners(GridKernalContext ctx,
+    public static Collection<CacheStoreSessionListener> startStoreSessionListeners(GridKernalContext ctx,
         Factory<CacheStoreSessionListener>[] factories) throws IgniteCheckedException {
         if (factories == null)
             return null;
@@ -1823,4 +1824,24 @@ public class GridCacheUtils {
 
         return lsnrs;
     }
+
+    /**
+     * Stops store session listeners.
+     *
+     * @param ctx Kernal context.
+     * @param sesLsnrs Session listeners.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public static void stopStoreSessionListeners(GridKernalContext ctx, Collection<CacheStoreSessionListener> sesLsnrs)
+        throws IgniteCheckedException {
+        if (sesLsnrs == null)
+            return;
+
+        for (CacheStoreSessionListener lsnr : sesLsnrs) {
+            if (lsnr instanceof LifecycleAware)
+                ((LifecycleAware)lsnr).stop();
+
+            ctx.resource().cleanupGeneric(lsnr);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 11d711c..bc5a0a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -70,6 +70,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     /** */
     private Collection<CacheStoreSessionListener> sesLsnrs;
 
+    /** */
+    private boolean globalSesLsnrs;
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void initialize(@Nullable CacheStore cfgStore, Map sesHolders) throws IgniteCheckedException {
@@ -166,10 +169,13 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 "Persistence store is configured, but both read-through and write-through are disabled.");
         }
 
-        sesLsnrs = CU.createStoreSessionListeners(cctx.kernalContext(), cfg.getCacheStoreSessionListenerFactories());
+        sesLsnrs = CU.startStoreSessionListeners(cctx.kernalContext(), cfg.getCacheStoreSessionListenerFactories());
 
-        if (sesLsnrs == null)
+        if (sesLsnrs == null) {
             sesLsnrs = cctx.shared().storeSessionListeners();
+
+            globalSesLsnrs = true;
+        }
     }
 
     /** {@inheritDoc} */
@@ -187,18 +193,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             }
         }
 
-        if (sesLsnrs != null) {
-            for (CacheStoreSessionListener lsnr : sesLsnrs) {
-                if (lsnr instanceof LifecycleAware)
-                    ((LifecycleAware)lsnr).stop();
-
-                try {
-                    cctx.kernalContext().resource().cleanupGeneric(lsnr);
-                }
-                catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to remove injected resources from store session listener (ignoring): " +
-                        lsnr, e);
-                }
+        if (!globalSesLsnrs) {
+            try {
+                CU.stopStoreSessionListeners(cctx.kernalContext(), sesLsnrs);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to stop store session listeners for cache: " + cctx.name(), e);
             }
         }
     }
@@ -721,7 +721,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                     lsnr.onSessionEnd(locSes, commit);
             }
 
-            if (!sesHolder.get().storeEnded(store))
+            if (!sesHolder.get().ended(store))
                 store.sessionEnd(commit);
         }
         catch (Throwable e) {
@@ -788,13 +788,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
         sesHolder.set(ses);
 
-        if (!ses.started()) {
-            if (sesLsnrs != null) {
-                for (CacheStoreSessionListener lsnr : sesLsnrs)
-                    lsnr.onSessionStart(locSes);
-            }
-
-            ses.onStarted();
+        if (sesLsnrs != null && !ses.started(this)) {
+            for (CacheStoreSessionListener lsnr : sesLsnrs)
+                lsnr.onSessionStart(locSes);
         }
     }
 
@@ -809,7 +805,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                         lsnr.onSessionEnd(locSes, !threwEx);
                 }
 
-                assert !sesHolder.get().storeEnded(store);
+                assert !sesHolder.get().ended(store);
 
                 store.sessionEnd(!threwEx);
             }
@@ -858,10 +854,11 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         private Object attachment;
 
         /** */
-        private boolean started;
+        private final Set<CacheStoreManager> started =
+            new GridSetWrapper<>(new IdentityHashMap<CacheStoreManager, Object>());
 
         /** */
-        private final Set<CacheStore> endedStores = new GridSetWrapper<>(new IdentityHashMap<CacheStore, Object>());
+        private final Set<CacheStore> ended = new GridSetWrapper<>(new IdentityHashMap<CacheStore, Object>());
 
         /**
          * @param tx Current transaction.
@@ -918,24 +915,18 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         }
 
         /**
-         */
-        private void onStarted() {
-            started = true;
-        }
-
-        /**
          * @return If session is started.
          */
-        private boolean started() {
-            return started;
+        private boolean started(CacheStoreManager mgr) {
+            return !started.add(mgr);
         }
 
         /**
          * @param store Cache store.
          * @return Whether session already ended on this store instance.
          */
-        private boolean storeEnded(CacheStore store) {
-            return !endedStores.add(store);
+        private boolean ended(CacheStore store) {
+            return !ended.add(store);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifeCycleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifeCycleSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifeCycleSelfTest.java
new file mode 100644
index 0000000..814c8a5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifeCycleSelfTest.java
@@ -0,0 +1,395 @@
+/*
+ * 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.cache.store;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * Store session listeners test.
+ */
+public class CacheStoreSessionListenerLifecycleSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final Queue<String> evts = new ConcurrentLinkedDeque<>();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCacheStoreSessionListenerFactories(
+            new SessionListenerFactory("Shared 1"),
+            new SessionListenerFactory("Shared 2")
+        );
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        evts.clear();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoCaches() throws Exception {
+        try {
+            startGrid();
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList("Shared 1 START", "Shared 2 START", "Shared 1 STOP", "Shared 2 STOP"),
+            evts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoOverride() throws Exception {
+        try {
+            Ignite ignite = startGrid();
+
+            for (int i = 0; i < 2; i++) {
+                CacheConfiguration<Integer, Integer> cacheCfg = cacheConfiguration("cache-" + i);
+
+                cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+                ignite.createCache(cacheCfg);
+            }
+
+            ignite.cache("cache-0").put(1, 1);
+            ignite.cache("cache-1").put(1, 1);
+
+            try (Transaction tx = ignite.transactions().txStart()) {
+                ignite.cache("cache-0").put(2, 2);
+                ignite.cache("cache-0").put(3, 3);
+                ignite.cache("cache-1").put(2, 2);
+                ignite.cache("cache-1").put(3, 3);
+
+                tx.commit();
+            }
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList(
+            "Shared 1 START",
+            "Shared 2 START",
+
+            // Put to cache-0.
+            "Shared 1 SESSION START cache-0",
+            "Shared 2 SESSION START cache-0",
+            "Shared 1 SESSION END cache-0",
+            "Shared 2 SESSION END cache-0",
+
+            // Put to cache-1.
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            // Transaction.
+            "Shared 1 SESSION START cache-0",
+            "Shared 2 SESSION START cache-0",
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "Shared 1 SESSION END cache-0",
+            "Shared 2 SESSION END cache-0",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            "Shared 1 STOP",
+            "Shared 2 STOP"
+        ), evts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartialOverride() throws Exception {
+        try {
+            Ignite ignite = startGrid();
+
+            for (int i = 0; i < 2; i++) {
+                String name = "cache-" + i;
+
+                CacheConfiguration cacheCfg = cacheConfiguration(name);
+
+                cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+                if (i == 0) {
+                    cacheCfg.setCacheStoreSessionListenerFactories(
+                        new SessionListenerFactory(name + " 1"),
+                        new SessionListenerFactory(name + " 2")
+                    );
+                }
+
+                ignite.createCache(cacheCfg);
+            }
+
+            ignite.cache("cache-0").put(1, 1);
+            ignite.cache("cache-1").put(1, 1);
+
+            try (Transaction tx = ignite.transactions().txStart()) {
+                ignite.cache("cache-0").put(2, 2);
+                ignite.cache("cache-0").put(3, 3);
+                ignite.cache("cache-1").put(2, 2);
+                ignite.cache("cache-1").put(3, 3);
+
+                tx.commit();
+            }
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList(
+            "Shared 1 START",
+            "Shared 2 START",
+            "cache-0 1 START",
+            "cache-0 2 START",
+
+            // Put to cache-0.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+
+            // Put to cache-1.
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            // Transaction.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            "cache-0 1 STOP",
+            "cache-0 2 STOP",
+            "Shared 1 STOP",
+            "Shared 2 STOP"
+        ), evts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOverride() throws Exception {
+        try {
+            Ignite ignite = startGrid();
+
+            for (int i = 0; i < 2; i++) {
+                String name = "cache-" + i;
+
+                CacheConfiguration cacheCfg = cacheConfiguration(name);
+
+                cacheCfg.setCacheStoreSessionListenerFactories(new SessionListenerFactory(name + " 1"), new SessionListenerFactory(name + " 2"));
+
+                ignite.createCache(cacheCfg);
+            }
+
+            ignite.cache("cache-0").put(1, 1);
+            ignite.cache("cache-1").put(1, 1);
+
+            try (Transaction tx = ignite.transactions().txStart()) {
+                ignite.cache("cache-0").put(2, 2);
+                ignite.cache("cache-0").put(3, 3);
+                ignite.cache("cache-1").put(2, 2);
+                ignite.cache("cache-1").put(3, 3);
+
+                tx.commit();
+            }
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList(
+            "Shared 1 START",
+            "Shared 2 START",
+            "cache-0 1 START",
+            "cache-0 2 START",
+            "cache-1 1 START",
+            "cache-1 2 START",
+
+            // Put to cache-0.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+
+            // Put to cache-1.
+            "cache-1 1 SESSION START cache-1",
+            "cache-1 2 SESSION START cache-1",
+            "cache-1 1 SESSION END cache-1",
+            "cache-1 2 SESSION END cache-1",
+
+            // Transaction.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "cache-1 1 SESSION START cache-1",
+            "cache-1 2 SESSION START cache-1",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+            "cache-1 1 SESSION END cache-1",
+            "cache-1 2 SESSION END cache-1",
+
+            "cache-0 1 STOP",
+            "cache-0 2 STOP",
+            "cache-1 1 STOP",
+            "cache-1 2 STOP",
+            "Shared 1 STOP",
+            "Shared 2 STOP"
+        ), evts);
+    }
+
+    /**
+     * @param name Cache name.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfiguration(String name) {
+        CacheConfiguration<Integer, Integer> cacheCfg = new CacheConfiguration<>(name);
+
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+        cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(Store.class));
+        cacheCfg.setWriteThrough(true);
+
+        return cacheCfg;
+    }
+
+    /**
+     */
+    private static class SessionListener implements CacheStoreSessionListener, LifecycleAware {
+        /** */
+        private final String name;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * @param name Name.
+         */
+        private SessionListener(String name) {
+            this.name = name;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void start() throws IgniteException {
+            assertNotNull(ignite);
+
+            evts.add(name + " START");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void stop() throws IgniteException {
+            assertNotNull(ignite);
+
+            evts.add(name + " STOP");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionStart(CacheStoreSession ses) {
+            assertNotNull(ignite);
+
+            evts.add(name + " SESSION START " + ses.cacheName());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+            assertNotNull(ignite);
+
+            evts.add(name + " SESSION END " + ses.cacheName());
+        }
+    }
+
+    /**
+     */
+    private static class SessionListenerFactory implements Factory<CacheStoreSessionListener> {
+        /** */
+        private String name;
+
+        /**
+         * @param name Name.
+         */
+        private SessionListenerFactory(String name) {
+            this.name = name;
+        }
+
+        @Override public CacheStoreSessionListener create() {
+            return new SessionListener(name);
+        }
+    }
+
+    /**
+     */
+    public static class Store extends CacheStoreAdapter<Integer, Integer> {
+        public Store() {
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
index f72ea47..f2de8ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
@@ -266,30 +266,6 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @param col1 Collection 1.
-     * @param col2 Collection 2.
-     */
-    private static void assertEqualsCollections(Collection<?> col1, Collection<?> col2) {
-        if (col1.size() != col2.size())
-            fail("Collections are not equal:\nExpected:\t" + col1 + "\nActual:\t" + col2);
-
-        Iterator<?> it1 = col1.iterator();
-        Iterator<?> it2 = col2.iterator();
-
-        int idx = 0;
-
-        while (it1.hasNext()) {
-            Object item1 = it1.next();
-            Object item2 = it2.next();
-
-            if (!F.eq(item1, item2))
-                fail("Collections are not equal (position " + idx + "):\nExpected: " + col1 + "\nActual:   " + col2);
-
-            idx++;
-        }
-    }
-
-    /**
      *
      */
     private static class TestStore implements CacheStore<Object, Object> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 5533897..a19ea23 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -858,4 +858,28 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
             ccfg.getAtomicWriteOrderMode() == CacheAtomicWriteOrderMode.CLOCK)
             U.sleep(50);
     }
+
+    /**
+     * @param exp Expected.
+     * @param act Actual.
+     */
+    protected void assertEqualsCollections(Collection<?> exp, Collection<?> act) {
+        if (exp.size() != act.size())
+            fail("Collections are not equal:\nExpected:\t" + exp + "\nActual:\t" + act);
+
+        Iterator<?> it1 = exp.iterator();
+        Iterator<?> it2 = act.iterator();
+
+        int idx = 0;
+
+        while (it1.hasNext()) {
+            Object item1 = it1.next();
+            Object item2 = it2.next();
+
+            if (!F.eq(item1, item2))
+                fail("Collections are not equal (position " + idx + "):\nExpected: " + exp + "\nActual:   " + act);
+
+            idx++;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ada1b2a7/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
index 81736cd..90431d7 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
@@ -154,7 +154,7 @@ public class CacheSpringStoreSessionListener implements CacheStoreSessionListene
 
     /** {@inheritDoc} */
     @Override public void onSessionStart(CacheStoreSession ses) {
-        if (ses.isWithinTransaction()) {
+        if (ses.isWithinTransaction() && ses.attachment() == null) {
             try {
                 TransactionDefinition def = definition(ses.transaction(), ses.cacheName());
 


[14/39] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-876-2
Commit: 68811e7f91649b6aa652340ee19b0cafda64dd07
Parents: 3e35b9a 4b4158f
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed May 20 21:42:05 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed May 20 21:42:05 2015 -0700

----------------------------------------------------------------------
 dev-tools/build.gradle                          |  35 +-
 dev-tools/gradle/wrapper/gradle-wrapper.jar     | Bin 0 -> 51017 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |   6 +
 dev-tools/gradlew                               | 164 ++++++++++
 dev-tools/src/main/groovy/jiraslurp.groovy      | 326 ++++++++++++++-----
 modules/core/src/main/java/META-INF/LICENSE     | 238 ++++++++++++++
 modules/core/src/main/java/META-INF/NOTICE      |  12 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 parent/pom.xml                                  |   5 +-
 pom.xml                                         |  41 ++-
 10 files changed, 722 insertions(+), 107 deletions(-)
----------------------------------------------------------------------



[35/39] incubator-ignite git commit: Merge branch 'ignite-891' into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch 'ignite-891' into ignite-sprint-5


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

Branch: refs/heads/ignite-876-2
Commit: 2f615220d132f6ebe48472b61bbc15e851681d59
Parents: 0527d6d 58918a8
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon Jun 1 20:38:02 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon Jun 1 20:38:02 2015 -0700

----------------------------------------------------------------------
 .../hibernate/CacheHibernatePersonStore.java    | 202 +---------
 .../hibernate/CacheHibernateStoreExample.java   |  17 +
 .../store/jdbc/CacheJdbcPersonStore.java        | 180 ++-------
 .../store/jdbc/CacheJdbcStoreExample.java       |  13 +
 .../store/spring/CacheSpringPersonStore.java    | 128 ++++++
 .../store/spring/CacheSpringStoreExample.java   | 143 +++++++
 .../datagrid/store/spring/package-info.java     |  22 ++
 .../apache/ignite/cache/store/CacheStore.java   |   2 +
 .../ignite/cache/store/CacheStoreSession.java   |  22 ++
 .../cache/store/CacheStoreSessionListener.java  | 133 +++++++
 .../jdbc/CacheJdbcStoreSessionListener.java     | 141 +++++++
 .../configuration/CacheConfiguration.java       |  32 ++
 .../configuration/IgniteConfiguration.java      |  35 ++
 .../processors/cache/GridCacheProcessor.java    |  12 +-
 .../cache/GridCacheSharedContext.java           |  47 ++-
 .../processors/cache/GridCacheUtils.java        |  54 +++
 .../cache/store/CacheOsStoreManager.java        |   1 -
 .../cache/store/CacheStoreManager.java          |   7 +-
 .../store/GridCacheStoreManagerAdapter.java     | 202 ++++++++--
 .../cache/transactions/IgniteTxAdapter.java     |  33 +-
 .../transactions/IgniteTxLocalAdapter.java      | 142 ++++---
 ...cheStoreSessionListenerAbstractSelfTest.java | 315 +++++++++++++++
 ...heStoreSessionListenerLifecycleSelfTest.java | 395 +++++++++++++++++++
 .../CacheJdbcStoreSessionListenerSelfTest.java  | 175 ++++++++
 .../IgniteCrossCacheTxStoreSelfTest.java        | 147 ++++---
 .../loadtests/hashmap/GridCacheTestContext.java |   3 +-
 .../junits/cache/TestCacheSession.java          |  18 +
 .../cache/TestThreadLocalCacheSession.java      |  15 +
 .../junits/common/GridCommonAbstractTest.java   |  24 ++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 .../CacheHibernateStoreSessionListener.java     | 216 ++++++++++
 ...heHibernateStoreSessionListenerSelfTest.java | 228 +++++++++++
 .../testsuites/IgniteHibernateTestSuite.java    |   2 +
 modules/spring/pom.xml                          |  14 +
 .../spring/CacheSpringStoreSessionListener.java | 207 ++++++++++
 ...CacheSpringStoreSessionListenerSelfTest.java | 197 +++++++++
 .../testsuites/IgniteSpringTestSuite.java       |   3 +
 37 files changed, 3050 insertions(+), 480 deletions(-)
----------------------------------------------------------------------



[32/39] incubator-ignite git commit: # IGNITE-883 Don't create FileSwapSpi for clients.

Posted by sb...@apache.org.
# IGNITE-883 Don't create FileSwapSpi for clients.


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

Branch: refs/heads/ignite-876-2
Commit: 541b1e0f67089c3e6038ef787fd145977c84084b
Parents: 97d0b04
Author: sevdokimov <se...@gridgain.com>
Authored: Mon Jun 1 20:49:06 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Mon Jun 1 20:49:06 2015 +0300

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/ignite/internal/IgnitionEx.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/541b1e0f/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 6e4efb5..21f6652 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1844,7 +1844,7 @@ public class IgnitionEx {
             if (cfg.getSwapSpaceSpi() == null) {
                 boolean needSwap = false;
 
-                if (cfg.getCacheConfiguration() != null) {
+                if (cfg.getCacheConfiguration() != null && !Boolean.TRUE.equals(cfg.isClientMode())) {
                     for (CacheConfiguration c : cfg.getCacheConfiguration()) {
                         if (c.isSwapEnabled()) {
                             needSwap = true;


[22/39] incubator-ignite git commit: Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Posted by sb...@apache.org.
Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891


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

Branch: refs/heads/ignite-876-2
Commit: b5b452354198cad9e188d80887127df15e5b95c6
Parents: 7ab0304 050f429
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri May 22 13:29:09 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri May 22 13:29:09 2015 -0700

----------------------------------------------------------------------
 .gitignore                                      |   3 +-
 dev-tools/slurp.sh                              |  74 +++++++++++
 dev-tools/src/main/groovy/jiraslurp.groovy      | 132 +++++++++++--------
 .../ClientAbstractConnectivitySelfTest.java     |  14 ++
 .../internal/interop/InteropException.java      |  71 ++++++++++
 .../interop/InteropNoCallbackException.java     |  50 +++++++
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../processors/hadoop/v2/HadoopV2Context.java   |  10 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../config/benchmark-client-mode.properties     |  89 +++++++++++++
 10 files changed, 384 insertions(+), 67 deletions(-)
----------------------------------------------------------------------



[19/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: e47f85acae51f609e0bd1a1465c38d4a1bc96576
Parents: 4a55d29
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri May 22 00:02:07 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri May 22 00:02:07 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/cache/store/CacheStore.java   |  2 +
 .../cache/store/CacheStoreSessionListener.java  | 99 +++++++++++++++++++-
 .../jdbc/CacheStoreSessionJdbcListener.java     | 44 ++++++++-
 .../configuration/CacheConfiguration.java       |  6 ++
 .../configuration/IgniteConfiguration.java      |  7 ++
 .../CacheStoreSessionHibernateListener.java     | 48 +++++++++-
 .../spring/CacheStoreSessionSpringListener.java | 32 ++++++-
 7 files changed, 230 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e47f85ac/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java
index d018298..5bfdda1 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java
@@ -94,6 +94,8 @@ public interface CacheStore<K, V> extends CacheLoader<K, V>, CacheWriter<K, V> {
      * @throws CacheWriterException If commit or rollback failed. Note that commit failure in some cases
      *      may bring cache transaction into {@link TransactionState#UNKNOWN} which will
      *      consequently cause all transacted entries to be invalidated.
+     * @deprecated Use {@link CacheStoreSessionListener} instead (refer to its JavaDoc for details).
      */
+    @Deprecated
     public void sessionEnd(boolean commit) throws CacheWriterException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e47f85ac/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
index cba66c3..8b7cd8f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
@@ -17,12 +17,104 @@
 
 package org.apache.ignite.cache.store;
 
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.configuration.*;
+
+import javax.cache.configuration.*;
+import javax.sql.*;
+
 /**
- * Store session listener.
+ * Cache store session listener that allows to implement callbacks
+ * for session lifecycle.
+ * <p>
+ * The most common use case for session listeners is database
+ * connection and transaction management. Store can be invoked one
+ * or several times during one session, depending on whether it's
+ * executed within cache transaction or not. In any case, you have
+ * to create a connection when session is started and commit it or
+ * rollback when session is finished.
+ * <p>
+ * Cache store session listener allows to implement this and other
+ * scenarios providing to callback methods:
+ * <ul>
+ *     <li>
+ *         {@link #onSessionStart(CacheStoreSession)} - called
+ *         before any store operation within a session is invoked.
+ *     </li>
+ *     <li>
+ *         {@link #onSessionEnd(CacheStoreSession, boolean)} - called
+ *         after all operations within a session are invoked.
+ *     </li>
+ * </ul>
+ * <h2>Implementations</h2>
+ * Ignites provides several out-of-the-box implementations
+ * of session listener (refer to individual JavaDocs for more
+ * details):
+ * <ul>
+ *     <li>
+ *         {@link CacheStoreSessionJdbcListener} - JDBC-based session
+ *         listener. For each session it gets a new JDBC connection from
+ *         provided {@link DataSource} and commits (or rolls back) it
+ *         when session ends.
+ *     </li>
+ *     <li>
+ *         {@ignitelink org.apache.ignite.cache.store.spring.CacheStoreSessionSpringListener} -
+ *         session listener based on Spring transaction management.
+ *         It starts a new DB transaction for each session and commits
+ *         (or rolls back) it when session ends. If there is no ongoing
+ *         cache transaction, this listener is no-op.
+ *     </li>
+ *     <li>
+ *         <@ignitelink org.apache.ignite.cache.store.hibernate.CacheStoreSessionHibernateListener} -
+ *         Hibernate-based session listener. It creates a new Hibernate
+ *         session for each Ignite session. If there is an ongoing cache
+ *         transaction, a corresponding Hibernate transaction is created
+ *         as well.
+ *     </li>
+ * </ul>
+ * <h2>Configuration</h2>
+ * There are two ways to configure a session listener:
+ * <ul>
+ *     <li>
+ *         Provide a global listener for all caches via
+ *         {@link IgniteConfiguration#setCacheStoreSessionListenerFactories(Factory[])}
+ *         configuration property. This will we called for any store
+ *         session, not depending on what caches participate in
+ *         transaction.
+ *     </li>
+ *     <li>
+ *         Provide a listener for a particular cache via
+ *         {@link CacheConfiguration#setCacheStoreSessionListenerFactories(Factory[])}
+ *         configuration property. This will be called only if the
+ *         cache participates in transaction.
+ *     </li>
+ * </ul>
+ * For example, here is how global {@link CacheStoreSessionJdbcListener}
+ * can be configured in Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;bean class="org.apache.ignite.configuration.IgniteConfiguration"&gt;
+ *     ...
+ *
+ *     &lt;property name="CacheStoreSessionListenerFactories"&gt;
+ *         &lt;list&gt;
+ *             &lt;bean class="javax.cache.configuration.FactoryBuilder$SingletonFactory"&gt;
+ *                 &lt;constructor-arg&gt;
+ *                     &lt;bean class="org.apache.ignite.cache.store.jdbc.CacheStoreSessionJdbcListener"&gt;
+ *                         &lt;!-- Inject external data source. --&gt;
+ *                         &lt;property name="dataSource" ref="jdbc-data-source"/&gt;
+ *                     &lt;/bean&gt;
+ *                 &lt;/constructor-arg&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/list&gt;
+ *     &lt;/property&gt;
+ * &lt;/bean&gt;
+ * </pre>
  */
 public interface CacheStoreSessionListener {
     /**
      * On session start callback.
+     * <p>
+     * Called before any store operation within a session is invoked.
      *
      * @param ses Current session.
      */
@@ -30,9 +122,12 @@ public interface CacheStoreSessionListener {
 
     /**
      * On session end callback.
+     * <p>
+     * Called after all operations within a session are invoked.
      *
      * @param ses Current session.
-     * @param commit {@code True} if transaction should commit, {@code false} for rollback.
+     * @param commit {@code True} if persistence store transaction
+     *      should commit, {@code false} for rollback.
      */
     public void onSessionEnd(CacheStoreSession ses, boolean commit);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e47f85ac/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
index e4cd617..7920fae 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lifecycle.*;
 
+import javax.cache.*;
 import javax.cache.integration.*;
 import javax.sql.*;
 import java.sql.*;
@@ -29,6 +30,44 @@ import java.util.*;
 
 /**
  * Cache store session listener based on JDBC connection.
+ * <p>
+ * For each session this listener gets a new JDBC connection
+ * from provided {@link DataSource} and commits (or rolls
+ * back) it when session ends.
+ * <p>
+ * The connection is stored in store session
+ * {@link CacheStoreSession#properties() properties} and can
+ * be accessed at any moment by {@link #JDBC_CONN_KEY} key.
+ * The listener guarantees that the connection will be
+ * available for any store operation. If there is an
+ * ongoing cache transaction, all operations within this
+ * transaction will be committed or rolled back only when
+ * session ends.
+ * <p>
+ * As an example, here is how the {@link CacheStore#write(Cache.Entry)}
+ * method can be implemented if {@link CacheStoreSessionJdbcListener}
+ * is configured:
+ * <pre name="code" class="java">
+ * private static class Store extends CacheStoreAdapter&lt;Integer, Integer&gt; {
+ *     &#64;CacheStoreSessionResource
+ *     private CacheStoreSession ses;
+ *
+ *     &#64;Override public void write(Cache.Entry&lt;? extends Integer, ? extends Integer&gt; entry) throws CacheWriterException {
+ *         // Get connection from the current session.
+ *         Connection conn = ses.<String, Connection>properties().get(CacheStoreSessionJdbcListener.JDBC_CONN_KEY);
+ *
+ *         // Execute update SQL query.
+ *         try {
+ *             conn.createStatement().executeUpdate("...");
+ *         }
+ *         catch (SQLException e) {
+ *             throw new CacheWriterException("Failed to update the store.", e);
+ *         }
+ *     }
+ * }
+ * </pre>
+ * JDBC connection will be automatically created by the listener
+ * at the start of the session and closed when it ends.
  */
 public class CacheStoreSessionJdbcListener implements CacheStoreSessionListener, LifecycleAware {
     /** Session key for JDBC connection. */
@@ -39,12 +78,13 @@ public class CacheStoreSessionJdbcListener implements CacheStoreSessionListener,
 
     /**
      * Sets data source.
+     * <p>
+     * This is a required parameter. If data source is not set,
+     * exception will be thrown on startup.
      *
      * @param dataSrc Data source.
      */
     public void setDataSource(DataSource dataSrc) {
-        A.notNull(dataSrc, "dataSrc");
-
         this.dataSrc = dataSrc;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e47f85ac/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 33a5711c..0018218 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -1742,6 +1742,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Gets cache store session listener factories.
      *
      * @return Cache store session listener factories.
+     * @see CacheStoreSessionListener
      */
     public Factory<? extends CacheStoreSessionListener>[] getCacheStoreSessionListenerFactories() {
         return storeSesLsnrs;
@@ -1749,9 +1750,14 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
     /**
      * Cache store session listener factories.
+     * <p>
+     * These listeners override global listeners provided in
+     * {@link IgniteConfiguration#setCacheStoreSessionListenerFactories(Factory[])}
+     * configuration property.
      *
      * @param storeSesLsnrs Cache store session listener factories.
      * @return {@code this} for chaining.
+     * @see CacheStoreSessionListener
      */
     public CacheConfiguration setCacheStoreSessionListenerFactories(
         Factory<? extends CacheStoreSessionListener>... storeSesLsnrs) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e47f85ac/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 96ac7e3..5d4c98f 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -2249,6 +2249,7 @@ public class IgniteConfiguration {
      * Gets cache store session listener factories.
      *
      * @return Cache store session listener factories.
+     * @see CacheStoreSessionListener
      */
     public Factory<CacheStoreSessionListener>[] getCacheStoreSessionListenerFactories() {
         return storeSesLsnrs;
@@ -2256,9 +2257,15 @@ public class IgniteConfiguration {
 
     /**
      * Cache store session listener factories.
+     * <p>
+     * These are global store session listeners, so they are applied to
+     * all caches. If you need to override listeners for a
+     * particular cache, use {@link CacheConfiguration#setCacheStoreSessionListenerFactories(Factory[])}
+     * configuration property.
      *
      * @param storeSesLsnrs Cache store session listener factories.
      * @return {@code this} for chaining.
+     * @see CacheStoreSessionListener
      */
     public IgniteConfiguration setCacheStoreSessionListenerFactories(
         Factory<CacheStoreSessionListener>... storeSesLsnrs) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e47f85ac/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
index fe0960e..ea1214a 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
@@ -32,7 +32,45 @@ import java.net.*;
 import java.util.*;
 
 /**
- * Cache store session listener based on Hibernate session.
+ * Hibernate-based cache store session listener.
+ * <p>
+ * This listener creates a new Hibernate session for each store
+ * session. If there is an ongoing cache transaction, a corresponding
+ * Hibernate transaction is created as well.
+ * <p>
+ * The Hibernate session is stored in store session
+ * {@link CacheStoreSession#properties() properties} and can
+ * be accessed at any moment by {@link #HIBERNATE_SES_KEY} key.
+ * The listener guarantees that the session will be
+ * available for any store operation. If there is an
+ * ongoing cache transaction, all operations within this
+ * transaction will share a DB transaction.
+ * <p>
+ * As an example, here is how the {@link CacheStore#write(javax.cache.Cache.Entry)}
+ * method can be implemented if {@link CacheStoreSessionHibernateListener}
+ * is configured:
+ * <pre name="code" class="java">
+ * private static class Store extends CacheStoreAdapter&lt;Integer, Integer&gt; {
+ *     &#64;CacheStoreSessionResource
+ *     private CacheStoreSession ses;
+ *
+ *     &#64;Override public void write(Cache.Entry&lt;? extends Integer, ? extends Integer&gt; entry) throws CacheWriterException {
+ *         // Get Hibernate session from the current store session.
+ *         Session hibSes = ses.<String, Session>properties().get(CacheStoreSessionHibernateListener.HIBERNATE_SES_KEY);
+ *
+ *         // Persist the value.
+ *         hibSes.persist(entry.getValue());
+ *     }
+ * }
+ * </pre>
+ * Hibernate session will be automatically created by the listener
+ * at the start of the session and closed when it ends.
+ * <p>
+ * {@link CacheStoreSessionHibernateListener} requires that either
+ * {@link #setSessionFactory(SessionFactory)} session factory}
+ * or {@link #setHibernateConfigurationPath(String) Hibernate configuration file}
+ * is provided. If non of them is set, exception is thrown. Is both are provided,
+ * session factory will be used.
  */
 public class CacheStoreSessionHibernateListener implements CacheStoreSessionListener, LifecycleAware {
     /** Session key for JDBC connection. */
@@ -53,12 +91,13 @@ public class CacheStoreSessionHibernateListener implements CacheStoreSessionList
 
     /**
      * Sets Hibernate session factory.
+     * <p>
+     * Either session factory or configuration file is required.
+     * If none is provided, exception will be thrown on startup.
      *
      * @param sesFactory Session factory.
      */
     public void setSessionFactory(SessionFactory sesFactory) {
-        A.notNull(sesFactory, "sesFactory");
-
         this.sesFactory = sesFactory;
     }
 
@@ -73,6 +112,9 @@ public class CacheStoreSessionHibernateListener implements CacheStoreSessionList
 
     /**
      * Sets hibernate configuration path.
+     * <p>
+     * Either session factory or configuration file is required.
+     * If none is provided, exception will be thrown on startup.
      *
      * @param hibernateCfgPath Hibernate configuration path.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e47f85ac/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
index 2fab4f0..e5201ba 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.transactions.*;
+import org.springframework.jdbc.core.*;
 import org.springframework.jdbc.datasource.*;
 import org.springframework.transaction.*;
 import org.springframework.transaction.support.*;
@@ -31,7 +32,28 @@ import javax.cache.integration.*;
 import javax.sql.*;
 
 /**
- * Cache store session listener based on Spring cache manager.
+ * Cache store session listener based on Spring transaction management.
+ * <p>
+ * This listener starts a new DB transaction for each session and commits
+ * or rolls it back when session ends. If there is no ongoing
+ * cache transaction, this listener is no-op.
+ * <p>
+ * Store implementation can use any Spring APIs like {@link JdbcTemplate}
+ * and others. The listener will guarantee that if there is an
+ * ongoing cache transaction, all store operations within this
+ * transaction will be automatically enlisted in the same database
+ * transaction.
+ * <p>
+ * {@link CacheStoreSessionSpringListener} requires that either
+ * {@link #setTransactionManager(PlatformTransactionManager) transaction manager}
+ * or {@link #setDataSource(DataSource) data source} is configured. If non of them is
+ * provided, exception is thrown. Is both are provided, data source will be
+ * ignored.
+ * <p>
+ * If there is a transaction, a {@link TransactionStatus} object will be stored
+ * in store session {@link CacheStoreSession#properties() properties} and can be
+ * accessed at any moment by {@link #TX_STATUS_KEY} key. This can be used to
+ * acquire current DB transaction status.
  */
 public class CacheStoreSessionSpringListener implements CacheStoreSessionListener, LifecycleAware {
     /** Session key for transaction status. */
@@ -52,6 +74,9 @@ public class CacheStoreSessionSpringListener implements CacheStoreSessionListene
 
     /**
      * Sets transaction manager.
+     * <p>
+     * Either transaction manager or data source is required.
+     * If none is provided, exception will be thrown on startup.
      *
      * @param txMgr Transaction manager.
      */
@@ -70,6 +95,9 @@ public class CacheStoreSessionSpringListener implements CacheStoreSessionListene
 
     /**
      * Sets data source.
+     * <p>
+     * Either transaction manager or data source is required.
+     * If none is provided, exception will be thrown on startup.
      *
      * @param dataSrc Data source.
      */
@@ -88,6 +116,8 @@ public class CacheStoreSessionSpringListener implements CacheStoreSessionListene
 
     /**
      * Sets propagation behavior.
+     * <p>
+     * This parameter is optional.
      *
      * @param propagation Propagation behavior.
      */


[34/39] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-5' into ignite-sprint-5


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

Branch: refs/heads/ignite-876-2
Commit: 0527d6d1560ce1982ae0be33f05c7db7899097b5
Parents: 9e10c03 541b1e0
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon Jun 1 16:44:48 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon Jun 1 16:44:48 2015 -0700

----------------------------------------------------------------------
 dev-tools/gradle/wrapper/gradle-wrapper.jar     |  Bin 51017 -> 0 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |   18 +-
 dev-tools/gradlew                               |  163 +-
 dev-tools/slurp.sh                              |   14 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  313 +-
 examples/pom.xml                                |    2 +-
 modules/aop/pom.xml                             |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/clients/pom.xml                         |    2 +-
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 modules/core/pom.xml                            |    2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    8 +-
 .../java/org/apache/ignite/IgniteServices.java  |    5 +-
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |  187 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   27 +-
 .../configuration/IgniteConfiguration.java      |   10 +-
 .../org/apache/ignite/igfs/IgfsUserContext.java |  119 +
 .../igfs/secondary/IgfsSecondaryFileSystem.java |    7 +
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   27 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   22 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../internal/igfs/common/IgfsMarshaller.java    |   35 +-
 .../igfs/common/IgfsPathControlRequest.java     |   22 +
 .../internal/managers/GridManagerAdapter.java   |   59 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   48 +
 .../discovery/GridDiscoveryManager.java         |  214 +-
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../processors/cache/CacheMetricsImpl.java      |  367 +-
 .../cache/CacheMetricsMXBeanImpl.java           |  100 +
 .../processors/cache/CacheMetricsSnapshot.java  |  380 +-
 .../cache/DynamicCacheChangeBatch.java          |   20 +-
 .../cache/DynamicCacheDescriptor.java           |    2 +
 .../processors/cache/GridCacheAdapter.java      |   29 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |    6 +-
 .../processors/cache/GridCacheEntryEx.java      |    6 +
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMapEntry.java     |    5 +
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   71 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |   74 +-
 .../processors/cache/GridCacheProxyImpl.java    |   12 +
 .../cache/GridCacheSharedContext.java           |    1 +
 .../processors/cache/GridCacheSwapManager.java  |  118 +-
 .../processors/cache/GridCacheUtils.java        |  234 +-
 .../processors/cache/IgniteCacheProxy.java      |    3 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedCacheEntry.java  |    7 -
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../GridDistributedTxRemoteAdapter.java         |   10 +-
 .../dht/GridClientPartitionTopology.java        |    8 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    6 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   22 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   78 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  313 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   28 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 +-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   83 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |   12 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   15 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   54 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  836 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../datastreamer/DataStreamerImpl.java          |    2 +
 .../internal/processors/hadoop/HadoopJob.java   |    2 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |    8 +-
 .../internal/processors/igfs/IgfsImpl.java      |    8 +-
 .../processors/igfs/IgfsIpcHandler.java         |  184 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |    9 +-
 .../internal/processors/igfs/IgfsServer.java    |    4 +-
 .../internal/processors/igfs/IgfsUtils.java     |   16 +
 .../processors/query/GridQueryIndexing.java     |   16 +
 .../processors/query/GridQueryProcessor.java    |   52 +-
 .../service/GridServiceProcessor.java           |  121 +-
 .../ignite/internal/util/GridJavaProcess.java   |   30 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |    2 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |   80 +
 .../apache/ignite/plugin/PluginProvider.java    |   26 +-
 .../org/apache/ignite/services/Service.java     |    5 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   37 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   47 -
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |   40 +
 .../spi/discovery/DiscoverySpiListener.java     |    5 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1478 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4766 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  170 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5799 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   10 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   24 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   67 +
 .../messages/TcpDiscoveryClientPingRequest.java |   56 +
 .../TcpDiscoveryClientPingResponse.java         |   67 +
 .../TcpDiscoveryCustomEventMessage.java         |   41 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   28 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   43 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |    2 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    6 +
 .../tcp/messages/TcpDiscoveryPingResponse.java  |   15 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    8 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 .../ignite/internal/GridAffinitySelfTest.java   |    1 +
 .../internal/GridDiscoveryEventSelfTest.java    |    7 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    7 +-
 .../internal/GridProjectionAbstractTest.java    |   16 +
 .../GridProjectionForCachesSelfTest.java        |   11 +-
 .../internal/GridReleaseTypeSelfTest.java       |   77 +-
 .../apache/ignite/internal/GridSelfTest.java    |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   62 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |  122 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   46 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    1 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    3 +
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../processors/cache/GridCacheTestEntryEx.java  |    4 +
 .../GridCacheVariableTopologySelfTest.java      |   12 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |    2 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   29 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   13 +-
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   20 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   47 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |   25 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java |  412 ++
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   93 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../DataStreamerMultiThreadedSelfTest.java      |    2 -
 .../igfs/IgfsClientCacheSelfTest.java           |    3 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |  251 +
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    8 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1171 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../testframework/GridSpiTestContext.java       |   25 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/common/GridCommonAbstractTest.java   |   59 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    4 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |    1 +
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    3 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 .../query/h2/GridH2IndexingGeoSelfTest.java     |   20 +-
 modules/hadoop/pom.xml                          |    2 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  165 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  107 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   32 +-
 .../internal/processors/hadoop/HadoopUtils.java |   10 +-
 .../hadoop/SecondaryFileSystemProvider.java     |   53 +-
 .../hadoop/fs/HadoopDistributedFileSystem.java  |   91 -
 .../hadoop/fs/HadoopFileSystemsUtils.java       |   17 -
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |  204 +
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |    6 +
 .../hadoop/igfs/HadoopIgfsInProc.java           |  170 +-
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |    2 +-
 .../hadoop/igfs/HadoopIgfsOutProc.java          |   33 +-
 .../hadoop/igfs/HadoopIgfsWrapper.java          |   19 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |    4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   56 +-
 ...oopSecondaryFileSystemConfigurationTest.java |    4 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   63 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |    2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |    2 +
 .../hadoop/HadoopFileSystemsTest.java           |   23 +-
 .../collections/HadoopSkipListSelfTest.java     |    4 +-
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   17 +
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |   18 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |    4 +-
 .../query/h2/sql/GridSqlOperationType.java      |    2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    4 +
 .../h2/twostep/GridReduceQueryExecutor.java     |    2 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |    2 +-
 .../local/IgniteCacheLocalQuerySelfTest.java    |    6 +
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   16 +
 modules/jcl/pom.xml                             |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/mesos/README.txt                        |   28 +
 modules/mesos/licenses/apache-2.0.txt           |  202 +
 modules/mesos/licenses/jetty-epl-license.txt    |   69 +
 modules/mesos/pom.xml                           |  101 +
 .../apache/ignite/mesos/ClusterProperties.java  |  519 ++
 .../apache/ignite/mesos/IgniteFramework.java    |  119 +
 .../apache/ignite/mesos/IgniteScheduler.java    |  361 ++
 .../org/apache/ignite/mesos/IgniteTask.java     |   86 +
 .../org/apache/ignite/mesos/package-info.java   |   22 +
 .../ignite/mesos/resource/IgniteProvider.java   |  234 +
 .../ignite/mesos/resource/JettyServer.java      |   61 +
 .../ignite/mesos/resource/ResourceHandler.java  |  142 +
 .../ignite/mesos/resource/ResourceProvider.java |  120 +
 .../ignite/mesos/resource/package-info.java     |   22 +
 .../main/resources/ignite-default-config.xml    |   35 +
 .../org/apache/ignite/IgniteMesosTestSuite.java |   38 +
 .../ignite/mesos/IgniteSchedulerSelfTest.java   |  464 ++
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    2 +-
 modules/schema-import/pom.xml                   |    2 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 .../licenses/jcraft-revised-bsd.txt             |   28 -
 modules/visor-console/pom.xml                   |    2 +-
 .../visor/commands/node/VisorNodeCommand.scala  |    2 +-
 .../commands/tasks/VisorTasksCommand.scala      |    2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   63 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web/pom.xml                             |    2 +-
 modules/yardstick/pom.xml                       |    2 +-
 parent/pom.xml                                  |    4 +
 pom.xml                                         |   14 +-
 scripts/git-format-patch.sh                     |   14 +-
 scripts/git-patch-prop.sh                       |    2 +-
 347 files changed, 23556 insertions(+), 11344 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0527d6d1/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------


[36/39] incubator-ignite git commit: # ignite-876 renamed test

Posted by sb...@apache.org.
# ignite-876 renamed test


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

Branch: refs/heads/ignite-876-2
Commit: c1f0ce2df5087051b07ce1552b8a2a85097f0fa5
Parents: 21b6a03
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 10:45:01 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 10:45:01 2015 +0300

----------------------------------------------------------------------
 ...QueryMultiThreadedOffHeapTieredSelfTest.java | 37 ++++++++++++++++++++
 ...eQueryMultiThreadedOffHeapTiredSelfTest.java | 37 --------------------
 .../IgniteCacheQuerySelfTestSuite.java          |  2 +-
 3 files changed, 38 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1f0ce2d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java
new file mode 100644
index 0000000..df4c01d
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.cache;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ * Test queries in off-heap tiered mode.
+ */
+public class IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest extends IgniteCacheQueryMultiThreadedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration() {
+        CacheConfiguration ccfg = super.cacheConfiguration();
+
+        ccfg.setCacheMode(CacheMode.REPLICATED);
+        ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+        ccfg.setOffHeapMaxMemory(0);
+
+        return ccfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1f0ce2d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java
deleted file mode 100644
index 8b09d0f..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-
-/**
- * Test queries in off-heap tired mode.
- */
-public class IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest extends IgniteCacheQueryMultiThreadedSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration() {
-        CacheConfiguration ccfg = super.cacheConfiguration();
-
-        ccfg.setCacheMode(CacheMode.REPLICATED);
-        ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
-        ccfg.setOffHeapMaxMemory(0);
-
-        return ccfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1f0ce2d/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index f42963a..12134d6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -59,7 +59,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
         suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.class);
+        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);


[24/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: 990bf9e38b405e97d0be1891be7680423208b279
Parents: b5b4523
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri May 22 18:49:04 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri May 22 18:49:04 2015 -0700

----------------------------------------------------------------------
 .../hibernate/CacheHibernatePersonStore.java    | 183 +--------------
 .../hibernate/CacheHibernateStoreExample.java   |  16 ++
 .../store/jdbc/CacheJdbcPersonStore.java        |  18 +-
 .../store/jdbc/CacheJdbcStoreExample.java       |   2 +-
 .../ignite/cache/store/CacheStoreSession.java   |   5 +
 .../cache/store/CacheStoreSessionListener.java  |  10 +-
 .../jdbc/CacheJdbcStoreSessionListener.java     | 143 +++++++++++
 .../jdbc/CacheStoreSessionJdbcListener.java     | 148 ------------
 .../store/GridCacheStoreManagerAdapter.java     |  32 +++
 .../CacheJdbcStoreSessionListenerSelfTest.java  | 175 ++++++++++++++
 .../CacheStoreSessionJdbcListenerSelfTest.java  | 182 --------------
 .../junits/cache/TestCacheSession.java          |  14 ++
 .../cache/TestThreadLocalCacheSession.java      |  16 ++
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +-
 .../CacheHibernateStoreSessionListener.java     | 218 +++++++++++++++++
 .../CacheStoreSessionHibernateListener.java     | 223 ------------------
 ...heHibernateStoreSessionListenerSelfTest.java | 228 ++++++++++++++++++
 ...heStoreSessionHibernateListenerSelfTest.java | 235 -------------------
 .../testsuites/IgniteHibernateTestSuite.java    |   2 +-
 .../spring/CacheSpringStoreSessionListener.java | 233 ++++++++++++++++++
 .../spring/CacheStoreSessionSpringListener.java | 235 -------------------
 ...CacheSpringStoreSessionListenerSelfTest.java | 197 ++++++++++++++++
 ...CacheStoreSessionSpringListenerSelfTest.java | 204 ----------------
 .../testsuites/IgniteSpringTestSuite.java       |   2 +-
 24 files changed, 1303 insertions(+), 1420 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
index 577301c..557ec6f 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
@@ -21,10 +21,7 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.examples.datagrid.store.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.apache.ignite.transactions.Transaction;
 import org.hibernate.*;
-import org.hibernate.cfg.*;
-import org.jetbrains.annotations.*;
 
 import javax.cache.integration.*;
 import java.util.*;
@@ -34,57 +31,30 @@ import java.util.*;
  * and deals with maps {@link UUID} to {@link Person}.
  */
 public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
-    /** Default hibernate configuration resource path. */
-    private static final String DFLT_HIBERNATE_CFG = "/org/apache/ignite/examples/datagrid/store/hibernate" +
-        "/hibernate.cfg.xml";
-
-    /** Session attribute name. */
-    private static final String ATTR_SES = "HIBERNATE_STORE_SESSION";
-
-    /** Session factory. */
-    private SessionFactory sesFactory;
-
     /** Auto-injected store session. */
     @CacheStoreSessionResource
     private CacheStoreSession ses;
 
-    /**
-     * Default constructor.
-     */
-    public CacheHibernatePersonStore() {
-        sesFactory = new Configuration().configure(DFLT_HIBERNATE_CFG).buildSessionFactory();
-    }
-
     /** {@inheritDoc} */
     @Override public Person load(Long key) {
-        Transaction tx = transaction();
-
-        System.out.println(">>> Store load [key=" + key + ", xid=" + (tx == null ? null : tx.xid()) + ']');
+        System.out.println(">>> Store load [key=" + key + ']');
 
-        Session ses = session(tx);
+        Session hibSes = ses.attachment();
 
         try {
-            return (Person) ses.get(Person.class, key);
+            return (Person)hibSes.get(Person.class, key);
         }
         catch (HibernateException e) {
-            rollback(ses, tx);
-
-            throw new CacheLoaderException("Failed to load value from cache store with key: " + key, e);
-        }
-        finally {
-            end(ses, tx);
+            throw new CacheLoaderException("Failed to load value from cache store [key=" + key + ']', e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void write(javax.cache.Cache.Entry<? extends Long, ? extends Person> entry) {
-        Transaction tx = transaction();
-
         Long key = entry.getKey();
-
         Person val = entry.getValue();
 
-        System.out.println(">>> Store put [key=" + key + ", val=" + val + ", xid=" + (tx == null ? null : tx.xid()) + ']');
+        System.out.println(">>> Store put [key=" + key + ", val=" + val + ']');
 
         if (val == null) {
             delete(key);
@@ -92,41 +62,29 @@ public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
             return;
         }
 
-        Session ses = session(tx);
+        Session hibSes = ses.attachment();
 
         try {
-            ses.saveOrUpdate(val);
+            hibSes.saveOrUpdate(val);
         }
         catch (HibernateException e) {
-            rollback(ses, tx);
-
             throw new CacheWriterException("Failed to put value to cache store [key=" + key + ", val" + val + "]", e);
         }
-        finally {
-            end(ses, tx);
-        }
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings({"JpaQueryApiInspection"})
     @Override public void delete(Object key) {
-        Transaction tx = transaction();
-
-        System.out.println(">>> Store remove [key=" + key + ", xid=" + (tx == null ? null : tx.xid()) + ']');
+        System.out.println(">>> Store remove [key=" + key + ']');
 
-        Session ses = session(tx);
+        Session hibSes = ses.attachment();
 
         try {
-            ses.createQuery("delete " + Person.class.getSimpleName() + " where key = :key")
+            hibSes.createQuery("delete " + Person.class.getSimpleName() + " where key = :key")
                 .setParameter("key", key).setFlushMode(FlushMode.ALWAYS).executeUpdate();
         }
         catch (HibernateException e) {
-            rollback(ses, tx);
-
-            throw new CacheWriterException("Failed to remove value from cache store with key: " + key, e);
-        }
-        finally {
-            end(ses, tx);
+            throw new CacheWriterException("Failed to remove value from cache store [key=" + key + ']', e);
         }
     }
 
@@ -137,12 +95,12 @@ public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
 
         final int entryCnt = (Integer)args[0];
 
-        Session ses = session(null);
+        Session hibSes = ses.attachment();
 
         try {
             int cnt = 0;
 
-            List res = ses.createCriteria(Person.class).list();
+            List res = hibSes.createCriteria(Person.class).list();
 
             if (res != null) {
                 Iterator iter = res.iterator();
@@ -161,120 +119,5 @@ public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
         catch (HibernateException e) {
             throw new CacheLoaderException("Failed to load values from cache store.", e);
         }
-        finally {
-            end(ses, null);
-        }
-    }
-
-    /**
-     * Rolls back hibernate session.
-     *
-     * @param ses Hibernate session.
-     * @param tx Cache ongoing transaction.
-     */
-    private void rollback(Session ses, Transaction tx) {
-        // Rollback only if there is no cache transaction,
-        // otherwise sessionEnd() will do all required work.
-        if (tx == null) {
-            org.hibernate.Transaction hTx = ses.getTransaction();
-
-            if (hTx != null && hTx.isActive())
-                hTx.rollback();
-        }
-    }
-
-    /**
-     * Ends hibernate session.
-     *
-     * @param ses Hibernate session.
-     * @param tx Cache ongoing transaction.
-     */
-    private void end(Session ses, @Nullable Transaction tx) {
-        // Commit only if there is no cache transaction,
-        // otherwise sessionEnd() will do all required work.
-        if (tx == null) {
-            org.hibernate.Transaction hTx = ses.getTransaction();
-
-            if (hTx != null && hTx.isActive())
-                hTx.commit();
-
-            ses.close();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sessionEnd(boolean commit) {
-        Transaction tx = ses.transaction();
-
-        Map<String, Session> props = ses.properties();
-
-        Session ses = props.remove(ATTR_SES);
-
-        if (ses != null) {
-            org.hibernate.Transaction hTx = ses.getTransaction();
-
-            if (hTx != null) {
-                try {
-                    if (commit) {
-                        ses.flush();
-
-                        hTx.commit();
-                    }
-                    else
-                        hTx.rollback();
-
-                    System.out.println("Transaction ended [xid=" + tx.xid() + ", commit=" + commit + ']');
-                }
-                catch (HibernateException e) {
-                    throw new CacheWriterException("Failed to end transaction [xid=" + tx.xid() +
-                        ", commit=" + commit + ']', e);
-                }
-                finally {
-                    ses.close();
-                }
-            }
-        }
-    }
-
-    /**
-     * Gets Hibernate session.
-     *
-     * @param tx Cache transaction.
-     * @return Session.
-     */
-    private Session session(@Nullable Transaction tx) {
-        Session hbSes;
-
-        if (tx != null) {
-            Map<String, Session> props = ses.properties();
-
-            hbSes = props.get(ATTR_SES);
-
-            if (hbSes == null) {
-                hbSes = sesFactory.openSession();
-
-                hbSes.beginTransaction();
-
-                // Store session in session properties, so it can be accessed
-                // for other operations on the same transaction.
-                props.put(ATTR_SES, hbSes);
-
-                System.out.println("Hibernate session open [ses=" + hbSes + ", tx=" + tx.xid() + "]");
-            }
-        }
-        else {
-            hbSes = sesFactory.openSession();
-
-            hbSes.beginTransaction();
-        }
-
-        return hbSes;
-    }
-
-    /**
-     * @return Current transaction.
-     */
-    @Nullable private Transaction transaction() {
-        return ses != null ? ses.transaction() : null;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
index 5a0cd0a..ac19072 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.examples.datagrid.store.hibernate;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.hibernate.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.examples.*;
 import org.apache.ignite.examples.datagrid.store.*;
@@ -37,6 +39,10 @@ import static org.apache.ignite.cache.CacheAtomicityMode.*;
  * start node with {@code examples/config/example-ignite.xml} configuration.
  */
 public class CacheHibernateStoreExample {
+    /** Hibernate configuration resource path. */
+    private static final String HIBERNATE_CFG =
+        "/org/apache/ignite/examples/datagrid/store/hibernate/hibernate.cfg.xml";
+
     /** Cache name. */
     private static final String CACHE_NAME = CacheHibernateStoreExample.class.getSimpleName();
 
@@ -71,6 +77,16 @@ public class CacheHibernateStoreExample {
             // Configure Hibernate store.
             cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(CacheHibernatePersonStore.class));
 
+            cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
+                @Override public CacheStoreSessionListener create() {
+                    CacheHibernateStoreSessionListener lsnr = new CacheHibernateStoreSessionListener();
+
+                    lsnr.setHibernateConfigurationPath(HIBERNATE_CFG);
+
+                    return lsnr;
+                }
+            });
+
             cacheCfg.setReadThrough(true);
             cacheCfg.setWriteThrough(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
index 856512b..6eb0386 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
@@ -19,7 +19,6 @@ package org.apache.ignite.examples.datagrid.store.jdbc;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
-import org.apache.ignite.cache.store.jdbc.*;
 import org.apache.ignite.examples.datagrid.store.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
@@ -70,7 +69,7 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
         System.out.println(">>> Loading key: " + key);
 
         try {
-            Connection conn = connection();
+            Connection conn = ses.attachment();
 
             try (PreparedStatement st = conn.prepareStatement("select * from PERSONS where id=?")) {
                 st.setString(1, key.toString());
@@ -97,7 +96,7 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
         System.out.println(">>> Putting [key=" + key + ", val=" + val +  ']');
 
         try {
-            Connection conn = connection();
+            Connection conn = ses.attachment();
 
             int updated;
 
@@ -134,7 +133,7 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
         System.out.println(">>> Removing key: " + key);
 
         try {
-            Connection conn = connection();
+            Connection conn = ses.attachment();
 
             try (PreparedStatement st = conn.prepareStatement("delete from PERSONS where id=?")) {
                 st.setLong(1, (Long)key);
@@ -154,7 +153,7 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
 
         final int entryCnt = (Integer)args[0];
 
-        Connection conn = connection();
+        Connection conn = ses.attachment();
 
         try (
             PreparedStatement st = conn.prepareStatement("select * from PERSONS");
@@ -176,13 +175,4 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
             throw new CacheLoaderException("Failed to load values from cache store.", e);
         }
     }
-
-    /**
-     * Gets JDBC connection attached to current session.
-     *
-     * @return Connection.
-     */
-    private Connection connection() {
-        return ses.<String, Connection>properties().get(CacheStoreSessionJdbcListener.JDBC_CONN_KEY);
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
index 82e1079..74e262c 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
@@ -77,7 +77,7 @@ public class CacheJdbcStoreExample {
             // Configure JDBC session listener.
             cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
                 @Override public CacheStoreSessionListener create() {
-                    CacheStoreSessionJdbcListener lsnr = new CacheStoreSessionJdbcListener();
+                    CacheJdbcStoreSessionListener lsnr = new CacheJdbcStoreSessionListener();
 
                     lsnr.setDataSource(JdbcConnectionPool.create("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1", "", ""));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
index 640d4a3..980c6df 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
@@ -19,6 +19,7 @@ package org.apache.ignite.cache.store;
 
 import org.apache.ignite.resources.*;
 import org.apache.ignite.transactions.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -51,6 +52,10 @@ public interface CacheStoreSession {
      */
     public boolean isWithinTransaction();
 
+    public void attach(@Nullable Object attachment);
+
+    @Nullable public <T> T attachment();
+
     /**
      * Gets current session properties. You can add properties directly to the
      * returned map.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
index 8b7cd8f..b319e55 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
@@ -52,20 +52,20 @@ import javax.sql.*;
  * details):
  * <ul>
  *     <li>
- *         {@link CacheStoreSessionJdbcListener} - JDBC-based session
+ *         {@link CacheJdbcStoreSessionListener} - JDBC-based session
  *         listener. For each session it gets a new JDBC connection from
  *         provided {@link DataSource} and commits (or rolls back) it
  *         when session ends.
  *     </li>
  *     <li>
- *         {@ignitelink org.apache.ignite.cache.store.spring.CacheStoreSessionSpringListener} -
+ *         {@ignitelink org.apache.ignite.cache.store.spring.CacheSpringStoreSessionListener} -
  *         session listener based on Spring transaction management.
  *         It starts a new DB transaction for each session and commits
  *         (or rolls back) it when session ends. If there is no ongoing
  *         cache transaction, this listener is no-op.
  *     </li>
  *     <li>
- *         <@ignitelink org.apache.ignite.cache.store.hibernate.CacheStoreSessionHibernateListener} -
+ *         <@ignitelink org.apache.ignite.cache.store.hibernate.CacheHibernateStoreSessionListener} -
  *         Hibernate-based session listener. It creates a new Hibernate
  *         session for each Ignite session. If there is an ongoing cache
  *         transaction, a corresponding Hibernate transaction is created
@@ -89,7 +89,7 @@ import javax.sql.*;
  *         cache participates in transaction.
  *     </li>
  * </ul>
- * For example, here is how global {@link CacheStoreSessionJdbcListener}
+ * For example, here is how global {@link CacheJdbcStoreSessionListener}
  * can be configured in Spring XML configuration file:
  * <pre name="code" class="xml">
  * &lt;bean class="org.apache.ignite.configuration.IgniteConfiguration"&gt;
@@ -99,7 +99,7 @@ import javax.sql.*;
  *         &lt;list&gt;
  *             &lt;bean class="javax.cache.configuration.FactoryBuilder$SingletonFactory"&gt;
  *                 &lt;constructor-arg&gt;
- *                     &lt;bean class="org.apache.ignite.cache.store.jdbc.CacheStoreSessionJdbcListener"&gt;
+ *                     &lt;bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListener"&gt;
  *                         &lt;!-- Inject external data source. --&gt;
  *                         &lt;property name="dataSource" ref="jdbc-data-source"/&gt;
  *                     &lt;/bean&gt;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
new file mode 100644
index 0000000..c59e86f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
@@ -0,0 +1,143 @@
+/*
+ * 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.cache.store.jdbc;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lifecycle.*;
+
+import javax.cache.*;
+import javax.cache.integration.*;
+import javax.sql.*;
+import java.sql.*;
+
+/**
+ * Cache store session listener based on JDBC connection.
+ * <p>
+ * For each session this listener gets a new JDBC connection
+ * from provided {@link DataSource} and commits (or rolls
+ * back) it when session ends.
+ * <p>
+ * The connection is saved as a store session
+  * {@link CacheStoreSession#attachment() attachment}.
+ * The listener guarantees that the connection will be
+ * available for any store operation. If there is an
+ * ongoing cache transaction, all operations within this
+ * transaction will be committed or rolled back only when
+ * session ends.
+ * <p>
+ * As an example, here is how the {@link CacheStore#write(Cache.Entry)}
+ * method can be implemented if {@link CacheJdbcStoreSessionListener}
+ * is configured:
+ * <pre name="code" class="java">
+ * private static class Store extends CacheStoreAdapter&lt;Integer, Integer&gt; {
+ *     &#64;CacheStoreSessionResource
+ *     private CacheStoreSession ses;
+ *
+ *     &#64;Override public void write(Cache.Entry&lt;? extends Integer, ? extends Integer&gt; entry) throws CacheWriterException {
+ *         // Get connection from the current session.
+ *         Connection conn = ses.attachment();
+ *
+ *         // Execute update SQL query.
+ *         try {
+ *             conn.createStatement().executeUpdate("...");
+ *         }
+ *         catch (SQLException e) {
+ *             throw new CacheWriterException("Failed to update the store.", e);
+ *         }
+ *     }
+ * }
+ * </pre>
+ * JDBC connection will be automatically created by the listener
+ * at the start of the session and closed when it ends.
+ */
+public class CacheJdbcStoreSessionListener implements CacheStoreSessionListener, LifecycleAware {
+    /** Data source. */
+    private DataSource dataSrc;
+
+    /**
+     * Sets data source.
+     * <p>
+     * This is a required parameter. If data source is not set,
+     * exception will be thrown on startup.
+     *
+     * @param dataSrc Data source.
+     */
+    public void setDataSource(DataSource dataSrc) {
+        this.dataSrc = dataSrc;
+    }
+
+    /**
+     * Gets data source.
+     *
+     * @return Data source.
+     */
+    public DataSource getDataSource() {
+        return dataSrc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteException {
+        if (dataSrc == null)
+            throw new IgniteException("Data source is required by " + getClass().getSimpleName() + '.');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionStart(CacheStoreSession ses) {
+        if (ses.attachment() == null) {
+            try {
+                Connection conn = dataSrc.getConnection();
+
+                conn.setAutoCommit(false);
+
+                ses.attach(conn);
+            }
+            catch (SQLException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+        Connection conn = ses.attachment();
+
+        if (conn != null) {
+            ses.attach(null);
+
+            try {
+                if (commit)
+                    conn.commit();
+                else
+                    conn.rollback();
+            }
+            catch (SQLException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+            finally {
+                U.closeQuiet(conn);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
deleted file mode 100644
index 7920fae..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.store.jdbc;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.store.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lifecycle.*;
-
-import javax.cache.*;
-import javax.cache.integration.*;
-import javax.sql.*;
-import java.sql.*;
-import java.util.*;
-
-/**
- * Cache store session listener based on JDBC connection.
- * <p>
- * For each session this listener gets a new JDBC connection
- * from provided {@link DataSource} and commits (or rolls
- * back) it when session ends.
- * <p>
- * The connection is stored in store session
- * {@link CacheStoreSession#properties() properties} and can
- * be accessed at any moment by {@link #JDBC_CONN_KEY} key.
- * The listener guarantees that the connection will be
- * available for any store operation. If there is an
- * ongoing cache transaction, all operations within this
- * transaction will be committed or rolled back only when
- * session ends.
- * <p>
- * As an example, here is how the {@link CacheStore#write(Cache.Entry)}
- * method can be implemented if {@link CacheStoreSessionJdbcListener}
- * is configured:
- * <pre name="code" class="java">
- * private static class Store extends CacheStoreAdapter&lt;Integer, Integer&gt; {
- *     &#64;CacheStoreSessionResource
- *     private CacheStoreSession ses;
- *
- *     &#64;Override public void write(Cache.Entry&lt;? extends Integer, ? extends Integer&gt; entry) throws CacheWriterException {
- *         // Get connection from the current session.
- *         Connection conn = ses.<String, Connection>properties().get(CacheStoreSessionJdbcListener.JDBC_CONN_KEY);
- *
- *         // Execute update SQL query.
- *         try {
- *             conn.createStatement().executeUpdate("...");
- *         }
- *         catch (SQLException e) {
- *             throw new CacheWriterException("Failed to update the store.", e);
- *         }
- *     }
- * }
- * </pre>
- * JDBC connection will be automatically created by the listener
- * at the start of the session and closed when it ends.
- */
-public class CacheStoreSessionJdbcListener implements CacheStoreSessionListener, LifecycleAware {
-    /** Session key for JDBC connection. */
-    public static final String JDBC_CONN_KEY = "__jdbc_conn_";
-
-    /** Data source. */
-    private DataSource dataSrc;
-
-    /**
-     * Sets data source.
-     * <p>
-     * This is a required parameter. If data source is not set,
-     * exception will be thrown on startup.
-     *
-     * @param dataSrc Data source.
-     */
-    public void setDataSource(DataSource dataSrc) {
-        this.dataSrc = dataSrc;
-    }
-
-    /**
-     * Gets data source.
-     *
-     * @return Data source.
-     */
-    public DataSource getDataSource() {
-        return dataSrc;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void start() throws IgniteException {
-        if (dataSrc == null)
-            throw new IgniteException("Data source is required by " + getClass().getSimpleName() + '.');
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stop() throws IgniteException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionStart(CacheStoreSession ses) {
-        Map<String, Connection> props = ses.properties();
-
-        if (!props.containsKey(JDBC_CONN_KEY)) {
-            try {
-                Connection conn = dataSrc.getConnection();
-
-                conn.setAutoCommit(false);
-
-                props.put(JDBC_CONN_KEY, conn);
-            }
-            catch (SQLException e) {
-                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
-        Connection conn = ses.<String, Connection>properties().remove(JDBC_CONN_KEY);
-
-        if (conn != null) {
-            try {
-                if (commit)
-                    conn.commit();
-                else
-                    conn.rollback();
-            }
-            catch (SQLException e) {
-                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
-            }
-            finally {
-                U.closeQuiet(conn);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 8096291..11d711c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -855,6 +855,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         private Map<Object, Object> props;
 
         /** */
+        private Object attachment;
+
+        /** */
         private boolean started;
 
         /** */
@@ -887,6 +890,20 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         }
 
         /**
+         * @param attachment Attachment.
+         */
+        private void attach(Object attachment) {
+            this.attachment = attachment;
+        }
+
+        /**
+         * @return Attachment.
+         */
+        private Object attachment() {
+            return attachment;
+        }
+
+        /**
          * @return Cache name.
          */
         private String cacheName() {
@@ -954,6 +971,21 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         }
 
         /** {@inheritDoc} */
+        @Override public void attach(@Nullable Object attachment) {
+            SessionData ses0 = sesHolder.get();
+
+            if (ses0 != null)
+                ses0.attach(attachment);
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public <T> T attachment() {
+            SessionData ses0 = sesHolder.get();
+
+            return ses0 != null ? (T)ses0.attachment() : null;
+        }
+
+        /** {@inheritDoc} */
         @Override public <K1, V1> Map<K1, V1> properties() {
             SessionData ses0 = sesHolder.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListenerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListenerSelfTest.java
new file mode 100644
index 0000000..64af249
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListenerSelfTest.java
@@ -0,0 +1,175 @@
+/*
+ * 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.cache.store.jdbc;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.h2.jdbcx.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.sql.*;
+import java.util.*;
+
+/**
+ * Tests for {@link CacheJdbcStoreSessionListener}.
+ */
+public class CacheJdbcStoreSessionListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
+        return new Factory<CacheStore<Integer, Integer>>() {
+            @Override public CacheStore<Integer, Integer> create() {
+                return new Store();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
+        return new Factory<CacheStoreSessionListener>() {
+            @Override public CacheStoreSessionListener create() {
+                CacheJdbcStoreSessionListener lsnr = new CacheJdbcStoreSessionListener();
+
+                lsnr.setDataSource(JdbcConnectionPool.create(URL, "", ""));
+
+                return lsnr;
+            }
+        };
+    }
+
+    /**
+     */
+    private static class Store extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private static String SES_CONN_KEY = "ses_conn";
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
+            loadCacheCnt.incrementAndGet();
+
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            loadCnt.incrementAndGet();
+
+            checkConnection();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            writeCnt.incrementAndGet();
+
+            checkConnection();
+
+            if (write.get()) {
+                Connection conn = ses.attachment();
+
+                try {
+                    String table;
+
+                    switch (ses.cacheName()) {
+                        case "cache1":
+                            table = "Table1";
+
+                            break;
+
+                        case "cache2":
+                            if (fail.get())
+                                throw new CacheWriterException("Expected failure.");
+
+                            table = "Table2";
+
+                            break;
+
+                        default:
+                            throw new CacheWriterException("Wring cache: " + ses.cacheName());
+                    }
+
+                    PreparedStatement stmt = conn.prepareStatement(
+                        "INSERT INTO " + table + " (key, value) VALUES (?, ?)");
+
+                    stmt.setInt(1, entry.getKey());
+                    stmt.setInt(2, entry.getValue());
+
+                    stmt.executeUpdate();
+                }
+                catch (SQLException e) {
+                    throw new CacheWriterException(e);
+                }
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            deleteCnt.incrementAndGet();
+
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            assertNull(ses.attachment());
+        }
+
+        /**
+         */
+        private void checkConnection() {
+            Connection conn = ses.attachment();
+
+            assertNotNull(conn);
+
+            try {
+                assertFalse(conn.isClosed());
+                assertFalse(conn.getAutoCommit());
+            }
+            catch (SQLException e) {
+                throw new RuntimeException(e);
+            }
+
+            verifySameInstance(conn);
+        }
+
+        /**
+         * @param conn Connection.
+         */
+        private void verifySameInstance(Connection conn) {
+            Map<String, Connection> props = ses.properties();
+
+            Connection sesConn = props.get(SES_CONN_KEY);
+
+            if (sesConn == null)
+                props.put(SES_CONN_KEY, conn);
+            else {
+                assertSame(conn, sesConn);
+
+                reuseCnt.incrementAndGet();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
deleted file mode 100644
index e4dac88..0000000
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListenerSelfTest.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.store.jdbc;
-
-import org.apache.ignite.cache.store.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.h2.jdbcx.*;
-
-import javax.cache.*;
-import javax.cache.configuration.*;
-import javax.cache.integration.*;
-import java.sql.*;
-import java.util.*;
-
-/**
- * Tests for {@link CacheStoreSessionJdbcListener}.
- */
-public class CacheStoreSessionJdbcListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
-        return new Factory<CacheStore<Integer, Integer>>() {
-            @Override public CacheStore<Integer, Integer> create() {
-                return new Store();
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
-        return new Factory<CacheStoreSessionListener>() {
-            @Override public CacheStoreSessionListener create() {
-                CacheStoreSessionJdbcListener lsnr = new CacheStoreSessionJdbcListener();
-
-                lsnr.setDataSource(JdbcConnectionPool.create(URL, "", ""));
-
-                return lsnr;
-            }
-        };
-    }
-
-    /**
-     */
-    private static class Store extends CacheStoreAdapter<Integer, Integer> {
-        /** */
-        private static String SES_CONN_KEY = "ses_conn";
-
-        /** */
-        @CacheStoreSessionResource
-        private CacheStoreSession ses;
-
-        /** {@inheritDoc} */
-        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
-            loadCacheCnt.incrementAndGet();
-
-            checkConnection();
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer load(Integer key) throws CacheLoaderException {
-            loadCnt.incrementAndGet();
-
-            checkConnection();
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
-            throws CacheWriterException {
-            writeCnt.incrementAndGet();
-
-            checkConnection();
-
-            if (write.get()) {
-                Connection conn = connection();
-
-                try {
-                    String table;
-
-                    switch (ses.cacheName()) {
-                        case "cache1":
-                            table = "Table1";
-
-                            break;
-
-                        case "cache2":
-                            if (fail.get())
-                                throw new CacheWriterException("Expected failure.");
-
-                            table = "Table2";
-
-                            break;
-
-                        default:
-                            throw new CacheWriterException("Wring cache: " + ses.cacheName());
-                    }
-
-                    PreparedStatement stmt = conn.prepareStatement(
-                        "INSERT INTO " + table + " (key, value) VALUES (?, ?)");
-
-                    stmt.setInt(1, entry.getKey());
-                    stmt.setInt(2, entry.getValue());
-
-                    stmt.executeUpdate();
-                }
-                catch (SQLException e) {
-                    throw new CacheWriterException(e);
-                }
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void delete(Object key) throws CacheWriterException {
-            deleteCnt.incrementAndGet();
-
-            checkConnection();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void sessionEnd(boolean commit) {
-            assertNull(connection());
-        }
-
-        /**
-         */
-        private void checkConnection() {
-            Connection conn = connection();
-
-            assertNotNull(conn);
-
-            try {
-                assertFalse(conn.isClosed());
-                assertFalse(conn.getAutoCommit());
-            }
-            catch (SQLException e) {
-                throw new RuntimeException(e);
-            }
-
-            verifySameInstance(conn);
-        }
-
-        /**
-         * @param conn Connection.
-         */
-        private void verifySameInstance(Connection conn) {
-            Map<String, Connection> props = ses.properties();
-
-            Connection sesConn = props.get(SES_CONN_KEY);
-
-            if (sesConn == null)
-                props.put(SES_CONN_KEY, conn);
-            else {
-                assertSame(conn, sesConn);
-
-                reuseCnt.incrementAndGet();
-            }
-        }
-
-        /**
-         * @return Connection.
-         */
-        private Connection connection() {
-            return ses.<String, Connection>properties().get(CacheStoreSessionJdbcListener.JDBC_CONN_KEY);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
index 0709880..bc7bf7d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
@@ -34,6 +34,9 @@ public class TestCacheSession implements CacheStoreSession {
     /** */
     private Map<Object, Object> props;
 
+    /** */
+    private Object attachment;
+
     /**
      *
      * @param tx Transaction.
@@ -55,6 +58,17 @@ public class TestCacheSession implements CacheStoreSession {
     }
 
     /** {@inheritDoc} */
+    @Override public void attach(@Nullable Object attachment) {
+        this.attachment = attachment;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T attachment() {
+        return (T)attachment;
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <K, V> Map<K, V> properties() {
         if (props == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
index 2bbcf1b..2047600 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
@@ -54,6 +54,22 @@ public class TestThreadLocalCacheSession implements CacheStoreSession {
     }
 
     /** {@inheritDoc} */
+    @Override public void attach(@Nullable Object attachment) {
+        TestCacheSession ses = sesHolder.get();
+
+        if (ses != null)
+            ses.attach(attachment);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T attachment() {
+        TestCacheSession ses = sesHolder.get();
+
+        return ses!= null ? (T)ses.attachment() : null;
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <K, V> Map<K, V> properties() {
         TestCacheSession ses = sesHolder.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index afb67f5..60c7a0a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -131,7 +131,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(CacheOffheapMapEntrySelfTest.class);
 
-        suite.addTestSuite(CacheStoreSessionJdbcListenerSelfTest.class);
+        suite.addTestSuite(CacheJdbcStoreSessionListenerSelfTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
new file mode 100644
index 0000000..61f7265
--- /dev/null
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
@@ -0,0 +1,218 @@
+/*
+ * 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.cache.store.hibernate;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
+import org.hibernate.*;
+import org.hibernate.cfg.*;
+
+import javax.cache.integration.*;
+import java.io.*;
+import java.net.*;
+
+/**
+ * Hibernate-based cache store session listener.
+ * <p>
+ * This listener creates a new Hibernate session for each store
+ * session. If there is an ongoing cache transaction, a corresponding
+ * Hibernate transaction is created as well.
+ * <p>
+ * The Hibernate session is saved as a store session
+ * {@link CacheStoreSession#attachment() attachment}.
+ * The listener guarantees that the session will be
+ * available for any store operation. If there is an
+ * ongoing cache transaction, all operations within this
+ * transaction will share a DB transaction.
+ * <p>
+ * As an example, here is how the {@link CacheStore#write(javax.cache.Cache.Entry)}
+ * method can be implemented if {@link CacheHibernateStoreSessionListener}
+ * is configured:
+ * <pre name="code" class="java">
+ * private static class Store extends CacheStoreAdapter&lt;Integer, Integer&gt; {
+ *     &#64;CacheStoreSessionResource
+ *     private CacheStoreSession ses;
+ *
+ *     &#64;Override public void write(Cache.Entry&lt;? extends Integer, ? extends Integer&gt; entry) throws CacheWriterException {
+ *         // Get Hibernate session from the current store session.
+ *         Session hibSes = ses.attachment();
+ *
+ *         // Persist the value.
+ *         hibSes.persist(entry.getValue());
+ *     }
+ * }
+ * </pre>
+ * Hibernate session will be automatically created by the listener
+ * at the start of the session and closed when it ends.
+ * <p>
+ * {@link CacheHibernateStoreSessionListener} requires that either
+ * {@link #setSessionFactory(SessionFactory)} session factory}
+ * or {@link #setHibernateConfigurationPath(String) Hibernate configuration file}
+ * is provided. If non of them is set, exception is thrown. Is both are provided,
+ * session factory will be used.
+ */
+public class CacheHibernateStoreSessionListener implements CacheStoreSessionListener, LifecycleAware {
+    /** Hibernate session factory. */
+    private SessionFactory sesFactory;
+
+    /** Hibernate configuration file path. */
+    private String hibernateCfgPath;
+
+    /** Logger. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Whether to close session on stop. */
+    private boolean closeSesOnStop;
+
+    /**
+     * Sets Hibernate session factory.
+     * <p>
+     * Either session factory or configuration file is required.
+     * If none is provided, exception will be thrown on startup.
+     *
+     * @param sesFactory Session factory.
+     */
+    public void setSessionFactory(SessionFactory sesFactory) {
+        this.sesFactory = sesFactory;
+    }
+
+    /**
+     * Gets Hibernate session factory.
+     *
+     * @return Session factory.
+     */
+    public SessionFactory getSessionFactory() {
+        return sesFactory;
+    }
+
+    /**
+     * Sets hibernate configuration path.
+     * <p>
+     * Either session factory or configuration file is required.
+     * If none is provided, exception will be thrown on startup.
+     *
+     * @param hibernateCfgPath Hibernate configuration path.
+     */
+    public void setHibernateConfigurationPath(String hibernateCfgPath) {
+        this.hibernateCfgPath = hibernateCfgPath;
+    }
+
+    /**
+     * Gets hibernate configuration path.
+     *
+     * @return Hibernate configuration path.
+     */
+    public String getHibernateConfigurationPath() {
+        return hibernateCfgPath;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public void start() throws IgniteException {
+        if (sesFactory == null && F.isEmpty(hibernateCfgPath))
+            throw new IgniteException("Either session factory or Hibernate configuration file is required by " +
+                getClass().getSimpleName() + '.');
+
+        if (!F.isEmpty(hibernateCfgPath)) {
+            if (sesFactory == null) {
+                try {
+                    URL url = new URL(hibernateCfgPath);
+
+                    sesFactory = new Configuration().configure(url).buildSessionFactory();
+                }
+                catch (MalformedURLException ignored) {
+                    // No-op.
+                }
+
+                if (sesFactory == null) {
+                    File cfgFile = new File(hibernateCfgPath);
+
+                    if (cfgFile.exists())
+                        sesFactory = new Configuration().configure(cfgFile).buildSessionFactory();
+                }
+
+                if (sesFactory == null)
+                    sesFactory = new Configuration().configure(hibernateCfgPath).buildSessionFactory();
+
+                if (sesFactory == null)
+                    throw new IgniteException("Failed to resolve Hibernate configuration file: " + hibernateCfgPath);
+
+                closeSesOnStop = true;
+            }
+            else
+                U.warn(log, "Hibernate configuration file configured in " + getClass().getSimpleName() +
+                    " will be ignored (session factory is already set).");
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        if (closeSesOnStop && sesFactory != null && !sesFactory.isClosed())
+            sesFactory.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionStart(CacheStoreSession ses) {
+        if (ses.attachment() == null) {
+            try {
+                Session hibSes = sesFactory.openSession();
+
+                ses.attach(hibSes);
+
+                if (ses.isWithinTransaction())
+                    hibSes.beginTransaction();
+            }
+            catch (HibernateException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+        Session hibSes = ses.attachment();
+
+        if (hibSes != null) {
+            ses.attach(null);
+
+            try {
+                Transaction tx = hibSes.getTransaction();
+
+                if (commit) {
+                    hibSes.flush();
+
+                    if (tx.isActive())
+                        tx.commit();
+                }
+                else if (tx.isActive())
+                    tx.rollback();
+            }
+            catch (HibernateException e) {
+                throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e);
+            }
+            finally {
+                hibSes.close();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
deleted file mode 100644
index ea1214a..0000000
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.store.hibernate;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.store.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lifecycle.*;
-import org.apache.ignite.resources.*;
-import org.hibernate.*;
-import org.hibernate.cfg.*;
-
-import javax.cache.integration.*;
-import java.io.*;
-import java.net.*;
-import java.util.*;
-
-/**
- * Hibernate-based cache store session listener.
- * <p>
- * This listener creates a new Hibernate session for each store
- * session. If there is an ongoing cache transaction, a corresponding
- * Hibernate transaction is created as well.
- * <p>
- * The Hibernate session is stored in store session
- * {@link CacheStoreSession#properties() properties} and can
- * be accessed at any moment by {@link #HIBERNATE_SES_KEY} key.
- * The listener guarantees that the session will be
- * available for any store operation. If there is an
- * ongoing cache transaction, all operations within this
- * transaction will share a DB transaction.
- * <p>
- * As an example, here is how the {@link CacheStore#write(javax.cache.Cache.Entry)}
- * method can be implemented if {@link CacheStoreSessionHibernateListener}
- * is configured:
- * <pre name="code" class="java">
- * private static class Store extends CacheStoreAdapter&lt;Integer, Integer&gt; {
- *     &#64;CacheStoreSessionResource
- *     private CacheStoreSession ses;
- *
- *     &#64;Override public void write(Cache.Entry&lt;? extends Integer, ? extends Integer&gt; entry) throws CacheWriterException {
- *         // Get Hibernate session from the current store session.
- *         Session hibSes = ses.<String, Session>properties().get(CacheStoreSessionHibernateListener.HIBERNATE_SES_KEY);
- *
- *         // Persist the value.
- *         hibSes.persist(entry.getValue());
- *     }
- * }
- * </pre>
- * Hibernate session will be automatically created by the listener
- * at the start of the session and closed when it ends.
- * <p>
- * {@link CacheStoreSessionHibernateListener} requires that either
- * {@link #setSessionFactory(SessionFactory)} session factory}
- * or {@link #setHibernateConfigurationPath(String) Hibernate configuration file}
- * is provided. If non of them is set, exception is thrown. Is both are provided,
- * session factory will be used.
- */
-public class CacheStoreSessionHibernateListener implements CacheStoreSessionListener, LifecycleAware {
-    /** Session key for JDBC connection. */
-    public static final String HIBERNATE_SES_KEY = "__hibernate_ses_";
-
-    /** Hibernate session factory. */
-    private SessionFactory sesFactory;
-
-    /** Hibernate configuration file path. */
-    private String hibernateCfgPath;
-
-    /** Logger. */
-    @LoggerResource
-    private IgniteLogger log;
-
-    /** Whether to close session on stop. */
-    private boolean closeSesOnStop;
-
-    /**
-     * Sets Hibernate session factory.
-     * <p>
-     * Either session factory or configuration file is required.
-     * If none is provided, exception will be thrown on startup.
-     *
-     * @param sesFactory Session factory.
-     */
-    public void setSessionFactory(SessionFactory sesFactory) {
-        this.sesFactory = sesFactory;
-    }
-
-    /**
-     * Gets Hibernate session factory.
-     *
-     * @return Session factory.
-     */
-    public SessionFactory getSessionFactory() {
-        return sesFactory;
-    }
-
-    /**
-     * Sets hibernate configuration path.
-     * <p>
-     * Either session factory or configuration file is required.
-     * If none is provided, exception will be thrown on startup.
-     *
-     * @param hibernateCfgPath Hibernate configuration path.
-     */
-    public void setHibernateConfigurationPath(String hibernateCfgPath) {
-        this.hibernateCfgPath = hibernateCfgPath;
-    }
-
-    /**
-     * Gets hibernate configuration path.
-     *
-     * @return Hibernate configuration path.
-     */
-    public String getHibernateConfigurationPath() {
-        return hibernateCfgPath;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override public void start() throws IgniteException {
-        if (sesFactory == null && F.isEmpty(hibernateCfgPath))
-            throw new IgniteException("Either session factory or Hibernate configuration file is required by " +
-                getClass().getSimpleName() + '.');
-
-        if (!F.isEmpty(hibernateCfgPath)) {
-            if (sesFactory == null) {
-                try {
-                    URL url = new URL(hibernateCfgPath);
-
-                    sesFactory = new Configuration().configure(url).buildSessionFactory();
-                }
-                catch (MalformedURLException ignored) {
-                    // No-op.
-                }
-
-                if (sesFactory == null) {
-                    File cfgFile = new File(hibernateCfgPath);
-
-                    if (cfgFile.exists())
-                        sesFactory = new Configuration().configure(cfgFile).buildSessionFactory();
-                }
-
-                if (sesFactory == null)
-                    sesFactory = new Configuration().configure(hibernateCfgPath).buildSessionFactory();
-
-                if (sesFactory == null)
-                    throw new IgniteException("Failed to resolve Hibernate configuration file: " + hibernateCfgPath);
-
-                closeSesOnStop = true;
-            }
-            else
-                U.warn(log, "Hibernate configuration file configured in " + getClass().getSimpleName() +
-                    " will be ignored (session factory is already set).");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stop() throws IgniteException {
-        if (closeSesOnStop && sesFactory != null && !sesFactory.isClosed())
-            sesFactory.close();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionStart(CacheStoreSession ses) {
-        Map<String, Session> props = ses.properties();
-
-        if (!props.containsKey(HIBERNATE_SES_KEY)) {
-            try {
-                Session hibSes = sesFactory.openSession();
-
-                props.put(HIBERNATE_SES_KEY, hibSes);
-
-                if (ses.isWithinTransaction())
-                    hibSes.beginTransaction();
-            }
-            catch (HibernateException e) {
-                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
-        Session hibSes = ses.<String, Session>properties().remove(HIBERNATE_SES_KEY);
-
-        if (hibSes != null) {
-            try {
-                Transaction tx = hibSes.getTransaction();
-
-                if (commit) {
-                    hibSes.flush();
-
-                    if (tx.isActive())
-                        tx.commit();
-                }
-                else if (tx.isActive())
-                    tx.rollback();
-            }
-            catch (HibernateException e) {
-                throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e);
-            }
-            finally {
-                hibSes.close();
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListenerSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListenerSelfTest.java
new file mode 100644
index 0000000..c30e216
--- /dev/null
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListenerSelfTest.java
@@ -0,0 +1,228 @@
+/*
+ * 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.cache.store.hibernate;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.hibernate.*;
+import org.hibernate.cfg.Configuration;
+
+import javax.cache.Cache;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import javax.persistence.*;
+import java.io.*;
+import java.util.*;
+
+/**
+ * Tests for {@link CacheJdbcStoreSessionListener}.
+ */
+public class CacheHibernateStoreSessionListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
+        return new Factory<CacheStore<Integer, Integer>>() {
+            @Override public CacheStore<Integer, Integer> create() {
+                return new Store();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
+        return new Factory<CacheStoreSessionListener>() {
+            @Override public CacheStoreSessionListener create() {
+                CacheHibernateStoreSessionListener lsnr = new CacheHibernateStoreSessionListener();
+
+                SessionFactory sesFactory = new Configuration().
+                    setProperty("hibernate.connection.url", URL).
+                    addAnnotatedClass(Table1.class).
+                    addAnnotatedClass(Table2.class).
+                    buildSessionFactory();
+
+                lsnr.setSessionFactory(sesFactory);
+
+                return lsnr;
+            }
+        };
+    }
+
+    /**
+     */
+    private static class Store extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private static String SES_CONN_KEY = "ses_conn";
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
+            loadCacheCnt.incrementAndGet();
+
+            checkSession();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            loadCnt.incrementAndGet();
+
+            checkSession();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            writeCnt.incrementAndGet();
+
+            checkSession();
+
+            if (write.get()) {
+                Session hibSes = ses.attachment();
+
+                switch (ses.cacheName()) {
+                    case "cache1":
+                        hibSes.save(new Table1(entry.getKey(), entry.getValue()));
+
+                        break;
+
+                    case "cache2":
+                        if (fail.get())
+                            throw new CacheWriterException("Expected failure.");
+
+                        hibSes.save(new Table2(entry.getKey(), entry.getValue()));
+
+                        break;
+
+                    default:
+                        throw new CacheWriterException("Wring cache: " + ses.cacheName());
+                }
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            deleteCnt.incrementAndGet();
+
+            checkSession();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            assertNull(ses.attachment());
+        }
+
+        /**
+         */
+        private void checkSession() {
+            Session hibSes = ses.attachment();
+
+            assertNotNull(hibSes);
+
+            assertTrue(hibSes.isOpen());
+
+            Transaction tx = hibSes.getTransaction();
+
+            assertNotNull(tx);
+
+            if (ses.isWithinTransaction())
+                assertTrue(tx.isActive());
+            else
+                assertFalse(tx.isActive());
+
+            verifySameInstance(hibSes);
+        }
+
+        /**
+         * @param hibSes Session.
+         */
+        private void verifySameInstance(Session hibSes) {
+            Map<String, Session> props = ses.properties();
+
+            Session sesConn = props.get(SES_CONN_KEY);
+
+            if (sesConn == null)
+                props.put(SES_CONN_KEY, hibSes);
+            else {
+                assertSame(hibSes, sesConn);
+
+                reuseCnt.incrementAndGet();
+            }
+        }
+    }
+
+    /**
+     */
+    @Entity
+    @Table(name = "Table1")
+    private static class Table1 implements Serializable {
+        /** */
+        @Id @GeneratedValue
+        @Column(name = "id")
+        private Integer id;
+
+        /** */
+        @Column(name = "key")
+        private int key;
+
+        /** */
+        @Column(name = "value")
+        private int value;
+
+        /**
+         * @param key Key.
+         * @param value Value.
+         */
+        private Table1(int key, int value) {
+            this.key = key;
+            this.value = value;
+        }
+    }
+
+    /**
+     */
+    @Entity
+    @Table(name = "Table2")
+    private static class Table2 implements Serializable {
+        /** */
+        @Id @GeneratedValue
+        @Column(name = "id")
+        private Integer id;
+
+        /** */
+        @Column(name = "key")
+        private int key;
+
+        /** */
+        @Column(name = "value")
+        private int value;
+
+        /**
+         * @param key Key.
+         * @param value Value.
+         */
+        private Table2(int key, int value) {
+            this.key = key;
+            this.value = value;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
deleted file mode 100644
index a9d465e..0000000
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.store.hibernate;
-
-import org.apache.ignite.cache.store.*;
-import org.apache.ignite.cache.store.jdbc.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-import org.hibernate.*;
-import org.hibernate.cfg.Configuration;
-
-import javax.cache.Cache;
-import javax.cache.configuration.*;
-import javax.cache.integration.*;
-import javax.persistence.*;
-import java.io.*;
-import java.util.*;
-
-/**
- * Tests for {@link CacheStoreSessionJdbcListener}.
- */
-public class CacheStoreSessionHibernateListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
-        return new Factory<CacheStore<Integer, Integer>>() {
-            @Override public CacheStore<Integer, Integer> create() {
-                return new Store();
-            }
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
-        return new Factory<CacheStoreSessionListener>() {
-            @Override public CacheStoreSessionListener create() {
-                CacheStoreSessionHibernateListener lsnr = new CacheStoreSessionHibernateListener();
-
-                SessionFactory sesFactory = new Configuration().
-                    setProperty("hibernate.connection.url", URL).
-                    addAnnotatedClass(Table1.class).
-                    addAnnotatedClass(Table2.class).
-                    buildSessionFactory();
-
-                lsnr.setSessionFactory(sesFactory);
-
-                return lsnr;
-            }
-        };
-    }
-
-    /**
-     */
-    private static class Store extends CacheStoreAdapter<Integer, Integer> {
-        /** */
-        private static String SES_CONN_KEY = "ses_conn";
-
-        /** */
-        @CacheStoreSessionResource
-        private CacheStoreSession ses;
-
-        /** {@inheritDoc} */
-        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
-            loadCacheCnt.incrementAndGet();
-
-            checkSession();
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer load(Integer key) throws CacheLoaderException {
-            loadCnt.incrementAndGet();
-
-            checkSession();
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
-            throws CacheWriterException {
-            writeCnt.incrementAndGet();
-
-            checkSession();
-
-            if (write.get()) {
-                Session hibSes = session();
-
-                switch (ses.cacheName()) {
-                    case "cache1":
-                        hibSes.save(new Table1(entry.getKey(), entry.getValue()));
-
-                        break;
-
-                    case "cache2":
-                        if (fail.get())
-                            throw new CacheWriterException("Expected failure.");
-
-                        hibSes.save(new Table2(entry.getKey(), entry.getValue()));
-
-                        break;
-
-                    default:
-                        throw new CacheWriterException("Wring cache: " + ses.cacheName());
-                }
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void delete(Object key) throws CacheWriterException {
-            deleteCnt.incrementAndGet();
-
-            checkSession();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void sessionEnd(boolean commit) {
-            assertNull(session());
-        }
-
-        /**
-         */
-        private void checkSession() {
-            Session hibSes = session();
-
-            assertNotNull(hibSes);
-
-            assertTrue(hibSes.isOpen());
-
-            Transaction tx = hibSes.getTransaction();
-
-            assertNotNull(tx);
-
-            if (ses.isWithinTransaction())
-                assertTrue(tx.isActive());
-            else
-                assertFalse(tx.isActive());
-
-            verifySameInstance(hibSes);
-        }
-
-        /**
-         * @param hibSes Session.
-         */
-        private void verifySameInstance(Session hibSes) {
-            Map<String, Session> props = ses.properties();
-
-            Session sesConn = props.get(SES_CONN_KEY);
-
-            if (sesConn == null)
-                props.put(SES_CONN_KEY, hibSes);
-            else {
-                assertSame(hibSes, sesConn);
-
-                reuseCnt.incrementAndGet();
-            }
-        }
-
-        /**
-         * @return Connection.
-         */
-        private Session session() {
-            return ses.<String, Session>properties().get(CacheStoreSessionHibernateListener.HIBERNATE_SES_KEY);
-        }
-    }
-
-    /**
-     */
-    @Entity
-    @Table(name = "Table1")
-    private static class Table1 implements Serializable {
-        /** */
-        @Id @GeneratedValue
-        @Column(name = "id")
-        private Integer id;
-
-        /** */
-        @Column(name = "key")
-        private int key;
-
-        /** */
-        @Column(name = "value")
-        private int value;
-
-        /**
-         * @param key Key.
-         * @param value Value.
-         */
-        private Table1(int key, int value) {
-            this.key = key;
-            this.value = value;
-        }
-    }
-
-    /**
-     */
-    @Entity
-    @Table(name = "Table2")
-    private static class Table2 implements Serializable {
-        /** */
-        @Id @GeneratedValue
-        @Column(name = "id")
-        private Integer id;
-
-        /** */
-        @Column(name = "key")
-        private int key;
-
-        /** */
-        @Column(name = "value")
-        private int value;
-
-        /**
-         * @param key Key.
-         * @param value Value.
-         */
-        private Table2(int key, int value) {
-            this.key = key;
-            this.value = value;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/990bf9e3/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
index ed782e2..655e801 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
@@ -41,7 +41,7 @@ public class IgniteHibernateTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheHibernateBlobStoreNodeRestartTest.class);
 
-        suite.addTestSuite(CacheStoreSessionHibernateListenerSelfTest.class);
+        suite.addTestSuite(CacheHibernateStoreSessionListenerSelfTest.class);
 
         return suite;
     }


[28/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: f0b980cb500a77617cfe42fe166ada086d7464af
Parents: e6cc139
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Sun May 24 23:53:44 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Sun May 24 23:53:44 2015 -0700

----------------------------------------------------------------------
 .../ignite/cache/store/CacheStoreSession.java    | 19 ++++++++++++++++++-
 .../jdbc/CacheJdbcStoreSessionListener.java      |  4 +---
 .../store/GridCacheStoreManagerAdapter.java      | 11 +++++++----
 .../junits/cache/TestCacheSession.java           |  6 +++++-
 .../cache/TestThreadLocalCacheSession.java       |  5 ++---
 .../CacheHibernateStoreSessionListener.java      |  4 +---
 .../spring/CacheSpringStoreSessionListener.java  |  4 +---
 7 files changed, 35 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0b980cb/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
index 980c6df..329e994 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
@@ -52,8 +52,25 @@ public interface CacheStoreSession {
      */
     public boolean isWithinTransaction();
 
-    public void attach(@Nullable Object attachment);
+    /**
+     * Attaches the given object to this session.
+     * <p>
+     * An attached object may later be retrieved via the {@link #attachment()}
+     * method. Invoking this method causes any previous attachment to be
+     * discarded. To attach additional objects use {@link #properties()} map.
+     * <p>
+     * The current attachment may be discarded by attaching {@code null}.
+     *
+     * @param attachment The object to be attached (or {@code null} to discard current attachment).
+     * @return Previously attached object, if any.
+     */
+    @Nullable public <T> T attach(@Nullable Object attachment);
 
+    /**
+     * Retrieves the current attachment or {@code null} if there is no attachment.
+     *
+     * @return Currently attached object, if any.
+     */
     @Nullable public <T> T attachment();
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0b980cb/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
index c59e86f..a20e535 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
@@ -121,11 +121,9 @@ public class CacheJdbcStoreSessionListener implements CacheStoreSessionListener,
 
     /** {@inheritDoc} */
     @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
-        Connection conn = ses.attachment();
+        Connection conn = ses.attach(null);
 
         if (conn != null) {
-            ses.attach(null);
-
             try {
                 if (commit)
                     conn.commit();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0b980cb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index bc5a0a2..b4a146a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -889,8 +889,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         /**
          * @param attachment Attachment.
          */
-        private void attach(Object attachment) {
+        private Object attach(Object attachment) {
+            Object prev = this.attachment;
+
             this.attachment = attachment;
+
+            return prev;
         }
 
         /**
@@ -962,11 +966,10 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         }
 
         /** {@inheritDoc} */
-        @Override public void attach(@Nullable Object attachment) {
+        @Override public Object attach(@Nullable Object attachment) {
             SessionData ses0 = sesHolder.get();
 
-            if (ses0 != null)
-                ses0.attach(attachment);
+            return ses0 != null ? ses0.attach(attachment) : null;
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0b980cb/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
index bc7bf7d..31cbefa 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
@@ -58,8 +58,12 @@ public class TestCacheSession implements CacheStoreSession {
     }
 
     /** {@inheritDoc} */
-    @Override public void attach(@Nullable Object attachment) {
+    @Override public Object attach(@Nullable Object attachment) {
+        Object prev = this.attachment;
+
         this.attachment = attachment;
+
+        return prev;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0b980cb/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
index 2047600..dc876d3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
@@ -54,11 +54,10 @@ public class TestThreadLocalCacheSession implements CacheStoreSession {
     }
 
     /** {@inheritDoc} */
-    @Override public void attach(@Nullable Object attachment) {
+    @Override public Object attach(@Nullable Object attachment) {
         TestCacheSession ses = sesHolder.get();
 
-        if (ses != null)
-            ses.attach(attachment);
+        return ses != null ? ses.attach(attachment) : null;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0b980cb/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
index 61f7265..cfad322 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
@@ -190,11 +190,9 @@ public class CacheHibernateStoreSessionListener implements CacheStoreSessionList
 
     /** {@inheritDoc} */
     @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
-        Session hibSes = ses.attachment();
+        Session hibSes = ses.attach(null);
 
         if (hibSes != null) {
-            ses.attach(null);
-
             try {
                 Transaction tx = hibSes.getTransaction();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0b980cb/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
index 90431d7..53fc599 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
@@ -169,11 +169,9 @@ public class CacheSpringStoreSessionListener implements CacheStoreSessionListene
     /** {@inheritDoc} */
     @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
         if (ses.isWithinTransaction()) {
-            TransactionStatus tx = ses.attachment();
+            TransactionStatus tx = ses.attach(null);
 
             if (tx != null) {
-                ses.attach(null);
-
                 try {
                     if (commit)
                         txMgr.commit(tx);


[17/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: b37d00465f1a162436082660ec69a1f765492373
Parents: 80ebfe0
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu May 21 18:52:15 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu May 21 18:52:15 2015 -0700

----------------------------------------------------------------------
 .../jdbc/CacheStoreSessionJdbcListener.java     | 16 +++-
 .../processors/cache/GridCacheProcessor.java    | 10 ++-
 .../cache/GridCacheSharedContext.java           | 15 +++-
 .../processors/cache/GridCacheUtils.java        | 33 +++++++
 .../store/GridCacheStoreManagerAdapter.java     | 44 +++-------
 .../cache/transactions/IgniteTxAdapter.java     | 13 ++-
 .../transactions/IgniteTxLocalAdapter.java      | 27 ++----
 .../loadtests/hashmap/GridCacheTestContext.java |  3 +-
 .../CacheStoreSessionHibernateListener.java     | 82 +++++++++++++++++-
 modules/spring/pom.xml                          | 12 +--
 .../spring/CacheStoreSessionSpringListener.java | 90 ++++++++++++++++++--
 ...CacheStoreSessionSpringListenerSelfTest.java |  2 +-
 12 files changed, 271 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
index c683abe..e4cd617 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
@@ -17,9 +17,10 @@
 
 package org.apache.ignite.cache.store.jdbc;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.resources.*;
+import org.apache.ignite.lifecycle.*;
 
 import javax.cache.integration.*;
 import javax.sql.*;
@@ -29,7 +30,7 @@ import java.util.*;
 /**
  * Cache store session listener based on JDBC connection.
  */
-public class CacheStoreSessionJdbcListener implements CacheStoreSessionListener {
+public class CacheStoreSessionJdbcListener implements CacheStoreSessionListener, LifecycleAware {
     /** Session key for JDBC connection. */
     public static final String JDBC_CONN_KEY = "__jdbc_conn_";
 
@@ -57,6 +58,17 @@ public class CacheStoreSessionJdbcListener implements CacheStoreSessionListener
     }
 
     /** {@inheritDoc} */
+    @Override public void start() throws IgniteException {
+        if (dataSrc == null)
+            throw new IgniteException("Data source is required by " + getClass().getSimpleName() + '.');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void onSessionStart(CacheStoreSession ses) {
         Map<String, Connection> props = ses.properties();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 0e1a9c2..5b57817 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -567,7 +567,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheConfiguration[] cfgs = ctx.config().getCacheConfiguration();
 
-        sharedCtx = createSharedContext(ctx);
+        sharedCtx = createSharedContext(ctx, CU.createStoreSessionListeners(ctx,
+            ctx.config().getCacheStoreSessionListenerFactories()));
 
         ctx.performance().add("Disable serializable transactions (set 'txSerializableEnabled' to false)",
             !ctx.config().getTransactionConfiguration().isTxSerializableEnabled());
@@ -1562,10 +1563,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * Creates shared context.
      *
      * @param kernalCtx Kernal context.
+     * @param storeSesLsnrs Store session listeners.
      * @return Shared context.
      */
     @SuppressWarnings("unchecked")
-    private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx) {
+    private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx,
+        Collection<CacheStoreSessionListener> storeSesLsnrs) {
         IgniteTxManager tm = new IgniteTxManager();
         GridCacheMvccManager mvccMgr = new GridCacheMvccManager();
         GridCacheVersionManager verMgr = new GridCacheVersionManager();
@@ -1580,7 +1583,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             mvccMgr,
             depMgr,
             exchMgr,
-            ioMgr
+            ioMgr,
+            storeSesLsnrs
         );
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index b16885e..45634b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -77,6 +78,9 @@ public class GridCacheSharedContext<K, V> {
     /** Preloaders start future. */
     private IgniteInternalFuture<Object> preloadersStartFut;
 
+    /** Store session listeners. */
+    private Collection<CacheStoreSessionListener> storeSesLsnrs;
+
     /**
      * @param txMgr Transaction manager.
      * @param verMgr Version manager.
@@ -89,7 +93,8 @@ public class GridCacheSharedContext<K, V> {
         GridCacheMvccManager mvccMgr,
         GridCacheDeploymentManager<K, V> depMgr,
         GridCachePartitionExchangeManager<K, V> exchMgr,
-        GridCacheIoManager ioMgr
+        GridCacheIoManager ioMgr,
+        Collection<CacheStoreSessionListener> storeSesLsnrs
     ) {
         this.kernalCtx = kernalCtx;
         this.mvccMgr = add(mvccMgr);
@@ -98,6 +103,7 @@ public class GridCacheSharedContext<K, V> {
         this.depMgr = add(depMgr);
         this.exchMgr = add(exchMgr);
         this.ioMgr = add(ioMgr);
+        this.storeSesLsnrs = storeSesLsnrs;
 
         txMetrics = new TransactionMetricsAdapter();
 
@@ -524,6 +530,13 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * @return Store session listeners.
+     */
+    @Nullable public Collection<CacheStoreSessionListener> storeSessionListeners() {
+        return storeSesLsnrs;
+    }
+
+    /**
      * @param mgr Manager to add.
      * @return Added manager.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 549f42f..6968fcb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -34,12 +35,14 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.plugin.*;
 import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
 import javax.cache.*;
+import javax.cache.configuration.*;
 import javax.cache.expiry.*;
 import javax.cache.integration.*;
 import java.io.*;
@@ -1790,4 +1793,34 @@ public class GridCacheUtils {
 
         return res;
     }
+
+    /**
+     * Creates store session listeners.
+     *
+     * @param ctx Kernal context.
+     * @param factories Factories.
+     * @return Listeners.
+     */
+    public static Collection<CacheStoreSessionListener> createStoreSessionListeners(GridKernalContext ctx,
+        Factory<CacheStoreSessionListener>[] factories) throws IgniteCheckedException {
+        if (factories == null)
+            return null;
+
+        Collection<CacheStoreSessionListener> lsnrs = new ArrayList<>(factories.length);
+
+        for (Factory<CacheStoreSessionListener> factory : factories) {
+            CacheStoreSessionListener lsnr = factory.create();
+
+            if (lsnr != null) {
+                ctx.resource().injectGeneric(lsnr);
+
+                if (lsnr instanceof LifecycleAware)
+                    ((LifecycleAware)lsnr).start();
+
+                lsnrs.add(lsnr);
+            }
+        }
+
+        return lsnrs;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 79ac86d..8096291 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -35,7 +35,6 @@ import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
-import javax.cache.configuration.*;
 import javax.cache.integration.*;
 import java.util.*;
 
@@ -167,39 +166,10 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 "Persistence store is configured, but both read-through and write-through are disabled.");
         }
 
-        sesLsnrs = createSessionListeners(cfg.getCacheStoreSessionListenerFactories());
+        sesLsnrs = CU.createStoreSessionListeners(cctx.kernalContext(), cfg.getCacheStoreSessionListenerFactories());
 
         if (sesLsnrs == null)
-            sesLsnrs = createSessionListeners(cctx.kernalContext().config().getCacheStoreSessionListenerFactories());
-    }
-
-    /**
-     * Creates session listeners.
-     *
-     * @param factories Factories.
-     * @return Listeners.
-     */
-    private Collection<CacheStoreSessionListener> createSessionListeners(Factory<CacheStoreSessionListener>[] factories)
-        throws IgniteCheckedException {
-        if (factories == null)
-            return null;
-
-        Collection<CacheStoreSessionListener> lsnrs = new ArrayList<>(factories.length);
-
-        for (Factory<CacheStoreSessionListener> factory : factories) {
-            CacheStoreSessionListener lsnr = factory.create();
-
-            if (lsnr != null) {
-                cctx.kernalContext().resource().injectGeneric(lsnr);
-
-                if (lsnr instanceof LifecycleAware)
-                    ((LifecycleAware)lsnr).start();
-
-                lsnrs.add(lsnr);
-            }
-        }
-
-        return lsnrs;
+            sesLsnrs = cctx.shared().storeSessionListeners();
     }
 
     /** {@inheritDoc} */
@@ -754,6 +724,11 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             if (!sesHolder.get().storeEnded(store))
                 store.sessionEnd(commit);
         }
+        catch (Throwable e) {
+            last = true;
+
+            throw e;
+        }
         finally {
             if (last && sesHolder != null) {
                 sesHolder.set(null);
@@ -834,8 +809,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                         lsnr.onSessionEnd(locSes, !threwEx);
                 }
 
-                if (!sesHolder.get().storeEnded(store))
-                    store.sessionEnd(!threwEx);
+                assert !sesHolder.get().storeEnded(store);
+
+                store.sessionEnd(!threwEx);
             }
         }
         catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index f6d5d90..adc1c86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -408,9 +408,18 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         if (!storeEnabled())
             return false;
 
-        Collection<CacheStoreManager> stores = stores();
+        Collection<Integer> cacheIds = activeCacheIds();
+
+        if (!cacheIds.isEmpty()) {
+            for (int cacheId : cacheIds) {
+                CacheStoreManager store = cctx.cacheContext(cacheId).store();
 
-        return stores != null && !stores.isEmpty();
+                if (store.configured())
+                    return true;
+            }
+        }
+
+        return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 1bed2da..fdaef47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -18,10 +18,10 @@
 package org.apache.ignite.internal.processors.cache.transactions;
 
 import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.processors.cache.dr.*;
@@ -1010,24 +1010,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 cctx.tm().resetContext();
             }
         }
-        else if (!internal()) {
-            Collection<CacheStoreManager> stores = stores();
-
-            if (stores != null && !stores.isEmpty()) {
-                try {
-                    sessionEnd(stores, true);
-                }
-                catch (IgniteCheckedException e) {
-                    commitError(e);
-
-                    setRollbackOnly();
-
-                    cctx.tm().removeCommittedTx(this);
-
-                    throw e;
-                }
-            }
-        }
 
         // Do not unlock transaction entries if one-phase commit.
         if (!onePhaseCommit()) {
@@ -1119,7 +1101,12 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 if (!internal()) {
                     Collection<CacheStoreManager> stores = stores();
 
-                    if (stores != null && !stores.isEmpty() && (near() || F.first(stores).isWriteToStoreFromDht()))
+                    assert isWriteToStoreFromDhtValid(stores) :
+                        "isWriteToStoreFromDht can't be different within one transaction";
+
+                    boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
+
+                    if (stores != null && !stores.isEmpty() && (near() || isWriteToStoreFromDht))
                         sessionEnd(stores, false);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 0b0f099..1c85ed3 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -54,7 +54,8 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
                 new GridCacheMvccManager(),
                 new GridCacheDeploymentManager<K, V>(),
                 new GridCachePartitionExchangeManager<K, V>(),
-                new GridCacheIoManager()
+                new GridCacheIoManager(),
+                null
             ),
             defaultCacheConfiguration(),
             CacheType.USER,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
index fc9eb91..fe0960e 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
@@ -17,23 +17,40 @@
 
 package org.apache.ignite.cache.store.hibernate;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
 import org.hibernate.*;
+import org.hibernate.cfg.*;
 
 import javax.cache.integration.*;
+import java.io.*;
+import java.net.*;
 import java.util.*;
 
 /**
  * Cache store session listener based on Hibernate session.
  */
-public class CacheStoreSessionHibernateListener implements CacheStoreSessionListener {
+public class CacheStoreSessionHibernateListener implements CacheStoreSessionListener, LifecycleAware {
     /** Session key for JDBC connection. */
     public static final String HIBERNATE_SES_KEY = "__hibernate_ses_";
 
     /** Hibernate session factory. */
     private SessionFactory sesFactory;
 
+    /** Hibernate configuration file path. */
+    private String hibernateCfgPath;
+
+    /** Logger. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Whether to close session on stop. */
+    private boolean closeSesOnStop;
+
     /**
      * Sets Hibernate session factory.
      *
@@ -54,6 +71,69 @@ public class CacheStoreSessionHibernateListener implements CacheStoreSessionList
         return sesFactory;
     }
 
+    /**
+     * Sets hibernate configuration path.
+     *
+     * @param hibernateCfgPath Hibernate configuration path.
+     */
+    public void setHibernateConfigurationPath(String hibernateCfgPath) {
+        this.hibernateCfgPath = hibernateCfgPath;
+    }
+
+    /**
+     * Gets hibernate configuration path.
+     *
+     * @return Hibernate configuration path.
+     */
+    public String getHibernateConfigurationPath() {
+        return hibernateCfgPath;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public void start() throws IgniteException {
+        if (sesFactory == null && F.isEmpty(hibernateCfgPath))
+            throw new IgniteException("Either session factory or Hibernate configuration file is required by " +
+                getClass().getSimpleName() + '.');
+
+        if (!F.isEmpty(hibernateCfgPath)) {
+            if (sesFactory == null) {
+                try {
+                    URL url = new URL(hibernateCfgPath);
+
+                    sesFactory = new Configuration().configure(url).buildSessionFactory();
+                }
+                catch (MalformedURLException ignored) {
+                    // No-op.
+                }
+
+                if (sesFactory == null) {
+                    File cfgFile = new File(hibernateCfgPath);
+
+                    if (cfgFile.exists())
+                        sesFactory = new Configuration().configure(cfgFile).buildSessionFactory();
+                }
+
+                if (sesFactory == null)
+                    sesFactory = new Configuration().configure(hibernateCfgPath).buildSessionFactory();
+
+                if (sesFactory == null)
+                    throw new IgniteException("Failed to resolve Hibernate configuration file: " + hibernateCfgPath);
+
+                closeSesOnStop = true;
+            }
+            else
+                U.warn(log, "Hibernate configuration file configured in " + getClass().getSimpleName() +
+                    " will be ignored (session factory is already set).");
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        if (closeSesOnStop && sesFactory != null && !sesFactory.isClosed())
+            sesFactory.close();
+    }
+
     /** {@inheritDoc} */
     @Override public void onSessionStart(CacheStoreSession ses) {
         Map<String, Session> props = ses.properties();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 2633c83..f49a23d 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -77,6 +77,12 @@
         </dependency>
 
         <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-jdbc</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>commons-logging</groupId>
             <artifactId>commons-logging</artifactId>
             <version>1.1.1</version>
@@ -104,12 +110,6 @@
             <scope>test</scope>
         </dependency>
 
-        <dependency>
-            <groupId>org.springframework</groupId>
-            <artifactId>spring-jdbc</artifactId>
-            <version>${spring.version}</version>
-            <scope>test</scope>
-        </dependency>
 
         <dependency>
             <groupId>com.h2database</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
index e0caad5..2fab4f0 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListener.java
@@ -17,32 +17,45 @@
 
 package org.apache.ignite.cache.store.spring;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
 import org.apache.ignite.transactions.*;
+import org.springframework.jdbc.datasource.*;
 import org.springframework.transaction.*;
 import org.springframework.transaction.support.*;
 
 import javax.cache.integration.*;
+import javax.sql.*;
 
 /**
  * Cache store session listener based on Spring cache manager.
  */
-public class CacheStoreSessionSpringListener implements CacheStoreSessionListener {
+public class CacheStoreSessionSpringListener implements CacheStoreSessionListener, LifecycleAware {
     /** Session key for transaction status. */
     public static final String TX_STATUS_KEY = "__spring_tx_status_";
 
     /** Transaction manager. */
     private PlatformTransactionManager txMgr;
 
+    /** Data source. */
+    private DataSource dataSrc;
+
+    /** Propagation behavior. */
+    private int propagation = TransactionDefinition.PROPAGATION_REQUIRED;
+
+    /** Logger. */
+    @LoggerResource
+    private IgniteLogger log;
+
     /**
      * Sets transaction manager.
      *
      * @param txMgr Transaction manager.
      */
     public void setTransactionManager(PlatformTransactionManager txMgr) {
-        A.notNull(txMgr, "txMgr");
-
         this.txMgr = txMgr;
     }
 
@@ -55,11 +68,71 @@ public class CacheStoreSessionSpringListener implements CacheStoreSessionListene
         return txMgr;
     }
 
+    /**
+     * Sets data source.
+     *
+     * @param dataSrc Data source.
+     */
+    public void setDataSource(DataSource dataSrc) {
+        this.dataSrc = dataSrc;
+    }
+
+    /**
+     * Gets data source.
+     *
+     * @return Data source.
+     */
+    public DataSource getDataSource() {
+        return dataSrc;
+    }
+
+    /**
+     * Sets propagation behavior.
+     *
+     * @param propagation Propagation behavior.
+     */
+    public void setPropagationBehavior(int propagation) {
+        this.propagation = propagation;
+    }
+
+    /**
+     * Gets propagation behavior.
+     *
+     * @return Propagation behavior.
+     */
+    public int getPropagationBehavior() {
+        return propagation;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteException {
+        if (txMgr == null && dataSrc == null)
+            throw new IgniteException("Either transaction manager or data source is required by " +
+                getClass().getSimpleName() + '.');
+
+        if (dataSrc != null) {
+            if (txMgr == null)
+                txMgr = new DataSourceTransactionManager(dataSrc);
+            else
+                U.warn(log, "Data source configured in " + getClass().getSimpleName() +
+                    " will be ignored (transaction manager is already set).");
+        }
+
+        assert txMgr != null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        // No-op.
+    }
+
     /** {@inheritDoc} */
     @Override public void onSessionStart(CacheStoreSession ses) {
         if (ses.isWithinTransaction()) {
             try {
-                ses.properties().put(TX_STATUS_KEY, txMgr.getTransaction(definition(ses.transaction())));
+                TransactionDefinition def = definition(ses.transaction(), ses.cacheName());
+
+                ses.properties().put(TX_STATUS_KEY, txMgr.getTransaction(def));
             }
             catch (TransactionException e) {
                 throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
@@ -91,12 +164,19 @@ public class CacheStoreSessionSpringListener implements CacheStoreSessionListene
      *
      * @return DB transaction isolation.
      */
-    private TransactionDefinition definition(Transaction tx) {
+    private TransactionDefinition definition(Transaction tx, String cacheName) {
         assert tx != null;
 
         DefaultTransactionDefinition def = new DefaultTransactionDefinition();
 
+        def.setName("Ignite Tx [cache=" + (cacheName != null ? cacheName : "<default>") + ", id=" + tx.xid() + ']');
         def.setIsolationLevel(isolationLevel(tx.isolation()));
+        def.setPropagationBehavior(propagation);
+
+        long timeoutSec = (tx.timeout() + 500) / 1000;
+
+        if (timeoutSec > 0 && timeoutSec < Integer.MAX_VALUE)
+            def.setTimeout((int)timeoutSec);
 
         return def;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b37d0046/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
index 79d5b5e..83ed249 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheStoreSessionSpringListenerSelfTest.java
@@ -54,7 +54,7 @@ public class CacheStoreSessionSpringListenerSelfTest extends CacheStoreSessionLi
             @Override public CacheStoreSessionListener create() {
                 CacheStoreSessionSpringListener lsnr = new CacheStoreSessionSpringListener();
 
-                lsnr.setTransactionManager(new DataSourceTransactionManager(DATA_SRC));
+                lsnr.setDataSource(DATA_SRC);
 
                 return lsnr;
             }


[38/39] incubator-ignite git commit: GG-10385 Interop .Net: Leak in GridMessagingTest

Posted by sb...@apache.org.
GG-10385 Interop .Net: Leak in GridMessagingTest


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

Branch: refs/heads/ignite-876-2
Commit: 4f3788d58157f97c097bbdbf517fadf142a31662
Parents: 6ed4eca
Author: ptupitsyn <pt...@gridgain.com>
Authored: Tue Jun 2 11:30:27 2015 +0300
Committer: ptupitsyn <pt...@gridgain.com>
Committed: Tue Jun 2 11:30:27 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/communication/GridIoManager.java  | 12 +++++-------
 1 file changed, 5 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f3788d5/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index c877d57..6e8d457 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -1211,6 +1211,11 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     public void addUserMessageListener(@Nullable final Object topic, @Nullable final IgniteBiPredicate<UUID, ?> p) {
         if (p != null) {
             try {
+                if (p instanceof GridLifecycleAwareMessageFilter)
+                    ((GridLifecycleAwareMessageFilter)p).initialize(ctx);
+                else
+                    ctx.resource().injectGeneric(p);
+
                 addMessageListener(TOPIC_COMM_USER,
                     new GridUserMessageListener(topic, (IgniteBiPredicate<UUID, Object>)p));
             }
@@ -1695,13 +1700,6 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             throws IgniteCheckedException {
             this.topic = topic;
             this.predLsnr = predLsnr;
-
-            if (predLsnr != null) {
-                if (predLsnr instanceof GridLifecycleAwareMessageFilter)
-                    ((GridLifecycleAwareMessageFilter)predLsnr).initialize(ctx);
-                else
-                    ctx.resource().injectGeneric(predLsnr);
-            }
         }
 
         /** {@inheritDoc} */


[39/39] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-876-2

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-876-2


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

Branch: refs/heads/ignite-876-2
Commit: c74ddc421efafddae3eced31183f989b3803cc49
Parents: c1f0ce2 4f3788d
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 11:56:55 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 11:56:55 2015 +0300

----------------------------------------------------------------------
 .../hibernate/CacheHibernatePersonStore.java    | 202 +---------
 .../hibernate/CacheHibernateStoreExample.java   |  17 +
 .../store/jdbc/CacheJdbcPersonStore.java        | 180 ++-------
 .../store/jdbc/CacheJdbcStoreExample.java       |  13 +
 .../store/spring/CacheSpringPersonStore.java    | 128 ++++++
 .../store/spring/CacheSpringStoreExample.java   | 143 +++++++
 .../datagrid/store/spring/package-info.java     |  22 ++
 .../apache/ignite/cache/store/CacheStore.java   |   2 +
 .../ignite/cache/store/CacheStoreSession.java   |  22 ++
 .../cache/store/CacheStoreSessionListener.java  | 133 +++++++
 .../jdbc/CacheJdbcStoreSessionListener.java     | 141 +++++++
 .../configuration/CacheConfiguration.java       |  32 ++
 .../configuration/IgniteConfiguration.java      |  38 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   2 +-
 .../managers/communication/GridIoManager.java   |  12 +-
 .../processors/cache/GridCacheProcessor.java    |  12 +-
 .../cache/GridCacheSharedContext.java           |  47 ++-
 .../processors/cache/GridCacheUtils.java        |  54 +++
 .../cache/store/CacheOsStoreManager.java        |   1 -
 .../cache/store/CacheStoreManager.java          |   7 +-
 .../store/GridCacheStoreManagerAdapter.java     | 202 ++++++++--
 .../cache/transactions/IgniteTxAdapter.java     |  33 +-
 .../transactions/IgniteTxLocalAdapter.java      | 142 ++++---
 ...cheStoreSessionListenerAbstractSelfTest.java | 315 +++++++++++++++
 ...heStoreSessionListenerLifecycleSelfTest.java | 395 +++++++++++++++++++
 .../CacheJdbcStoreSessionListenerSelfTest.java  | 175 ++++++++
 .../IgniteCrossCacheTxStoreSelfTest.java        | 147 ++++---
 .../loadtests/hashmap/GridCacheTestContext.java |   3 +-
 .../junits/cache/TestCacheSession.java          |  18 +
 .../cache/TestThreadLocalCacheSession.java      |  15 +
 .../junits/common/GridCommonAbstractTest.java   |  24 ++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 .../CacheHibernateStoreSessionListener.java     | 216 ++++++++++
 ...heHibernateStoreSessionListenerSelfTest.java | 228 +++++++++++
 .../testsuites/IgniteHibernateTestSuite.java    |   2 +
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +-
 modules/spring/pom.xml                          |  14 +
 .../spring/CacheSpringStoreSessionListener.java | 207 ++++++++++
 ...CacheSpringStoreSessionListenerSelfTest.java | 197 +++++++++
 .../testsuites/IgniteSpringTestSuite.java       |   3 +
 40 files changed, 3057 insertions(+), 492 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c74ddc42/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 12134d6,e435ed7..2d5fed5
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@@ -59,10 -59,10 +59,10 @@@ public class IgniteCacheQuerySelfTestSu
          suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
          suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
 -        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.class);
 +        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-         suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
+         // suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
          suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
  //        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class); TODO IGNITE-484


[16/39] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891


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

Branch: refs/heads/ignite-876-2
Commit: 80ebfe0b0387ace6438bce6704df2cfded89292c
Parents: 1e3238c 4929c7d
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu May 21 15:54:58 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu May 21 15:54:58 2015 -0700

----------------------------------------------------------------------
 dev-tools/build.gradle                          |   2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      | 213 ++++++++++++++-----
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |  72 +++++++
 .../IgniteCacheEvictionSelfTestSuite.java       |   1 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   4 +-
 36 files changed, 265 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/80ebfe0b/modules/spring/pom.xml
----------------------------------------------------------------------


[30/39] incubator-ignite git commit: # IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
# IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: 0d9f1a7c053fa00e39f231a83e5972643698dbdd
Parents: 084b689
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed May 27 21:33:49 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed May 27 21:33:49 2015 -0700

----------------------------------------------------------------------
 .../cache/store/CacheStoreSessionListener.java  |  2 +-
 .../spring/CacheSpringStoreSessionListener.java | 24 --------------------
 2 files changed, 1 insertion(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0d9f1a7c/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
index b319e55..1543bf9 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
@@ -65,7 +65,7 @@ import javax.sql.*;
  *         cache transaction, this listener is no-op.
  *     </li>
  *     <li>
- *         <@ignitelink org.apache.ignite.cache.store.hibernate.CacheHibernateStoreSessionListener} -
+ *         {@ignitelink org.apache.ignite.cache.store.hibernate.CacheHibernateStoreSessionListener} -
  *         Hibernate-based session listener. It creates a new Hibernate
  *         session for each Ignite session. If there is an ongoing cache
  *         transaction, a corresponding Hibernate transaction is created

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0d9f1a7c/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
index 53fc599..0a32816 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
@@ -61,9 +61,6 @@ public class CacheSpringStoreSessionListener implements CacheStoreSessionListene
     /** Data source. */
     private DataSource dataSrc;
 
-    /** Propagation behavior. */
-    private int propagation = TransactionDefinition.PROPAGATION_REQUIRED;
-
     /** Logger. */
     @LoggerResource
     private IgniteLogger log;
@@ -110,26 +107,6 @@ public class CacheSpringStoreSessionListener implements CacheStoreSessionListene
         return dataSrc;
     }
 
-    /**
-     * Sets propagation behavior.
-     * <p>
-     * This parameter is optional.
-     *
-     * @param propagation Propagation behavior.
-     */
-    public void setPropagationBehavior(int propagation) {
-        this.propagation = propagation;
-    }
-
-    /**
-     * Gets propagation behavior.
-     *
-     * @return Propagation behavior.
-     */
-    public int getPropagationBehavior() {
-        return propagation;
-    }
-
     /** {@inheritDoc} */
     @Override public void start() throws IgniteException {
         if (txMgr == null && dataSrc == null)
@@ -197,7 +174,6 @@ public class CacheSpringStoreSessionListener implements CacheStoreSessionListene
 
         def.setName("Ignite Tx [cache=" + (cacheName != null ? cacheName : "<default>") + ", id=" + tx.xid() + ']');
         def.setIsolationLevel(isolationLevel(tx.isolation()));
-        def.setPropagationBehavior(propagation);
 
         long timeoutSec = (tx.timeout() + 500) / 1000;
 


[13/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: 1e3238c76a3f32de31d8297082bd3eb9c7b4c332
Parents: d4bcd19
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed May 20 18:53:45 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed May 20 18:53:45 2015 -0700

----------------------------------------------------------------------
 ...cheStoreSessionListenerAbstractSelfTest.java |  4 +-
 .../CacheStoreSessionHibernateListener.java     |  5 +-
 ...heStoreSessionHibernateListenerSelfTest.java | 97 +++++++++++++++++++-
 3 files changed, 96 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3238c7/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
index 5df8f68..adac0b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
@@ -93,8 +93,8 @@ public abstract class CacheStoreSessionListenerAbstractSelfTest extends GridComm
             conn.createStatement().executeUpdate("DROP TABLE IF EXISTS Table1");
             conn.createStatement().executeUpdate("DROP TABLE IF EXISTS Table2");
 
-            conn.createStatement().executeUpdate("CREATE TABLE Table1 (key INT, value INT)");
-            conn.createStatement().executeUpdate("CREATE TABLE Table2 (key INT, value INT)");
+            conn.createStatement().executeUpdate("CREATE TABLE Table1 (id INT AUTO_INCREMENT, key INT, value INT)");
+            conn.createStatement().executeUpdate("CREATE TABLE Table2 (id INT AUTO_INCREMENT, key INT, value INT)");
         }
 
         loadCacheCnt.set(0);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3238c7/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
index 06af03a..fc9eb91 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
@@ -19,7 +19,6 @@ package org.apache.ignite.cache.store.hibernate;
 
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.resources.*;
 import org.hibernate.*;
 
 import javax.cache.integration.*;
@@ -85,10 +84,10 @@ public class CacheStoreSessionHibernateListener implements CacheStoreSessionList
                 if (commit) {
                     hibSes.flush();
 
-                    if (tx != null)
+                    if (tx.isActive())
                         tx.commit();
                 }
-                else if (tx != null)
+                else if (tx.isActive())
                     tx.rollback();
             }
             catch (HibernateException e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1e3238c7/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
index d631393..a9d465e 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListenerSelfTest.java
@@ -27,6 +27,8 @@ import org.hibernate.cfg.Configuration;
 import javax.cache.Cache;
 import javax.cache.configuration.*;
 import javax.cache.integration.*;
+import javax.persistence.*;
+import java.io.*;
 import java.util.*;
 
 /**
@@ -48,10 +50,13 @@ public class CacheStoreSessionHibernateListenerSelfTest extends CacheStoreSessio
             @Override public CacheStoreSessionListener create() {
                 CacheStoreSessionHibernateListener lsnr = new CacheStoreSessionHibernateListener();
 
-                Configuration cfg = new Configuration().
-                    setProperty("hibernate.connection.url", URL);
+                SessionFactory sesFactory = new Configuration().
+                    setProperty("hibernate.connection.url", URL).
+                    addAnnotatedClass(Table1.class).
+                    addAnnotatedClass(Table2.class).
+                    buildSessionFactory();
 
-                lsnr.setSessionFactory(cfg.buildSessionFactory());
+                lsnr.setSessionFactory(sesFactory);
 
                 return lsnr;
             }
@@ -90,6 +95,28 @@ public class CacheStoreSessionHibernateListenerSelfTest extends CacheStoreSessio
             writeCnt.incrementAndGet();
 
             checkSession();
+
+            if (write.get()) {
+                Session hibSes = session();
+
+                switch (ses.cacheName()) {
+                    case "cache1":
+                        hibSes.save(new Table1(entry.getKey(), entry.getValue()));
+
+                        break;
+
+                    case "cache2":
+                        if (fail.get())
+                            throw new CacheWriterException("Expected failure.");
+
+                        hibSes.save(new Table2(entry.getKey(), entry.getValue()));
+
+                        break;
+
+                    default:
+                        throw new CacheWriterException("Wring cache: " + ses.cacheName());
+                }
+            }
         }
 
         /** {@inheritDoc} */
@@ -113,10 +140,14 @@ public class CacheStoreSessionHibernateListenerSelfTest extends CacheStoreSessio
 
             assertTrue(hibSes.isOpen());
 
+            Transaction tx = hibSes.getTransaction();
+
+            assertNotNull(tx);
+
             if (ses.isWithinTransaction())
-                assertNotNull(hibSes.getTransaction());
+                assertTrue(tx.isActive());
             else
-                assertNull(hibSes.getTransaction());
+                assertFalse(tx.isActive());
 
             verifySameInstance(hibSes);
         }
@@ -145,4 +176,60 @@ public class CacheStoreSessionHibernateListenerSelfTest extends CacheStoreSessio
             return ses.<String, Session>properties().get(CacheStoreSessionHibernateListener.HIBERNATE_SES_KEY);
         }
     }
+
+    /**
+     */
+    @Entity
+    @Table(name = "Table1")
+    private static class Table1 implements Serializable {
+        /** */
+        @Id @GeneratedValue
+        @Column(name = "id")
+        private Integer id;
+
+        /** */
+        @Column(name = "key")
+        private int key;
+
+        /** */
+        @Column(name = "value")
+        private int value;
+
+        /**
+         * @param key Key.
+         * @param value Value.
+         */
+        private Table1(int key, int value) {
+            this.key = key;
+            this.value = value;
+        }
+    }
+
+    /**
+     */
+    @Entity
+    @Table(name = "Table2")
+    private static class Table2 implements Serializable {
+        /** */
+        @Id @GeneratedValue
+        @Column(name = "id")
+        private Integer id;
+
+        /** */
+        @Column(name = "key")
+        private int key;
+
+        /** */
+        @Column(name = "value")
+        private int value;
+
+        /**
+         * @param key Key.
+         * @param value Value.
+         */
+        private Table2(int key, int value) {
+            this.key = key;
+            this.value = value;
+        }
+    }
 }


[07/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: 463883d289ed7decf4534c4788c8a092931a77b7
Parents: f57b6f9
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon May 18 17:49:44 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon May 18 17:49:44 2015 -0700

----------------------------------------------------------------------
 .../cache/GridCacheSharedContext.java           | 30 +++++++++++++++-----
 .../cache/store/CacheStoreManager.java          |  5 ++++
 .../store/GridCacheStoreManagerAdapter.java     |  7 ++++-
 .../transactions/IgniteTxLocalAdapter.java      | 27 +++++++++++++++---
 4 files changed, 57 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/463883d2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index dacd1aa..b16885e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.store.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.processors.timeout.*;
@@ -427,23 +428,38 @@ public class GridCacheSharedContext<K, V> {
      * @param tx Transaction to check.
      * @param activeCacheIds Active cache IDs.
      * @param cacheCtx Cache context.
-     * @return {@code True} if cross-cache transaction can include this new cache.
+     * @return Error message if transactions are incompatible.
      */
-    public boolean txCompatible(IgniteInternalTx tx, Iterable<Integer> activeCacheIds, GridCacheContext<K, V> cacheCtx) {
-        if (cacheCtx.systemTx() ^ tx.system())
-            return false;
+    @Nullable public String verifyTxCompatibility(IgniteInternalTx tx, Iterable<Integer> activeCacheIds,
+        GridCacheContext<K, V> cacheCtx) {
+        if (cacheCtx.systemTx() && !tx.system())
+            return "system cache can be enlisted only in system transaction";
+
+        if (!cacheCtx.systemTx() && tx.system())
+            return "non-system cache can't be enlisted in system transaction";
 
         for (Integer cacheId : activeCacheIds) {
             GridCacheContext<K, V> activeCacheCtx = cacheContext(cacheId);
 
-            // System transactions may sap only one cache.
             if (cacheCtx.systemTx()) {
                 if (activeCacheCtx.cacheId() != cacheCtx.cacheId())
-                    return false;
+                    return "system transaction can include only one cache";
             }
+
+            CacheStoreManager store = cacheCtx.store();
+            CacheStoreManager activeStore = activeCacheCtx.store();
+
+            if (store.isLocal() != activeStore.isLocal())
+                return "caches with local and non-local stores can't be enlisted in one transaction";
+
+            if (store.isWriteBehind() != activeStore.isWriteBehind())
+                return "caches with different write-behind setting can't be enlisted in one transaction";
+
+            // If local and write-behind validations passed, this must be true.
+            assert store.isWriteToStoreFromDht() == activeStore.isWriteToStoreFromDht();
         }
 
-        return true;
+        return null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/463883d2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
index 327b879..a14df6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
@@ -68,6 +68,11 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
     public boolean isWriteThrough();
 
     /**
+     * @return {@code True} is write-behind is enabled.
+     */
+    public boolean isWriteBehind();
+
+    /**
      * @return Whether DHT transaction can write to store from DHT.
      */
     public boolean isWriteToStoreFromDht();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/463883d2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index aeca58f..b608bb6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -295,8 +295,13 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isWriteBehind() {
+        return cctx.config().isWriteBehindEnabled();
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isWriteToStoreFromDht() {
-        return cctx.config().isWriteBehindEnabled() || locStore;
+        return isWriteBehind() || locStore;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/463883d2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 854448d..4d6a544 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -488,6 +488,8 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         if (stores == null || stores.isEmpty())
             return;
 
+        assert isWriteToStoreFromDhtValid(stores) : "isWriteToStoreFromDht can't be different within one transaction";
+
         boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
 
         if (near() || isWriteToStoreFromDht) {
@@ -659,6 +661,21 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         }
     }
 
+    /**
+     * @param stores Store managers.
+     * @return If {@code isWriteToStoreFromDht} value same for all stores.
+     */
+    private boolean isWriteToStoreFromDhtValid(Collection<CacheStoreManager> stores) {
+        boolean exp = F.first(stores).isWriteToStoreFromDht();
+
+        for (CacheStoreManager store : stores) {
+            if (store.isWriteToStoreFromDht() != exp)
+                return false;
+        }
+
+        return true;
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings({"CatchGenericClass"})
     @Override public void userCommit() throws IgniteCheckedException {
@@ -3017,7 +3034,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
         // Check if we can enlist new cache to transaction.
         if (!activeCacheIds.contains(cacheId)) {
-            if (!cctx.txCompatible(this, activeCacheIds, cacheCtx)) {
+            String err = cctx.verifyTxCompatibility(this, activeCacheIds, cacheCtx);
+
+            if (err != null) {
                 StringBuilder cacheNames = new StringBuilder();
 
                 int idx = 0;
@@ -3029,9 +3048,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                         cacheNames.append(", ");
                 }
 
-                throw new IgniteCheckedException("Failed to enlist new cache to existing transaction " +
-                    "(cache configurations are not compatible) [" +
-                    "activeCaches=[" + cacheNames + "]" +
+                throw new IgniteCheckedException("Failed to enlist new cache to existing transaction (" +
+                    err +
+                    ") [activeCaches=[" + cacheNames + "]" +
                     ", cacheName=" + cacheCtx.name() +
                     ", cacheSystem=" + cacheCtx.systemTx() +
                     ", txSystem=" + system() + ']');


[26/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: 9f88b05cc8be60e69d746330b0d6617ff5f424bd
Parents: ada1b2a
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Sun May 24 23:35:44 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Sun May 24 23:35:44 2015 -0700

----------------------------------------------------------------------
 ...heStoreSessionListenerLifeCycleSelfTest.java | 395 -------------------
 1 file changed, 395 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9f88b05c/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifeCycleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifeCycleSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifeCycleSelfTest.java
deleted file mode 100644
index 814c8a5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifeCycleSelfTest.java
+++ /dev/null
@@ -1,395 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.store;
-
-import org.apache.ignite.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.lifecycle.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.junits.common.*;
-import org.apache.ignite.transactions.*;
-
-import javax.cache.*;
-import javax.cache.configuration.*;
-import javax.cache.integration.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-
-/**
- * Store session listeners test.
- */
-public class CacheStoreSessionListenerLifecycleSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final Queue<String> evts = new ConcurrentLinkedDeque<>();
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setCacheStoreSessionListenerFactories(
-            new SessionListenerFactory("Shared 1"),
-            new SessionListenerFactory("Shared 2")
-        );
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(disco);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        evts.clear();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNoCaches() throws Exception {
-        try {
-            startGrid();
-        }
-        finally {
-            stopGrid();
-        }
-
-        assertEqualsCollections(Arrays.asList("Shared 1 START", "Shared 2 START", "Shared 1 STOP", "Shared 2 STOP"),
-            evts);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNoOverride() throws Exception {
-        try {
-            Ignite ignite = startGrid();
-
-            for (int i = 0; i < 2; i++) {
-                CacheConfiguration<Integer, Integer> cacheCfg = cacheConfiguration("cache-" + i);
-
-                cacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-                ignite.createCache(cacheCfg);
-            }
-
-            ignite.cache("cache-0").put(1, 1);
-            ignite.cache("cache-1").put(1, 1);
-
-            try (Transaction tx = ignite.transactions().txStart()) {
-                ignite.cache("cache-0").put(2, 2);
-                ignite.cache("cache-0").put(3, 3);
-                ignite.cache("cache-1").put(2, 2);
-                ignite.cache("cache-1").put(3, 3);
-
-                tx.commit();
-            }
-        }
-        finally {
-            stopGrid();
-        }
-
-        assertEqualsCollections(Arrays.asList(
-            "Shared 1 START",
-            "Shared 2 START",
-
-            // Put to cache-0.
-            "Shared 1 SESSION START cache-0",
-            "Shared 2 SESSION START cache-0",
-            "Shared 1 SESSION END cache-0",
-            "Shared 2 SESSION END cache-0",
-
-            // Put to cache-1.
-            "Shared 1 SESSION START cache-1",
-            "Shared 2 SESSION START cache-1",
-            "Shared 1 SESSION END cache-1",
-            "Shared 2 SESSION END cache-1",
-
-            // Transaction.
-            "Shared 1 SESSION START cache-0",
-            "Shared 2 SESSION START cache-0",
-            "Shared 1 SESSION START cache-1",
-            "Shared 2 SESSION START cache-1",
-            "Shared 1 SESSION END cache-0",
-            "Shared 2 SESSION END cache-0",
-            "Shared 1 SESSION END cache-1",
-            "Shared 2 SESSION END cache-1",
-
-            "Shared 1 STOP",
-            "Shared 2 STOP"
-        ), evts);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPartialOverride() throws Exception {
-        try {
-            Ignite ignite = startGrid();
-
-            for (int i = 0; i < 2; i++) {
-                String name = "cache-" + i;
-
-                CacheConfiguration cacheCfg = cacheConfiguration(name);
-
-                cacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-                if (i == 0) {
-                    cacheCfg.setCacheStoreSessionListenerFactories(
-                        new SessionListenerFactory(name + " 1"),
-                        new SessionListenerFactory(name + " 2")
-                    );
-                }
-
-                ignite.createCache(cacheCfg);
-            }
-
-            ignite.cache("cache-0").put(1, 1);
-            ignite.cache("cache-1").put(1, 1);
-
-            try (Transaction tx = ignite.transactions().txStart()) {
-                ignite.cache("cache-0").put(2, 2);
-                ignite.cache("cache-0").put(3, 3);
-                ignite.cache("cache-1").put(2, 2);
-                ignite.cache("cache-1").put(3, 3);
-
-                tx.commit();
-            }
-        }
-        finally {
-            stopGrid();
-        }
-
-        assertEqualsCollections(Arrays.asList(
-            "Shared 1 START",
-            "Shared 2 START",
-            "cache-0 1 START",
-            "cache-0 2 START",
-
-            // Put to cache-0.
-            "cache-0 1 SESSION START cache-0",
-            "cache-0 2 SESSION START cache-0",
-            "cache-0 1 SESSION END cache-0",
-            "cache-0 2 SESSION END cache-0",
-
-            // Put to cache-1.
-            "Shared 1 SESSION START cache-1",
-            "Shared 2 SESSION START cache-1",
-            "Shared 1 SESSION END cache-1",
-            "Shared 2 SESSION END cache-1",
-
-            // Transaction.
-            "cache-0 1 SESSION START cache-0",
-            "cache-0 2 SESSION START cache-0",
-            "Shared 1 SESSION START cache-1",
-            "Shared 2 SESSION START cache-1",
-            "cache-0 1 SESSION END cache-0",
-            "cache-0 2 SESSION END cache-0",
-            "Shared 1 SESSION END cache-1",
-            "Shared 2 SESSION END cache-1",
-
-            "cache-0 1 STOP",
-            "cache-0 2 STOP",
-            "Shared 1 STOP",
-            "Shared 2 STOP"
-        ), evts);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOverride() throws Exception {
-        try {
-            Ignite ignite = startGrid();
-
-            for (int i = 0; i < 2; i++) {
-                String name = "cache-" + i;
-
-                CacheConfiguration cacheCfg = cacheConfiguration(name);
-
-                cacheCfg.setCacheStoreSessionListenerFactories(new SessionListenerFactory(name + " 1"), new SessionListenerFactory(name + " 2"));
-
-                ignite.createCache(cacheCfg);
-            }
-
-            ignite.cache("cache-0").put(1, 1);
-            ignite.cache("cache-1").put(1, 1);
-
-            try (Transaction tx = ignite.transactions().txStart()) {
-                ignite.cache("cache-0").put(2, 2);
-                ignite.cache("cache-0").put(3, 3);
-                ignite.cache("cache-1").put(2, 2);
-                ignite.cache("cache-1").put(3, 3);
-
-                tx.commit();
-            }
-        }
-        finally {
-            stopGrid();
-        }
-
-        assertEqualsCollections(Arrays.asList(
-            "Shared 1 START",
-            "Shared 2 START",
-            "cache-0 1 START",
-            "cache-0 2 START",
-            "cache-1 1 START",
-            "cache-1 2 START",
-
-            // Put to cache-0.
-            "cache-0 1 SESSION START cache-0",
-            "cache-0 2 SESSION START cache-0",
-            "cache-0 1 SESSION END cache-0",
-            "cache-0 2 SESSION END cache-0",
-
-            // Put to cache-1.
-            "cache-1 1 SESSION START cache-1",
-            "cache-1 2 SESSION START cache-1",
-            "cache-1 1 SESSION END cache-1",
-            "cache-1 2 SESSION END cache-1",
-
-            // Transaction.
-            "cache-0 1 SESSION START cache-0",
-            "cache-0 2 SESSION START cache-0",
-            "cache-1 1 SESSION START cache-1",
-            "cache-1 2 SESSION START cache-1",
-            "cache-0 1 SESSION END cache-0",
-            "cache-0 2 SESSION END cache-0",
-            "cache-1 1 SESSION END cache-1",
-            "cache-1 2 SESSION END cache-1",
-
-            "cache-0 1 STOP",
-            "cache-0 2 STOP",
-            "cache-1 1 STOP",
-            "cache-1 2 STOP",
-            "Shared 1 STOP",
-            "Shared 2 STOP"
-        ), evts);
-    }
-
-    /**
-     * @param name Cache name.
-     * @return Cache configuration.
-     */
-    private CacheConfiguration<Integer, Integer> cacheConfiguration(String name) {
-        CacheConfiguration<Integer, Integer> cacheCfg = new CacheConfiguration<>(name);
-
-        cacheCfg.setAtomicityMode(TRANSACTIONAL);
-        cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(Store.class));
-        cacheCfg.setWriteThrough(true);
-
-        return cacheCfg;
-    }
-
-    /**
-     */
-    private static class SessionListener implements CacheStoreSessionListener, LifecycleAware {
-        /** */
-        private final String name;
-
-        /** */
-        @IgniteInstanceResource
-        private Ignite ignite;
-
-        /**
-         * @param name Name.
-         */
-        private SessionListener(String name) {
-            this.name = name;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void start() throws IgniteException {
-            assertNotNull(ignite);
-
-            evts.add(name + " START");
-        }
-
-        /** {@inheritDoc} */
-        @Override public void stop() throws IgniteException {
-            assertNotNull(ignite);
-
-            evts.add(name + " STOP");
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionStart(CacheStoreSession ses) {
-            assertNotNull(ignite);
-
-            evts.add(name + " SESSION START " + ses.cacheName());
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
-            assertNotNull(ignite);
-
-            evts.add(name + " SESSION END " + ses.cacheName());
-        }
-    }
-
-    /**
-     */
-    private static class SessionListenerFactory implements Factory<CacheStoreSessionListener> {
-        /** */
-        private String name;
-
-        /**
-         * @param name Name.
-         */
-        private SessionListenerFactory(String name) {
-            this.name = name;
-        }
-
-        @Override public CacheStoreSessionListener create() {
-            return new SessionListener(name);
-        }
-    }
-
-    /**
-     */
-    public static class Store extends CacheStoreAdapter<Integer, Integer> {
-        public Store() {
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer load(Integer key) throws CacheLoaderException {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
-            throws CacheWriterException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void delete(Object key) throws CacheWriterException {
-            // No-op.
-        }
-    }
-}


[06/39] incubator-ignite git commit: IGNITE-891 - Cache store improvements

Posted by sb...@apache.org.
IGNITE-891 - Cache store improvements


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

Branch: refs/heads/ignite-876-2
Commit: f57b6f98585a85cf0147d75eaaf43d85b23b7e65
Parents: d13b8f9
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon May 18 17:21:16 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon May 18 17:21:16 2015 -0700

----------------------------------------------------------------------
 .../ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java   | 4 ----
 .../store/hibernate/CacheStoreSessionHibernateListener.java      | 4 ----
 .../org/apache/ignite/testsuites/IgniteHibernateTestSuite.java   | 2 ++
 3 files changed, 2 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f57b6f98/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
index 87ed5c1..c683abe 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheStoreSessionJdbcListener.java
@@ -36,10 +36,6 @@ public class CacheStoreSessionJdbcListener implements CacheStoreSessionListener
     /** Data source. */
     private DataSource dataSrc;
 
-    /** Store session. */
-    @CacheStoreSessionResource
-    private CacheStoreSession ses;
-
     /**
      * Sets data source.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f57b6f98/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
index e1fd4b2..06af03a 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheStoreSessionHibernateListener.java
@@ -35,10 +35,6 @@ public class CacheStoreSessionHibernateListener implements CacheStoreSessionList
     /** Hibernate session factory. */
     private SessionFactory sesFactory;
 
-    /** Store session. */
-    @CacheStoreSessionResource
-    private CacheStoreSession ses;
-
     /**
      * Sets Hibernate session factory.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f57b6f98/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
index da741f8..ed782e2 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
@@ -41,6 +41,8 @@ public class IgniteHibernateTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheHibernateBlobStoreNodeRestartTest.class);
 
+        suite.addTestSuite(CacheStoreSessionHibernateListenerSelfTest.class);
+
         return suite;
     }
 }


[02/39] incubator-ignite git commit: Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Posted by sb...@apache.org.
Merge branches 'ignite-891' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-891

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java


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

Branch: refs/heads/ignite-876-2
Commit: 0c528986a8436f9c904e3049cd1383f81ff75606
Parents: b97441f da5a228
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri May 15 10:41:40 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri May 15 10:41:40 2015 -0700

----------------------------------------------------------------------
 .../socket/WordsSocketStreamerServer.java       |   2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 136 +++++----------
 .../internal/interop/InteropBootstrap.java      |  34 ++++
 .../interop/InteropBootstrapFactory.java        |  39 +++++
 .../internal/interop/InteropIgnition.java       | 103 ++++++++++++
 .../internal/interop/InteropProcessor.java      |  25 +++
 .../processors/cache/GridCacheAdapter.java      |   8 +-
 .../processors/cache/GridCacheMapEntry.java     |  35 +---
 .../distributed/GridDistributedLockRequest.java | 111 +++----------
 .../GridDistributedTxFinishRequest.java         |  70 ++------
 .../GridDistributedTxPrepareRequest.java        | 112 +++----------
 .../GridDistributedTxRemoteAdapter.java         |  20 +--
 .../distributed/dht/GridDhtLockFuture.java      |   2 -
 .../distributed/dht/GridDhtLockRequest.java     |  45 +++--
 .../dht/GridDhtTransactionalCacheAdapter.java   |   6 -
 .../distributed/dht/GridDhtTxFinishFuture.java  |   3 -
 .../distributed/dht/GridDhtTxFinishRequest.java |  43 +++--
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 -
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  68 +-------
 .../distributed/dht/GridDhtTxPrepareFuture.java |  18 +-
 .../dht/GridDhtTxPrepareRequest.java            |  60 ++++---
 .../cache/distributed/dht/GridDhtTxRemote.java  |   8 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   6 -
 .../distributed/near/GridNearLockFuture.java    |   6 -
 .../distributed/near/GridNearLockRequest.java   |  61 +++----
 .../near/GridNearOptimisticTxPrepareFuture.java |  15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   2 -
 .../near/GridNearTransactionalCache.java        |   4 -
 .../near/GridNearTxFinishRequest.java           |  28 ++--
 .../cache/distributed/near/GridNearTxLocal.java |  20 +--
 .../near/GridNearTxPrepareRequest.java          |  52 +++---
 .../distributed/near/GridNearTxRemote.java      |  24 +--
 .../cache/transactions/IgniteInternalTx.java    |  10 --
 .../transactions/IgniteTransactionsImpl.java    |   4 +-
 .../cache/transactions/IgniteTxAdapter.java     |  72 +-------
 .../cache/transactions/IgniteTxEntry.java       |  48 +-----
 .../cache/transactions/IgniteTxHandler.java     |   6 -
 .../transactions/IgniteTxLocalAdapter.java      | 165 ++-----------------
 .../cache/transactions/IgniteTxLocalEx.java     |  21 +--
 .../cache/transactions/IgniteTxManager.java     |  62 +------
 .../communication/tcp/TcpCommunicationSpi.java  |   2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  26 ---
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   8 +-
 .../processors/cache/jta/CacheJtaManager.java   |   4 +-
 pom.xml                                         |   2 +
 45 files changed, 487 insertions(+), 1115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0c528986/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0c528986/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 40bb36e,609108f..fa64e12
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@@ -496,17 -480,10 +480,17 @@@ public abstract class IgniteTxLocalAdap
       */
      @SuppressWarnings({"CatchGenericClass"})
      protected void batchStoreCommit(Iterable<IgniteTxEntry> writeEntries) throws IgniteCheckedException {
-         if (!storeEnabled() || (internal() && !groupLock()))
 -        CacheStoreManager store = store();
++        if (!storeEnabled() || internal())
 +            return;
  
 -        if (store != null && store.isWriteThrough() && storeEnabled() &&
 -            !internal() && (near() || store.isWriteToStoreFromDht())) {
 +        Collection<CacheStoreManager> stores = stores();
 +
 +        if (stores == null || stores.isEmpty())
 +            return;
 +
 +        boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
 +
 +        if (near() || isWriteToStoreFromDht) {
              try {
                  if (writeEntries != null) {
                      Map<Object, IgniteBiTuple<Object, GridCacheVersion>> putMap = null;
@@@ -1008,12 -971,11 +985,12 @@@
              }
          }
          else {
 -            CacheStoreManager store = store();
 +            Collection<CacheStoreManager> stores = stores();
  
-             if (stores != null && !stores.isEmpty() && (!internal() || groupLock())) {
 -            if (store != null && !internal()) {
++            if (stores != null && !stores.isEmpty() && !internal()) {
                  try {
 -                    store.sessionEnd(this, true);
 +                    for (CacheStoreManager store : stores)
 +                        store.sessionEnd(this, true);
                  }
                  catch (IgniteCheckedException e) {
                      commitError(e);
@@@ -1114,13 -1076,11 +1091,13 @@@
  
                  cctx.tm().rollbackTx(this);
  
 -                CacheStoreManager store = store();
 +                Collection<CacheStoreManager> stores = stores();
  
 -                if (store != null && (near() || store.isWriteToStoreFromDht())) {
 -                    if (!internal())
 -                        store.sessionEnd(this, false);
 +                if (stores != null && !stores.isEmpty() && (near() || F.first(stores).isWriteToStoreFromDht())) {
-                     if (!internal() || groupLock()) {
++                    if (!internal()) {
 +                        for (CacheStoreManager store : stores)
 +                            store.sessionEnd(this, false);
 +                    }
                  }
              }
              catch (Error | IgniteCheckedException | RuntimeException e) {


[37/39] incubator-ignite git commit: # ignite-sprint-5 disabled hanging test

Posted by sb...@apache.org.
# ignite-sprint-5 disabled hanging test


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

Branch: refs/heads/ignite-876-2
Commit: 6ed4ecaa459492e3caa983b9d3889bef8855e516
Parents: 2f61522
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 2 10:58:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 2 10:58:52 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6ed4ecaa/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index f42963a..e435ed7 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -62,7 +62,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
+        // suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
         suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
 //        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class); TODO IGNITE-484