You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/05/31 07:35:43 UTC

[12/19] ignite git commit: IGNITE-5030 Support Spring @Cacheable(sync=true) annotation

IGNITE-5030 Support Spring @Cacheable(sync=true) annotation


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

Branch: refs/heads/ignite-5075
Commit: b0e49ad9ad9cba8901b5ffad37b4fe171246a9d0
Parents: b7f2021
Author: mcherkasov <mc...@gridgain.com>
Authored: Tue May 30 14:57:58 2017 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue May 30 14:57:58 2017 +0300

----------------------------------------------------------------------
 modules/spring/pom.xml                          |   7 +
 .../apache/ignite/cache/spring/SpringCache.java |  60 +++++++--
 .../ignite/cache/spring/SpringCacheManager.java |  51 ++++++-
 .../GridSpringCacheManagerMultiJvmSelfTest.java | 134 +++++++++++++++++++
 .../GridSpringDynamicCacheTestService.java      |  13 ++
 .../ignite/cache/spring/SpringCacheTest.java    |   2 +-
 .../ignite/cache/spring/spring-caching1.xml     |  56 ++++++++
 .../ignite/cache/spring/spring-caching2.xml     |  56 ++++++++
 8 files changed, 368 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index edaa013..e739118 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -123,6 +123,13 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>com.thoughtworks.xstream</groupId>
+            <artifactId>xstream</artifactId>
+            <version>1.4.8</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
index f35ed2a..9a8f2a8 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
@@ -20,6 +20,7 @@ package org.apache.ignite.cache.spring;
 import java.io.Serializable;
 import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteLock;
 import org.springframework.cache.Cache;
 import org.springframework.cache.support.SimpleValueWrapper;
 
