You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/14 15:13:44 UTC

[27/50] [abbrv] ignite git commit: ignite-4950 Do not allow AffinityKeyMapped annotation on methods

ignite-4950 Do not allow AffinityKeyMapped annotation on methods


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

Branch: refs/heads/ignite-4587
Commit: 1743f27d65bbdd5e098626cd668d21751eb09b18
Parents: 42b1580
Author: Konstantin Dudkov <kd...@ya.ru>
Authored: Fri Apr 14 12:34:50 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Apr 14 12:34:50 2017 +0300

----------------------------------------------------------------------
 .../cache/affinity/AffinityKeyMapped.java       | 14 ++---
 .../GridCacheDefaultAffinityKeyMapper.java      | 38 +-------------
 .../datastructures/DataStructuresProcessor.java |  1 -
 .../datastructures/GridCacheQueueAdapter.java   | 49 ++----------------
 .../datastructures/GridCacheSetImpl.java        | 54 +-------------------
 .../internal/GridMultipleJobsSelfTest.java      | 12 ++---
 .../GridCacheConcurrentTxMultiNodeTest.java     |  9 +++-
 .../loadtests/colocation/GridTestKey.java       | 28 ++++++++--
 .../ignite/loadtests/dsi/GridDsiPerfJob.java    |  7 ++-
 9 files changed, 56 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1743f27d/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java
index dd42d5c..8b19338 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/AffinityKeyMapped.java
@@ -27,9 +27,9 @@ import java.util.concurrent.Callable;
 /**
  * Optional annotation to specify custom key-to-node affinity. Affinity key is a key
  * which will be used to determine a node on which given cache key will be stored. This
- * annotation allows to mark a field or a method in the cache key object that will be
+ * annotation allows to mark a field in the cache key object that will be
  * used as an affinity key (instead of the entire cache key object that is used for
- * affinity by default). Note that a class can have only one field or method annotated
+ * affinity by default). Note that a class can have only one field annotated
  * with {@code @AffinityKeyMapped} annotation.
  * <p>
  * One of the major use cases for this annotation is the routing of grid computations
@@ -39,11 +39,11 @@ import java.util.concurrent.Callable;
  * <h1 class="header">Mapping Cache Keys</h1>
  * The default implementation of {@link AffinityKeyMapper}, which will be used
  * if no explicit affinity mapper is specified in cache configuration, will first look
- * for any field or method annotated with {@code @AffinityKeyMapped} annotation.
- * If such field or method is not found, then the cache key itself will be used for
+ * for any field annotated with {@code @AffinityKeyMapped} annotation.
+ * If such field is not found, then the cache key itself will be used for
  * key-to-node affinity (this means that all objects with the same cache key will always
- * be routed to the same node). If such field or method is found, then the value of this
- * field or method will be used for key-to-node affinity. This allows to specify alternate
+ * be routed to the same node). If such field is found, then the value of this
+ * field will be used for key-to-node affinity. This allows to specify alternate
  * affinity key, other than the cache key itself, whenever needed.
  * <p>
  * For example, if a {@code Person} object is always accessed together with a {@code Company} object
@@ -108,7 +108,7 @@ import java.util.concurrent.Callable;
  */
 @Documented
 @Retention(RetentionPolicy.RUNTIME)
