You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by an...@apache.org on 2015/05/15 05:03:45 UTC

[40/50] incubator-ignite git commit: # IGNITE-894 Revert adding @InjectRecursively.

# IGNITE-894 Revert adding @InjectRecursively.


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

Branch: refs/heads/ignite-843
Commit: 56e67e8f7ce808c190346d9228987f25b54b4e6f
Parents: 593e3ee
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 14 16:38:45 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 14 16:40:00 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridInternalWrapper.java    | 30 ++++++++++++++++
 .../closure/GridClosureProcessor.java           | 25 ++++++++++----
 .../internal/processors/igfs/IgfsJobImpl.java   | 10 ++++--
 .../processors/resource/GridResourceIoc.java    | 36 ++++++++++----------
 .../resource/GridResourceProcessor.java         |  8 ++++-
 .../processors/resource/GridResourceUtils.java  | 15 ++++++++
 .../processors/resource/InjectRecursively.java  | 30 ----------------
 .../resources/META-INF/classnames.properties    |  1 -
 8 files changed, 96 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/56e67e8f/modules/core/src/main/java/org/apache/ignite/internal/GridInternalWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridInternalWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/GridInternalWrapper.java
new file mode 100644
index 0000000..76563e7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridInternalWrapper.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.internal;
+
+/**
+ * Internal wrapper interface for custom resource injection logic.
+ */
+public interface GridInternalWrapper<T> {
+    /**
+     * Get user object where resources must be injected.
+     *
+     * @return User object.
+     */
+    public T userObject();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/56e67e8f/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
index 8f5afbf..658557e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
@@ -1584,12 +1584,12 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     /**
      *
      */
-    private static class C1<T, R> implements ComputeJob, Externalizable {
+    private static class C1<T, R> implements ComputeJob, Externalizable, GridNoImplicitInjection,
+        GridInternalWrapper<IgniteClosure> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** */
-        @InjectRecursively
         protected IgniteClosure<T, R> job;
 
         /** */
@@ -1635,6 +1635,11 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
+        @Override public IgniteClosure userObject() {
+            return job;
+        }
+
+        /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(C1.class, this);
         }
@@ -1676,12 +1681,11 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     /**
      *
      */
-    private static class C2<R> implements ComputeJob, Externalizable {
+    private static class C2<R> implements ComputeJob, Externalizable, GridNoImplicitInjection, GridInternalWrapper<Callable> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** */
-        @InjectRecursively
         protected Callable<R> c;
 
         /**
@@ -1724,6 +1728,11 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
+        @Override public Callable userObject() {
+            return c;
+        }
+
+        /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(C2.class, this);
         }
@@ -1763,12 +1772,11 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
     /**
      */
-    private static class C4 implements ComputeJob, Externalizable {
+    private static class C4 implements ComputeJob, Externalizable, GridNoImplicitInjection, GridInternalWrapper<Runnable> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** */
-        @InjectRecursively
         protected Runnable r;
 
         /**
@@ -1808,6 +1816,11 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
+        @Override public Runnable userObject() {
+            return r;
+        }
+
+        /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(C4.class, this);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/56e67e8f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java
index 8f2cfd2..fa90e21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java
@@ -21,7 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.igfs.mapreduce.*;
-import org.apache.ignite.internal.processors.resource.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.resources.*;
 
 import java.io.*;
@@ -29,12 +29,11 @@ import java.io.*;
 /**
  * IGFS job implementation.
  */
-public class IgfsJobImpl implements ComputeJob {
+public class IgfsJobImpl implements ComputeJob, GridInternalWrapper<IgfsJob> {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** IGFS job. */
-    @InjectRecursively
     private IgfsJob job;
 
     /** IGFS name. */
@@ -110,4 +109,9 @@ public class IgfsJobImpl implements ComputeJob {
     @Override public void cancel() {
         job.cancel();
     }
+
+    /** {@inheritDoc} */
+    @Override public IgfsJob userObject() {
+        return job;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/56e67e8f/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java
index ce19664..1e85ecd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceIoc.java
@@ -272,29 +272,29 @@ class GridResourceIoc {
 
             for (Class cls0 = cls; !cls0.equals(Object.class); cls0 = cls0.getSuperclass()) {
                 for (Field field : cls0.getDeclaredFields()) {
-                    InjectRecursively injectRecursively = field.getAnnotation(InjectRecursively.class);
+                    Annotation[] fieldAnns = field.getAnnotations();
 
-                    if (injectRecursively != null
-                        || (allowImplicitInjection && field.getName().startsWith("this$")
-                            || field.getName().startsWith("val$"))) {
-                        field.setAccessible(true);
+                    for (Annotation ann : fieldAnns) {
+                        T2<List<GridResourceField>, List<GridResourceMethod>> t2 = annMap.get(ann.annotationType());
 
-                        recursiveFieldsList.add(field);
-                    }
-                    else {
-                        for (Annotation ann : field.getAnnotations()) {
-                            T2<List<GridResourceField>, List<GridResourceMethod>> t2 = annMap.get(ann.annotationType());
+                        if (t2 == null) {
+                            t2 = new T2<List<GridResourceField>, List<GridResourceMethod>>(
+                                new ArrayList<GridResourceField>(),
+                                new ArrayList<GridResourceMethod>());
 
-                            if (t2 == null) {
-                                t2 = new T2<List<GridResourceField>, List<GridResourceMethod>>(
-                                    new ArrayList<GridResourceField>(),
-                                    new ArrayList<GridResourceMethod>());
+                            annMap.put(ann.annotationType(), t2);
+                        }
 
-                                annMap.put(ann.annotationType(), t2);
-                            }
+                        t2.get1().add(new GridResourceField(field, ann));
+                    }
 
-                            t2.get1().add(new GridResourceField(field, ann));
-                        }
+                    if (allowImplicitInjection
+                        && fieldAnns.length == 0
+                        && GridResourceUtils.mayRequireResources(field)) {
+                        field.setAccessible(true);
+
+                        // Account for anonymous inner classes.
+                        recursiveFieldsList.add(field);
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/56e67e8f/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
index 5b51592..f5ba492 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
@@ -23,7 +23,6 @@ import org.apache.ignite.compute.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.processors.*;
-import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.resources.*;
@@ -281,6 +280,13 @@ public class GridResourceProcessor extends GridProcessorAdapter {
         Object obj = unwrapTarget(job);
 
         injectToJob(dep, taskCls, obj, ses, jobCtx);
+
+        if (obj instanceof GridInternalWrapper) {
+            Object usrObj = ((GridInternalWrapper)obj).userObject();
+
+            if (usrObj != null)
+                injectToJob(dep, taskCls, usrObj, ses, jobCtx);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/56e67e8f/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceUtils.java
index 254f171..660d6ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceUtils.java
@@ -88,4 +88,19 @@ final class GridResourceUtils {
                 ", target=" + target + ", rsrc=" + rsrc + ']', e);
         }
     }
+
+    /**
+     * Checks if specified field requires recursive inspection to find resource annotations.
+     *
+     * @param f Field.
+     * @return {@code true} if requires, {@code false} if doesn't.
+     */
+    static boolean mayRequireResources(Field f) {
+        assert f != null;
+
+        // Need to inspect anonymous classes, callable and runnable instances.
+        return f.getName().startsWith("this$") || f.getName().startsWith("val$") ||
+            Callable.class.isAssignableFrom(f.getType()) || Runnable.class.isAssignableFrom(f.getType()) ||
+            IgniteClosure.class.isAssignableFrom(f.getType());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/56e67e8f/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/InjectRecursively.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/InjectRecursively.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/InjectRecursively.java
deleted file mode 100644
index 383ee03..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/InjectRecursively.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.resource;
-
-import java.lang.annotation.*;
-
-/**
- * Indicates that resource injection should be performed for field value too.
- */
-@Documented
-@Retention(RetentionPolicy.RUNTIME)
-@Target(ElementType.FIELD)
-public @interface InjectRecursively {
-    // No-op.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/56e67e8f/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index f130840..b3eed46 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1140,7 +1140,6 @@ org.apache.ignite.internal.util.lang.GridAbsClosure
 org.apache.ignite.internal.util.lang.GridAbsClosureX
 org.apache.ignite.internal.util.lang.GridCloseableIterator
 org.apache.ignite.internal.util.lang.GridClosureException
-org.apache.ignite.internal.util.lang.GridComputeJobWrapper
 org.apache.ignite.internal.util.lang.GridFunc$1
 org.apache.ignite.internal.util.lang.GridFunc$10
 org.apache.ignite.internal.util.lang.GridFunc$100