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/12/10 10:50:24 UTC

[17/26] ignite git commit: IGNITE-2064 Fixed deserialize. Added more tests.

IGNITE-2064 Fixed deserialize. Added more tests.


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

Branch: refs/heads/ignite-1537
Commit: be17c9eef8a1afa647214089e3d7689e97bb5bea
Parents: 9413232
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Dec 9 17:33:59 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 10 11:17:25 2015 +0300

----------------------------------------------------------------------
 .../internal/portable/BinaryObjectImpl.java     |  28 ++--
 .../internal/portable/PortableContext.java      |   7 +
 .../processors/cache/CacheLazyEntry.java        |  12 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   2 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |  69 ++++++----
 ...yAtomicEntryProcessorDeploymentSelfTest.java | 129 +++++++++++++++++++
 ...ctionalEntryProcessorDeploymentSelfTest.java |  31 +++++
 .../IgniteBinaryObjectsCacheTestSuite3.java     |   9 +-
 ...cheDeploymentBinaryObjectEntryProcessor.java |  36 ++++++
 9 files changed, 284 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
index cf5a659..c8528d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/BinaryObjectImpl.java
@@ -17,6 +17,16 @@
 
 package org.apache.ignite.internal.portable;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.util.Date;
+import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectException;
@@ -33,17 +43,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.jetbrains.annotations.Nullable;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.sql.Timestamp;
-import java.util.Date;
-import java.util.UUID;
-
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
@@ -543,6 +542,8 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     /**
      * Runs value deserialization regardless of whether obj already has the deserialized value.
      * Will set obj if descriptor is configured to keep deserialized values.
+     * @param coCtx CacheObjectContext.
+     * @return Object.
      */
     private Object deserializeValue(@Nullable CacheObjectContext coCtx) {
         // TODO: IGNITE-1272 - Deserialize with proper class loader.
@@ -575,6 +576,9 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
      * @return Reader.
      */
     private BinaryReaderExImpl reader(@Nullable BinaryReaderHandles rCtx) {
-        return new BinaryReaderExImpl(ctx, PortableHeapInputStream.create(arr, start), null, rCtx);
+        return new BinaryReaderExImpl(ctx,
+            PortableHeapInputStream.create(arr, start),
+            ctx.configuration().getClassLoader(),
+            rCtx);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 8bad737..2c7e4c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -234,6 +234,13 @@ public class PortableContext implements Externalizable {
     }
 
     /**
+     * @return Ignite configuration.
+     */
+    public IgniteConfiguration configuration(){
+        return igniteCfg;
+    }
+
+    /**
      * @param marsh Portable marshaller.
      * @param cfg Configuration.
      * @throws BinaryObjectException In case of error.

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
index 2b9efa3..713bb3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
@@ -77,7 +77,7 @@ public class CacheLazyEntry<K, V> implements Cache.Entry<K, V> {
      * @param valObj Cache object
      * @param val Cache value.
      */
-    public CacheLazyEntry(GridCacheContext<K, V> ctx, 
+    public CacheLazyEntry(GridCacheContext<K, V> ctx,
         KeyCacheObject keyObj,
         K key,
         CacheObject valObj,
@@ -102,6 +102,16 @@ public class CacheLazyEntry<K, V> implements Cache.Entry<K, V> {
 
     /** {@inheritDoc} */
     @Override public V getValue() {
+        return getValue(keepPortable);
+    }
+
+    /**
+     * Returns the value stored in the cache when this entry was created.
+     *
+     * @param keepPortable Flag to keep portable if needed.
+     * @return the value corresponding to this entry
+     */
+    public V getValue(boolean keepPortable) {
         if (val == null)
             val = (V)cctx.unwrapPortableIfNeeded(valObj, keepPortable, false);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 3829e28..40399b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -382,7 +382,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                                     procRes = processor.process(invokeEntry, t.get2());
 
-                                    val = cacheCtx.toCacheObject(invokeEntry.getValue());
+                                    val = cacheCtx.toCacheObject(invokeEntry.getValue(true));
                                 }
                                 catch (Exception e) {
                                     err = e;

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java
index 35ee6cc..137e90f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicEntryProcessorDeploymentSelfTest.java
@@ -45,27 +45,24 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
-    /** Entry processor */
-    protected static String TEST_ENT_PROCESSOR =
-        GridTestProperties.getProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME) != null ?
-            GridTestProperties.getProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME) :
-            "org.apache.ignite.tests.p2p.CacheDeploymentEntryProcessor";
-
     /** Test value. */
     protected static String TEST_VALUE = "org.apache.ignite.tests.p2p.CacheDeploymentTestValue";
 
     /** */
-    private DeploymentMode depMode;
+    protected DeploymentMode depMode;
 
     /** */
-    private boolean cliendMode;
+    protected boolean clientMode;
+
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        if (cliendMode)
-            cfg.setClientMode(cliendMode);
+        cfg.setClientMode(clientMode);
+
+        if (clientMode)
+            cfg.setClassLoader(getExternalClassLoader());
 
         cfg.setDeploymentMode(depMode);
 
@@ -99,6 +96,22 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
         return cfg;
     }
 
+    /**
+     * @return Cache.
+     */
+    protected IgniteCache getCache(){
+        return grid(1).cache(null);
+    }
+
+    /**
+     * @return Entry Processor.
+     */
+    protected String getEntryProcessor(){
+       return GridTestProperties.getProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME) != null ?
+            GridTestProperties.getProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME) :
+            "org.apache.ignite.tests.p2p.CacheDeploymentEntryProcessor";
+    }
+
     protected CacheAtomicityMode atomicityMode() {
         return ATOMIC;
     }
@@ -144,26 +157,28 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
      */
     private void doTestInvoke() throws Exception {
         try {
-            cliendMode = false;
+            clientMode = false;
             startGrid(0);
 
-            cliendMode = true;
+            clientMode = true;
             startGrid(1);
 
-            ClassLoader ldr = getExternalClassLoader();
-
-            Class procCls = ldr.loadClass(TEST_ENT_PROCESSOR);
-            Class valCls = ldr.loadClass(TEST_VALUE);
+            Class procCls = grid(1).configuration().getClassLoader().loadClass(getEntryProcessor());
+            Class valCls = grid(1).configuration().getClassLoader().loadClass(TEST_VALUE);
 
             assertTrue(grid(1).configuration().isClientMode());
+            assertFalse(grid(0).configuration().isClientMode());
 
-            IgniteCache cache = grid(1).cache(null);
+            IgniteCache cache = getCache();
 
             cache.put("key", valCls.newInstance());
 
             Boolean res = (Boolean)cache.invoke("key", (CacheEntryProcessor)procCls.newInstance());
 
             assertTrue(res);
+
+            // Checks that get produces no exceptions.
+            cache.get("key");
         }
         finally {
             stopAllGrids();
@@ -175,20 +190,19 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
      */
     private void doTestInvokeAll() throws Exception {
         try {
-            cliendMode = false;
+            clientMode = false;
             startGrid(0);
 
-            cliendMode = true;
+            clientMode = true;
             startGrid(1);
 
-            ClassLoader ldr = getExternalClassLoader();
-
-            Class procCls = ldr.loadClass(TEST_ENT_PROCESSOR);
-            Class valCls = ldr.loadClass(TEST_VALUE);
+            Class procCls = grid(1).configuration().getClassLoader().loadClass(getEntryProcessor());
+            Class valCls = grid(1).configuration().getClassLoader().loadClass(TEST_VALUE);
 
             assertTrue(grid(1).configuration().isClientMode());
+            assertFalse(grid(0).configuration().isClientMode());
 
-            IgniteCache cache = grid(1).cache(null);
+            IgniteCache cache = getCache();
 
             HashSet keys = new HashSet();
 
@@ -207,6 +221,13 @@ public class GridCacheAtomicEntryProcessorDeploymentSelfTest extends GridCommonA
 
             for (EntryProcessorResult result : res.values())
                 assertTrue((Boolean)result.get());
+
+            // Checks that get produces no exceptions.
+            for (int i = 0; i < 3; i++) {
+                String key = "key" + i;
+
+                cache.get(key);
+            }
         }
         finally {
             stopAllGrids();

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.java
new file mode 100644
index 0000000..1004e7a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.java
@@ -0,0 +1,129 @@
+/*
+ *  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.portable;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.configuration.DeploymentMode;
+import org.apache.ignite.internal.processors.cache.GridCacheAtomicEntryProcessorDeploymentSelfTest;
+
+/**
+ * Cache EntryProcessor + Deployment.
+ */
+public class GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest
+    extends GridCacheAtomicEntryProcessorDeploymentSelfTest {
+    /** {@inheritDoc} */
+    protected IgniteCache getCache() {
+        return grid(1).cache(null).withKeepBinary();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String getEntryProcessor() {
+        return "org.apache.ignite.tests.p2p.CacheDeploymentBinaryObjectEntryProcessor";
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetDeployment() throws Exception {
+        depMode = DeploymentMode.CONTINUOUS;
+
+        doTestGet(false);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetDeployment2() throws Exception {
+        depMode = DeploymentMode.SHARED;
+
+        doTestGet(false);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetDeploymentWithKeepBinary() throws Exception {
+        depMode = DeploymentMode.CONTINUOUS;
+
+        doTestGet(true);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetDeployment2WithKeepBinary() throws Exception {
+        depMode = DeploymentMode.SHARED;
+
+        doTestGet(true);
+    }
+
+    /**
+     * @throws Exception Exception.
+     */
+    private void doTestGet(boolean withKeepBinary) throws Exception {
+        try {
+            clientMode = false;
+            startGrid(0);
+
+            clientMode = true;
+            startGrid(1);
+
+            Class valCls = grid(1).configuration().getClassLoader().loadClass(TEST_VALUE);
+
+            assertTrue(grid(1).configuration().isClientMode());
+            assertFalse(grid(0).configuration().isClientMode());
+
+            IgniteCache cache1 = grid(1).cache(null);
+            IgniteCache cache0 = grid(0).cache(null);
+
+            if (withKeepBinary) {
+                cache1 = cache1.withKeepBinary();
+                cache0 = cache0.withKeepBinary();
+            }
+
+            cache1.put("key", valCls.newInstance());
+
+            if (withKeepBinary) {
+                BinaryObject obj = (BinaryObject)(cache0.get("key"));
+
+                try {
+                    obj.deserialize();
+
+                    fail("Exception did not happened.");
+                }
+                catch (BinaryInvalidTypeException e) {
+                    // No-op.
+                }
+            }
+            else
+                try {
+                    cache0.get("key");
+
+                    fail("Exception did not happened.");
+                }
+                catch (BinaryInvalidTypeException e) {
+                    // No-op.
+                }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.java
new file mode 100644
index 0000000..c115754
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.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.internal.processors.cache.portable;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+
+/**
+ * Cache EntryProcessor + Deployment for transactional cache.
+ */
+public class GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest extends
+    GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return CacheAtomicityMode.TRANSACTIONAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java
index 3d25645..3aecda3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryObjectsCacheTestSuite3.java
@@ -19,6 +19,8 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.portable.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest;
 import org.apache.ignite.testframework.config.GridTestProperties;
 
 /**
@@ -34,6 +36,11 @@ public class IgniteBinaryObjectsCacheTestSuite3 {
         GridTestProperties.setProperty(GridTestProperties.ENTRY_PROCESSOR_CLASS_NAME,
             "org.apache.ignite.tests.p2p.CacheDeploymentPortableEntryProcessor");
 
-        return IgniteCacheTestSuite3.suite();
+        TestSuite suite = IgniteCacheTestSuite3.suite();
+
+        suite.addTestSuite(GridCacheBinaryAtomicEntryProcessorDeploymentSelfTest.class);
+        suite.addTestSuite(GridCacheBinaryTransactionalEntryProcessorDeploymentSelfTest.class);
+
+        return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/be17c9ee/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentBinaryObjectEntryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentBinaryObjectEntryProcessor.java b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentBinaryObjectEntryProcessor.java
new file mode 100644
index 0000000..7c6895c
--- /dev/null
+++ b/modules/extdata/p2p/src/main/java/org/apache/ignite/tests/p2p/CacheDeploymentBinaryObjectEntryProcessor.java
@@ -0,0 +1,36 @@
+/*
+ *  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.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CacheEntryProcessor;
+
+/**
+ * Entry processor for {@code GridCacheEntryProcessorDeploymentSelfTest}.
+ */
+public class CacheDeploymentBinaryObjectEntryProcessor implements CacheEntryProcessor<String, BinaryObject, Boolean> {
+    /** {@inheritDoc} */
+    @Override public Boolean process(MutableEntry<String, BinaryObject> entry,
+        Object... arguments) throws EntryProcessorException {
+        BinaryObject val = entry.getValue();
+
+        return true;
+    }
+}