@@ -33,13 +34,18 @@ class SpringCache implements Cache {
     /** */
     private final IgniteCache<Object, Object> cache;
 
+    /** */
+    private final SpringCacheManager mgr;
+
     /**
      * @param cache Cache.
+     * @param mgr Manager
      */
-    SpringCache(IgniteCache<Object, Object> cache) {
+    SpringCache(IgniteCache<Object, Object> cache, SpringCacheManager mgr) {
         assert cache != null;
 
         this.cache = cache;
+        this.mgr = mgr;
     }
 
     /** {@inheritDoc} */
@@ -75,6 +81,40 @@ class SpringCache implements Cache {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T get(final Object key, final Callable<T> valLdr) {
+        Object val = cache.get(key);
+
+        if (val == null) {
+            IgniteLock lock = mgr.getSyncLock(cache.getName(), key);
+
+            lock.lock();
+
+            try {
+                val = cache.get(key);
+
+                if (val == null) {
+                    try {
+                        T retVal = valLdr.call();
+
+                        val = wrapNull(retVal);
+
+                        cache.put(key, val);
+                    }
+                    catch (Exception e) {
+                        throw new ValueRetrievalException(key, valLdr, e);
+                    }
+                }
+            }
+            finally {
+                lock.unlock();
+            }
+        }
+
+        return (T)unwrapNull(val);
+    }
+
+    /** {@inheritDoc} */
     @Override public void put(Object key, Object val) {
         if (val == null)
             cache.withSkipStore().put(key, NULL);
@@ -83,11 +123,6 @@ class SpringCache implements Cache {
     }
 
     /** {@inheritDoc} */
-    @Override public <T> T get(Object key, Callable<T> valLdr) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
     @Override public ValueWrapper putIfAbsent(Object key, Object val) {
         Object old;
 
@@ -116,11 +151,18 @@ class SpringCache implements Cache {
     private static ValueWrapper fromValue(Object val) {
         assert val != null;
 
-        return new SimpleValueWrapper(NULL.equals(val) ? null : val);
+        return new SimpleValueWrapper(unwrapNull(val));
     }
 
-    /**
-     */
+    private static Object unwrapNull(Object val) {
+        return NULL.equals(val) ? null : val;
+    }
+
+    private <T> Object wrapNull(T val) {
+        return val == null ? NULL : val;
+    }
+
+    /** */
     private static class NullValue implements Serializable {
         /** {@inheritDoc} */
         @Override public boolean equals(Object o) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
index 2a24b42..c04eb26 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
@@ -19,8 +19,10 @@ package org.apache.ignite.cache.spring;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Objects;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteLock;
 import org.apache.ignite.IgniteSpring;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -138,6 +140,12 @@ import org.springframework.context.ApplicationContextAware;
  * in caching the data.
  */
 public class SpringCacheManager implements CacheManager, InitializingBean, ApplicationContextAware {
+    /** Default locks count. */
+    private static final int DEFAULT_LOCKS_COUNT = 512;
+
+    /** IgniteLock name prefix. */
+    private static final String SPRING_LOCK_NAME_PREFIX = "springSync";
+
     /** Caches map. */
     private final ConcurrentMap<String, SpringCache> caches = new ConcurrentHashMap8<>();
 
@@ -150,6 +158,9 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
     /** Ignite instance name. */
     private String igniteInstanceName;
 
+    /** Count of IgniteLocks are used for sync get */
+    private int locksCnt = DEFAULT_LOCKS_COUNT;
+
     /** Dynamic cache configuration template. */
     private CacheConfiguration<Object, Object> dynamicCacheCfg;
 
@@ -162,6 +173,9 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
     /** Spring context. */
     private ApplicationContext springCtx;
 
+    /** Locks for value loading to support sync option. */
+    private ConcurrentHashMap8<Integer, IgniteLock> locks = new ConcurrentHashMap8<>();
+
     /** {@inheritDoc} */
     @Override public void setApplicationContext(ApplicationContext ctx) {
         this.springCtx = ctx;
@@ -244,6 +258,22 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
     }
 
     /**
+     * Gets locks count.
+     *
+     * @return locks count.
+     */
+    public int getLocksCount() {
+        return locksCnt;
+    }
+
+    /**
+     * @param locksCnt locks count.
+     */
+    public void setLocksCount(int locksCnt) {
+        this.locksCnt = locksCnt;
+    }
+
+    /**
      * Gets dynamic cache configuration template.
      *
      * @return Dynamic cache configuration template.
@@ -314,7 +344,7 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
             cacheCfg.setName(name);
 
             cache = new SpringCache(nearCacheCfg != null ? ignite.getOrCreateCache(cacheCfg, nearCacheCfg) :
-                ignite.getOrCreateCache(cacheCfg));
+                ignite.getOrCreateCache(cacheCfg), this);
 
             SpringCache old = caches.putIfAbsent(name, cache);
 
@@ -331,4 +361,23 @@ public class SpringCacheManager implements CacheManager, InitializingBean, Appli
 
         return new ArrayList<>(caches.keySet());
     }
+
+    /**
+     * Provides {@link org.apache.ignite.IgniteLock} for specified cache name and key.
+     *
+     * @param name cache name
+     * @param key  key
+     * @return {@link org.apache.ignite.IgniteLock}
+     */
+    IgniteLock getSyncLock(String name, Object key) {
+        int hash = Objects.hash(name, key);
+
+        final int idx = hash % getLocksCount();
+
+        return locks.computeIfAbsent(idx, new ConcurrentHashMap8.Fun<Integer, IgniteLock>() {
+            @Override public IgniteLock apply(Integer integer) {
+                return ignite.reentrantLock(SPRING_LOCK_NAME_PREFIX + idx, true, false, true);
+            }
+        });
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerMultiJvmSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerMultiJvmSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerMultiJvmSelfTest.java
new file mode 100644
index 0000000..39483fc
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringCacheManagerMultiJvmSelfTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.spring;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy;
+import org.springframework.beans.factory.BeanFactory;
+import org.springframework.context.support.ClassPathXmlApplicationContext;
+
+/**
+ * Spring cache test in multi jvm environment.
+ */
+public class GridSpringCacheManagerMultiJvmSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean isMultiJvm() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTestIgniteInstanceName(int idx) {
+        return getTestIgniteInstanceName() + idx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTestIgniteInstanceName() {
+        return "testGrid";
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSyncCache() throws Exception {
+        IgniteEx loc = startGrid(0);
+
+        final int threads = 4;
+        final int entries = 100_000;
+        final int remoteNum = 2;
+
+        final CountDownLatch latch = new CountDownLatch(1);
+
+        List<IgniteInternalFuture<Integer>> futures = new ArrayList<>(remoteNum);
+
+        for (int i = 0; i < remoteNum; i++) {
+            final int gridIdx = i + 1;
+
+            final IgniteEx remote = startGrid(gridIdx);
+
+            IgniteInternalFuture<Integer> calledCntFut = GridTestUtils.runAsync(new Callable<Integer>() {
+                @Override public Integer call() throws Exception {
+                    latch.await();
+
+                    return executeRemotely((IgniteProcessProxy)remote, new TestIgniteCallable<Integer>() {
+                        @Override public Integer call(Ignite ignite) throws Exception {
+                            BeanFactory factory =
+                                new ClassPathXmlApplicationContext(
+                                    "org/apache/ignite/cache/spring/spring-caching" + gridIdx + ".xml");
+
+                            final GridSpringDynamicCacheTestService dynamicSvc =
+                                (GridSpringDynamicCacheTestService)factory.getBean("dynamicTestService");
+
+                            final CyclicBarrier barrier = new CyclicBarrier(threads);
+
+                            GridTestUtils.runMultiThreaded(
+                                new Callable() {
+                                    @Override public Object call() throws Exception {
+                                        for (int i = 0; i < entries; i++) {
+                                            barrier.await();
+
+                                            assertEquals("value" + i, dynamicSvc.cacheableSync(i));
+                                            assertEquals("value" + i, dynamicSvc.cacheableSync(i));
+                                        }
+
+                                        return null;
+                                    }
+                                },
+                                threads,
+                                "get-sync");
+
+                            return dynamicSvc.called();
+                        }
+                    });
+
+                }
+            });
+
+            futures.add(calledCntFut);
+        }
+
+        latch.countDown();
+
+        int totalCalledCnt = 0;
+
+        for (IgniteInternalFuture<Integer> future : futures)
+            totalCalledCnt += future.get();
+
+        IgniteCache<Object, Object> cache = loc.cache("dynamicCache");
+
+        assertEquals(entries, cache.size());
+        assertEquals(entries, totalCalledCnt);
+
+        for (int i = 0; i < entries; i++)
+            assertEquals("value" + i, cache.get(i));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java
index 6584277..b15a9c0 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/GridSpringDynamicCacheTestService.java
@@ -46,6 +46,19 @@ public class GridSpringDynamicCacheTestService {
      * @param key Key.
      * @return Value.
      */
+    @Cacheable(value = "dynamicCache", sync = true)
+    public String cacheableSync(Integer key) {
+        assert key != null;
+
+        cnt.incrementAndGet();
+
+        return "value" + key;
+    }
+
+    /**
+     * @param key Key.
+     * @return Value.
+     */
     @CachePut("dynamicCache")
     public String cachePut(Integer key) {
         assert key != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java
index 2324235..06dc263 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/SpringCacheTest.java
@@ -59,7 +59,7 @@ public class SpringCacheTest extends GridCommonAbstractTest {
 
         cacheName = String.valueOf(System.currentTimeMillis());
         nativeCache = ignite.getOrCreateCache(cacheName);
-        springCache = new SpringCache(nativeCache);
+        springCache = new SpringCache(nativeCache, null);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching1.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching1.xml b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching1.xml
new file mode 100644
index 0000000..679fd97
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching1.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"
+       xmlns:cache="http://www.springframework.org/schema/cache"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/cache http://www.springframework.org/schema/cache/spring-cache.xsd">
+    <!--
+        Test service with cacheable methods.
+    -->
+    <bean id="testService" class="org.apache.ignite.cache.spring.GridSpringCacheTestService"/>
+
+    <!--
+        Test service with cacheable methods (dynamic cache).
+    -->
+    <bean id="dynamicTestService" class="org.apache.ignite.cache.spring.GridSpringDynamicCacheTestService"/>
+
+    <!--
+        Cache manager.
+    -->
+    <bean id="cacheManager" class="org.apache.ignite.cache.spring.SpringCacheManager">
+        <property name="igniteInstanceName" value="testGrid1"/>
+        <property name="dynamicCacheConfiguration">
+            <bean class="org.apache.ignite.configuration.CacheConfiguration">
+            </bean>
+        </property>
+    </bean>
+
+    <!--
+        Key generator.
+    -->
+    <bean id="keyGenerator" class="org.apache.ignite.cache.spring.GridSpringCacheTestKeyGenerator"/>
+
+    <!--
+        Enable annotation-driver configuration for caching.
+    -->
+    <cache:annotation-driven key-generator="keyGenerator"/>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b0e49ad9/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching2.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching2.xml b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching2.xml
new file mode 100644
index 0000000..6a9e25a
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/spring/spring-caching2.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"
+       xmlns:cache="http://www.springframework.org/schema/cache"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/cache http://www.springframework.org/schema/cache/spring-cache.xsd">
+    <!--
+        Test service with cacheable methods.
+    -->
+    <bean id="testService" class="org.apache.ignite.cache.spring.GridSpringCacheTestService"/>
+
+    <!--
+        Test service with cacheable methods (dynamic cache).
+    -->
+    <bean id="dynamicTestService" class="org.apache.ignite.cache.spring.GridSpringDynamicCacheTestService"/>
+
+    <!--
+        Cache manager.
+    -->
+    <bean id="cacheManager" class="org.apache.ignite.cache.spring.SpringCacheManager">
+        <property name="igniteInstanceName" value="testGrid2"/>
+        <property name="dynamicCacheConfiguration">
+            <bean class="org.apache.ignite.configuration.CacheConfiguration">
+            </bean>
+        </property>
+    </bean>
+
+    <!--
+        Key generator.
+    -->
+    <bean id="keyGenerator" class="org.apache.ignite.cache.spring.GridSpringCacheTestKeyGenerator"/>
+
+    <!--
+        Enable annotation-driver configuration for caching.
+    -->
+    <cache:annotation-driven key-generator="keyGenerator"/>
+</beans>