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

[16/27] incubator-ignite git commit: #IGNITE-960 Fixed review notes. Added tests. Added check that CacheConfiguration is serializable.

#IGNITE-960 Fixed review notes. Added tests. Added check that CacheConfiguration is serializable.


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

Branch: refs/heads/ignite-gg-10411
Commit: a7ca45bb5fb9afa782d2b2ec134cc6524c7c22f8
Parents: b037baf
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Jun 16 17:15:49 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Jun 16 17:15:49 2015 +0300

----------------------------------------------------------------------
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |  22 +-
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   | 287 +++++++++++++++++++
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   3 +
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   | 145 ++++++++++
 .../processors/cache/GridCacheProcessor.java    |  18 ++
 .../util/spring/IgniteSpringHelper.java         |  10 +
 .../hibernate/CacheHibernateBlobStore.java      |  87 +-----
 .../CacheHibernateBlobStoreFactory.java         | 180 ++++++++++++
 .../CacheHibernateStoreFactorySelfTest.java     |  66 +++++
 .../testsuites/IgniteHibernateTestSuite.java    |   2 +
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   | 237 ---------------
 .../util/spring/IgniteSpringHelperImpl.java     |  17 ++
 .../src/test/config/incorrect-store-cache.xml   |   2 +-
 .../test/config/pojo-incorrect-store-cache.xml  |  56 ++++
 modules/spring/src/test/config/store-cache.xml  |   2 +-
 modules/spring/src/test/config/store-cache1.xml |   2 +-
 .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java |   2 +-
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java | 194 +++++++++++++
 .../testsuites/IgniteSpringTestSuite.java       |   5 +
 19 files changed, 991 insertions(+), 346 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java
