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 2016/02/12 14:09:26 UTC

[20/22] ignite git commit: IGNITE-2603

IGNITE-2603


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

Branch: refs/heads/ignite-2407
Commit: a32dfc41ea9301f8b98c6a666e4b72c65c892659
Parents: 725d6cb
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Feb 12 14:30:08 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Fri Feb 12 14:30:08 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |   8 +-
 .../GridCacheReplicatedPreloadSelfTest.java     | 121 ++++++++++++++-----
 .../p2p/CacheDeploymentAffinityKeyMapper.java   |  35 ++++++
 .../CacheDeploymentAlwaysTruePredicate2.java    |  30 +++++
 ...oymentCacheEntryEventSerializableFilter.java |  32 +++++
 .../p2p/CacheDeploymentCacheEntryListener.java  |  31 +++++
 ...CacheDeploymentCachePluginConfiguration.java |  74 ++++++++++++
 ...heDeploymentStoreSessionListenerFactory.java |  83 +++++++++++++
 8 files changed, 383 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a32dfc41/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 5acad6c..7a36e73 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
@@ -3425,8 +3425,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         try {
             if (val.getCacheStoreFactory() != null) {
                 try {
-                    marshaller.unmarshal(marshaller.marshal(val.getCacheStoreFactory()),
-                        val.getCacheStoreFactory().getClass().getClassLoader());
+                    ClassLoader ldr = ctx.config().getClassLoader();
+
+                    if (ldr == null)
+                        ldr = val.getCacheStoreFactory().getClass().getClassLoader();
+
+                    marshaller.unmarshal(marshaller.marshal(val.getCacheStoreFactory()), ldr);
                 }
                 catch (IgniteCheckedException e) {
                     throw new IgniteCheckedException("Failed to validate cache configuration. " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/a32dfc41/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
index 887fea4..1fae875 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/preloader/GridCacheReplicatedPreloadSelfTest.java
@@ -26,13 +26,18 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.UUID;
+import javax.cache.configuration.CacheEntryListenerConfiguration;
 import javax.cache.configuration.Factory;
+import javax.cache.configuration.MutableCacheEntryListenerConfiguration;
+import javax.cache.event.CacheEntryListener;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.AffinityFunctionContext;
+import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -43,6 +48,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.plugin.CachePluginConfiguration;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -71,7 +78,7 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     private int poolSize = 2;
 
     /** */
-    private volatile boolean needStore = false;
+    private volatile boolean extClassloadingAtCfg = false;
 
     /** */
     private volatile boolean isClient = false;
@@ -136,21 +143,47 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
         cacheCfg.setRebalanceBatchSize(batchSize);
         cacheCfg.setRebalanceThreadPoolSize(poolSize);
 
-        if (needStore) {
-            Object sf = null;
+        if (extClassloadingAtCfg) {
+            loadExternalClassesToCfg(cacheCfg);
+        }
 
-            try {
-                sf = getExternalClassLoader().
-                    loadClass("org.apache.ignite.tests.p2p.CacheDeploymentTestStoreFactory").newInstance();
-            }
-            catch (Exception e) {
-                throw new RuntimeException(e);
-            }
+        return cacheCfg;
+    }
+
+    /**
+     *
+     * @param cacheCfg Configuration.
+     */
+    private void loadExternalClassesToCfg(CacheConfiguration cacheCfg) {
+        try {
+            Object sf = getExternalClassLoader().
+                loadClass("org.apache.ignite.tests.p2p.CacheDeploymentTestStoreFactory").newInstance();
 
             cacheCfg.setCacheStoreFactory((Factory)sf);
-        }
 
-        return cacheCfg;
+            Object sslf = getExternalClassLoader().
+                loadClass("org.apache.ignite.tests.p2p.CacheDeploymentStoreSessionListenerFactory").newInstance();
+
+            cacheCfg.setCacheStoreSessionListenerFactories((Factory)sslf);
+
+            Object cpc = getExternalClassLoader().
+                loadClass("org.apache.ignite.tests.p2p.CacheDeploymentCachePluginConfiguration").newInstance();
+
+            cacheCfg.setPluginConfigurations((CachePluginConfiguration)cpc);
+
+            Object akm = getExternalClassLoader().
+                loadClass("org.apache.ignite.tests.p2p.CacheDeploymentAffinityKeyMapper").newInstance();
+
+            cacheCfg.setAffinityMapper((AffinityKeyMapper)akm);
+
+            Object pred = getExternalClassLoader().
+                loadClass("org.apache.ignite.tests.p2p.CacheDeploymentAlwaysTruePredicate2").newInstance();
+
+            cacheCfg.setNodeFilter((IgnitePredicate)pred);
+        }
+        catch (Exception e) {
+            throw new RuntimeException(e);
+        }
     }
 
     /**
@@ -299,9 +332,9 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If test failed.
      */
-    public void testStore() throws Exception {
+    public void testExternalClassesAtConfiguration() throws Exception {
         try {
-            needStore = true;
+            extClassloadingAtCfg = true;
             useExtClassLoader = true;
 
             Ignite g1 = startGrid(1);
@@ -316,13 +349,47 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
             IgniteCache<Integer, Object> cache2 = g2.cache(null);
             IgniteCache<Integer, Object> cache3 = g3.cache(null);
 
+            final Class<CacheEntryListener> cls1 = (Class<CacheEntryListener>) getExternalClassLoader().
+                loadClass("org.apache.ignite.tests.p2p.CacheDeploymentCacheEntryListener");
+            final Class<CacheEntryEventSerializableFilter> cls2 = (Class<CacheEntryEventSerializableFilter>) getExternalClassLoader().
+                loadClass("org.apache.ignite.tests.p2p.CacheDeploymentCacheEntryEventSerializableFilter");
+
+            CacheEntryListenerConfiguration<Integer, Object> lsnrCfg = new MutableCacheEntryListenerConfiguration<>(
+                new Factory<CacheEntryListener<Integer, Object>>() {
+                    @Override public CacheEntryListener<Integer, Object> create() {
+                        try {
+                            return cls1.newInstance();
+                        }
+                        catch (Exception e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                },
+                new Factory<CacheEntryEventSerializableFilter<Integer, Object>>() {
+                    /** {@inheritDoc} */
+                    @Override public CacheEntryEventSerializableFilter<Integer, Object> create() {
+                        try {
+
+                            return cls2.newInstance();
+                        }
+                        catch (Exception e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                },
+                true,
+                true
+            );
+
+            cache1.registerCacheEntryListener(lsnrCfg);
+
             cache1.put(1, 1);
 
             assertEquals(1, cache2.get(1));
             assertEquals(1, cache3.get(1));
         }
         finally {
-            needStore = false;
+            extClassloadingAtCfg = false;
             isClient = false;
             useExtClassLoader = false;
         }
@@ -331,9 +398,9 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If test failed.
      */
-    public void testStoreDynamicStart() throws Exception {
+    public void testExternalClassesAtConfigurationDynamicStart() throws Exception {
         try {
-            needStore = false;
+            extClassloadingAtCfg = false;
             useExtClassLoader = true;
 
             Ignite g1 = startGrid(1);
@@ -343,12 +410,10 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
 
             Ignite g3 = startGrid(3);
 
-            Object sf = getExternalClassLoader().loadClass(
-                "org.apache.ignite.tests.p2p.CacheDeploymentTestStoreFactory").newInstance();
-
             CacheConfiguration cfg = defaultCacheConfiguration();
 
-            cfg.setCacheStoreFactory((Factory)sf);
+            loadExternalClassesToCfg(cfg);
+
             cfg.setName("customStore");
 
             IgniteCache<Integer, Object> cache1 = g1.createCache(cfg);
@@ -362,7 +427,7 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
             assertEquals(1, cache3.get(1));
         }
         finally {
-            needStore = false;
+            extClassloadingAtCfg = false;
             isClient = false;
             useExtClassLoader = false;
         }
@@ -371,9 +436,9 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If test failed.
      */
-    public void testStoreDynamicStart2() throws Exception {
+    public void testExternalClassesAtConfigurationDynamicStart2() throws Exception {
         try {
-            needStore = false;
+            extClassloadingAtCfg = false;
             useExtClassLoader = true;
 
             Ignite g1 = startGrid(1);
@@ -383,12 +448,10 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
 
             Ignite g3 = startGrid(3);
 
-            Object sf = getExternalClassLoader().loadClass(
-                "org.apache.ignite.tests.p2p.CacheDeploymentTestStoreFactory").newInstance();
-
             CacheConfiguration cfg = defaultCacheConfiguration();
 
-            cfg.setCacheStoreFactory((Factory)sf);
+            loadExternalClassesToCfg(cfg);
+
             cfg.setName("customStore");
 
             IgniteCache<Integer, Object> cache1 = g1.getOrCreateCache(cfg);
@@ -402,7 +465,7 @@ public class GridCacheReplicatedPreloadSelfTest extends GridCommonAbstractTest {
             assertEquals(1, cache3.get(1));
         }
         finally {
-            needStore = false;
+            extClassloadingAtCfg = false;
             isClient = false;
             useExtClassLoader = false;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a32dfc41/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentAffinityKeyMapper.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentAffinityKeyMapper.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentAffinityKeyMapper.java
new file mode 100644
index 0000000..fbb74d2
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentAffinityKeyMapper.java
@@ -0,0 +1,35 @@
+/*
+ * 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.affinity.AffinityKeyMapper;
+
+/**
+ * Test affinity ley mapper for cache deployment tests.
+ */
+public class CacheDeploymentAffinityKeyMapper implements AffinityKeyMapper {
+    /** {@inheritDoc} */
+    @Override public Object affinityKey(Object key) {
+        return key;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a32dfc41/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentAlwaysTruePredicate2.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentAlwaysTruePredicate2.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentAlwaysTruePredicate2.java
new file mode 100644
index 0000000..d88c7bf
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentAlwaysTruePredicate2.java
@@ -0,0 +1,30 @@
+/*
+ * 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.lang.IgnitePredicate;
+
+/**
+ *
+ */
+public class CacheDeploymentAlwaysTruePredicate2 implements IgnitePredicate<Object> {
+    /** */
+    @Override public boolean apply(Object o) {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a32dfc41/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCacheEntryEventSerializableFilter.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCacheEntryEventSerializableFilter.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCacheEntryEventSerializableFilter.java
new file mode 100644
index 0000000..c29c1a4
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCacheEntryEventSerializableFilter.java
@@ -0,0 +1,32 @@
+/*
+ * 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.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
+import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
+
+/**
+ *
+ */
+public class CacheDeploymentCacheEntryEventSerializableFilter implements CacheEntryEventSerializableFilter {
+    /** {@inheritDoc} */
+    @Override public boolean evaluate(CacheEntryEvent event) throws CacheEntryListenerException {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a32dfc41/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCacheEntryListener.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCacheEntryListener.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCacheEntryListener.java
new file mode 100644
index 0000000..64c13fb
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCacheEntryListener.java
@@ -0,0 +1,31 @@
+/*
+ * 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.event.CacheEntryCreatedListener;
+import javax.cache.event.CacheEntryListenerException;
+
+/**
+ *
+ */
+public class CacheDeploymentCacheEntryListener implements CacheEntryCreatedListener {
+    /** {@inheritDoc} */
+    @Override public void onCreated(Iterable iterable) throws CacheEntryListenerException {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a32dfc41/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCachePluginConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCachePluginConfiguration.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCachePluginConfiguration.java
new file mode 100644
index 0000000..bb37c25
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentCachePluginConfiguration.java
@@ -0,0 +1,74 @@
+/*
+ * 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.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.plugin.CachePluginConfiguration;
+import org.apache.ignite.plugin.CachePluginContext;
+import org.apache.ignite.plugin.CachePluginProvider;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test cache plugin configuration for cache deployment tests.
+ */
+public class CacheDeploymentCachePluginConfiguration<K, V> implements CachePluginConfiguration<K, V> {
+    /** {@inheritDoc} */
+    @Override public CachePluginProvider createProvider(CachePluginContext ctx) {
+        return new CacheDeploymentCachePluginProvider();
+    }
+
+    private static class CacheDeploymentCachePluginProvider implements CachePluginProvider {
+        /** {@inheritDoc} */
+        @Nullable @Override public Object createComponent(Class cls) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void start() throws IgniteCheckedException {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void stop(boolean cancel) {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIgniteStart() throws IgniteCheckedException {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIgniteStop(boolean cancel) {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void validate() throws IgniteCheckedException {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void validateRemote(CacheConfiguration locCfg, CachePluginConfiguration locPluginCcfg,
+            CacheConfiguration rmtCfg, ClusterNode rmtNode) throws IgniteCheckedException {
+
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a32dfc41/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentStoreSessionListenerFactory.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentStoreSessionListenerFactory.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentStoreSessionListenerFactory.java
new file mode 100644
index 0000000..74d9d21
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentStoreSessionListenerFactory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.Factory;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.store.CacheStoreSession;
+import org.apache.ignite.cache.store.CacheStoreSessionListener;
+import org.apache.ignite.lifecycle.LifecycleAware;
+
+/**
+ * Test store session listener factory for cache deployment tests.
+ */
+public class CacheDeploymentStoreSessionListenerFactory implements Factory<CacheStoreSessionListener> {
+    /** */
+    private String name;
+
+    /**
+     *
+     */
+    public CacheDeploymentStoreSessionListenerFactory() {
+    }
+
+    /**
+     * @param name Name.
+     */
+    public CacheDeploymentStoreSessionListenerFactory(String name) {
+        this.name = name;
+    }
+
+    @Override public CacheStoreSessionListener create() {
+        return new CacheDeploymentSessionListener(name);
+    }
+
+    /**
+     */
+    private static class CacheDeploymentSessionListener implements CacheStoreSessionListener, LifecycleAware {
+        /** */
+        private final String name;
+
+        /**
+         * @param name Name.
+         */
+        private CacheDeploymentSessionListener(String name) {
+            this.name = name;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void start() throws IgniteException {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void stop() throws IgniteException {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionStart(CacheStoreSession ses) {
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+
+        }
+    }
+}
\ No newline at end of file