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

incubator-ignite git commit: IGNITE-50 Added support user's class loader for execution context.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-50 [created] f46f266e9


IGNITE-50 Added support user's class loader for execution context.


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

Branch: refs/heads/ignite-50
Commit: f46f266e968705471e061161f56f8c8c623fe4e5
Parents: 61a70d9
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Fri Jan 30 16:18:30 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Feb 2 13:37:50 2015 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |  26 +++
 .../org/apache/ignite/internal/IgnitionEx.java  |   1 +
 .../cache/GridCacheDeploymentManager.java       |  19 +-
 .../processors/cache/GridCacheProcessor.java    |  11 +-
 .../cache/GridCacheProjectionImpl.java          |  23 ++-
 .../GridCacheContinuousQueryManager.java        |  14 +-
 ...IgniteCacheAbstractExecutionContextTest.java | 197 +++++++++++++++++++
 .../IgniteCacheAtomicExecutionContextTest.java  |  49 +++++
 ...iteCachePartitionedExecutionContextTest.java |  49 +++++
 ...niteCacheReplicatedExecutionContextTest.java |  49 +++++
 .../IgniteCacheTxExecutionContextTest.java      |  49 +++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   7 +
 .../p2p/CacheEntryListenerConfiguration.java    |  60 ++++++
 .../tests/p2p/CacheEntryProcessorNoop.java      |  37 ++++
 .../ignite/tests/p2p/CacheExpirePolicyNoop.java |  45 +++++
 .../apache/ignite/tests/p2p/CacheStoreNoop.java |  88 +++++++++
 16 files changed, 719 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 732aafc..4675aa8 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -50,6 +50,10 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import javax.management.*;
+import javax.cache.processor.*;
+import javax.cache.expiry.*;
+import javax.cache.integration.*;
+import javax.cache.event.*;
 import java.lang.management.*;
 import java.net.*;
 import java.util.*;