-@Target({ElementType.FIELD, ElementType.METHOD})
+@Target({ElementType.FIELD})
 public @interface AffinityKeyMapped {
     // No-op.
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/1743f27d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDefaultAffinityKeyMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDefaultAffinityKeyMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDefaultAffinityKeyMapper.java
index 4a2f039..54c40ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDefaultAffinityKeyMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDefaultAffinityKeyMapper.java
@@ -17,9 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.lang.annotation.Annotation;
 import java.lang.reflect.Field;
-import java.lang.reflect.Method;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
@@ -29,7 +27,6 @@ import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.GridArgumentCheck;
 import org.apache.ignite.internal.util.GridReflectionCache;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.resources.IgniteInstanceResource;
@@ -38,7 +35,7 @@ import org.jetbrains.annotations.Nullable;
 
 /**
  * Default key affinity mapper. If key class has annotation {@link AffinityKeyMapped},
- * then the value of annotated method or field will be used to get affinity value instead
+ * then the value of annotated field will be used to get affinity value instead
  * of the key itself. If there is no annotation, then the key is used as is.
  * <p>
  * Convenience affinity key adapter, {@link AffinityKey} can be used in
@@ -62,22 +59,7 @@ public class GridCacheDefaultAffinityKeyMapper implements AffinityKeyMapper {
                 return f.getAnnotation(AffinityKeyMapped.class) != null;
             }
         },
-        new P1<Method>() {
-            @Override public boolean apply(Method m) {
-                // Account for anonymous inner classes.
-                Annotation ann = m.getAnnotation(AffinityKeyMapped.class);
-
-                if (ann != null) {
-                    if (!F.isEmpty(m.getParameterTypes()))
-                        throw new IllegalStateException("Method annotated with @AffinityKeyMapped annotation " +
-                            "cannot have parameters: " + m);
-
-                    return true;
-                }
-
-                return false;
-            }
-        }
+        null
     );
 
     /** Logger. */
@@ -106,17 +88,6 @@ public class GridCacheDefaultAffinityKeyMapper implements AffinityKeyMapper {
                 reflectCache.firstField(key.getClass()) + ", key=" + key + ']', e);
         }
 
-        try {
-            Object o = reflectCache.firstMethodValue(key);
-
-            if (o != null)
-                return o;
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to invoke affinity method for key [mtd=" +
-                reflectCache.firstMethod(key.getClass()) + ", key=" + key + ']', e);
-        }
-
         return key;
     }
 