index 6852260..e01c879 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStore.java
@@ -19,6 +19,7 @@ package org.apache.ignite.cache.store.jdbc;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -65,27 +66,8 @@ import java.util.concurrent.atomic.*;
  *     <li>Insert entry query (see {@link #setInsertQuery(String)})</li>
  *     <li>Delete entry query (see {@link #setDeleteQuery(String)})</li>
  * </ul>
- * <h2 class="header">Java Example</h2>
- * <pre name="code" class="java">
- *     ...
- *     GridCacheJdbcBlobStore&lt;String, String&gt; store = new GridCacheJdbcBlobStore&lt;String, String&gt;();
- *     ...
- * </pre>
- * <h2 class="header">Spring Example</h2>
- * <pre name="code" class="xml">
- *     ...
- *     &lt;bean id=&quot;cache.jdbc.store&quot;
- *         class=&quot;org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStore&quot;&gt;
- *         &lt;property name=&quot;connectionUrl&quot; value=&quot;jdbc:h2:mem:&quot;/&gt;
- *         &lt;property name=&quot;createTableQuery&quot;
- *             value=&quot;create table if not exists ENTRIES (key other, val other)&quot;/&gt;
- *     &lt;/bean&gt;
- *     ...
- * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
- * <br>
- * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ * Use {@link CacheJdbcBlobStoreFactory} factory to pass {@link CacheJdbcBlobStore} to {@link CacheConfiguration}.
  */
 public class CacheJdbcBlobStore<K, V> extends CacheStoreAdapter<K, V> {
     /** Default connection URL (value is <tt>jdbc:h2:mem:jdbcCacheStore;DB_CLOSE_DELAY=-1</tt>). */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
new file mode 100644
index 0000000..29a83c9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
@@ -0,0 +1,287 @@
+/*
+ * 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.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.spring.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
+
+import javax.cache.configuration.*;
+import javax.sql.*;
+
+/**
+ * {@link Factory} implementation for {@link CacheJdbcBlobStore}.
+ *
+ * Use this factory to pass {@link CacheJdbcBlobStore} to {@link CacheConfiguration}.
+ *
+ * <h2 class="header">Spring Example</h2>
+ * <pre name="code" class="xml">
+ *     &lt;bean id= "simpleDataSource" class="org.h2.jdbcx.JdbcDataSource"/&gt;
+ *
+ *     &lt;bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration"&gt;
+ *          ...
+ *          &lt;property name="cacheConfiguration"&gt;
+ *               &lt;list&gt;
+ *                  &lt;bean class="org.apache.ignite.configuration.CacheConfiguration"&gt;
+ *                      ...
+ *                      &lt;property name="cacheStoreFactory"&gt;
+ *                          &lt;bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory"&gt;
+ *                              &lt;property name="user" value = "GridGain" /&gt;
+ *                              &lt;property name="dataSourceBean" value = "simpleDataSource" /&gt;
+ *                          &lt;/bean&gt;
+ *                      &lt;/property&gt;
+ *                  &lt;/bean&gt;
+ *               &lt;/list&gt;
+ *          &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * </pre>
+ * <p>
+ * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <br>
+ * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ */
+public class CacheJdbcBlobStoreFactory<K, V> implements Factory<CacheJdbcBlobStore<K, V>> {
+    /** Connection URL. */
+    private String connUrl = CacheJdbcBlobStore.DFLT_CONN_URL;
+
+    /** Query to create table. */
+    private String createTblQry = CacheJdbcBlobStore.DFLT_CREATE_TBL_QRY;
+
+    /** Query to load entry. */
+    private String loadQry = CacheJdbcBlobStore.DFLT_LOAD_QRY;
+
+    /** Query to update entry. */
+    private String updateQry = CacheJdbcBlobStore.DFLT_UPDATE_QRY;
+
+    /** Query to insert entries. */
+    private String insertQry = CacheJdbcBlobStore.DFLT_INSERT_QRY;
+
+    /** Query to delete entries. */
+    private String delQry = CacheJdbcBlobStore.DFLT_DEL_QRY;
+
+    /** User name for database access. */
+    private String user;
+
+    /** Password for database access. */
+    @GridToStringExclude
+    private String passwd;
+
+    /** Flag for schema initialization. */
+    private boolean initSchema = true;
+
+    /** Name of data source bean. */
+    private String dataSrcBean;
+
+    /** Data source. */
+    private DataSource dataSrc;
+
+    /** Application context. */
+    @SpringApplicationContextResource
+    private Object appContext;
+
+    /** {@inheritDoc} */
+    @Override public CacheJdbcBlobStore<K, V> create() {
+        CacheJdbcBlobStore<K, V> store = new CacheJdbcBlobStore<>();
+
+        store.setInitSchema(initSchema);
+        store.setConnectionUrl(connUrl);
+        store.setCreateTableQuery(createTblQry);
+        store.setLoadQuery(loadQry);
+        store.setUpdateQuery(updateQry);
+        store.setInsertQuery(insertQry);
+        store.setDeleteQuery(delQry);
+        store.setUser(user);
+        store.setPassword(passwd);
+
+        if (dataSrc != null)
+            store.setDataSource(dataSrc);
+        else if (dataSrcBean != null) {
+            if (appContext == null)
+                throw new IgniteException("Spring application context resource is not injected.");
+
+            IgniteSpringHelper spring;
+
+            try {
+                spring = IgniteComponentType.SPRING.create(false);
+
+                DataSource data = spring.loadBeanFromAppContext(appContext, dataSrcBean);
+
+                store.setDataSource(data);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to load bean in application context [beanName=" + dataSrcBean +
+                    ", igniteConfig=" + appContext + ']');
+            }
+        }
+
+        return store;
+    }
+
+    /**
+     * Flag indicating whether DB schema should be initialized by Ignite (default behaviour) or
+     * was explicitly created by user.
+     *
+     * @param initSchema Initialized schema flag.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setInitSchema(boolean)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setInitSchema(boolean initSchema) {
+        this.initSchema = initSchema;
+
+        return this;
+    }
+
+    /**
+     * Sets connection URL.
+     *
+     * @param connUrl Connection URL.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setConnectionUrl(String)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setConnectionUrl(String connUrl) {
+        this.connUrl = connUrl;
+
+        return this;
+    }
+
+    /**
+     * Sets create table query.
+     *
+     * @param createTblQry Create table query.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setCreateTableQuery(String)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setCreateTableQuery(String createTblQry) {
+        this.createTblQry = createTblQry;
+
+        return this;
+    }
+
+    /**
+     * Sets load query.
+     *
+     * @param loadQry Load query
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setLoadQuery(String)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setLoadQuery(String loadQry) {
+        this.loadQry = loadQry;
+
+        return this;
+    }
+
+    /**
+     * Sets update entry query.
+     *
+     * @param updateQry Update entry query.
+     * @return {@code This} for chaining.
+     * @see  CacheJdbcBlobStore#setUpdateQuery(String)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setUpdateQuery(String updateQry) {
+        this.updateQry = updateQry;
+
+        return this;
+    }
+
+    /**
+     * Sets insert entry query.
+     *
+     * @param insertQry Insert entry query.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setInsertQuery(String)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setInsertQuery(String insertQry) {
+        this.insertQry = insertQry;
+
+        return this;
+    }
+
+    /**
+     * Sets delete entry query.
+     *
+     * @param delQry Delete entry query.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setDeleteQuery(String)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setDeleteQuery(String delQry) {
+        this.delQry = delQry;
+
+        return this;
+    }
+
+    /**
+     * Sets user name for database access.
+     *
+     * @param user User name.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setUser(String)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setUser(String user) {
+        this.user = user;
+
+        return this;
+    }
+
+    /**
+     * Sets password for database access.
+     *
+     * @param passwd Password.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setPassword(String)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setPassword(String passwd) {
+        this.passwd = passwd;
+
+        return this;
+    }
+
+    /**
+     * Sets name of the data source bean.
+     *
+     * @param dataSrcBean Data source bean name.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setDataSource(DataSource)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setDataSourceBean(String dataSrcBean) {
+        this.dataSrcBean = dataSrcBean;
+
+        return this;
+    }
+
+    /**
+     * Sets data source. Data source should be fully configured and ready-to-use.
+     *
+     * @param dataSrc Data source.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcBlobStore#setDataSource(DataSource)
+     */
+    public CacheJdbcBlobStoreFactory<K, V> setDataSource(DataSource dataSrc) {
+        this.dataSrc = dataSrc;
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheJdbcBlobStoreFactory.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
index 682f018..da00d95 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
@@ -19,6 +19,7 @@ package org.apache.ignite.cache.store.jdbc;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
@@ -33,6 +34,8 @@ import java.util.*;
  * Implementation of {@link CacheStore} backed by JDBC and POJO via reflection.
  *
  * This implementation stores objects in underlying database using java beans mapping description via reflection.
+ * <p>
+ * Use {@link CacheJdbcPojoStoreFactory} factory to pass {@link CacheJdbcPojoStore} to {@link CacheConfiguration}.
  *
  * @param <K> The type of keys handled by this loader.
  * @param <V> The type of values generated by this loader.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
new file mode 100644
index 0000000..4cb680a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.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.*;
+import org.apache.ignite.cache.store.jdbc.dialect.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.spring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
+
+import javax.cache.configuration.*;
+import javax.sql.*;
+
+/**
+ * {@link Factory} implementation for {@link CacheJdbcPojoStore}.
+ *
+ * Use this factory to pass {@link CacheJdbcPojoStore} to {@link CacheConfiguration}.
+ *
+ * <h2 class="header">Spring Example</h2>
+ * <pre name="code" class="xml">
+ *     &lt;bean id= "simpleDataSource" class="org.h2.jdbcx.JdbcDataSource"/&gt;
+ *
+ *     &lt;bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration"&gt;
+ *          ...
+ *          &lt;property name="cacheConfiguration"&gt;
+ *               &lt;list&gt;
+ *                  &lt;bean class="org.apache.ignite.configuration.CacheConfiguration"&gt;
+ *                      ...
+ *                      &lt;property name="cacheStoreFactory"&gt;
+ *                          &lt;bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory"&gt;
+ *                              &lt;property name="dataSourceBean" value = "simpleDataSource" /&gt;
+ *                          &lt;/bean&gt;
+ *                      &lt;/property&gt;
+ *                  &lt;/bean&gt;
+ *               &lt;/list&gt;
+ *          &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * </pre>
+ * <p>
+ * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <br>
+ * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ */
+public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheJdbcPojoStore<K, V>> {
+    /** Name of data source bean. */
+    private String dataSrcBean;
+
+    /** Data source. */
+    private DataSource dataSrc;
+
+    /** Database dialect. */
+    protected JdbcDialect dialect;
+
+    /** Application context. */
+    @SpringApplicationContextResource
+    private Object appContext;
+
+    /** {@inheritDoc} */
+    @Override public CacheJdbcPojoStore<K, V> create() {
+        CacheJdbcPojoStore<K, V> store = new CacheJdbcPojoStore<>();
+
+        store.setDialect(dialect);
+
+        if (dataSrc != null)
+            store.setDataSource(dataSrc);
+        else if (dataSrcBean != null) {
+            if (appContext == null)
+                throw new IgniteException("Spring application context resource is not injected.");
+
+            IgniteSpringHelper spring;
+
+            try {
+                spring = IgniteComponentType.SPRING.create(false);
+
+                DataSource data = spring.loadBeanFromAppContext(appContext, dataSrcBean);
+
+                store.setDataSource(data);
+            }
+            catch (Exception e) {
+                throw new IgniteException("Failed to load bean in application context [beanName=" + dataSrcBean +
+                    ", igniteConfig=" + appContext + ']');
+            }
+        }
+
+        return store;
+    }
+
+    /**
+     * Sets name of the data source bean.
+     *
+     * @param dataSrcBean Data source bean name.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcPojoStore#setDataSource(DataSource)
+     */
+    public CacheJdbcPojoStoreFactory<K, V> setDataSourceBean(String dataSrcBean) {
+        this.dataSrcBean = dataSrcBean;
+
+        return this;
+    }
+
+    /**
+     * Sets data source. Data source should be fully configured and ready-to-use.
+     *
+     * @param dataSrc Data source.
+     * @return {@code This} for chaining.
+     * @see CacheJdbcPojoStore#setDataSource(DataSource)
+     */
+    public CacheJdbcPojoStoreFactory<K, V> setDataSource(DataSource dataSrc) {
+        this.dataSrc = dataSrc;
+
+        return this;
+    }
+
+    /**
+     * Set database dialect.
+     *
+     * @param dialect Database dialect.
+     * @see CacheJdbcPojoStore#setDialect(JdbcDialect)
+     */
+    public void setDialect(JdbcDialect dialect) {
+        this.dialect = dialect;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheJdbcPojoStoreFactory.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/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 ab7c48d..2f7f22c 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
@@ -1888,6 +1888,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         req.failIfExists(failIfExists);
 
         if (ccfg != null) {
+            try {
+                checkSerializable(ccfg);
+            }
+            catch (IgniteCheckedException e) {
+                return new GridFinishedFuture<>(e);
+            }
+
             if (desc != null && !desc.cancelled()) {
                 if (failIfExists)
                     return new GridFinishedFuture<>(new CacheExistsException("Failed to start cache " +
@@ -2962,6 +2969,17 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (val == null)
             return;
 
+        if (val.getCacheStoreFactory() != null) {
+            try {
+                marshaller.unmarshal(marshaller.marshal(val.getCacheStoreFactory()),
+                    val.getCacheStoreFactory().getClass().getClassLoader());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteCheckedException("Failed to validate cache configuration. " +
+                    "Cache store factory is not serializable. Cache name: " + U.maskName(val.getName()), e);
+            }
+        }
+
         try {
             marshaller.unmarshal(marshaller.marshal(val), val.getClass().getClassLoader());
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/core/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelper.java b/modules/core/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelper.java
index a9e9e93..345bb7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelper.java
@@ -133,6 +133,16 @@ public interface IgniteSpringHelper {
     public <T> T loadBean(InputStream stream, String beanName) throws IgniteCheckedException;
 
     /**
+     * Loads bean instance by name from application context.
+     *
+     * @param appContext Sprint application context.
+     * @param beanName Bean name.
+     * @return Bean instance.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public <T> T loadBeanFromAppContext(Object appContext, String beanName) throws IgniteCheckedException;
+
+    /**
      * Gets user version for given class loader by checking
      * {@code META-INF/ignite.xml} file for {@code userVersion} attribute. If
      * {@code ignite.xml} file is not found, or user version is not specified there,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java
index 77093b5..e6c542c 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java
@@ -19,6 +19,7 @@ package org.apache.ignite.cache.store.hibernate;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -59,92 +60,8 @@ import java.util.concurrent.atomic.*;
  * <p>
  * If hibernate properties are provided, mapping
  * {@code GridCacheHibernateStoreEntry.hbm.xml} is included automatically.
- *
- * <h2 class="header">Java Example</h2>
- * In this example existing session factory is provided.
- * <pre name="code" class="java">
- *     ...
- *     CacheHibernateBlobStore&lt;String, String&gt; store = new CacheHibernateBlobStore&lt;String, String&gt;();
- *
- *     store.setSessionFactory(sesFactory);
- *     ...
- * </pre>
- *
- * <h2 class="header">Spring Example (using Spring ORM)</h2>
- * <pre name="code" class="xml">
- *   ...
- *   &lt;bean id=&quot;cache.hibernate.store&quot;
- *       class=&quot;org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStore&quot;&gt;
- *       &lt;property name=&quot;sessionFactory&quot;&gt;
- *           &lt;bean class=&quot;org.springframework.orm.hibernate3.LocalSessionFactoryBean&quot;&gt;
- *               &lt;property name=&quot;hibernateProperties&quot;&gt;
- *                   &lt;value&gt;
- *                       connection.url=jdbc:h2:mem:
- *                       show_sql=true
- *                       hbm2ddl.auto=true
- *                       hibernate.dialect=org.hibernate.dialect.H2Dialect
- *                   &lt;/value&gt;
- *               &lt;/property&gt;
- *               &lt;property name=&quot;mappingResources&quot;&gt;
- *                   &lt;list&gt;
- *                       &lt;value&gt;
- *                           org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.hbm.xml
- *                       &lt;/value&gt;
- *                   &lt;/list&gt;
- *               &lt;/property&gt;
- *           &lt;/bean&gt;
- *       &lt;/property&gt;
- *   &lt;/bean&gt;
- *   ...
- * </pre>
- *
- * <h2 class="header">Spring Example (using Spring ORM and persistent annotations)</h2>
- * <pre name="code" class="xml">
- *     ...
- *     &lt;bean id=&quot;cache.hibernate.store1&quot;
- *         class=&quot;org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStore&quot;&gt;
- *         &lt;property name=&quot;sessionFactory&quot;&gt;
- *             &lt;bean class=&quot;org.springframework.orm.hibernate3.annotation.AnnotationSessionFactoryBean&quot;&gt;
- *                 &lt;property name=&quot;hibernateProperties&quot;&gt;
- *                     &lt;value&gt;
- *                         connection.url=jdbc:h2:mem:
- *                         show_sql=true
- *                         hbm2ddl.auto=true
- *                         hibernate.dialect=org.hibernate.dialect.H2Dialect
- *                     &lt;/value&gt;
- *                 &lt;/property&gt;
- *                 &lt;property name=&quot;annotatedClasses&quot;&gt;
- *                     &lt;list&gt;
- *                         &lt;value&gt;
- *                             org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreEntry
- *                         &lt;/value&gt;
- *                     &lt;/list&gt;
- *                 &lt;/property&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- *     ...
- * </pre>
- *
- * <h2 class="header">Spring Example</h2>
- * <pre name="code" class="xml">
- *     ...
- *     &lt;bean id=&quot;cache.hibernate.store2&quot;
- *         class=&quot;org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStore&quot;&gt;
- *         &lt;property name=&quot;hibernateProperties&quot;&gt;
- *             &lt;props&gt;
- *                 &lt;prop key=&quot;connection.url&quot;&gt;jdbc:h2:mem:&lt;/prop&gt;
- *                 &lt;prop key=&quot;hbm2ddl.auto&quot;&gt;update&lt;/prop&gt;
- *                 &lt;prop key=&quot;show_sql&quot;&gt;true&lt;/prop&gt;
- *             &lt;/props&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- *     ...
- * </pre>
  * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
- * <br>
- * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ * Use {@link CacheHibernateBlobStoreFactory} factory to pass {@link CacheHibernateBlobStore} to {@link CacheConfiguration}.
  */
 public class CacheHibernateBlobStore<K, V> extends CacheStoreAdapter<K, V> {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
new file mode 100644
index 0000000..6b89a9e
--- /dev/null
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
@@ -0,0 +1,180 @@
+/*
+ * 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.configuration.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.hibernate.*;
+
+import javax.cache.configuration.*;
+import java.util.*;
+
+/**
+ * {@link Factory} implementation for {@link CacheHibernateBlobStore}.
+ *
+ * Use this factory to pass {@link CacheHibernateBlobStore} to {@link CacheConfiguration}.
+ *
+ * <h2 class="header">Java Example</h2>
+ * In this example existing session factory is provided.
+ * <pre name="code" class="java">
+ *     ...
+ *     CacheHibernateBlobStoreFactory&lt;String, String&gt; factory = new CacheHibernateBlobStoreFactory&lt;String, String&gt;();
+ *
+ *     factory.setSessionFactory(sesFactory);
+ *     ...
+ * </pre>
+ *
+ * <h2 class="header">Spring Example (using Spring ORM)</h2>
+ * <pre name="code" class="xml">
+ *   ...
+ *   &lt;bean id=&quot;cache.hibernate.store.factory&quot;
+ *       class=&quot;org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory&quot;&gt;
+ *       &lt;property name=&quot;sessionFactory&quot;&gt;
+ *           &lt;bean class=&quot;org.springframework.orm.hibernate3.LocalSessionFactoryBean&quot;&gt;
+ *               &lt;property name=&quot;hibernateProperties&quot;&gt;
+ *                   &lt;value&gt;
+ *                       connection.url=jdbc:h2:mem:
+ *                       show_sql=true
+ *                       hbm2ddl.auto=true
+ *                       hibernate.dialect=org.hibernate.dialect.H2Dialect
+ *                   &lt;/value&gt;
+ *               &lt;/property&gt;
+ *               &lt;property name=&quot;mappingResources&quot;&gt;
+ *                   &lt;list&gt;
+ *                       &lt;value&gt;
+ *                           org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.hbm.xml
+ *                       &lt;/value&gt;
+ *                   &lt;/list&gt;
+ *               &lt;/property&gt;
+ *           &lt;/bean&gt;
+ *       &lt;/property&gt;
+ *   &lt;/bean&gt;
+ *   ...
+ * </pre>
+ *
+ * <h2 class="header">Spring Example (using Spring ORM and persistent annotations)</h2>
+ * <pre name="code" class="xml">
+ *     ...
+ *     &lt;bean id=&quot;cache.hibernate.store.factory1&quot;
+ *         class=&quot;org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory&quot;&gt;
+ *         &lt;property name=&quot;sessionFactory&quot;&gt;
+ *             &lt;bean class=&quot;org.springframework.orm.hibernate3.annotation.AnnotationSessionFactoryBean&quot;&gt;
+ *                 &lt;property name=&quot;hibernateProperties&quot;&gt;
+ *                     &lt;value&gt;
+ *                         connection.url=jdbc:h2:mem:
+ *                         show_sql=true
+ *                         hbm2ddl.auto=true
+ *                         hibernate.dialect=org.hibernate.dialect.H2Dialect
+ *                     &lt;/value&gt;
+ *                 &lt;/property&gt;
+ *                 &lt;property name=&quot;annotatedClasses&quot;&gt;
+ *                     &lt;list&gt;
+ *                         &lt;value&gt;
+ *                             org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreEntry
+ *                         &lt;/value&gt;
+ *                     &lt;/list&gt;
+ *                 &lt;/property&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ *     ...
+ * </pre>
+ *
+ * <h2 class="header">Spring Example</h2>
+ * <pre name="code" class="xml">
+ *     ...
+ *     &lt;bean id=&quot;cache.hibernate.store.factory2&quot;
+ *         class=&quot;org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory&quot;&gt;
+ *         &lt;property name=&quot;hibernateProperties&quot;&gt;
+ *             &lt;props&gt;
+ *                 &lt;prop key=&quot;connection.url&quot;&gt;jdbc:h2:mem:&lt;/prop&gt;
+ *                 &lt;prop key=&quot;hbm2ddl.auto&quot;&gt;update&lt;/prop&gt;
+ *                 &lt;prop key=&quot;show_sql&quot;&gt;true&lt;/prop&gt;
+ *             &lt;/props&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ *     ...
+ * </pre>
+ * <p>
+ * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <br>
+ * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ */
+public class CacheHibernateBlobStoreFactory<K, V> implements Factory<CacheHibernateBlobStore<K, V>> {
+    /** Session factory. */
+    @GridToStringExclude
+    private SessionFactory sesFactory;
+
+    /** Path to hibernate configuration file. */
+    private String hibernateCfgPath;
+
+    /** Hibernate properties. */
+    @GridToStringExclude
+    private Properties hibernateProps;
+
+
+    /** {@inheritDoc} */
+    @Override public CacheHibernateBlobStore<K, V> create() {
+        CacheHibernateBlobStore<K, V> store = new CacheHibernateBlobStore<>();
+
+        store.setSessionFactory(sesFactory);
+        store.setHibernateConfigurationPath(hibernateCfgPath);
+        store.setHibernateProperties(hibernateProps);
+
+        return store;
+    }
+
+    /**
+     * Sets session factory.
+     *
+     * @param sesFactory Session factory.
+     * @see CacheHibernateBlobStore#setSessionFactory(SessionFactory)
+     */
+    public void setSessionFactory(SessionFactory sesFactory) {
+        this.sesFactory = sesFactory;
+    }
+
+    /**
+     * Sets hibernate configuration path.
+     * <p>
+     * This may be either URL or file path or classpath resource.
+     *
+     * @param hibernateCfgPath URL or file path or classpath resource
+     *      pointing to hibernate configuration XML file.
+     * @see CacheHibernateBlobStore#setHibernateConfigurationPath(String)
+     */
+    public void setHibernateConfigurationPath(String hibernateCfgPath) {
+        this.hibernateCfgPath = hibernateCfgPath;
+    }
+
+    /**
+     * Sets Hibernate properties.
+     *
+     * @param hibernateProps Hibernate properties.
+     * @see CacheHibernateBlobStore#setHibernateProperties(Properties)
+     */
+    public void setHibernateProperties(Properties hibernateProps) {
+        this.hibernateProps = hibernateProps;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheHibernateBlobStoreFactory.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java
new file mode 100644
index 0000000..225fc11
--- /dev/null
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.configuration.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+/**
+ * Test for Cache jdbc blob store factory.
+ */
+public class CacheHibernateStoreFactorySelfTest extends GridCommonAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheConfiguration() throws Exception {
+        try (Ignite ignite1 = startGrid(0)) {
+            IgniteCache<Integer, String> cache1 = ignite1.getOrCreateCache(cacheConfiguration());
+
+            checkStore(cache1);
+        }
+    }
+
+    /**
+     * @return Cache configuration with store.
+     */
+    private CacheConfiguration<Integer, String> cacheConfiguration() {
+        CacheConfiguration<Integer, String> cfg = new CacheConfiguration<>();
+
+        CacheHibernateBlobStoreFactory<Integer, String> factory = new CacheHibernateBlobStoreFactory();
+
+        factory.setHibernateConfigurationPath("/org/apache/ignite/cache/store/hibernate/hibernate.cfg.xml");
+
+        cfg.setCacheStoreFactory(factory);
+
+        return cfg;
+    }
+
+    /**
+     * @param cache Ignite cache.
+     * @throws Exception If store parameters is not the same as in configuration xml.
+     */
+    private void checkStore(IgniteCache<Integer, String> cache) throws Exception {
+        CacheHibernateBlobStore store = (CacheHibernateBlobStore)cache.getConfiguration(CacheConfiguration.class)
+            .getCacheStoreFactory().create();
+
+        assertEquals("/org/apache/ignite/cache/store/hibernate/hibernate.cfg.xml",
+            GridTestUtils.getFieldValue(store, CacheHibernateBlobStore.class, "hibernateCfgPath"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/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 3ac5ec4..2965250 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
@@ -43,6 +43,8 @@ public class IgniteHibernateTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheHibernateStoreSessionListenerSelfTest.class);
 
+        suite.addTestSuite(CacheHibernateStoreFactorySelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/spring/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
deleted file mode 100644
index b33b3b6..0000000
--- a/modules/spring/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
+++ /dev/null
@@ -1,237 +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.configuration.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.resources.*;
-import org.springframework.context.*;
-
-import javax.cache.configuration.*;
-import javax.sql.*;
-
-/**
- * {@link Factory} implementation for {@link CacheJdbcBlobStore}.
- *
- * Use this factory to pass {@link CacheJdbcBlobStore} to {@link CacheConfiguration}.
- *
- * <h2 class="header">Spring Example</h2>
- * <pre name="code" class="xml"> *
- *     &lt;bean id= "simpleDataSource" class="org.h2.jdbcx.JdbcDataSource"/&gt;
- *
- *     &lt;bean id=&quot;cache.jdbc.store&quot;
- *         class=&quot;org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStore&quot;&gt;
- *         &lt;property name=&quot;connectionUrl&quot; value=&quot;jdbc:h2:mem:&quot;/&gt;
- *         &lt;property name=&quot;createTableQuery&quot;
- *             value=&quot;create table if not exists ENTRIES (key other, val other)&quot;/&gt;
- *     &lt;/bean&gt;
- *
- *     &lt;bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration"&gt;
- *          ...
- *          &lt;property name="cacheConfiguration"&gt;
- *               &lt;list&gt;
- *                  &lt;bean class="org.apache.ignite.configuration.CacheConfiguration"&gt;
- *                      ...
- *                      &lt;property name="cacheStoreFactory"&gt;
- *                          &lt;bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory"&gt;
- *                              &lt;property name="user" value = "GridGain" /&gt;
- *                              &lt;property name="dataSourceBean" value = "simpleDataSource" /&gt;
- *                          &lt;/bean&gt;
- *                      &lt;/property&gt;
- *                  &lt;/bean&gt;
- *               &lt;/list&gt;
- *          &lt;/property&gt;
- *     &lt;/bean&gt;
- * </pre>
- * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
- * <br>
- * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
- */
-public class CacheJdbcBlobStoreFactory<K, V> implements Factory<CacheJdbcBlobStore<K, V>> {
-    /** Connection URL. */
-    private String connUrl = CacheJdbcBlobStore.DFLT_CONN_URL;
-
-    /** Query to create table. */
-    private String createTblQry = CacheJdbcBlobStore.DFLT_CREATE_TBL_QRY;
-
-    /** Query to load entry. */
-    private String loadQry = CacheJdbcBlobStore.DFLT_LOAD_QRY;
-
-    /** Query to update entry. */
-    private String updateQry = CacheJdbcBlobStore.DFLT_UPDATE_QRY;
-
-    /** Query to insert entries. */
-    private String insertQry = CacheJdbcBlobStore.DFLT_INSERT_QRY;
-
-    /** Query to delete entries. */
-    private String delQry = CacheJdbcBlobStore.DFLT_DEL_QRY;
-
-    /** User name for database access. */
-    private String user;
-
-    /** Password for database access. */
-    @GridToStringExclude
-    private String passwd;
-
-    /** Flag for schema initialization. */
-    private boolean initSchema = true;
-
-    /** Name of data source bean. */
-    private String dataSrcBean;
-
-    /** Application context. */
-    @SpringApplicationContextResource
-    private ApplicationContext appContext;
-
-    /** {@inheritDoc} */
-    @Override public CacheJdbcBlobStore<K, V> create() {
-        CacheJdbcBlobStore<K, V> store = new CacheJdbcBlobStore<>();
-
-        store.setInitSchema(initSchema);
-        store.setConnectionUrl(connUrl);
-        store.setCreateTableQuery(createTblQry);
-        store.setLoadQuery(loadQry);
-        store.setUpdateQuery(updateQry);
-        store.setInsertQuery(insertQry);
-        store.setDeleteQuery(delQry);
-        store.setUser(user);
-        store.setPassword(passwd);
-
-        if (dataSrcBean != null) {
-            if (appContext == null)
-                throw new IgniteException("Spring application context resource is not injected.");
-
-            if (!appContext.containsBean(dataSrcBean))
-                throw new IgniteException("Failed to find bean in application context [beanName=" + dataSrcBean +
-                    ", appContext=" + appContext + ']');
-
-            DataSource data = (DataSource)appContext.getBean(dataSrcBean);
-
-            store.setDataSource(data);
-        }
-
-        return store;
-    }
-
-    /**
-     * Flag indicating whether DB schema should be initialized by Ignite (default behaviour) or
-     * was explicitly created by user.
-     *
-     * @param initSchema Initialized schema flag.
-     * @return {@code This} for chaining.
-     * @see CacheJdbcBlobStore#setInitSchema(boolean)
-     */
-    public CacheJdbcBlobStoreFactory setInitSchema(boolean initSchema) {
-        this.initSchema = initSchema;
-
-        return this;
-    }
-
-    /**
-     * Sets connection URL.
-     *
-     * @param connUrl Connection URL.
-     * @see CacheJdbcBlobStore#setConnectionUrl(String)
-     */
-    public void setConnectionUrl(String connUrl) {
-        this.connUrl = connUrl;
-    }
-
-    /**
-     * See {@link CacheJdbcBlobStore#setCreateTableQuery(String)}.
-     *
-     * @param createTblQry Create table query.
-     */
-    public void setCreateTableQuery(String createTblQry) {
-        this.createTblQry = createTblQry;
-    }
-
-    /**
-     * See {@link CacheJdbcBlobStore#setLoadQuery(String)}.
-     *
-     * @param loadQry Load query
-     */
-    public void setLoadQuery(String loadQry) {
-        this.loadQry = loadQry;
-    }
-
-    /**
-     * See {@link CacheJdbcBlobStore#setUpdateQuery(String)}.
-     *
-     * @param updateQry Update entry query.
-     */
-    public void setUpdateQuery(String updateQry) {
-        this.updateQry = updateQry;
-    }
-
-    /**
-     * See {@link CacheJdbcBlobStore#setInsertQuery(String)}.
-     *
-     * @param insertQry Insert entry query.
-     */
-    public void setInsertQuery(String insertQry) {
-        this.insertQry = insertQry;
-    }
-
-    /**
-     * See {@link CacheJdbcBlobStore#setDeleteQuery(String)}.
-     *
-     * @param delQry Delete entry query.
-     */
-    public void setDeleteQuery(String delQry) {
-        this.delQry = delQry;
-    }
-
-    /**
-     * See {@link CacheJdbcBlobStore#setUser(String)}.
-     *
-     * @param user User name.
-     */
-    public void setUser(String user) {
-        this.user = user;
-    }
-
-    /**
-     * See {@link CacheJdbcBlobStore#setPassword(String)}.
-     *
-     * @param passwd Password.
-     */
-    public void setPassword(String passwd) {
-        this.passwd = passwd;
-    }
-
-    /**
-     * Sets name of the data source bean.
-     *
-     * See {@link CacheJdbcBlobStore#setDataSource(DataSource)}
-     * for more information.
-     *
-     * @param dataSrcBean Data source bean name.
-     */
-    public void setDataSourceBean(String dataSrcBean) {
-        this.dataSrcBean = dataSrcBean;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(CacheJdbcBlobStoreFactory.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java b/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java
index 2c7c7e1..cbb200d 100644
--- a/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java
+++ b/modules/spring/src/main/java/org/apache/ignite/internal/util/spring/IgniteSpringHelperImpl.java
@@ -192,6 +192,23 @@ public class IgniteSpringHelperImpl implements IgniteSpringHelper {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public <T> T loadBeanFromAppContext(Object appContext, String beanName) throws IgniteCheckedException {
+        ApplicationContext springCtx = (ApplicationContext)appContext;
+
+        try {
+            return (T)springCtx.getBean(beanName);
+        }
+        catch (NoSuchBeanDefinitionException e) {
+            throw new IgniteCheckedException("Spring bean with provided name doesn't exist " +
+                    ", beanName=" + beanName + ']');
+        }
+        catch (BeansException e) {
+            throw new IgniteCheckedException("Failed to load Spring bean with provided name " +
+                    ", beanName=" + beanName + ']', e);
+        }
+    }
+
     /**
      * @param stream Input stream containing Spring XML configuration.
      * @return Context.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/spring/src/test/config/incorrect-store-cache.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/incorrect-store-cache.xml b/modules/spring/src/test/config/incorrect-store-cache.xml
index 9a0b7c6..2ccda21 100644
--- a/modules/spring/src/test/config/incorrect-store-cache.xml
+++ b/modules/spring/src/test/config/incorrect-store-cache.xml
@@ -43,7 +43,7 @@
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="ipFinder">
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
                         <property name="addresses">
                             <list>
                                 <value>127.0.0.1:47500..47509</value>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/spring/src/test/config/pojo-incorrect-store-cache.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/pojo-incorrect-store-cache.xml b/modules/spring/src/test/config/pojo-incorrect-store-cache.xml
new file mode 100644
index 0000000..5627bd0
--- /dev/null
+++ b/modules/spring/src/test/config/pojo-incorrect-store-cache.xml
@@ -0,0 +1,56 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="test"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory">
+                            <property name="dataSourceBean" value = "simpleDataSource"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/spring/src/test/config/store-cache.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/store-cache.xml b/modules/spring/src/test/config/store-cache.xml
index 992a22c..07a4ab0 100644
--- a/modules/spring/src/test/config/store-cache.xml
+++ b/modules/spring/src/test/config/store-cache.xml
@@ -45,7 +45,7 @@
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="ipFinder">
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
                         <property name="addresses">
                             <list>
                                 <value>127.0.0.1:47500..47509</value>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/spring/src/test/config/store-cache1.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/store-cache1.xml b/modules/spring/src/test/config/store-cache1.xml
index 9209e52..bff88e9 100644
--- a/modules/spring/src/test/config/store-cache1.xml
+++ b/modules/spring/src/test/config/store-cache1.xml
@@ -48,7 +48,7 @@
         <property name="discoverySpi">
             <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
                 <property name="ipFinder">
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
                         <property name="addresses">
                             <list>
                                 <value>127.0.0.1:47500..47509</value>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java
index 4a693a7..6d3d3fd 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java
@@ -78,7 +78,7 @@ public class CacheJdbcBlobStoreFactorySelfTest extends GridCommonAbstractTest {
                 }
                 return null;
             }
-        }, IgniteException.class, "Cannot find bean in application context.");
+        }, IgniteException.class, "Failed to load bean in application context");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
new file mode 100644
index 0000000..7b709d5
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
@@ -0,0 +1,194 @@
+/*
+ * 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.jdbc.dialect.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.h2.jdbcx.*;
+import sun.jdbc.odbc.ee.*;
+
+import javax.cache.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Test for Cache jdbc blob store factory.
+ */
+public class CacheJdbcPojoStoreFactorySelfTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "test";
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheConfiguration() throws Exception {
+        try (Ignite ignite = Ignition.start("modules/spring/src/test/config/node.xml")) {
+            try (Ignite ignite1 = Ignition.start("modules/spring/src/test/config/node1.xml")) {
+                try (IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheConfiguration())) {
+                    try (IgniteCache<Integer, String> cache1 = ignite1.getOrCreateCache(cacheConfiguration())) {
+                        checkStore(cache, JdbcDataSource.class);
+
+                        checkStore(cache1, ConnectionPoolDataSource.class);
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSerializable() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try (Ignite ignite = Ignition.start("modules/spring/src/test/config/node.xml")) {
+                    try (IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheConfigurationH2Dialect())) {
+                        checkStore(cache, JdbcDataSource.class);
+                    }
+                }
+
+                return null;
+            }
+        }, CacheException.class, "Failed to validate cache configuration. Cache store factory is not serializable.");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIncorrectBeanConfiguration() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try(Ignite ignite = Ignition.start("modules/spring/src/test/config/pojo-incorrect-store-cache.xml")) {
+                    ignite.cache(CACHE_NAME).getConfiguration(CacheConfiguration.class).
+                        getCacheStoreFactory().create();
+                }
+                return null;
+            }
+        }, IgniteException.class, "Failed to load bean in application context");
+    }
+
+    /**
+     * @return Cache configuration with store.
+     */
+    private CacheConfiguration<Integer, String> cacheConfiguration() {
+        CacheConfiguration<Integer, String> cfg = new CacheConfiguration<>();
+
+        CacheJdbcPojoStoreFactory<Integer, String> factory = new CacheJdbcPojoStoreFactory<>();
+
+        factory.setDataSourceBean("simpleDataSource");
+
+        factory.setDialect(new DummyDialect());
+
+        cfg.setCacheStoreFactory(factory);
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache configuration with store.
+     */
+    private CacheConfiguration<Integer, String> cacheConfigurationH2Dialect() {
+        CacheConfiguration<Integer, String> cfg = new CacheConfiguration<>();
+
+        CacheJdbcPojoStoreFactory<Integer, String> factory = new CacheJdbcPojoStoreFactory<>();
+
+        factory.setDataSourceBean("simpleDataSource");
+
+        factory.setDialect(new H2Dialect());
+
+        cfg.setCacheStoreFactory(factory);
+
+        return cfg;
+    }
+
+    /**
+     * @param cache Ignite cache.
+     * @param dataSrcClass Data source class.
+     * @throws Exception If store parameters is not the same as in configuration xml.
+     */
+    private void checkStore(IgniteCache<Integer, String> cache, Class<?> dataSrcClass) throws Exception {
+        CacheJdbcPojoStore store = (CacheJdbcPojoStore)cache.getConfiguration(CacheConfiguration.class).
+            getCacheStoreFactory().create();
+
+        assertEquals(dataSrcClass,
+            GridTestUtils.getFieldValue(store, CacheAbstractJdbcStore.class, "dataSrc").getClass());
+    }
+
+    /**
+     *
+     */
+    public static class DummyDialect implements JdbcDialect, Serializable {
+        /** {@inheritDoc} */
+        @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection<String> keyCols) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String loadCacheRangeQuery(String fullTblName, Collection<String> keyCols,
+            Iterable<String> uniqCols, boolean appendLowerBound, boolean appendUpperBound) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String loadCacheQuery(String fullTblName, Iterable<String> uniqCols) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String loadQuery(String fullTblName, Collection<String> keyCols, Iterable<String> cols,
+            int keyCnt) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String insertQuery(String fullTblName, Collection<String> keyCols,
+            Collection<String> valCols) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String updateQuery(String fullTblName, Collection<String> keyCols, Iterable<String> valCols) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasMerge() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String mergeQuery(String fullTblName, Collection<String> keyCols,
+            Collection<String> uniqCols) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String removeQuery(String fullTblName, Iterable<String> keyCols) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int getMaxParameterCount() {
+            return 0;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a7ca45bb/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 12dd494..0c2e99e 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.jdbc.*;
 import org.apache.ignite.cache.store.spring.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.p2p.*;
@@ -50,6 +51,10 @@ public class IgniteSpringTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheSpringStoreSessionListenerSelfTest.class);
 
+        suite.addTestSuite(CacheJdbcBlobStoreFactorySelfTest.class);
+
+        suite.addTestSuite(CacheJdbcPojoStoreFactorySelfTest.class);
+
         return suite;
     }
 }