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

[1/4] incubator-ignite git commit: IGNITE-884 - Spring autowiring

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-884 [created] d41af24d2


IGNITE-884 - Spring autowiring


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

Branch: refs/heads/ignite-884
Commit: c3459a2dfd31de86b10aa8ab5abcc0ce5588e9d5
Parents: 40f826b
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 12 22:08:21 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 12 22:08:21 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/CacheStartContext.java     | 104 +++++++++++
 .../processors/cache/GridCacheProcessor.java    |  17 +-
 .../resource/GridResourceProcessor.java         |   8 +
 .../resource/GridSpringResourceContext.java     |   6 +
 .../resource/GridSpringResourceContextImpl.java |  15 +-
 .../spring/autowired/SpringAutowiredBean.java   |  40 +++++
 .../autowired/SpringAutowiredSelfTest.java      | 180 +++++++++++++++++++
 .../autowired/SpringAutowiredTestService.java   |  72 ++++++++
 .../autowired/SpringAutowiredTestStore.java     |  76 ++++++++
 .../spring/autowired/autowired-service.xml      |  64 +++++++
 .../ignite/spring/autowired/autowired.xml       |  36 ++++
 11 files changed, 613 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
new file mode 100644
index 0000000..f305a47
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
@@ -0,0 +1,104 @@
+/*
+ * 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.store.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+
+import javax.cache.configuration.*;
+import javax.cache.expiry.*;
+import java.util.*;
+
+/**
+ * Cache start context.
+ */
+public class CacheStartContext {
+    /** */
+    private CacheStore store;
+
+    /** */
+    private ExpiryPolicy expPlc;
+
+    /** */
+    private CacheStoreSessionListener[] sesLsnrs;
+
+    /**
+     * @param cfg Configuration.
+     */
+    public CacheStartContext(GridKernalContext ctx, CacheConfiguration<?, ?> cfg) {
+        assert ctx != null;
+        assert cfg != null;
+
+        store = create(ctx, cfg.getCacheStoreFactory());
+        expPlc = create(ctx, cfg.getExpiryPolicyFactory());
+        //sesLsnrs = create(ctx, cfg.getCacheStoreSessionListenerFactories());
+    }
+
+    /**
+     * @return Cache store.
+     */
+    public CacheStore store() {
+        return store;
+    }
+
+    /**
+     * @return Expiry policy.
+     */
+    public ExpiryPolicy expiryPolicy() {
+        return expPlc;
+    }
+
+    /**
+     * @return Store session listeners.
+     */
+    public CacheStoreSessionListener[] storeSessionListeners() {
+        return sesLsnrs;
+    }
+
+    /**
+     * @param ctx Context.
+     * @param factory Factory.
+     * @return Object.
+     */
+    private <T> T create(GridKernalContext ctx, Factory<T> factory) {
+        T obj = factory != null ? factory.create() : null;
+
+        ctx.resource().autowireSpringBean(obj);
+
+        return obj;
+    }
+
+    /**
+     * @param ctx Context.
+     * @param factories Factories.
+     * @return Objects.
+     */
+    private <T> T[] create(GridKernalContext ctx, Factory<T>[] factories) {
+        Collection<T> col = new ArrayList<>(factories.length);
+
+        for (Factory<T> factory : factories) {
+            T obj = create(ctx, factory);
+
+            if (obj != null)
+                col.add(obj);
+        }
+
+        return (T[])col.toArray();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/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 4fdec33..b20d32c 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
@@ -57,6 +57,7 @@ import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.spi.*;
 import org.jetbrains.annotations.*;
+import org.jsr166.*;
 
 import javax.cache.configuration.*;
 import javax.cache.integration.*;
@@ -118,6 +119,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Cache templates. */
     private ConcurrentMap<String, DynamicCacheDescriptor> registeredTemplates = new ConcurrentHashMap<>();
 
+    /** Start contexts. */
+    private ConcurrentMap<String, CacheStartContext> startCtxs = new ConcurrentHashMap8<>();
+
     /** */
     private IdentityHashMap<CacheStore, ThreadLocal> sesHolders = new IdentityHashMap<>();
 
@@ -222,7 +226,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
 
         if (cfg.getCacheStoreFactory() == null) {
-            Factory<CacheLoader> ldrFactory = cfg.getCacheLoaderFactory();
+            Factory<CacheLoader> ldrFactory = cfg.isReadThrough() ? cfg.getCacheLoaderFactory() : null;
             Factory<CacheWriter> writerFactory = cfg.isWriteThrough() ? cfg.getCacheWriterFactory() : null;
 
             if (ldrFactory != null || writerFactory != null)
@@ -1072,7 +1076,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     {
         assert cfg != null;
 
-        CacheStore cfgStore = cfg.getCacheStoreFactory() != null ? cfg.getCacheStoreFactory().create() : null;
+        CacheStartContext startCtx = startCtxs.remove(maskNull(cfg.getName()));
+
+        if (startCtx == null)
+            startCtx = new CacheStartContext(ctx, cfg);
+
+        CacheStore cfgStore = startCtx.store();
 
         validate(ctx.config(), cfg, cacheType, cfgStore);
 
@@ -1958,6 +1967,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         req.cacheType(cacheType);
 
+        startCtxs.putIfAbsent(maskNull(cacheName), new CacheStartContext(ctx, req.startCacheConfiguration()));
+
         return F.first(initiateCacheChanges(F.asList(req), failIfExists));
     }
 
@@ -3040,6 +3051,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
             // Make sure to remove future before completion.
             pendingFuts.remove(maskNull(cacheName), this);
+            startCtxs.remove(maskNull(cacheName));
 
             return super.onDone(res, err);
         }
@@ -3143,4 +3155,3 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
     }
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
index f5ba492..faa7142 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
@@ -556,6 +556,14 @@ public class GridResourceProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param bean Spring bean.
+     */
+    public void autowireSpringBean(@Nullable Object bean) {
+        if (rsrcCtx != null)
+            rsrcCtx.autowireBean(bean);
+    }
+
+    /**
      * Returns GridResourceIoc object. For tests only!!!
      *
      * @return GridResourceIoc object.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java
index f0d17ec..dfce905 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.resource;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
+import org.jetbrains.annotations.*;
 
 /**
  * Interface was introduced to avoid compile-time dependency on spring framework. Spring resource context
@@ -44,5 +45,10 @@ public interface GridSpringResourceContext {
      * @throws IgniteCheckedException If unwrap failed.
      */
     public Object unwrapTarget(Object target) throws IgniteCheckedException;
+
+    /**
+     * @param bean Spring bean.
+     */
+    public void autowireBean(@Nullable Object bean);
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/spring/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContextImpl.java b/modules/spring/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContextImpl.java
index c49d891..024dd2f 100644
--- a/modules/spring/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContextImpl.java
+++ b/modules/spring/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContextImpl.java
@@ -26,16 +26,21 @@ import org.springframework.context.*;
  * Implementation of {@link GridSpringResourceContext}.
  */
 public class GridSpringResourceContextImpl implements GridSpringResourceContext {
+    /** Spring context. */
+    private final ApplicationContext springCtx;
+
     /** Spring application context injector. */
-    private GridResourceInjector springCtxInjector;
+    private final GridResourceInjector springCtxInjector;
 
     /** Spring bean resources injector. */
-    private GridResourceInjector springBeanInjector;
+    private final GridResourceInjector springBeanInjector;
 
     /**
      * @param springCtx Spring application context.
      */
     public GridSpringResourceContextImpl(@Nullable ApplicationContext springCtx) {
+        this.springCtx = springCtx;
+
         springCtxInjector = new GridResourceBasicInjector<>(springCtx);
         springBeanInjector = new GridResourceSpringBeanInjector(springCtx);
     }
@@ -64,4 +69,10 @@ public class GridSpringResourceContextImpl implements GridSpringResourceContext
 
         return target;
     }
+
+    /** {@inheritDoc} */
+    @Override public void autowireBean(@Nullable Object bean) {
+        if (springCtx != null && bean != null)
+            springCtx.getAutowireCapableBeanFactory().autowireBean(bean);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredBean.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredBean.java b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredBean.java
new file mode 100644
index 0000000..b7d85c0
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredBean.java
@@ -0,0 +1,40 @@
+/*
+ * 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.spring.autowired;
+
+/**
+ * Autowired bean.
+ */
+public class SpringAutowiredBean {
+    /** */
+    private String name;
+
+    /**
+     * @param name Name.
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return Name.
+     */
+    public String getName() {
+        return name;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredSelfTest.java
new file mode 100644
index 0000000..43c45ef
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredSelfTest.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.spring.autowired;
+
+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.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.springframework.context.*;
+import org.springframework.context.support.*;
+import org.springframework.core.io.*;
+
+import javax.cache.configuration.*;
+import java.net.*;
+import java.util.*;
+
+/**
+ * Spring autowiring test.
+ */
+public class SpringAutowiredSelfTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CACHE_NAME = SpringAutowiredSelfTest.class.getSimpleName();
+
+    /** */
+    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder();
+
+    static {
+        IP_FINDER.setAddresses(Arrays.asList("127.0.0.1:47500"));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        SpringAutowiredTestStore.load.clear();
+        SpringAutowiredTestStore.write.clear();
+        SpringAutowiredTestStore.delete.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStoreAtomic() throws Exception {
+        testStore(CacheAtomicityMode.ATOMIC);
+
+        assertEquals(3, SpringAutowiredTestStore.load.size());
+        assertEquals(3, SpringAutowiredTestStore.write.size());
+        assertEquals(3, SpringAutowiredTestStore.delete.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStoreTransactional() throws Exception {
+        testStore(CacheAtomicityMode.TRANSACTIONAL);
+
+        assertEquals(3, SpringAutowiredTestStore.load.size());
+        assertEquals(1, SpringAutowiredTestStore.write.size());
+        assertEquals(1, SpringAutowiredTestStore.delete.size());
+    }
+
+    /**
+     * @param mode Atomicity mode.
+     * @throws Exception In case of error.
+     */
+    private void testStore(CacheAtomicityMode mode) throws Exception {
+        ApplicationContext appCtx = new ClassPathXmlApplicationContext(
+            "org/apache/ignite/spring/autowired/autowired.xml");
+
+        for (int i = 0; i < 3; i++)
+            IgniteSpring.start(configuration("server-" + i), appCtx);
+
+        Ignition.setClientMode(true);
+
+        Ignite client = IgniteSpring.start(configuration("client"), appCtx);
+
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
+
+        ccfg.setName(CACHE_NAME);
+        ccfg.setAtomicityMode(mode);
+        ccfg.setCacheStoreFactory(FactoryBuilder.factoryOf(SpringAutowiredTestStore.class));
+        ccfg.setReadThrough(true);
+        ccfg.setWriteThrough(true);
+
+        IgniteCache<Integer, Integer> cache = client.createCache(ccfg);
+
+        for (int i = 0; i < 100; i++)
+            cache.get(i);
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        for (int i = 0; i < 100; i++)
+            cache.remove(i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServiceAtomic() throws Exception {
+        ApplicationContext appCtx = new ClassPathXmlApplicationContext(
+            "org/apache/ignite/spring/autowired/autowired.xml");
+
+        for (int i = 0; i < 3; i++)
+            IgniteSpring.start(configuration("server-" + i), appCtx);
+
+        SpringAutowiredTestService.mode = CacheAtomicityMode.ATOMIC;
+
+        URL url = new ClassPathResource("org/apache/ignite/spring/autowired/autowired-service.xml").getURL();
+
+        SpringAutowiredTestService svc = Ignition.loadSpringBean(url, "test-service");
+
+        svc.run();
+
+        assertEquals(3, SpringAutowiredTestStore.load.size());
+        assertEquals(3, SpringAutowiredTestStore.write.size());
+        assertEquals(3, SpringAutowiredTestStore.delete.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServiceTransactional() throws Exception {
+        ApplicationContext appCtx = new ClassPathXmlApplicationContext(
+            "org/apache/ignite/spring/autowired/autowired.xml");
+
+        for (int i = 0; i < 3; i++)
+            IgniteSpring.start(configuration("server-" + i), appCtx);
+
+        SpringAutowiredTestService.mode = CacheAtomicityMode.TRANSACTIONAL;
+
+        URL url = new ClassPathResource("org/apache/ignite/spring/autowired/autowired-service.xml").getURL();
+
+        SpringAutowiredTestService svc = Ignition.loadSpringBean(url, "test-service");
+
+        svc.run();
+
+        assertEquals(3, SpringAutowiredTestStore.load.size());
+        assertEquals(1, SpringAutowiredTestStore.write.size());
+        assertEquals(1, SpringAutowiredTestStore.delete.size());
+    }
+
+    /**
+     * @param name Name.
+     * @return Configuration.
+     */
+    private IgniteConfiguration configuration(String name) {
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setGridName(name);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredTestService.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredTestService.java b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredTestService.java
new file mode 100644
index 0000000..e011cae
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredTestService.java
@@ -0,0 +1,72 @@
+/*
+ * 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.spring.autowired;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.springframework.beans.factory.*;
+
+import javax.cache.configuration.*;
+
+/**
+ * Test service.
+ */
+public class SpringAutowiredTestService implements InitializingBean {
+    /** */
+    public static CacheAtomicityMode mode;
+
+    /** */
+    private Ignite ignite;
+
+    /** */
+    private IgniteCache<Integer, Integer> cache;
+
+    /**
+     * @param ignite Ignite.
+     */
+    public void setIgnite(Ignite ignite) {
+        this.ignite = ignite;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void afterPropertiesSet() throws Exception {
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
+
+        ccfg.setName(SpringAutowiredSelfTest.CACHE_NAME);
+        ccfg.setAtomicityMode(mode);
+        ccfg.setCacheStoreFactory(FactoryBuilder.factoryOf(SpringAutowiredTestStore.class));
+        ccfg.setReadThrough(true);
+        ccfg.setWriteThrough(true);
+
+        cache = ignite.createCache(ccfg);
+    }
+
+    /**
+     */
+    public void run() {
+        for (int i = 0; i < 100; i++)
+            cache.get(i);
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        for (int i = 0; i < 100; i++)
+            cache.remove(i);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredTestStore.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredTestStore.java b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredTestStore.java
new file mode 100644
index 0000000..d51c767
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/SpringAutowiredTestStore.java
@@ -0,0 +1,76 @@
+/*
+ * 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.spring.autowired;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.resources.*;
+import org.springframework.beans.factory.annotation.*;
+
+import javax.cache.*;
+import javax.cache.integration.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Test store.
+ */
+public class SpringAutowiredTestStore extends CacheStoreAdapter<Integer, Integer> {
+    /** */
+    public static final Set<UUID> load = new ConcurrentSkipListSet<>();
+
+    /** */
+    public static final Set<UUID> write = new ConcurrentSkipListSet<>();
+
+    /** */
+    public static final Set<UUID> delete = new ConcurrentSkipListSet<>();
+
+    /** */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** */
+    @Autowired
+    private SpringAutowiredBean bean;
+
+    /** {@inheritDoc} */
+    @Override public Integer load(Integer key) throws CacheLoaderException {
+        load.add(ignite.cluster().localNode().id());
+
+        assert bean != null;
+        assert "test-bean".equals(bean.getName()) : bean.getName();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry) throws CacheWriterException {
+        write.add(ignite.cluster().localNode().id());
+
+        assert bean != null;
+        assert "test-bean".equals(bean.getName()) : bean.getName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void delete(Object key) throws CacheWriterException {
+        delete.add(ignite.cluster().localNode().id());
+
+        assert bean != null;
+        assert "test-bean".equals(bean.getName()) : bean.getName();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/spring/src/test/java/org/apache/ignite/spring/autowired/autowired-service.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/autowired/autowired-service.xml b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/autowired-service.xml
new file mode 100644
index 0000000..7062a2e
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/autowired-service.xml
@@ -0,0 +1,64 @@
+<?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.
+-->
+
+<!--
+    Ignite configuration with all defaults and enabled p2p deployment and enabled events.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:context="http://www.springframework.org/schema/context"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/context
+        http://www.springframework.org/schema/context/spring-context-2.5.xsd">
+    <context:annotation-config/>
+
+    <bean class="org.apache.ignite.spring.autowired.SpringAutowiredBean">
+        <property name="name" value="test-bean"/>
+    </bean>
+
+    <bean id="ignite" class="org.apache.ignite.IgniteSpringBean">
+        <property name="configuration">
+            <bean class="org.apache.ignite.configuration.IgniteConfiguration">
+                <property name="gridName" value="client"/>
+
+                <property name="clientMode" value="true"/>
+
+                <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</value>
+                                    </list>
+                                </property>
+                            </bean>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+
+    <bean id="test-service" class="org.apache.ignite.spring.autowired.SpringAutowiredTestService">
+        <property name="ignite" ref="ignite"/>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c3459a2d/modules/spring/src/test/java/org/apache/ignite/spring/autowired/autowired.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/spring/autowired/autowired.xml b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/autowired.xml
new file mode 100644
index 0000000..6bae746
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/spring/autowired/autowired.xml
@@ -0,0 +1,36 @@
+<?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.
+-->
+
+<!--
+    Ignite configuration with all defaults and enabled p2p deployment and enabled events.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:context="http://www.springframework.org/schema/context"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/context
+        http://www.springframework.org/schema/context/spring-context-2.5.xsd">
+    <context:annotation-config/>
+
+    <bean class="org.apache.ignite.spring.autowired.SpringAutowiredBean">
+        <property name="name" value="test-bean"/>
+    </bean>
+</beans>


[2/4] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-884

Posted by vk...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-884


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

Branch: refs/heads/ignite-884
Commit: db8eb5582bec4d5bdc853f5d49c121321df992f8
Parents: c3459a2 0907338
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon Jun 15 14:54:06 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon Jun 15 14:54:06 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/cache/query/ScanQuery.java  | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[4/4] incubator-ignite git commit: IGNITE-884 - Spring autowiring

Posted by vk...@apache.org.
IGNITE-884 - Spring autowiring


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

Branch: refs/heads/ignite-884
Commit: d41af24d2ff94a66325b40e6ea268871264caf61
Parents: 8c0cc4f
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon Jun 15 20:04:12 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon Jun 15 20:04:12 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheUtils.java | 11 +++++++++--
 1 file changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d41af24d/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 fda05ee..d7a6c31 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
@@ -1686,7 +1686,9 @@ public class GridCacheUtils {
      * @param factory Factory.
      * @return Object.
      */
-    public static <T> T create(GridKernalContext ctx, Factory<T> factory) {
+    @Nullable public static <T> T create(GridKernalContext ctx, @Nullable Factory<T> factory) {
+        assert ctx != null;
+
         T obj = factory != null ? factory.create() : null;
 
         ctx.resource().autowireSpringBean(obj);
@@ -1699,7 +1701,12 @@ public class GridCacheUtils {
      * @param factories Factories.
      * @return Objects.
      */
-    public static <T> Collection<T> create(GridKernalContext ctx, Factory[] factories) {
+    @Nullable public static <T> Collection<T> create(GridKernalContext ctx, @Nullable Factory[] factories) {
+        assert ctx != null;
+
+        if (factories == null)
+            return null;
+
         Collection<T> col = new ArrayList<>(factories.length);
 
         for (Factory<T> factory : factories) {


[3/4] incubator-ignite git commit: IGNITE-884 - Spring autowiring

Posted by vk...@apache.org.
IGNITE-884 - Spring autowiring


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

Branch: refs/heads/ignite-884
Commit: 8c0cc4f14294e43ffe1be172f56e5d06fe587c31
Parents: db8eb55
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Mon Jun 15 19:42:18 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon Jun 15 19:42:18 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/CacheStartContext.java     | 43 +++-------------
 .../processors/cache/GridCacheContext.java      | 21 ++++++--
 .../GridCacheLoaderWriterStoreFactory.java      | 17 ++++---
 .../processors/cache/GridCacheProcessor.java    | 12 +++--
 .../processors/cache/GridCacheUtils.java        | 53 ++++++++++++++------
 .../store/GridCacheStoreManagerAdapter.java     |  6 ++-
 .../optimized/OptimizedClassDescriptor.java     |  2 +-
 .../loadtests/hashmap/GridCacheTestContext.java |  1 +
 8 files changed, 85 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
index f305a47..3e69d5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStartContext.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 
-import javax.cache.configuration.*;
 import javax.cache.expiry.*;
 import java.util.*;
 
@@ -36,7 +36,7 @@ public class CacheStartContext {
     private ExpiryPolicy expPlc;
 
     /** */
-    private CacheStoreSessionListener[] sesLsnrs;
+    private Collection<CacheStoreSessionListener> sesLsnrs;
 
     /**
      * @param cfg Configuration.
@@ -45,9 +45,9 @@ public class CacheStartContext {
         assert ctx != null;
         assert cfg != null;
 
-        store = create(ctx, cfg.getCacheStoreFactory());
-        expPlc = create(ctx, cfg.getExpiryPolicyFactory());
-        //sesLsnrs = create(ctx, cfg.getCacheStoreSessionListenerFactories());
+        store = CU.create(ctx, cfg.getCacheStoreFactory());
+        expPlc = CU.create(ctx, cfg.getExpiryPolicyFactory());
+        sesLsnrs = CU.create(ctx, cfg.getCacheStoreSessionListenerFactories());
     }
 
     /**
@@ -67,38 +67,7 @@ public class CacheStartContext {
     /**
      * @return Store session listeners.
      */
-    public CacheStoreSessionListener[] storeSessionListeners() {
+    public Collection<CacheStoreSessionListener> storeSessionListeners() {
         return sesLsnrs;
     }
-
-    /**
-     * @param ctx Context.
-     * @param factory Factory.
-     * @return Object.
-     */
-    private <T> T create(GridKernalContext ctx, Factory<T> factory) {
-        T obj = factory != null ? factory.create() : null;
-
-        ctx.resource().autowireSpringBean(obj);
-
-        return obj;
-    }
-
-    /**
-     * @param ctx Context.
-     * @param factories Factories.
-     * @return Objects.
-     */
-    private <T> T[] create(GridKernalContext ctx, Factory<T>[] factories) {
-        Collection<T> col = new ArrayList<>(factories.length);
-
-        for (Factory<T> factory : factories) {
-            T obj = create(ctx, factory);
-
-            if (obj != null)
-                col.add(obj);
-        }
-
-        return (T[])col.toArray();
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 8a4e3b9..f18f62c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.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.*;
@@ -54,12 +55,11 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
-import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.plugin.security.SecurityException;
+import org.apache.ignite.plugin.security.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
-import javax.cache.configuration.*;
 import javax.cache.expiry.*;
 import java.io.*;
 import java.util.*;
@@ -199,6 +199,9 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Updates allowed flag. */
     private boolean updatesAllowed;
 
+    /** Session listeners. */
+    private Collection<CacheStoreSessionListener> sesLsnrs;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -231,6 +234,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     public GridCacheContext(
         GridKernalContext ctx,
         GridCacheSharedContext sharedCtx,
+        CacheStartContext startCtx,
         CacheConfiguration cacheCfg,
         CacheType cacheType,
         boolean affNode,
@@ -310,13 +314,13 @@ public class GridCacheContext<K, V> implements Externalizable {
 
         plc = cacheType.ioPolicy();
 
-        Factory<ExpiryPolicy> factory = cacheCfg.getExpiryPolicyFactory();
-
-        expiryPlc = factory != null ? factory.create() : null;
+        expiryPlc = startCtx.expiryPolicy();
 
         if (expiryPlc instanceof EternalExpiryPolicy)
             expiryPlc = null;
 
+        sesLsnrs = startCtx.storeSessionListeners();
+
         itHolder = new CacheWeakQueryIteratorsHolder(log);
     }
 
@@ -1029,6 +1033,13 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return Store session listeners.
+     */
+    public Collection<CacheStoreSessionListener> storeSessionListeners() {
+        return sesLsnrs;
+    }
+
+    /**
      * @param p Predicate.
      * @return {@code True} if given predicate is filter for {@code putIfAbsent} operation.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
index 3e27894..82a534c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheLoaderWriterStoreFactory.java
@@ -5,9 +5,9 @@
  * 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.
@@ -18,6 +18,8 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.configuration.*;
@@ -31,6 +33,9 @@ class GridCacheLoaderWriterStoreFactory<K, V> implements Factory<CacheStore<K, V
     private static final long serialVersionUID = 0L;
 
     /** */
+    private final GridKernalContext cctx;
+
+    /** */
     private final Factory<CacheLoader<K, V>> ldrFactory;
 
     /** */
@@ -40,8 +45,9 @@ class GridCacheLoaderWriterStoreFactory<K, V> implements Factory<CacheStore<K, V
      * @param ldrFactory Loader factory.
      * @param writerFactory Writer factory.
      */
-    GridCacheLoaderWriterStoreFactory(@Nullable Factory<CacheLoader<K, V>> ldrFactory,
+    GridCacheLoaderWriterStoreFactory(GridKernalContext cctx, @Nullable Factory<CacheLoader<K, V>> ldrFactory,
         @Nullable Factory<CacheWriter<K, V>> writerFactory) {
+        this.cctx = cctx;
         this.ldrFactory = ldrFactory;
         this.writerFactory = writerFactory;
 
@@ -50,9 +56,8 @@ class GridCacheLoaderWriterStoreFactory<K, V> implements Factory<CacheStore<K, V
 
     /** {@inheritDoc} */
     @Override public CacheStore<K, V> create() {
-        CacheLoader<K, V> ldr = ldrFactory == null ? null : ldrFactory.create();
-
-        CacheWriter<K, V> writer = writerFactory == null ? null : writerFactory.create();
+        CacheLoader<K, V> ldr = CU.create(cctx, ldrFactory);
+        CacheWriter<K, V> writer = CU.create(cctx, writerFactory);
 
         return new GridCacheLoaderWriterStore<>(ldr, writer);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/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 b20d32c..f806be7 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
@@ -230,7 +230,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             Factory<CacheWriter> writerFactory = cfg.isWriteThrough() ? cfg.getCacheWriterFactory() : null;
 
             if (ldrFactory != null || writerFactory != null)
-                cfg.setCacheStoreFactory(new GridCacheLoaderWriterStoreFactory(ldrFactory, writerFactory));
+                cfg.setCacheStoreFactory(new GridCacheLoaderWriterStoreFactory(ctx, ldrFactory, writerFactory));
         }
         else {
             if (cfg.getCacheLoaderFactory() != null)
@@ -571,8 +571,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheConfiguration[] cfgs = ctx.config().getCacheConfiguration();
 
-        sharedCtx = createSharedContext(ctx, CU.startStoreSessionListeners(ctx,
-            ctx.config().getCacheStoreSessionListenerFactories()));
+        Collection<CacheStoreSessionListener> sesLsnrs = CU.create(ctx,
+            ctx.config().getCacheStoreSessionListenerFactories());
+
+        CU.startStoreSessionListeners(ctx, sesLsnrs);
+
+        sharedCtx = createSharedContext(ctx, sesLsnrs);
 
         ctx.performance().add("Disable serializable transactions (set 'txSerializableEnabled' to false)",
             !ctx.config().getTransactionConfiguration().isTxSerializableEnabled());
@@ -1131,6 +1135,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         GridCacheContext<?, ?> cacheCtx = new GridCacheContext(
             ctx,
             sharedCtx,
+            startCtx,
             cfg,
             cacheType,
             ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cfg.getName()),
@@ -1261,6 +1266,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cacheCtx = new GridCacheContext(
                 ctx,
                 sharedCtx,
+                startCtx,
                 cfg,
                 cacheType,
                 ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cfg.getName()),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/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 3bd2a45..fda05ee 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
@@ -56,7 +56,6 @@ import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.CacheRebalanceMode.*;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 import static org.apache.ignite.internal.GridTopic.*;
-import static org.apache.ignite.internal.IgniteNodeAttributes.*;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
 
 /**
@@ -1644,31 +1643,22 @@ public class GridCacheUtils {
      * Creates and starts store session listeners.
      *
      * @param ctx Kernal context.
-     * @param factories Factories.
-     * @return Listeners.
+     * @param sesLsnrs Session 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();
+    public static void startStoreSessionListeners(GridKernalContext ctx, Collection<CacheStoreSessionListener> sesLsnrs)
+        throws IgniteCheckedException {
+        if (sesLsnrs == null)
+            return;
 
+        for (CacheStoreSessionListener lsnr : sesLsnrs) {
             if (lsnr != null) {
                 ctx.resource().injectGeneric(lsnr);
 
                 if (lsnr instanceof LifecycleAware)
                     ((LifecycleAware)lsnr).start();
-
-                lsnrs.add(lsnr);
             }
         }
-
-        return lsnrs;
     }
 
     /**
@@ -1690,4 +1680,35 @@ public class GridCacheUtils {
             ctx.resource().cleanupGeneric(lsnr);
         }
     }
+
+    /**
+     * @param ctx Context.
+     * @param factory Factory.
+     * @return Object.
+     */
+    public static <T> T create(GridKernalContext ctx, Factory<T> factory) {
+        T obj = factory != null ? factory.create() : null;
+
+        ctx.resource().autowireSpringBean(obj);
+
+        return obj;
+    }
+
+    /**
+     * @param ctx Context.
+     * @param factories Factories.
+     * @return Objects.
+     */
+    public static <T> Collection<T> create(GridKernalContext ctx, Factory[] factories) {
+        Collection<T> col = new ArrayList<>(factories.length);
+
+        for (Factory<T> factory : factories) {
+            T obj = create(ctx, factory);
+
+            if (obj != null)
+                col.add(obj);
+        }
+
+        return col;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/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 b4a146a..627aa12 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
@@ -169,9 +169,11 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 "Persistence store is configured, but both read-through and write-through are disabled.");
         }
 
-        sesLsnrs = CU.startStoreSessionListeners(cctx.kernalContext(), cfg.getCacheStoreSessionListenerFactories());
+        sesLsnrs = cctx.storeSessionListeners();
 
-        if (sesLsnrs == null) {
+        if (sesLsnrs != null)
+            CU.startStoreSessionListeners(cctx.kernalContext(), sesLsnrs);
+        else {
             sesLsnrs = cctx.shared().storeSessionListeners();
 
             globalSesLsnrs = true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
index 05ef534..b66de27 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
@@ -531,7 +531,7 @@ class OptimizedClassDescriptor {
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
     void write(OptimizedObjectOutputStream out, Object obj) throws IOException {
-        out.write(type);
+        out.writeByte(type);
 
         switch (type) {
             case BYTE:

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c0cc4f1/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 77d3905..efeb8f4 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
@@ -57,6 +57,7 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
                 new GridCacheIoManager(),
                 null
             ),
+            new CacheStartContext(ctx, defaultCacheConfiguration()),
             defaultCacheConfiguration(),
             CacheType.USER,
             true,