@@ -130,11 +101,6 @@ public class GridCacheDefaultAffinityKeyMapper implements AffinityKeyMapper {
         if (field != null)
             return field.getName();
 
-        Method mtd = reflectCache.firstMethod(cls);
-
-        if (mtd != null)
-            return mtd.getName();
-
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1743f27d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index b72a7fb..e4cb0ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -1041,7 +1041,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter implemen
                     hdr.id(),
                     name,
                     hdr.collocated(),
-                    cctx.binaryMarshaller(),
                     hdr.head(),
                     hdr.tail(),
                     0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1743f27d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
index 6e087e6..2f6abb6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheQueueAdapter.java
@@ -101,9 +101,6 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
     /** Access to affinityRun() and affinityCall() functions. */
     private final IgniteCompute compute;
 
-    /** */
-    private final boolean binaryMarsh;
-
     /**
      * @param queueName Queue name.
      * @param hdr Queue hdr.
@@ -118,7 +115,6 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
         collocated = hdr.collocated();
         queueKey = new GridCacheQueueHeaderKey(queueName);
         cache = cctx.kernalContext().cache().internalCache(cctx.name());
-        binaryMarsh = cctx.binaryMarshaller();
         this.compute = cctx.kernalContext().grid().compute();
 
         log = cctx.logger(getClass());
@@ -377,7 +373,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
 
             checkRemoved(t.get1());
 
-            removeKeys(cache, id, queueName, collocated, binaryMarsh, t.get1(), t.get2(), batchSize);
+            removeKeys(cache, id, queueName, collocated, t.get1(), t.get2(), batchSize);
         }
         catch (IgniteCheckedException e) {
             throw U.convertException(e);
@@ -433,7 +429,6 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
      * @param id Queue unique ID.
      * @param name Queue name.
      * @param collocated Collocation flag.
-     * @param binaryMarsh {@code True} if binary marshaller is configured.
      * @param startIdx Start item index.
      * @param endIdx End item index.
      * @param batchSize Batch size.
@@ -445,7 +440,6 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
         IgniteUuid id,
         String name,
         boolean collocated,
-        boolean binaryMarsh,
         long startIdx,
         long endIdx,
         int batchSize)
@@ -453,7 +447,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
         Set<QueueItemKey> keys = new HashSet<>(batchSize > 0 ? batchSize : 10);
 
         for (long idx = startIdx; idx < endIdx; idx++) {
-            keys.add(itemKey(id, name, collocated, binaryMarsh, idx));
+            keys.add(itemKey(id, name, collocated, idx));
 
             if (batchSize > 0 && keys.size() == batchSize) {
                 cache.removeAll(keys);
@@ -565,7 +559,7 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
      * @return Item key.
      */
     protected QueueItemKey itemKey(Long idx) {
-        return itemKey(id, queueName, collocated(), binaryMarsh, idx);
+        return itemKey(id, queueName, collocated(), idx);
     }
 
     /** {@inheritDoc} */
@@ -586,18 +580,15 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
      * @param id Queue unique ID.
      * @param queueName Queue name.
      * @param collocated Collocation flag.
-     * @param binaryMarsh {@code True} if binary marshaller is configured.
      * @param idx Item index.
      * @return Item key.
      */
     private static QueueItemKey itemKey(IgniteUuid id,
         String queueName,
         boolean collocated,
-        boolean binaryMarsh,
         long idx) {
         return collocated ?
-            (binaryMarsh ? new CollocatedQueueItemKey(id, queueName, idx) : new CollocatedItemKey(id, queueName, idx)) :
-            new GridCacheQueueItemKey(id, queueName, idx);
+            new CollocatedQueueItemKey(id, queueName, idx) : new GridCacheQueueItemKey(id, queueName, idx);
     }
 
     /**
@@ -695,38 +686,6 @@ public abstract class GridCacheQueueAdapter<T> extends AbstractCollection<T> imp
     }
 
     /**
-     * Item key for collocated queue.
-     */
-    private static class CollocatedItemKey extends GridCacheQueueItemKey {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         * Required by {@link Externalizable}.
-         */
-        public CollocatedItemKey() {
-            // No-op.
-        }
-
-        /**
-         * @param id Queue unique ID.
-         * @param queueName Queue name.
-         * @param idx Item index.
-         */
-        private CollocatedItemKey(IgniteUuid id, String queueName, long idx) {
-            super(id, queueName, idx);
-        }
-
-        /**
-         * @return Item affinity key.
-         */
-        @AffinityKeyMapped
-        public Object affinityKey() {
-            return queueName();
-        }
-    }
-
-    /**
      */
     protected static class ClearProcessor implements
         EntryProcessor<GridCacheQueueHeaderKey, GridCacheQueueHeader, IgniteBiTuple<Long, Long>>, Externalizable {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1743f27d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
index d5f277e..e336474 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
@@ -559,8 +559,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
      * @return Item key.
      */
     private SetItemKey itemKey(Object item) {
-        return collocated ? (binaryMarsh ? new CollocatedSetItemKey(name, id, item) : new CollocatedItemKey(name, id, item))
-            : new GridCacheSetItemKey(id, item);
+        return collocated ? new CollocatedSetItemKey(name, id, item) : new GridCacheSetItemKey(id, item);
     }
 
     /** {@inheritDoc} */
@@ -607,55 +606,4 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
             // No-op.
         }
     }
