You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by nt...@apache.org on 2016/02/01 15:21:19 UTC

[11/49] ignite git commit: Do not show 'you should implement Externalizable' warning

Do not show 'you should implement Externalizable' warning


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

Branch: refs/heads/ignite-2435
Commit: 457e8c11c813f208bc7e007aa1159dd04e68d0e8
Parents: a4cf78a
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Wed Jan 27 19:04:48 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Wed Jan 27 19:04:48 2016 -0800

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  78 +----------
 .../processors/cache/GridCacheUtils.java        |  36 +----
 .../cache/GridCacheUtilsSelfTest.java           | 136 +------------------
 3 files changed, 11 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/457e8c11/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 2582e6c..3081cfb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -216,9 +216,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /** */
     protected boolean keyCheck = !Boolean.getBoolean(IGNITE_CACHE_KEY_VALIDATION_DISABLED);
 
-    /** */
-    private boolean valCheck = true;
-
     /** Last asynchronous future. */
     protected ThreadLocal<FutureHolder> lastFut = new ThreadLocal<FutureHolder>() {
         @Override protected FutureHolder initialValue() {
@@ -1926,8 +1923,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         V prevVal = syncOp(new SyncOp<V>(true) {
             @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return (V)tx.putAsync(ctx, key, val, true, filter).get().value();
@@ -1981,8 +1976,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         return asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx) {
                 return tx.putAsync(ctx, key, val, true, filter)
@@ -2020,8 +2013,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         Boolean stored = syncOp(new SyncOp<Boolean>(true) {
             @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return tx.putAsync(ctx, key, val, false, filter).get().success();
@@ -2341,8 +2332,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx) {
                 return tx.putAsync(ctx, key, val, false, filter).chain(
@@ -2368,8 +2357,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         return syncOp(new SyncOp<V>(true) {
             @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return (V)tx.putAsync(ctx, key, val, true, ctx.noValArray()).get().value();
@@ -2392,8 +2379,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx) {
                 return tx.putAsync(ctx, key, val, true, ctx.noValArray())
@@ -2422,8 +2407,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         Boolean stored = syncOp(new SyncOp<Boolean>(true) {
             @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return tx.putAsync(ctx, key, val, false, ctx.noValArray()).get().success();
@@ -2451,8 +2434,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx) {
                 return tx.putAsync(ctx, key, val, false, ctx.noValArray()).chain(
@@ -2477,8 +2458,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         return syncOp(new SyncOp<V>(true) {
             @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return (V)tx.putAsync(ctx, key, val, true, ctx.hasValArray()).get().value();
@@ -2501,8 +2480,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx) {
                 return tx.putAsync(ctx, key, val, true, ctx.hasValArray()).chain(
@@ -2527,8 +2504,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         return syncOp(new SyncOp<Boolean>(true) {
             @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return tx.putAsync(ctx, key, val, false, ctx.hasValArray()).get().success();
@@ -2547,8 +2522,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx) {
                 return tx.putAsync(ctx, key, val, false, ctx.hasValArray()).chain(
@@ -2568,10 +2541,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(oldVal);
-
-        validateCacheValue(newVal);
-
         return syncOp(new SyncOp<Boolean>(true) {
             @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 // Register before hiding in the filter.
@@ -2599,10 +2568,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(oldVal);
-
-        validateCacheValue(newVal);
-
         IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx) {
                 // Register before hiding in the filter.
@@ -2642,8 +2607,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKeys(m.keySet());
 
-        validateCacheValues(m.values());
-
         syncOp(new SyncInOp(m.size() == 1) {
             @Override public void inOp(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 tx.putAllAsync(ctx, m, false, CU.empty0()).get();
@@ -2666,8 +2629,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKeys(m.keySet());
 
-        validateCacheValues(m.values());
-
         return asyncOp(new AsyncInOp(m.keySet()) {
             @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter tx) {
                 return tx.putAllAsync(ctx, m, false, CU.empty0()).chain(RET2NULL);
@@ -3067,8 +3028,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         boolean rmv = syncOp(new SyncOp<Boolean>(true) {
             @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 // Register before hiding in the filter.
@@ -3104,8 +3063,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        validateCacheValue(val);
-
         IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx) {
                 // Register before hiding in the filter.
@@ -4639,37 +4596,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-     * Validates that given cache value implements {@link Externalizable}.
-     *
-     * @param val Cache value.
-     */
-    private void validateCacheValue(Object val) {
-        if (valCheck) {
-            CU.validateCacheValue(log, val);
-
-            valCheck = false;
-        }
-    }
-
-    /**
-     * Validates that given cache values implement {@link Externalizable}.
-     *
-     * @param vals Cache values.
-     */
-    private void validateCacheValues(Iterable<?> vals) {
-        if (valCheck) {
-            for (Object val : vals) {
-                if (val == null)
-                    continue;
-
-                CU.validateCacheValue(log, val);
-            }
-
-            valCheck = false;
-        }
-    }
-
-    /**
      * Validates that given cache key has overridden equals and hashCode methods and
      * implements {@link Externalizable}.
      *
@@ -4678,7 +4604,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      */
     protected void validateCacheKey(Object key) {
         if (keyCheck) {
-            CU.validateCacheKey(log, key);
+            CU.validateCacheKey(key);
 
             keyCheck = false;
         }
@@ -4700,7 +4626,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 if (key == null || key instanceof GridCacheInternal)
                     continue;
 
-                CU.validateCacheKey(log, key);
+                CU.validateCacheKey(key);
 
                 keyCheck = false;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/457e8c11/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 51f6dcd..8723827 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -82,7 +82,6 @@ import org.apache.ignite.internal.util.typedef.P2;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteClosure;
@@ -1243,32 +1242,15 @@ public class GridCacheUtils {
     }
 
     /**
-     * Validates that cache value object implements {@link Externalizable}.
+     * Validates that cache key object has overridden equals and hashCode methods.
      *
-     * @param log Logger used to log warning message.
-     * @param val Value.
-     */
-    public static void validateCacheValue(IgniteLogger log, @Nullable Object val) {
-        if (val == null)
-            return;
-
-        validateExternalizable(log, val);
-    }
-
-    /**
-     * Validates that cache key object has overridden equals and hashCode methods and
-     * implements {@link Externalizable}.
-     *
-     * @param log Logger used to log warning message.
      * @param key Key.
      * @throws IllegalArgumentException If equals or hashCode is not implemented.
      */
-    public static void validateCacheKey(IgniteLogger log, @Nullable Object key) {
+    public static void validateCacheKey(@Nullable Object key) {
         if (key == null)
             return;
 
-        validateExternalizable(log, key);
-
         if (!U.overridesEqualsAndHashCode(key))
             throw new IllegalArgumentException("Cache key must override hashCode() and equals() methods: " +
                 key.getClass().getName());
@@ -1355,20 +1337,6 @@ public class GridCacheUtils {
             return 1;
     }
 
-    /**
-     * Validates that cache key or cache value implements {@link Externalizable}
-     *
-     * @param log Logger used to log warning message.
-     * @param obj Cache key or cache value.
-     */
-    private static void validateExternalizable(IgniteLogger log, Object obj) {
-        Class<?> cls = obj.getClass();
-
-        if (!cls.isArray() && !U.isJdk(cls) && !(obj instanceof Externalizable) && !(obj instanceof GridCacheInternal))
-            LT.warn(log, null, "For best performance you should implement " +
-                "java.io.Externalizable for all cache keys and values: " + cls.getName());
-    }
-
 //    /**
 //     * @param cfg Grid configuration.
 //     * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/457e8c11/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheUtilsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheUtilsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheUtilsSelfTest.java
index b601fe2..d5888e7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheUtilsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheUtilsSelfTest.java
@@ -17,14 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
 import java.util.concurrent.Callable;
-import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.testframework.GridStringLogger;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -32,10 +26,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
  * Grid cache utils test.
  */
 public class GridCacheUtilsSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final String EXTERNALIZABLE_WARNING = "For best performance you should implement " +
-        "java.io.Externalizable";
-
     /**
      * Does not override equals and hashCode.
      */
@@ -92,38 +82,6 @@ public class GridCacheUtilsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Overrides equals and hashCode and implements {@link Externalizable}.
-     */
-    private static class ExternalizableEqualsAndHashCode implements Externalizable {
-        /**
-         * Constructor required by {@link Externalizable}.
-         */
-        public ExternalizableEqualsAndHashCode() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return super.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return super.equals(obj);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            // No-op.
-        }
-    }
-
-    /**
      * Extends class which overrides equals and hashCode.
      */
     private static class ExtendsClassWithEqualsAndHashCode extends EqualsAndHashCode {
@@ -145,59 +103,19 @@ public class GridCacheUtilsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Does not implement {@link Externalizable}.
-     */
-    private static class NoImplExternalizable {
-    }
-
-    /**
-     * Implements {@link Externalizable}.
-     */
-    private static class ImplExternalizable implements Externalizable  {
-        /**
-         * Constructor required by {@link Externalizable}.
-         */
-        public ImplExternalizable() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            // No-op.
-        }
-    }
-
-    /**
-     * Extends class which implements {@link Externalizable}.
-     */
-    private static class ExtendsImplExternalizable extends ImplExternalizable {
-        /**
-         * Constructor required by {@link Externalizable}.
-         */
-        public ExtendsImplExternalizable() {
-            // No-op.
-        }
-    }
-
-    /**
      */
     public void testCacheKeyValidation() {
-        CU.validateCacheKey(log, "key");
+        CU.validateCacheKey("key");
 
-        CU.validateCacheKey(log, 1);
+        CU.validateCacheKey(1);
 
-        CU.validateCacheKey(log, 1L);
+        CU.validateCacheKey(1L);
 
-        CU.validateCacheKey(log, 1.0);
+        CU.validateCacheKey(1.0);
 
-        CU.validateCacheKey(log, new ExtendsClassWithEqualsAndHashCode());
+        CU.validateCacheKey(new ExtendsClassWithEqualsAndHashCode());
 
-        CU.validateCacheKey(log, new ExtendsClassWithEqualsAndHashCode2());
+        CU.validateCacheKey(new ExtendsClassWithEqualsAndHashCode2());
 
         assertThrowsForInvalidKey(new NoEqualsAndHashCode());
 
@@ -206,46 +124,6 @@ public class GridCacheUtilsSelfTest extends GridCommonAbstractTest {
         assertThrowsForInvalidKey(new NoHashCode());
 
         assertThrowsForInvalidKey(new WrongEquals());
-
-        IgniteLogger log = new GridStringLogger(false);
-
-        CU.validateCacheKey(log, new ExternalizableEqualsAndHashCode());
-
-        assertFalse(log.toString().contains(EXTERNALIZABLE_WARNING));
-
-        CU.validateCacheKey(log, "key");
-
-        assertFalse(log.toString().contains(EXTERNALIZABLE_WARNING));
-
-        CU.validateCacheKey(log, new EqualsAndHashCode());
-
-        assertTrue(log.toString().contains(EXTERNALIZABLE_WARNING));
-    }
-
-    /**
-     */
-    public void testCacheValueValidation() {
-        IgniteLogger log = new GridStringLogger(false);
-
-        CU.validateCacheValue(log, new ImplExternalizable());
-
-        assertFalse(log.toString().contains(EXTERNALIZABLE_WARNING));
-
-        CU.validateCacheValue(log, new ExtendsImplExternalizable());
-
-        assertFalse(log.toString().contains(EXTERNALIZABLE_WARNING));
-
-        CU.validateCacheValue(log, "val");
-
-        assertFalse(log.toString().contains(EXTERNALIZABLE_WARNING));
-
-        CU.validateCacheValue(log, new byte[10]);
-
-        assertFalse(log.toString().contains(EXTERNALIZABLE_WARNING));
-
-        CU.validateCacheValue(log, new NoImplExternalizable());
-
-        assertTrue(log.toString().contains(EXTERNALIZABLE_WARNING));
     }
 
     /**
@@ -254,7 +132,7 @@ public class GridCacheUtilsSelfTest extends GridCommonAbstractTest {
     private void assertThrowsForInvalidKey(final Object key) {
         GridTestUtils.assertThrows(log, new Callable<Void>() {
             @Override public Void call() throws Exception {
-                CU.validateCacheKey(log, key);
+                CU.validateCacheKey(key);
 
                 return null;
             }