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/06/03 10:34:46 UTC

incubator-ignite git commit: ignite-471-2: using optimizedmarshaller to marshal objects that redefine writeObj and readObj

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-471-2 f2a2daae2 -> 145a72120


ignite-471-2: using optimizedmarshaller to marshal objects that redefine writeObj and readObj


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

Branch: refs/heads/ignite-471-2
Commit: 145a72120c28955d6d2f463ee0afc2a9f8cdc599
Parents: f2a2daa
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jun 3 11:34:27 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jun 3 11:34:27 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/MarshallerContextImpl.java  | 10 ++-----
 .../GridCacheSwapScanQueryAbstractSelfTest.java | 22 +++++++--------
 .../closure/GridClosureProcessorSelfTest.java   | 26 +++++++++---------
 .../marshaller/MarshallerContextTestImpl.java   |  9 ++++--
 .../p2p/GridP2PLocalDeploymentSelfTest.java     |  6 ++--
 .../p2p/GridP2PRemoteClassLoadersSelfTest.java  | 29 ++++++++++----------
 6 files changed, 52 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/145a7212/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 85939a6..f964bec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -90,13 +90,9 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
         try {
             old = cache0.tryPutIfAbsent(id, clsName);
 
-            if (old != null && !old.equals(clsName)) {
-                U.quietAndWarn(log, "Type ID collision detected, may affect performance " +
-                    "(set idMapper property on marshaller to fix) [id=" + id + ", clsName1=" + clsName +
-                    "clsName2=" + old + ']');
-
-                return false;
-            }
+            if (old != null && !old.equals(clsName))
+                throw new IgniteCheckedException("Type ID collision detected [id=" + id + ", clsName1=" + clsName +
+                    ", clsName2=" + old + ']');
 
             failedCnt = 0;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/145a7212/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
index 931e3e5..07322ef 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
@@ -126,19 +126,19 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
         final int ENTRY_CNT = 500;
 
         for (int i = 0; i < ENTRY_CNT; i++)
-            cache.getAndPut(new Key(i), new Person("p-" + i, i));
+            cache.getAndPut(new Key(i), new Person2("p-" + i, i));
 
         try {
-            CacheQuery<Map.Entry<Key, Person>> qry =
+            CacheQuery<Map.Entry<Key, Person2>> qry =
                 cache.context().queries().createScanQuery(new QueryFilter(), false);
 
-            Collection<Map.Entry<Key, Person>> res = qry.execute().get();
+            Collection<Map.Entry<Key, Person2>> res = qry.execute().get();
 
             assertEquals(ENTRY_CNT / 2, res.size());
 
-            for (Map.Entry<Key, Person> e : res) {
+            for (Map.Entry<Key, Person2> e : res) {
                 Key k = e.getKey();
-                Person p = e.getValue();
+                Person2 p = e.getValue();
 
                 assertEquals(k.id, (Integer)p.salary);
                 assertEquals(0, k.id % 2);
@@ -169,11 +169,11 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
         GridTestUtils.runMultiThreaded(new Callable<Void>() {
             @SuppressWarnings("unchecked")
             @Override public Void call() throws Exception {
-                CacheQuery<Map.Entry<Key, Person>> qry = cache.context().queries().createScanQuery(
+                CacheQuery<Map.Entry<Key, Person2>> qry = cache.context().queries().createScanQuery(
                     new QueryFilter(), false);
 
                 for (int i = 0; i < 250; i++) {
-                    Collection<Map.Entry<Key, Person>> res = qry.execute().get();
+                    Collection<Map.Entry<Key, Person2>> res = qry.execute().get();
 
                     assertEquals(expCnt, res.size());
 
@@ -335,7 +335,7 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
      *
      */
     @SuppressWarnings("PublicInnerClass")
-    public static class Person {
+    public static class Person2 {
         /** */
         @SuppressWarnings("PublicField")
         public String name;
@@ -348,7 +348,7 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
          * @param name Name.
          * @param salary Salary.
          */
-        public Person(String name, int salary) {
+        public Person2(String name, int salary) {
             this.name = name;
             this.salary = salary;
         }
@@ -357,9 +357,9 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA
     /**
      *
      */
-    private static class QueryFilter implements IgniteBiPredicate<Key, Person> {
+    private static class QueryFilter implements IgniteBiPredicate<Key, Person2> {
         /** {@inheritDoc} */
-        @Override public boolean apply(Key key, Person p) {
+        @Override public boolean apply(Key key, Person2 p) {
             assertEquals(key.id, (Integer)p.salary);
 
             return key.id % 2 == 0;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/145a7212/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java
index ec5067e..381db74 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java
@@ -92,7 +92,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
     /**
      * Test runnable job.
      */
-    private static class TestRunnable implements IgniteRunnable {
+    private static class ClosureTestRunnable implements IgniteRunnable {
         /** */
         @IgniteInstanceResource
         private Ignite ignite;
@@ -127,7 +127,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
     /**
      * Test callable job.
      */
-    private static class TestCallable extends AbstractTestCallable {
+    private static class ClosureTestCallable extends AbstractTestCallable {
         /** {@inheritDoc} */
         @Override public Integer call() {
             log.info("Callable job executed on node: " + ignite.cluster().localNode().id());
@@ -141,11 +141,11 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
     /**
      * Test callable job which throws class not found exception.
      */
-    private static class TestCallableError extends AbstractTestCallable implements Externalizable {
+    private static class ClosureTestCallableError extends AbstractTestCallable implements Externalizable {
         /**
          *
          */
-        public TestCallableError() {
+        public ClosureTestCallableError() {
             // No-op.
         }
 
@@ -231,7 +231,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @return Future object.
      */
     private ComputeTaskFuture<?> runAsync(int idx,
-        Collection<TestRunnable> jobs,
+        Collection<ClosureTestRunnable> jobs,
         @Nullable IgnitePredicate<ClusterNode> p)
     {
         assert idx >= 0 && idx < NODES_CNT;
@@ -299,7 +299,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @param p Optional node predicate.
      * @return Future object.
      */
-    private ComputeTaskFuture<Collection<Integer>> callAsync(int idx, Collection<TestCallable> jobs,
+    private ComputeTaskFuture<Collection<Integer>> callAsync(int idx, Collection<ClosureTestCallable> jobs,
         @Nullable IgnitePredicate<ClusterNode> p) {
         assert idx >= 0 && idx < NODES_CNT;
         assert !F.isEmpty(jobs);
@@ -331,7 +331,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testRunAsyncSingle() throws Exception {
-        IgniteRunnable job = new TestRunnable();
+        IgniteRunnable job = new ClosureTestRunnable();
 
         ComputeTaskFuture<?> fut = broadcast(0, job, null);
 
@@ -357,7 +357,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testRunAsyncMultiple() throws Exception {
-        Collection<TestRunnable> jobs = F.asList(new TestRunnable(), new TestRunnable());
+        Collection<ClosureTestRunnable> jobs = F.asList(new ClosureTestRunnable(), new ClosureTestRunnable());
 
         ComputeTaskFuture<?> fut = runAsync(0, jobs, null);
 
@@ -371,7 +371,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCallAsyncSingle() throws Exception {
-        IgniteCallable<Integer> job = new TestCallable();
+        IgniteCallable<Integer> job = new ClosureTestCallable();
 
         ComputeTaskFuture<Collection<Integer>> fut1 = broadcast(0, job, null);
 
@@ -401,7 +401,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
     public void testCallAsyncErrorNoFailover() throws Exception {
         IgniteCompute comp = compute(grid(0).cluster().forPredicate(F.notEqualTo(grid(0).localNode()))).withAsync();
 
-        comp.withNoFailover().call(new TestCallableError());
+        comp.withNoFailover().call(new ClosureTestCallableError());
 
         ComputeTaskFuture<Integer> fut = comp.future();
 
@@ -419,7 +419,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testWithName() throws Exception {
-        grid(0).compute().withName("TestTaskName").call(new TestCallable());
+        grid(0).compute().withName("TestTaskName").call(new ClosureTestCallable());
     }
 
     /**
@@ -457,7 +457,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCallAsyncMultiple() throws Exception {
-        Collection<TestCallable> jobs = F.asList(new TestCallable(), new TestCallable());
+        Collection<ClosureTestCallable> jobs = F.asList(new ClosureTestCallable(), new ClosureTestCallable());
 
         ComputeTaskFuture<Collection<Integer>> fut = callAsync(0, jobs, null);
 
@@ -476,7 +476,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testReduceAsync() throws Exception {
-        Collection<TestCallable> jobs = F.asList(new TestCallable(), new TestCallable());
+        Collection<ClosureTestCallable> jobs = F.asList(new ClosureTestCallable(), new ClosureTestCallable());
 
         IgniteCompute comp = grid(0).compute().withAsync();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/145a7212/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java
index dce3829..4164fda 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.marshaller;
 
+import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.jsr166.*;
 
@@ -30,8 +31,12 @@ public class MarshallerContextTestImpl extends MarshallerContextAdapter {
     private final static ConcurrentMap<Integer, String> map = new ConcurrentHashMap8<>();
 
     /** {@inheritDoc} */
-    @Override protected boolean registerClassName(int id, String clsName) {
-        map.putIfAbsent(id, clsName);
+    @Override protected boolean registerClassName(int id, String clsName) throws IgniteCheckedException {
+        String oldClsName = map.putIfAbsent(id, clsName);
+
+        if (oldClsName != null && !oldClsName.equals(clsName))
+            throw new IgniteCheckedException("Duplicate ID [id=" + id + ", oldClsName=" + oldClsName + ", clsName=" +
+                clsName + ']');
 
         return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/145a7212/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PLocalDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PLocalDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PLocalDeploymentSelfTest.java
index 28eb687..51a7574 100644
--- a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PLocalDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PLocalDeploymentSelfTest.java
@@ -212,7 +212,7 @@ public class GridP2PLocalDeploymentSelfTest extends GridCommonAbstractTest {
 
             for (ClusterNode node : subgrid) {
                 if (node.id().equals(arg))
-                    return Collections.singletonMap(new TestJob(arg), node);
+                    return Collections.singletonMap(new DeployementTestJob(arg), node);
             }
 
             throw new IgniteException("Failed to find target node: " + arg);
@@ -230,7 +230,7 @@ public class GridP2PLocalDeploymentSelfTest extends GridCommonAbstractTest {
         /**
          * Simple job class.
          */
-        public static class TestJob extends ComputeJobAdapter {
+        public static class DeployementTestJob extends ComputeJobAdapter {
             /** Ignite instance. */
             @IgniteInstanceResource
             private Ignite ignite;
@@ -238,7 +238,7 @@ public class GridP2PLocalDeploymentSelfTest extends GridCommonAbstractTest {
             /**
              * @param nodeId Node ID for node this job is supposed to execute on.
              */
-            public TestJob(UUID nodeId) { super(nodeId); }
+            public DeployementTestJob(UUID nodeId) { super(nodeId); }
 
             /** {@inheritDoc} */
             @Override public Serializable execute() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/145a7212/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRemoteClassLoadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRemoteClassLoadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRemoteClassLoadersSelfTest.java
index a6e3411..086d887 100644
--- a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRemoteClassLoadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRemoteClassLoadersSelfTest.java
@@ -43,9 +43,9 @@ public class GridP2PRemoteClassLoadersSelfTest extends GridCommonAbstractTest {
 
         // Override P2P configuration to exclude Task and Job classes
         cfg.setPeerClassLoadingLocalClassPathExclude(
-            GridP2PTestTask.class.getName(),
-            GridP2PTestTask1.class.getName(),
-            GridP2PTestJob.class.getName(),
+            GridP2PRemoteTestTask.class.getName(),
+            GridP2PRemoteTestTask1.class.getName(),
+            GridP2PRemoteTestJob.class.getName(),
             GridP2PRemoteClassLoadersSelfTest.class.getName()
         );
 
@@ -73,13 +73,14 @@ public class GridP2PRemoteClassLoadersSelfTest extends GridCommonAbstractTest {
             ClassLoader tstClsLdr =
                 new GridTestClassLoader(
                     Collections.<String, String>emptyMap(), getClass().getClassLoader(),
-                    GridP2PTestTask.class.getName(), GridP2PTestTask1.class.getName(), GridP2PTestJob.class.getName());
+                    GridP2PRemoteTestTask.class.getName(), GridP2PRemoteTestTask1.class.getName(),
+                    GridP2PRemoteTestJob.class.getName());
 
             Class<? extends ComputeTask<?, ?>> task1 =
-                (Class<? extends ComputeTask<?, ?>>) tstClsLdr.loadClass(GridP2PTestTask.class.getName());
+                (Class<? extends ComputeTask<?, ?>>) tstClsLdr.loadClass(GridP2PRemoteTestTask.class.getName());
 
             Class<? extends ComputeTask<?, ?>> task2 =
-                (Class<? extends ComputeTask<?, ?>>) tstClsLdr.loadClass(GridP2PTestTask1.class.getName());
+                (Class<? extends ComputeTask<?, ?>>) tstClsLdr.loadClass(GridP2PRemoteTestTask1.class.getName());
 
             Object res1 = ignite1.compute().execute(task1.newInstance(), null);
 
@@ -124,19 +125,19 @@ public class GridP2PRemoteClassLoadersSelfTest extends GridCommonAbstractTest {
             ClassLoader tstClsLdr1 =
                 new GridTestClassLoader(
                     Collections.EMPTY_MAP, getClass().getClassLoader(),
-                    GridP2PTestTask.class.getName(), GridP2PTestJob.class.getName()
+                    GridP2PRemoteTestTask.class.getName(), GridP2PRemoteTestJob.class.getName()
                 );
 
             ClassLoader tstClsLdr2 =
                 new GridTestClassLoader(
                     Collections.EMPTY_MAP, getClass().getClassLoader(),
-                    GridP2PTestTask1.class.getName(), GridP2PTestJob.class.getName());
+                    GridP2PRemoteTestTask1.class.getName(), GridP2PRemoteTestJob.class.getName());
 
             Class<? extends ComputeTask<?, ?>> task1 =
-                (Class<? extends ComputeTask<?, ?>>) tstClsLdr1.loadClass(GridP2PTestTask.class.getName());
+                (Class<? extends ComputeTask<?, ?>>) tstClsLdr1.loadClass(GridP2PRemoteTestTask.class.getName());
 
             Class<? extends ComputeTask<?, ?>> task2 =
-                (Class<? extends ComputeTask<?, ?>>) tstClsLdr2.loadClass(GridP2PTestTask1.class.getName());
+                (Class<? extends ComputeTask<?, ?>>) tstClsLdr2.loadClass(GridP2PRemoteTestTask1.class.getName());
 
             Object res1 = ignite1.compute().execute(task1.newInstance(), null);
 
@@ -217,11 +218,11 @@ public class GridP2PRemoteClassLoadersSelfTest extends GridCommonAbstractTest {
     /**
      * P2P test job.
      */
-    public static class GridP2PTestJob extends ComputeJobAdapter {
+    public static class GridP2PRemoteTestJob extends ComputeJobAdapter {
         /**
          * @param arg Argument.
          */
-        public GridP2PTestJob(String arg) {
+        public GridP2PRemoteTestJob(String arg) {
             super(arg);
         }
 
@@ -235,7 +236,7 @@ public class GridP2PRemoteClassLoadersSelfTest extends GridCommonAbstractTest {
     /**
      * P2P test task.
      */
-    public static class GridP2PTestTask extends ComputeTaskAdapter<Serializable, Object> {
+    public static class GridP2PRemoteTestTask extends ComputeTaskAdapter<Serializable, Object> {
         /** */
         @LoggerResource
         private IgniteLogger log;
@@ -275,7 +276,7 @@ public class GridP2PRemoteClassLoadersSelfTest extends GridCommonAbstractTest {
     /**
      * P2p test task.
      */
-    public static class GridP2PTestTask1 extends GridP2PTestTask {
+    public static class GridP2PRemoteTestTask1 extends GridP2PRemoteTestTask {
         // No-op.
     }
 }