-
-    /**
-     * Item key for collocated set.
-     */
-    private static class CollocatedItemKey extends GridCacheSetItemKey {
-        /** */
-        private static final long serialVersionUID = -1400701398705953750L;
-
-        /** */
-        private String setName;
-
-        /**
-         * Required by {@link Externalizable}.
-         */
-        public CollocatedItemKey() {
-            // No-op.
-        }
-
-        /**
-         * @param setName Set name.
-         * @param setId Set unique ID.
-         * @param item Set item.
-         */
-        private CollocatedItemKey(String setName, IgniteUuid setId, Object item) {
-            super(setId, item);
-
-            this.setName = setName;
-        }
-
-        /**
-         * @return Item affinity key.
-         */
-        @AffinityKeyMapped
-        public Object affinityKey() {
-            return setName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            super.writeExternal(out);
-
-            U.writeString(out, setName);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            super.readExternal(in);
-
-            setName = U.readString(in);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1743f27d/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java
index 5776a15..bc6eca2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java
@@ -216,6 +216,10 @@ public class GridMultipleJobsSelfTest extends GridCommonAbstractTest {
         /** */
         private static AtomicInteger cnt = new AtomicInteger();
 
+        /** */
+        @AffinityKeyMapped
+        private String affKey = "key";
+
         /** {@inheritDoc} */
         @Override public Boolean call() throws Exception {
             int c = cnt.incrementAndGet();
@@ -227,13 +231,5 @@ public class GridMultipleJobsSelfTest extends GridCommonAbstractTest {
 
             return true;
         }
-
-        /**
-         * @return Affinity key.
-         */
-        @AffinityKeyMapped
-        public String affinityKey() {
-            return "key";
-        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/1743f27d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
index 4776456..3656270 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
@@ -427,14 +427,20 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
         private static final long MAX = 5000;
 
         /** */
+        @AffinityKeyMapped
+        private String affKey;
+
+        /** */
         @IgniteInstanceResource
         private Ignite ignite;
 
         /**
          * @param msg Message.
          */
-        PerfJob(@Nullable Message msg) {
+        PerfJob(Message msg) {
             super(msg);
+
+            affKey = msg.getTerminalId();
         }
 
         /**
@@ -447,7 +453,6 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
         /**
          * @return Terminal ID.
          */
-        @AffinityKeyMapped
         public String terminalId() {
             return message().getTerminalId();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1743f27d/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestKey.java b/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestKey.java
index 2060bde..688be24 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestKey.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestKey.java
@@ -28,8 +28,13 @@ import org.apache.ignite.cache.affinity.AffinityKeyMapped;
  * Accenture key.
  */
 public class GridTestKey implements Externalizable {
+    /** */
     private long id;
 
+    /** */
+    @AffinityKeyMapped
+    private int affKey;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -37,19 +42,33 @@ public class GridTestKey implements Externalizable {
         // No-op.
     }
 
-    public GridTestKey(long id) {
+    /**
+     * @param id ID.
+     */
+    GridTestKey(long id) {
         this.id = id;
+
+        affKey = affinityKey(id);
     }
 
+    /**
+     * @return ID.
+     */
     public long getId() {
         return id;
     }
 
-    @AffinityKeyMapped
+    /**
+     * @return Affinity key.
+     */
     public int affinityKey() {
-        return affinityKey(id);
+        return affKey;
     }
 
+    /**
+     * @param id ID.
+     * @return Affinity key.
+     */
     public static int affinityKey(long id) {
         return (int)(id % GridTestConstants.MOD_COUNT);
     }
@@ -76,6 +95,7 @@ public class GridTestKey implements Externalizable {
         out.writeLong(id);
     }
 
+    /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         if (this == o)
             return true;
@@ -85,10 +105,12 @@ public class GridTestKey implements Externalizable {
         return id == key.id;
     }
 
+    /** {@inheritDoc} */
     @Override public int hashCode() {
         return (int)(id ^ (id >>> 32));
     }
 
+    /** {@inheritDoc} */
     @Override public String toString() {
         return "AccentureKey [id=" + id + ']';
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1743f27d/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java b/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java
index e123947..6dd3df5 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java
@@ -51,6 +51,10 @@ public class GridDsiPerfJob extends ComputeJobAdapter {
         new ConcurrentHashMap8<>();
 
     /** */
+    @AffinityKeyMapped
+    private String affKey;
+
+    /** */
     private static final long PRINT_FREQ = 10000;
 
     /** */
@@ -71,6 +75,8 @@ public class GridDsiPerfJob extends ComputeJobAdapter {
      */
     public GridDsiPerfJob(@Nullable GridDsiMessage msg) {
         super(msg);
+
+        affKey = message().getTerminalId();
     }
 
     /**
@@ -83,7 +89,6 @@ public class GridDsiPerfJob extends ComputeJobAdapter {
     /**
      * @return Terminal ID.
      */
-    @AffinityKeyMapped
     @Nullable public String terminalId() {
         GridDsiMessage msg = message();