@@ -528,6 +532,9 @@ public class IgniteConfiguration {
     /** */
     private IgniteQueryConfiguration qryCfg;
 
+    /** User's class loader. */
+    private ClassLoader classLdr;
+
     /**
      * Creates valid grid configuration with all default values.
      */
@@ -653,6 +660,7 @@ public class IgniteConfiguration {
         userAttrs = cfg.getUserAttributes();
         waitForSegOnStart = cfg.isWaitForSegmentOnStart();
         warmupClos = cfg.getWarmupClosure();
+        classLdr = cfg.getClassLoader();
     }
 
     /**
@@ -3180,6 +3188,24 @@ public class IgniteConfiguration {
         this.qryCfg = qryCfg;
     }
 
+    /**
+     * Sets loader which will be used for instantiating execution context ({@link EntryProcessor EntryProcessors},
+     * {@link CacheEntryListener CacheEntryListeners}, {@link CacheLoader CacheLoaders} and
+     * {@link ExpiryPolicy ExpiryPolicys}).
+     *
+     * @param classLdr Class loader.
+     */
+    public void setClassLoader(ClassLoader classLdr) {
+        this.classLdr = classLdr;
+    }
+
+    /**
+     * @return User's class loader.
+     */
+    public ClassLoader getClassLoader() {
+        return classLdr;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgniteConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 7dd9e73..dd3bda6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1422,6 +1422,7 @@ public class IgnitionEx {
             myCfg.setPluginConfigurations(cfg.getPluginConfigurations());
             myCfg.setTransactionsConfiguration(new TransactionsConfiguration(cfg.getTransactionsConfiguration()));
             myCfg.setQueryConfiguration(cfg.getQueryConfiguration());
+            myCfg.setClassLoader(cfg.getClassLoader());
 
             ClientConnectionConfiguration clientCfg = cfg.getClientConnectionConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
index 35bcffb..c441bb5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
@@ -82,7 +82,7 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
 
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
-        globalLdr = new CacheClassLoader();
+        globalLdr = getCacheClassLoader();
 
         nodeFilter = new P1<ClusterNode>() {
             @Override public boolean apply(ClusterNode node) {
@@ -124,6 +124,14 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
         }
     }
 
+    /**
+     * @return If user's class loader is null then will be used default class loader.
+     */
+    private CacheClassLoader getCacheClassLoader() {
+        return cctx.gridConfig().getClassLoader() == null ? new CacheClassLoader()
+            : new CacheClassLoader(cctx.gridConfig().getClassLoader());
+    }
+
     /** {@inheritDoc} */
     @Override protected void stop0(boolean cancel) {
         if (discoLsnr != null)
@@ -761,7 +769,14 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
          * Sets context class loader as parent.
          */
         private CacheClassLoader() {
-            super(U.detectClassLoader(GridCacheDeploymentManager.class));
+            this(U.detectClassLoader(GridCacheDeploymentManager.class));
+        }
+
+        /**
+         * Sets context class loader as user's class loader.
+         */
+        private CacheClassLoader(ClassLoader classLdr) {
+            super(classLdr);
 
             p2pExclude = cctx.gridConfig().getPeerClassLoadingLocalClassPathExclude();
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/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 742ddb9..44b4e21 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
@@ -31,7 +31,8 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lifecycle.LifecycleAware;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.marshaller.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.internal.processors.cache.datastructures.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
@@ -598,6 +599,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             CacheStore cfgStore = cfg.getCacheStoreFactory() != null ? cfg.getCacheStoreFactory().create() : null;
 
+            if (ctx.config().getClassLoader() != null && cfg.getCacheStoreFactory() != null) {
+                IgniteMarshaller marsh = ctx.config().getMarshaller();
+
+                byte[] bytes = marsh.marshal(cfgStore);
+
+                cfgStore = marsh.unmarshal(bytes, ctx.config().getClassLoader());
+            }
+
             validate(ctx.config(), cfg, cfgStore);
 
             CacheJtaManagerAdapter jta = JTA.create(cfg.getTransactionManagerLookupClassName() == null);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
index 72b2505..0608852 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.marshaller.*;
 import org.apache.ignite.portables.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.dr.*;
@@ -35,6 +36,7 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
+import javax.cache.*;
 import javax.cache.expiry.*;
 import javax.cache.processor.*;
 import java.io.*;
@@ -143,7 +145,26 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
 
         this.keepPortable = keepPortable;
 
-        this.expiryPlc = expiryPlc;
+        if (cctx.gridConfig().getClassLoader() == null)
+            this.expiryPlc = expiryPlc;
+        else
+            this.expiryPlc = loadExpiryPolicy(expiryPlc);
+    }
+
+    private ExpiryPolicy loadExpiryPolicy(ExpiryPolicy expiryPlc) {
+        try {
+            if (expiryPlc == null)
+                return null;
+
+            IgniteMarshaller marsh = cctx.gridConfig().getMarshaller();
+
+            byte[] bytes = marsh.marshal(expiryPlc);
+
+            return marsh.unmarshal(bytes, cctx.gridConfig().getClassLoader());
+        }
+        catch (Exception e){
+            throw new CacheException("Failed to load expiry policy by user's class loader.", e);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryManager.java
index b3f81d7..59933b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryManager.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.marshaller.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -219,7 +220,18 @@ public class GridCacheContinuousQueryManager<K, V> extends GridCacheManagerAdapt
 
             A.notNull(factory, "cacheEntryListenerFactory");
 
-            CacheEntryListener lsnr = factory.create();
+            CacheEntryListener lsnr;
+
+            if (cctx.gridConfig().getClassLoader() == null)
+                lsnr = factory.create();
+            else {
+                IgniteMarshaller marsh = cctx.gridConfig().getMarshaller();
+
+                byte[] bytes = marsh.marshal(factory);
+
+                lsnr = ((Factory<CacheEntryListener<? super K, ? super V>>)marsh
+                    .unmarshal(bytes, cctx.gridConfig().getClassLoader())).create();
+            }
 
             A.notNull(lsnr, "lsnr");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAbstractExecutionContextTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAbstractExecutionContextTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAbstractExecutionContextTest.java
new file mode 100644
index 0000000..60802c6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAbstractExecutionContextTest.java
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.context;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.config.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.expiry.*;
+import javax.cache.processor.*;
+import java.net.*;
+import java.util.*;
+
+/**
+ *
+ */
+public abstract class IgniteCacheAbstractExecutionContextTest extends IgniteCacheAbstractTest {
+    /**  */
+    public static final int ITERATIONS = 1000;
+
+    /** */
+    public static final String EXPIRY_POLICY_CLASS = "org.apache.ignite.tests.p2p.CacheExpirePolicyNoop";
+
+    /** */
+    public static final String ENTRY_LISTENER_CLASS = "org.apache.ignite.tests.p2p.CacheEntryListenerConfiguration";
+
+    /** */
+    public static final String ENTRY_PROCESSOR_CLASS = "org.apache.ignite.tests.p2p.CacheEntryProcessorNoop";
+
+    /** */
+    public static final String CACHE_STORE_CLASS = "org.apache.ignite.tests.p2p.CacheStoreNoop";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setClassLoader(new UsersClassLoader());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheStore<?, ?> cacheStore() {
+        try {
+            return loadClass(new UsersClassLoader(), CACHE_STORE_CLASS);
+        }
+        catch (Exception e){
+            throw new CacheException(e);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEntryProcessor() throws Exception {
+        UsersClassLoader classLdr = (UsersClassLoader)ignite(0).configuration().getClassLoader();
+
+        IgniteCache<Object, Object> cache = ignite(0).jcache(null);
+
+        Set<Integer> keys = new TreeSet<>();
+
+        for (int i = 0; i < ITERATIONS; i++) {
+            cache.put(i, i);
+
+            keys.add(i);
+        }
+
+        Map<Object, EntryProcessorResult<Object>> res = cache.invokeAll(
+            keys,
+            this.<EntryProcessor<Object, Object, Object>>loadClass(classLdr, ENTRY_PROCESSOR_CLASS));
+
+        assertEquals(ITERATIONS, res.size());
+
+        for (EntryProcessorResult<Object> val : res.values())
+            assertEquals(42, (long)val.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheEntryListener() throws Exception {
+        UsersClassLoader classLdr = (UsersClassLoader)ignite(0).configuration().getClassLoader();
+
+        IgniteCache<Object, Object> cache = ignite(0).jcache(null);
+
+        CacheEntryListenerConfiguration<Object, Object> list = loadClass(ignite(0).configuration().getClassLoader(),
+            ENTRY_LISTENER_CLASS);
+
+        cache.registerCacheEntryListener(list);
+
+        for (int i = ITERATIONS; i < 2 * ITERATIONS; i++)
+            cache.put(i, i);
+
+        assertEquals(ITERATIONS, U.invoke(list.getClass(), list, "getCnt", null));
+        assertEquals(2, (int)classLdr.classUsedCnt(ENTRY_LISTENER_CLASS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExpirePolicies() throws Exception {
+        UsersClassLoader classLdr = (UsersClassLoader)ignite(0).configuration().getClassLoader();
+
+        IgniteCache<Object, Object> cache = ignite(0).jcache(null);
+
+        cache.withExpiryPolicy(this.<ExpiryPolicy>loadClass(classLdr, EXPIRY_POLICY_CLASS));
+
+        assertEquals(2, (int)classLdr.classUsedCnt(EXPIRY_POLICY_CLASS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheLoaderWriter() throws Exception {
+        IgniteCache<Object, Object> cache = ignite(0).jcache(null);
+
+        UsersClassLoader classLdr = (UsersClassLoader)ignite(0).configuration().getClassLoader();
+
+        assertEquals(42L, cache.get(99999999999999L));
+
+        assertEquals(1, (int)classLdr.classUsedCnt(CACHE_STORE_CLASS));
+    }
+
+    /**
+     * @return Loaded class.
+     * @throws Exception Thrown if any exception occurs.
+     */
+    private <T> T loadClass(ClassLoader usersClassLdr, String className)
+        throws Exception {
+        assertNotNull(usersClassLdr);
+
+        assertNotNull(usersClassLdr.loadClass(className));
+
+        return (T)usersClassLdr.loadClass(className).newInstance();
+    }
+
+    /**
+     *
+     */
+    private static class UsersClassLoader extends GridTestExternalClassLoader {
+        /**
+         *
+         */
+        private Map<String, Integer> loadedClasses = new HashMap<>();
+
+        /**
+         *
+         * @throws MalformedURLException If failed
+         */
+        public UsersClassLoader() throws MalformedURLException {
+            super(new URL[]{new URL(GridTestProperties.getProperty("p2p.uri.cls"))});
+        }
+
+        /**
+         *
+         */
+        @Override public Class<?> loadClass(String name) throws ClassNotFoundException {
+            int count = !loadedClasses.containsKey(name) ? 0 : loadedClasses.get(name);
+
+            ++count;
+
+            loadedClasses.put(name, count);
+
+            return super.loadClass(name);
+        }
+
+        public Integer classUsedCnt(String name){
+            return loadedClasses.get(name);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAtomicExecutionContextTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAtomicExecutionContextTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAtomicExecutionContextTest.java
new file mode 100644
index 0000000..da80b92
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheAtomicExecutionContextTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.context;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheDistributionMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicExecutionContextTest extends IgniteCacheAbstractExecutionContextTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.LOCAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheDistributionMode distributionMode() {
+        return PARTITIONED_ONLY;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCachePartitionedExecutionContextTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCachePartitionedExecutionContextTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCachePartitionedExecutionContextTest.java
new file mode 100644
index 0000000..26d6c18
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCachePartitionedExecutionContextTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.context;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheDistributionMode.*;
+
+/**
+ *
+ */
+public class IgniteCachePartitionedExecutionContextTest extends IgniteCacheAbstractExecutionContextTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheDistributionMode distributionMode() {
+        return PARTITIONED_ONLY;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheReplicatedExecutionContextTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheReplicatedExecutionContextTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheReplicatedExecutionContextTest.java
new file mode 100644
index 0000000..a93317b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheReplicatedExecutionContextTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.context;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheDistributionMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheReplicatedExecutionContextTest extends IgniteCacheAbstractExecutionContextTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.REPLICATED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheDistributionMode distributionMode() {
+        return PARTITIONED_ONLY;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheTxExecutionContextTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheTxExecutionContextTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheTxExecutionContextTest.java
new file mode 100644
index 0000000..9817904
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/context/IgniteCacheTxExecutionContextTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.context;
+
+import org.apache.ignite.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheDistributionMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheTxExecutionContextTest extends IgniteCacheAbstractExecutionContextTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.LOCAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+        return CLOCK;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheDistributionMode distributionMode() {
+        return PARTITIONED_ONLY;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 8f885fb..8368ab0 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -22,6 +22,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.fair.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.context.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 import org.apache.ignite.internal.processors.cache.expiry.*;
@@ -66,6 +67,12 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheTxLocalInvokeTest.class);
         suite.addTestSuite(IgniteCrossCacheTxStoreSelfTest.class);
 
+        // User's class loader tests.
+        suite.addTestSuite(IgniteCacheAtomicExecutionContextTest.class);
+        suite.addTestSuite(IgniteCachePartitionedExecutionContextTest.class);
+        suite.addTestSuite(IgniteCacheReplicatedExecutionContextTest.class);
+        suite.addTestSuite(IgniteCacheTxExecutionContextTest.class);
+
         // Affinity tests.
         suite.addTestSuite(GridCachePartitionFairAffinityNodesSelfTest.class);
         suite.addTestSuite(GridCacheAffinityBackupsSelfTest.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheEntryListenerConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheEntryListenerConfiguration.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheEntryListenerConfiguration.java
new file mode 100644
index 0000000..92d0d12
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheEntryListenerConfiguration.java
@@ -0,0 +1,60 @@
+/*
+ * 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.tests.p2p;
+
+import javax.cache.configuration.*;
+import javax.cache.event.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Entry processor for testing user's class loader.
+ */
+public class CacheEntryListenerConfiguration extends MutableCacheEntryListenerConfiguration<Integer, Integer> {
+    /** */
+    public final static AtomicInteger CNT = new AtomicInteger(0);
+
+    /**
+     *
+     */
+    public CacheEntryListenerConfiguration() {
+        super(new Factory<CacheEntryListener<Integer, Integer>>() {
+            /** {@inheritDoc} */
+            @Override public CacheEntryListener<Integer, Integer> create() {
+                return new EntryListener();
+            }
+        }, null, true, true);
+    }
+
+    /**
+     *
+     */
+    public static class EntryListener implements CacheEntryCreatedListener<Integer, Integer> {
+        /** {@inheritDoc} */
+        @Override public void onCreated(Iterable iterable) throws CacheEntryListenerException {
+            for (Object entry : iterable) {
+                assert entry != null;
+
+                CNT.incrementAndGet();
+            }
+        }
+    }
+
+    public static int getCnt() {
+        return CNT.get();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheEntryProcessorNoop.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheEntryProcessorNoop.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheEntryProcessorNoop.java
new file mode 100644
index 0000000..c59c531
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheEntryProcessorNoop.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.tests.p2p;
+
+import javax.cache.processor.*;
+
+/**
+ * Entry processor for testing user's class loader.
+ */
+public class CacheEntryProcessorNoop implements EntryProcessor<Object, Object, Object> {
+    /**
+     *
+     */
+    public CacheEntryProcessorNoop() {
+    }
+
+    @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+        throws EntryProcessorException {
+
+        return 42L;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheExpirePolicyNoop.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheExpirePolicyNoop.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheExpirePolicyNoop.java
new file mode 100644
index 0000000..18846b3
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheExpirePolicyNoop.java
@@ -0,0 +1,45 @@
+/*
+ * 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.tests.p2p;
+
+import javax.cache.expiry.*;
+/**
+ * Entry processor for testing user's class loader.
+ */
+public class CacheExpirePolicyNoop implements ExpiryPolicy {
+    /**
+     *
+     */
+    public CacheExpirePolicyNoop() {
+    }
+
+    /** {@inheritDoc} */
+    @Override public Duration getExpiryForCreation() {
+        return Duration.ETERNAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Duration getExpiryForAccess() {
+        return Duration.ETERNAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Duration getExpiryForUpdate() {
+        return Duration.ETERNAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f46f266e/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheStoreNoop.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheStoreNoop.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheStoreNoop.java
new file mode 100644
index 0000000..6e3f29f
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheStoreNoop.java
@@ -0,0 +1,88 @@
+/*
+ * 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.tests.p2p;
+
+import org.apache.ignite.cache.store.*;
+
+import javax.cache.*;
+import javax.cache.integration.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class CacheStoreNoop extends CacheStoreAdapter {
+    /** */
+    private static final AtomicInteger loadCnt = new AtomicInteger(0);
+
+    /** */
+    private static final AtomicInteger writeCnt = new AtomicInteger(0);
+
+    /** */
+    private static final AtomicInteger deleteCnt = new AtomicInteger(0);
+
+    /** */
+    private static boolean isInjected;
+
+    /** {@inheritDoc} */
+    @Override public Object load(Object key) throws CacheLoaderException {
+        loadCnt.incrementAndGet();
+
+        isInjected = ignite() != null;
+
+        return 42L;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(Cache.Entry entry) throws CacheWriterException {
+        writeCnt.incrementAndGet();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void delete(Object key) throws CacheWriterException {
+        deleteCnt.incrementAndGet();
+    }
+
+    /**
+     *
+     */
+    public static int getLoadCnt() {
+        return loadCnt.get();
+    }
+
+    /**
+     *
+     */
+    public static int getWriteCnt() {
+        return writeCnt.get();
+    }
+
+    /**
+     *
+     */
+    public static int getDeleteCnt() {
+        return deleteCnt.get();
+    }
+
+    /**
+     *
+     */
+    public boolean isInjected() {
+        return isInjected;
+    }
+}