You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/01/04 08:25:42 UTC

[1/9] ignite git commit: 2308: added test, added comments.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2308 [created] 42a6a8c27


2308: added test, added comments.


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

Branch: refs/heads/ignite-2308
Commit: 922a4daecadbd8b1485683274c36a86153f78f8e
Parents: e298eab
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Dec 30 22:47:31 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Wed Dec 30 22:47:31 2015 +0300

----------------------------------------------------------------------
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |   1 +
 .../processors/hadoop/HadoopClassLoader.java    | 591 +++++++++++++++----
 .../hadoop/HadoopClassLoaderTest.java           | 109 +++-
 .../hadoop/cls/CircularDependencyHadoop.java    |  14 +
 .../hadoop/cls/CircularDependencyNoHadoop.java  |   9 +
 .../hadoop/cls/DependencyNoHadoop.java          |   9 +
 .../processors/hadoop/cls/HadoopCasting.java    |  23 +
 .../hadoop/cls/HadoopClassAnnotation.java       |  10 +
 .../hadoop/cls/HadoopConstructorInvocation.java |  13 +
 .../HadoopDeclaredCheckedExceptionInMethod.java |  13 +
 .../HadoopDeclaredRuntimeExceptionInMethod.java |  13 +
 .../processors/hadoop/cls/HadoopExtends.java    |  10 +
 .../processors/hadoop/cls/HadoopField.java      |  13 +
 .../processors/hadoop/cls/HadoopImplements.java |  19 +
 .../hadoop/cls/HadoopInitializer.java           |  14 +
 .../processors/hadoop/cls/HadoopInnerClass.java |  15 +
 .../hadoop/cls/HadoopLocalVariableType.java     |  20 +
 .../hadoop/cls/HadoopMethodAnnotation.java      |  12 +
 .../hadoop/cls/HadoopMethodInvocation.java      |  13 +
 .../hadoop/cls/HadoopMethodParameter.java       |  13 +
 .../hadoop/cls/HadoopMethodReturnType.java      |  13 +
 .../processors/hadoop/cls/HadoopOuterClass.java |  20 +
 .../hadoop/cls/HadoopParameterAnnotation.java   |  11 +
 .../hadoop/cls/HadoopStaticField.java           |  11 +
 .../hadoop/cls/HadoopStaticInitializer.java     |  16 +
 .../processors/hadoop/cls/NoHadoop.java         |   7 +
 .../testsuites/IgniteHadoopTestSuite.java       |   3 +
 27 files changed, 875 insertions(+), 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index 5dce67f..a72452b 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -328,6 +328,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             if (initSecondary) {
                 Map<String, String> props = paths.properties();
+                //Object payload0 = paths.getPayload(getClass().getClassLoader());
 
                 String secUri = props.get(SECONDARY_FS_URI);
                 String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index f12af46..89a55e4 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -40,13 +40,16 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import org.objectweb.asm.AnnotationVisitor;
+import org.objectweb.asm.Attribute;
 import org.objectweb.asm.ClassReader;
 import org.objectweb.asm.ClassVisitor;
 import org.objectweb.asm.ClassWriter;
 import org.objectweb.asm.FieldVisitor;
+import org.objectweb.asm.Handle;
 import org.objectweb.asm.Label;
 import org.objectweb.asm.MethodVisitor;
 import org.objectweb.asm.Opcodes;
+import org.objectweb.asm.Type;
 import org.objectweb.asm.commons.Remapper;
 import org.objectweb.asm.commons.RemappingClassAdapter;
 
@@ -125,10 +128,14 @@ public class HadoopClassLoader extends URLClassLoader {
      * @return {@code true} if we need to check this class.
      */
     private static boolean isHadoopIgfs(String cls) {
-        String ignitePackagePrefix = "org.apache.ignite";
-        int len = ignitePackagePrefix.length();
+        String ignitePackPrefix = "org.apache.ignite";
 
-        return cls.startsWith(ignitePackagePrefix) && (cls.indexOf("igfs.", len) != -1 || cls.indexOf(".fs.", len) != -1 || cls.indexOf("hadoop.", len) != -1);
+        int len = ignitePackPrefix.length();
+
+        return cls.startsWith(ignitePackPrefix)
+            && (cls.indexOf("igfs.", len) != -1
+                || cls.indexOf(".fs.", len) != -1
+                || cls.indexOf("hadoop.", len) != -1);
     }
 
     /**
@@ -295,169 +302,527 @@ public class HadoopClassLoader extends URLClassLoader {
 
         final AtomicBoolean hasDeps = new AtomicBoolean();
 
-        rdr.accept(new ClassVisitor(Opcodes.ASM4) {
-            AnnotationVisitor av = new AnnotationVisitor(Opcodes.ASM4) {
-                // TODO
-            };
+        Collector c = new Collector(hasDeps, visited);
 
-            FieldVisitor fv = new FieldVisitor(Opcodes.ASM4) {
-                @Override public AnnotationVisitor visitAnnotation(String desc, boolean b) {
-                    onType(desc);
+        AnnotationVisitor annotationVisitor = new CollectingAnnotationVisitor(c);
 
-                    return av;
-                }
-            };
+        MethodVisitor methVisitor = new CollectingMethodVisitor(c, annotationVisitor);
 
-            MethodVisitor mv = new MethodVisitor(Opcodes.ASM4) {
-                @Override public AnnotationVisitor visitAnnotation(String desc, boolean b) {
-                    onType(desc);
+        FieldVisitor fieldVisitor = new CollectingFieldVisitor(c, annotationVisitor);
 
-                    return av;
-                }
+        ClassVisitor clsVisitor = new CollectingClassVisitor(c, annotationVisitor, methVisitor, fieldVisitor);
 
-                @Override public AnnotationVisitor visitParameterAnnotation(int i, String desc, boolean b) {
-                    onType(desc);
+        rdr.accept(clsVisitor, 0);
 
-                    return av;
-                }
+        if (hasDeps.get()) // We already know that we have dependencies, no need to check parent.
+            return true;
 
-                @Override public AnnotationVisitor visitAnnotationDefault() {
-                    return av;
-                }
+        // Here we are known to not have any dependencies but possibly we have a parent which has them.
+        int idx = clsName.lastIndexOf('$');
 
-                @Override public void visitFieldInsn(int i, String owner, String name, String desc) {
-                    onType(owner);
-                    onType(desc);
-                }
+        if (idx == -1) // No parent class.
+            return false;
 
-                @Override public void visitFrame(int i, int i2, Object[] locTypes, int i3, Object[] stackTypes) {
-                    for (Object o : locTypes) {
-                        if (o instanceof String)
-                            onType((String)o);
-                    }
+        String parentCls = clsName.substring(0, idx);
 
-                    for (Object o : stackTypes) {
-                        if (o instanceof String)
-                            onType((String)o);
-                    }
-                }
+        if (visited.contains(parentCls))
+            return false;
 
-                @Override public void visitLocalVariable(String name, String desc, String signature, Label lb,
-                    Label lb2, int i) {
-                    onType(desc);
-                }
+        Boolean res = cache.get(parentCls);
 
-                @Override public void visitMethodInsn(int i, String owner, String name, String desc) {
-                    onType(owner);
-                }
+        if (res == null)
+            res = hasExternalDependencies(parentCls, visited);
 
-                @Override public void visitMultiANewArrayInsn(String desc, int dim) {
-                    onType(desc);
-                }
+        return res;
+    }
 
-                @Override public void visitTryCatchBlock(Label lb, Label lb2, Label lb3, String e) {
-                    onType(e);
-                }
-            };
+    /**
+     * Implement business logic of the dependency analysis.
+     * Keeps the necessary state.
+     */
+    private class Collector {
+        /** Attribute gets 'true' if the positive answer is found. */
+        final AtomicBoolean hasDeps;
+
+        /** Collection of visited class names to prevent infinite loops in case of
+         * circular dependencies. */
+        final Set<String> visited;
+
+        /**
+         * Constructor.
+         *
+         * @param hasDeps has dependencies initial value.
+         * @param visitedSet visited set initial value.
+         */
+        Collector(AtomicBoolean hasDeps, Set<String> visitedSet) {
+            this.hasDeps = hasDeps;
+
+            this.visited = visitedSet;
+        }
 
-            void onClass(String depCls) {
-                assert validateClassName(depCls) : depCls;
+        /**
+         * Answers if the model travers should be finished.
+         *
+         * @return If it is done.
+         */
+        boolean isDone() {
+            return hasDeps.get();
+        }
 
-                if (depCls.startsWith("java.")) // Filter out platform classes.
-                    return;
+        /**
+         * Processes a method descriptor
+         * @param methDesc The method desc String.
+         */
+        void onMethodsDesc(final String methDesc) {
+            // Process method return type:
+            onType(Type.getReturnType(methDesc));
 
-                if (visited.contains(depCls))
-                    return;
+            if (isDone())
+                return;
 
-                Boolean res = cache.get(depCls);
+            // Process method argument types:
+            for (Type t: Type.getArgumentTypes(methDesc)) {
+                onType(t);
 
-                if (res == Boolean.TRUE || (res == null && hasExternalDependencies(depCls, visited)))
-                    hasDeps.set(true);
+                if (isDone())
+                    return;
             }
+        }
 
-            void onType(String type) {
-                if (type == null)
-                    return;
+        /**
+         * Processes dependencies of a class.
+         *
+         * @param depCls The class name as dot-notated FQN.
+         */
+        void onClass(final String depCls) {
+            assert depCls.indexOf('/') == -1 : depCls; // class name should be fully converted to dot notation.
+            assert depCls.charAt(0) != 'L' : depCls;
+            assert validateClassName(depCls) : depCls;
 
-                int off = 0;
+            if (depCls.startsWith("java.") || depCls.startsWith("javax.")) // Filter out platform classes.
+                return;
 
-                while (type.charAt(off) == '[')
-                    off++; // Handle arrays.
+            if (visited.contains(depCls))
+                return;
 
-                if (off != 0)
-                    type = type.substring(off);
+            Boolean res = cache.get(depCls);
 
-                if (type.length() == 1)
-                    return; // Get rid of primitives.
+            if (res == Boolean.TRUE
+                || (res == null && hasExternalDependencies(depCls, visited)))
+                hasDeps.set(true);
+        }
 
-                if (type.charAt(type.length() - 1) == ';') {
-                    assert type.charAt(0) == 'L' : type;
+        /**
+         * Analyses dependencies of given type.
+         *
+         * @param t The type to process.
+         */
+        void onType(Type t) {
+            if (t == null)
+                return;
 
-                    type = type.substring(1, type.length() - 1);
-                }
+            int sort = t.getSort();
 
-                type = type.replace('/', '.');
+            switch (sort) {
+                case Type.ARRAY:
+                    onType(t.getElementType());
 
-                onClass(type);
+                    break;
+
+                case Type.OBJECT:
+                    onClass(t.getClassName());
+
+                    break;
             }
+        }
 
-            @Override public void visit(int i, int i2, String name, String signature, String superName,
-                String[] ifaces) {
-                onType(superName);
+        /**
+         * Analyses dependencies of given object type.
+         *
+         * @param objType The object type to process.
+         */
+        void onInternalTypeName(String objType) {
+            if (objType == null)
+                return;
 
-                if (ifaces != null) {
-                    for (String iface : ifaces)
-                        onType(iface);
-                }
+            assert objType.length() > 1 : objType;
+
+            if (objType.charAt(0) == '[')
+                // handle array. In this case this is a type descriptor notation, like "[Ljava/lang/Object;"
+                onType(objType);
+            else {
+                assert objType.indexOf('.') == -1 : objType; // Must be slash-separated FQN.
+
+                String clsName = objType.replace('/', '.'); // Convert it to dot notation.
+
+                onClass(clsName); // Process.
             }
+        }
 
-            @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
-                onType(desc);
+        /**
+         * Type description analyser.
+         *
+         * @param desc The description.
+         */
+        void onType(String desc) {
+            if (!F.isEmpty(desc)) {
+                if (desc.length() <= 1)
+                    return; // Optimization: filter out primitive types in early stage.
 
-                return av;
+                Type t = Type.getType(desc);
+
+                onType(t);
             }
+        }
+    }
+
+    /**
+     * Annotation visitor.
+     */
+    static class CollectingAnnotationVisitor extends AnnotationVisitor {
+        /** */
+        final Collector c;
+
+        /**
+         *
+         * @param c The collector.
+         */
+        CollectingAnnotationVisitor(Collector c) {
+            super(Opcodes.ASM4);
+
+            this.c = c;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotation(String name, String desc) {
+            if (c.isDone())
+                return null;
+
+            c.onType(desc);
+
+            return this;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitEnum(String name, String desc, String val) {
+            if (c.isDone())
+                return;
+
+            c.onType(desc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitArray(String name) {
+            return this;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visit(String name, Object val) {
+            if (val instanceof Type)
+                c.onType((Type)val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitEnd() {
+            // noop
+        }
+    }
+
+    /**
+     * Field visitor.
+     */
+    static class CollectingFieldVisitor extends FieldVisitor {
+        /** */
+        private final Collector c;
+
+        /** */
+        private final AnnotationVisitor av;
+
+        /**
+         * Constructor.
+         */
+        CollectingFieldVisitor(Collector c, AnnotationVisitor av) {
+            super(Opcodes.ASM4);
+
+            this.c = c;
+
+            this.av = av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
+            if (c.isDone())
+                return null;
+
+            c.onType(desc);
+
+            return av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitAttribute(Attribute attr) {
+            // noop
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitEnd() {
+            // noop
+        }
+    }
 
-            @Override public void visitInnerClass(String name, String outerName, String innerName, int i) {
-                onType(name);
+    /**
+     * Class visitor.
+     */
+    static class CollectingClassVisitor extends ClassVisitor {
+        /** */
+        private final Collector c;
+
+        /** */
+        private final AnnotationVisitor av;
+
+        /** */
+        private final MethodVisitor mv;
+
+        /** */
+        private final FieldVisitor fv;
+
+        /**
+         * Constructor.
+         */
+        CollectingClassVisitor(Collector c, AnnotationVisitor av, MethodVisitor mv, FieldVisitor fv) {
+            super(Opcodes.ASM4);
+
+            this.c = c;
+            this.av = av;
+            this.mv = mv;
+            this.fv = fv;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visit(int i, int i2, String name, String signature, String superName,
+            String[] ifaces) {
+            if (c.isDone())
+                return;
+
+            c.onInternalTypeName(superName);
+
+            if (c.isDone())
+                return;
+
+            if (ifaces != null) {
+                for (String iface : ifaces) {
+                    c.onInternalTypeName(iface);
+
+                    if (c.isDone())
+                        return;
+                }
             }
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
+            if (c.isDone())
+                return null;
 
-            @Override public FieldVisitor visitField(int i, String name, String desc, String signature, Object val) {
-                onType(desc);
+            c.onType(desc);
+
+            return av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitInnerClass(String name, String outerName, String innerName, int i) {
+            if (c.isDone())
+                return;
+
+            c.onInternalTypeName(name);
+        }
 
-                return fv;
+        /** {@inheritDoc} */
+        @Override public FieldVisitor visitField(int i, String name, String desc, String signature, Object val) {
+            if (c.isDone())
+                return null;
+
+            c.onType(desc);
+
+            return fv;
+        }
+
+        /** {@inheritDoc} */
+        @Override public MethodVisitor visitMethod(int i, String name, String desc, String signature,
+            String[] exceptions) {
+            if (c.isDone())
+                return null;
+
+            c.onMethodsDesc(desc);
+
+            // Process declared method exceptions:
+            if (exceptions != null) {
+                for (String e : exceptions)
+                    c.onInternalTypeName(e);
             }
 
-            @Override public MethodVisitor visitMethod(int i, String name, String desc, String signature,
-                String[] exceptions) {
-                if (exceptions != null) {
-                    for (String e : exceptions)
-                        onType(e);
+            return mv;
+        }
+    }
+
+    /**
+     * Method visitor.
+     */
+    static class CollectingMethodVisitor extends MethodVisitor {
+        /** */
+        private final Collector c;
+
+        /** */
+        private final AnnotationVisitor av;
+
+        /**
+         * Constructor.
+         *
+         * @param c The collector.
+         * @param av The annotation visitor.
+         */
+        CollectingMethodVisitor(Collector c, AnnotationVisitor av) {
+            super(Opcodes.ASM4);
+
+            this.c = c;
+            this.av = av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
+            if (c.isDone())
+                return null;
+
+            c.onType(desc);
+
+            return av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitParameterAnnotation(int i, String desc, boolean b) {
+            if (c.isDone())
+                return null;
+
+            c.onType(desc);
+
+            return av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotationDefault() {
+            return av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitFieldInsn(int opcode, String owner, String name, String desc) {
+            if (c.isDone())
+                return;
+
+            c.onInternalTypeName(owner);
+
+            if (c.isDone())
+                return;
+
+            c.onType(desc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitInvokeDynamicInsn(String name, String desc, Handle bsm, Object... bsmArgs) {
+            if (c.isDone())
+                return;
+
+            c.onMethodsDesc(desc);
+
+            if (c.isDone())
+                return;
+
+            if (bsmArgs != null) {
+                for (Object b: bsmArgs) {
+                    if (b instanceof Type)
+                        c.onType((Type) b);
+
+                    if (c.isDone())
+                        return;
                 }
+            }
+        }
 
-                return mv;
+        /** {@inheritDoc} */
+        @Override public void visitFrame(int type, int nLoc, Object[] locTypes, int nStack,
+        Object[] stackTypes) {
+            // TODO: not sure we need that to detect dependencies. This seems to be superfluous.
+            // Optimizations:
+            // 1) Do not consider frames that are exactly equal to the previous;
+            // 2) Return if local and stack arrays are empty ('nLock' and 'nStack' correspond to the number
+            // of non-null array elements).
+            if (type == Opcodes.F_SAME || (nLoc == 0 && nStack == 0))
+                return;
+
+            if (c.isDone())
+                return;
+
+            for (Object o : locTypes) {
+                if (o instanceof String)
+                    c.onInternalTypeName((String) o);
+
+                if (c.isDone())
+                    return;
             }
-        }, 0);
 
-        if (hasDeps.get()) // We already know that we have dependencies, no need to check parent.
-            return true;
+            for (Object o : stackTypes) {
+                if (o instanceof String)
+                    c.onInternalTypeName((String) o);
 
-        // Here we are known to not have any dependencies but possibly we have a parent which have them.
-        int idx = clsName.lastIndexOf('$');
+                if (c.isDone())
+                    return;
+            }
+        }
 
-        if (idx == -1) // No parent class.
-            return false;
+        /** {@inheritDoc} */
+        @Override public void visitLocalVariable(String name, String desc, String signature, Label lb,
+            Label lb2, int i) {
+            if (c.isDone())
+                return;
 
-        String parentCls = clsName.substring(0, idx);
+            c.onType(desc);
+        }
 
-        if (visited.contains(parentCls))
-            return false;
+        /** {@inheritDoc} */
+        @Override public void visitMethodInsn(int i, String owner, String name, String desc) {
+            if (c.isDone())
+                return;
 
-        Boolean res = cache.get(parentCls);
+            c.onInternalTypeName(owner);
 
-        if (res == null)
-            res = hasExternalDependencies(parentCls, visited);
+            if (c.isDone())
+                return;
 
-        return res;
+            c.onMethodsDesc(desc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitMultiANewArrayInsn(String desc, int dim) {
+            if (c.isDone())
+                return;
+
+            c.onType(desc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitTryCatchBlock(Label start, Label end, Label hndl, String typeStr) {
+            if (c.isDone())
+                return;
+
+            c.onInternalTypeName(typeStr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitTypeInsn(int opcode, String type) {
+            if (c.isDone())
+                return;
+
+            c.onInternalTypeName(type);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
index 085dd45..e878d9a 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
@@ -17,53 +17,108 @@
 
 package org.apache.ignite.internal.processors.hadoop;
 
+import java.util.HashSet;
+import javax.security.auth.AuthPermission;
 import junit.framework.TestCase;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.Job;
+import org.apache.ignite.internal.processors.hadoop.cls.CircularDependencyHadoop;
+import org.apache.ignite.internal.processors.hadoop.cls.CircularDependencyNoHadoop;
+import org.apache.ignite.internal.processors.hadoop.cls.DependencyNoHadoop;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopCasting;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopClassAnnotation;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopConstructorInvocation;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopDeclaredCheckedExceptionInMethod;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopDeclaredRuntimeExceptionInMethod;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopExtends;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopField;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopImplements;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopInitializer;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopInnerClass;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopLocalVariableType;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopMethodAnnotation;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopMethodInvocation;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopMethodParameter;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopMethodReturnType;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopOuterClass;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopParameterAnnotation;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopStaticField;
+import org.apache.ignite.internal.processors.hadoop.cls.HadoopStaticInitializer;
+import org.apache.ignite.internal.processors.hadoop.cls.NoHadoop;
 
 /**
  *
  */
 public class HadoopClassLoaderTest extends TestCase {
     /** */
-    HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+    final HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
 
     /**
      * @throws Exception If failed.
      */
     public void testClassLoading() throws Exception {
-        assertNotSame(Test1.class, ldr.loadClass(Test1.class.getName()));
-        assertNotSame(Test2.class, ldr.loadClass(Test2.class.getName()));
-        assertSame(Test3.class, ldr.loadClass(Test3.class.getName()));
-    }
+        assertNotSame(CircularDependencyHadoop.class, ldr.loadClass(CircularDependencyHadoop.class.getName()));
+        assertNotSame(CircularDependencyNoHadoop.class, ldr.loadClass(CircularDependencyNoHadoop.class.getName()));
 
-//    public void testDependencySearch() {
-//        assertTrue(ldr.hasExternalDependencies(Test1.class.getName(), new HashSet<String>()));
-//        assertTrue(ldr.hasExternalDependencies(Test2.class.getName(), new HashSet<String>()));
-//    }
+        assertSame(NoHadoop.class, ldr.loadClass(NoHadoop.class.getName()));
+    }
 
     /**
      *
      */
-    private static class Test1 {
-        /** */
-        Test2 t2;
+    public void testDependencySearch() {
+        // Various positive cases of Hadoop classes dependency:
+        final Class[] positiveClasses = {
+            // Hadoop class itself:
+            Configuration.class,
+            // Class for that org.apache.ignite.internal.processors.hadoop.HadoopClassLoader.isHadoopIgfs returns true:
+            HadoopUtils.class,
 
-        /** */
-        Job[][] jobs = new Job[4][4];
-    }
+            HadoopStaticField.class,
+            HadoopCasting.class,
+            HadoopClassAnnotation.class,
+            HadoopConstructorInvocation.class,
+            HadoopDeclaredCheckedExceptionInMethod.class,
+            HadoopDeclaredRuntimeExceptionInMethod.class,
+            HadoopExtends.class,
+            HadoopField.class,
+            HadoopImplements.class,
+            HadoopInitializer.class,
 
-    /**
-     *
-     */
-    private static abstract class Test2 {
-        /** */
-        abstract Test1 t1();
-    }
+            // TODO: actually the 2 below classes do not depend on Hadoop, should not be detected as such.
+            // TODO: but for now they are, so this behavior is asserted in test:
+            HadoopInnerClass.class,
+            HadoopOuterClass.InnerNoHadoop.class,
 
-    /**
-     *
-     */
-    private static class Test3 {
-        // No-op.
+            HadoopLocalVariableType.class,
+            HadoopMethodAnnotation.class,
+            HadoopMethodInvocation.class,
+            HadoopMethodParameter.class,
+            HadoopMethodReturnType.class,
+            HadoopParameterAnnotation.class,
+            HadoopStaticField.class,
+            HadoopStaticInitializer.class,
+
+            DependencyNoHadoop.class,
+            CircularDependencyHadoop.class,
+            CircularDependencyNoHadoop.class,
+        };
+
+        for (Class c: positiveClasses)
+            assertTrue(c.getName(),
+                ldr.hasExternalDependencies(c.getName(), new HashSet<String>()));
+
+        // Negative cases:
+        final Class[] negativeClasses = {
+            // java.lang.*:
+            Object.class,
+            // javax.*:
+            AuthPermission.class,
+            NoHadoop.class,
+        };
+
+        for (Class c: negativeClasses)
+            assertFalse(c.getName(),
+                ldr.hasExternalDependencies(c.getName(), new HashSet<String>()));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyHadoop.java
new file mode 100644
index 0000000..02df9be
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyHadoop.java
@@ -0,0 +1,14 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.mapreduce.Job;
+
+/**
+ * Class has a direct Hadoop dependency and a circular dependency on another class.
+ */
+public class CircularDependencyHadoop {
+    /** */
+    Job[][] jobs = new Job[4][4];
+
+    /** */
+    private CircularDependencyNoHadoop y;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyNoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyNoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyNoHadoop.java
new file mode 100644
index 0000000..0f9b543
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyNoHadoop.java
@@ -0,0 +1,9 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+/**
+ * Does not have direct Hadoop dependency, but has a circular
+ */
+public class CircularDependencyNoHadoop {
+    /** */
+    private CircularDependencyHadoop x;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/DependencyNoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/DependencyNoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/DependencyNoHadoop.java
new file mode 100644
index 0000000..f13f1ce
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/DependencyNoHadoop.java
@@ -0,0 +1,9 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+/**
+ * Has a unidirected dependency on Hadoop-dependent class.
+ */
+public class DependencyNoHadoop {
+    /** */
+    HadoopField x;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopCasting.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopCasting.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopCasting.java
new file mode 100644
index 0000000..39a58f6
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopCasting.java
@@ -0,0 +1,23 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Class contains casting to a Hadoop type.
+ */
+public abstract class HadoopCasting <T> {
+    /** */
+    public abstract T create();
+
+    /** */
+    public void consume(T t) {
+        // noop
+    }
+
+    /** */
+    void test(HadoopCasting<FileSystem> c) {
+        FileSystem fs = c.create();
+
+        c.consume(fs);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopClassAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopClassAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopClassAnnotation.java
new file mode 100644
index 0000000..ff18f82
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopClassAnnotation.java
@@ -0,0 +1,10 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Class has Hadoop annotation.
+ */
+@InterfaceAudience.Public
+public class HadoopClassAnnotation {
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopConstructorInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopConstructorInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopConstructorInvocation.java
new file mode 100644
index 0000000..c7ac886
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopConstructorInvocation.java
@@ -0,0 +1,13 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Invokes a Hadoop type constructor.
+ */
+public class HadoopConstructorInvocation {
+    /** */
+    private void foo() {
+        Object x = new Configuration();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredCheckedExceptionInMethod.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredCheckedExceptionInMethod.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredCheckedExceptionInMethod.java
new file mode 100644
index 0000000..b9890f1
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredCheckedExceptionInMethod.java
@@ -0,0 +1,13 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.fs.ChecksumException;
+
+/**
+ * Method declares a checked Hadoop Exception.
+ */
+public class HadoopDeclaredCheckedExceptionInMethod {
+    /** */
+    void foo() throws ChecksumException {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredRuntimeExceptionInMethod.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredRuntimeExceptionInMethod.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredRuntimeExceptionInMethod.java
new file mode 100644
index 0000000..9c8c000
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredRuntimeExceptionInMethod.java
@@ -0,0 +1,13 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+
+/**
+ * Method declares a runtime Hadoop Exception.
+ */
+public class HadoopDeclaredRuntimeExceptionInMethod {
+    /** */
+    void foo() throws HadoopIllegalArgumentException {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopExtends.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopExtends.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopExtends.java
new file mode 100644
index 0000000..f0a082e
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopExtends.java
@@ -0,0 +1,10 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.fs.LocalFileSystem;
+
+/**
+ * Class extends a Hadoop class.
+ */
+public class HadoopExtends extends LocalFileSystem {
+    // noop
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopField.java
new file mode 100644
index 0000000..ccdeb0e
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopField.java
@@ -0,0 +1,13 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Has a Hadoop field.
+ */
+public class HadoopField {
+    /**
+     *
+     */
+    private Configuration conf;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopImplements.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopImplements.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopImplements.java
new file mode 100644
index 0000000..a143e85
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopImplements.java
@@ -0,0 +1,19 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Implements a Hadoop interface.
+ */
+public class HadoopImplements implements Configurable {
+    /** {@inheritDoc} */
+    @Override public void setConf(Configuration conf) {
+        // noop
+    }
+
+    /** {@inheritDoc} */
+    @Override public Configuration getConf() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInitializer.java
new file mode 100644
index 0000000..f153e57
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInitializer.java
@@ -0,0 +1,14 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+/**
+ * Has a field initialized with an expression invoking Hadoop method.
+ */
+public class HadoopInitializer {
+    /** */
+    private final Object x = org.apache.hadoop.fs.FileSystem.getDefaultUri(null);
+
+    /** */
+    HadoopInitializer() throws Exception {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInnerClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInnerClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInnerClass.java
new file mode 100644
index 0000000..15d0a4a
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInnerClass.java
@@ -0,0 +1,15 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.AbstractFileSystem;
+
+/**
+ * Has a *static* inner class depending on Hadoop.
+ */
+public class HadoopInnerClass {
+    /** */
+    private static abstract class Foo implements Configurable {
+        // nothing
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopLocalVariableType.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopLocalVariableType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopLocalVariableType.java
new file mode 100644
index 0000000..a82909c
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopLocalVariableType.java
@@ -0,0 +1,20 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Has a local variable of Hadoop type.
+ */
+public class HadoopLocalVariableType {
+    /** */
+    void foo() {
+        Configuration c = null;
+
+        moo(c);
+    }
+
+    /** */
+    void moo(Object x) {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodAnnotation.java
new file mode 100644
index 0000000..b62677b
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodAnnotation.java
@@ -0,0 +1,12 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+/**
+ * Method has a Hadoop annotation.
+ */
+public class HadoopMethodAnnotation {
+    /** */
+    @org.apache.hadoop.classification.InterfaceStability.Unstable
+    void foo() {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodInvocation.java
new file mode 100644
index 0000000..85a1e49
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodInvocation.java
@@ -0,0 +1,13 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Method contains a Hadoop type method invocation.
+ */
+public class HadoopMethodInvocation {
+    /** */
+    void foo(FileSystem fs) {
+        fs.getChildFileSystems();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodParameter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodParameter.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodParameter.java
new file mode 100644
index 0000000..427b771
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodParameter.java
@@ -0,0 +1,13 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Contains a formal parameter of Hadoop type.
+ */
+public class HadoopMethodParameter {
+    /** */
+    protected void paramaterMethod(Configuration c) {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodReturnType.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodReturnType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodReturnType.java
new file mode 100644
index 0000000..afb883d
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodReturnType.java
@@ -0,0 +1,13 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Contains a method return value of Hadoop type.
+ */
+public class HadoopMethodReturnType {
+    /** */
+    FileSystem fsMethod() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopOuterClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopOuterClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopOuterClass.java
new file mode 100644
index 0000000..8a453d7
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopOuterClass.java
@@ -0,0 +1,20 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Outer class depends on Hadoop, but Inner *static* one does not.
+ */
+public class HadoopOuterClass {
+    /** */
+    Configuration c;
+
+    /** */
+    public static class InnerNoHadoop {
+        /** */
+        int x;
+
+        /** */
+        void foo() {}
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopParameterAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopParameterAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopParameterAnnotation.java
new file mode 100644
index 0000000..03da8e2
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopParameterAnnotation.java
@@ -0,0 +1,11 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+/**
+ * Has a paramater annotated with a Hadoop annotation.
+ */
+public class HadoopParameterAnnotation {
+    /** */
+    void foo(@org.apache.hadoop.classification.InterfaceStability.Stable Object annotatedParam) {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticField.java
new file mode 100644
index 0000000..fd11093
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticField.java
@@ -0,0 +1,11 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Has a static field of Hadoop type.
+ */
+public class HadoopStaticField {
+    /** */
+    static FileSystem fs;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticInitializer.java
new file mode 100644
index 0000000..022493b
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticInitializer.java
@@ -0,0 +1,16 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+import java.util.List;
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Uses Hadoop type in a static initializer.
+ */
+public class HadoopStaticInitializer {
+    /** */
+    static final List x;
+
+    static {
+        x = FileSystem.getAllStatistics();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/NoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/NoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/NoHadoop.java
new file mode 100644
index 0000000..8bd13f9
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/NoHadoop.java
@@ -0,0 +1,7 @@
+package org.apache.ignite.internal.processors.hadoop.cls;
+
+/**
+ * Class that does not anyhow depend on Hadoop.
+ */
+public class NoHadoop {
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/922a4dae/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 6641bc8..1831085 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -54,6 +54,7 @@ import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalPrimarySelfT
 import org.apache.ignite.igfs.IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest;
 import org.apache.ignite.igfs.IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest;
 import org.apache.ignite.igfs.IgniteHadoopFileSystemSecondaryModeSelfTest;
+import org.apache.ignite.internal.processors.hadoop.HadoopClassLoaderTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommandLineTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopDefaultMapReducePlannerSelfTest;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileSystemsTest;
@@ -95,6 +96,8 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         TestSuite suite = new TestSuite("Ignite Hadoop MR Test Suite");
 
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopClassLoaderTest.class.getName())));
+
         suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfs20FileSystemLoopbackPrimarySelfTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfsDualSyncSelfTest.class.getName())));


[2/9] ignite git commit: Merge branch 'master' of https://github.com/apache/ignite into ignite-2308

Posted by vo...@apache.org.
Merge branch 'master' of https://github.com/apache/ignite into ignite-2308


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

Branch: refs/heads/ignite-2308
Commit: 683fea8e62544a3b67b56747626288716078874b
Parents: 922a4da dffcb83
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Dec 30 22:48:10 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Wed Dec 30 22:48:10 2015 +0300

----------------------------------------------------------------------
 doap_Ignite.rdf                                 |  67 ++++++
 .../version/GridCachePlainVersionedEntry.java   |   2 +-
 .../platform/PlatformAbstractTarget.java        |  17 +-
 .../processors/platform/PlatformTarget.java     |  22 ++
 .../platform/compute/PlatformCompute.java       |  34 ++-
 .../platform/utils/PlatformFutureUtils.java     | 119 +++++++---
 .../platform/utils/PlatformListenable.java      |  47 ++++
 .../ignite/internal/util/IgniteUtils.java       |  40 ++++
 .../ignite/internal/util/nio/GridNioServer.java |  16 +-
 .../cache/IgniteCacheStoreCollectionTest.java   | 163 +++++++++++++
 .../cpp/common/include/ignite/common/exports.h  |   7 +-
 .../cpp/common/include/ignite/common/java.h     |  13 +-
 .../platforms/cpp/common/project/vs/module.def  |   6 +-
 modules/platforms/cpp/common/src/exports.cpp    |  20 +-
 modules/platforms/cpp/common/src/java.cpp       |  70 +++++-
 .../Compute/ComputeApiTest.cs                   |  20 ++
 .../Apache.Ignite.Core.csproj                   |   2 +
 .../Common/IgniteFutureCancelledException.cs    |  65 ++++++
 .../Apache.Ignite.Core/Compute/ICompute.cs      | 233 +++++++++++++++++++
 .../Impl/Common/CancelledTask.cs                |  47 ++++
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  74 +++++-
 .../Apache.Ignite.Core/Impl/Compute/Compute.cs  | 157 ++++++++++++-
 .../Impl/Compute/ComputeImpl.cs                 |  15 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   4 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |  52 +++++
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |  16 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |  31 ++-
 27 files changed, 1276 insertions(+), 83 deletions(-)
----------------------------------------------------------------------



[6/9] ignite git commit: IGNITE-2308: Refactoring.

Posted by vo...@apache.org.
IGNITE-2308: Refactoring.


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

Branch: refs/heads/ignite-2308
Commit: 0de6ed9ad5d7251dddad6e576c7ea74cd8057ab6
Parents: e6f4455
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jan 4 10:03:51 2016 +0400
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jan 4 10:03:51 2016 +0400

----------------------------------------------------------------------
 .../processors/hadoop/HadoopClassLoader.java    | 89 +++++++++-----------
 1 file changed, 40 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0de6ed9a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index 010a6bd..55d87ea 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -274,7 +274,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
     /**
      * Check whether class has external dependencies on Hadoop.
-     *
+     * 
      * @param clsName Class name.
      * @return {@code True} if class has external dependencies.
      */
@@ -285,15 +285,15 @@ public class HadoopClassLoader extends URLClassLoader {
         ctx.mthdVisitor = new CollectingMethodVisitor(ctx, ctx.annVisitor);
         ctx.fldVisitor = new CollectingFieldVisitor(ctx, ctx.annVisitor);
         ctx.clsVisitor = new CollectingClassVisitor(ctx, ctx.annVisitor, ctx.mthdVisitor, ctx.fldVisitor);
-
+        
         return hasExternalDependencies(clsName, ctx);
     }
-
+        
     /**
      * Check whether class has external dependencies on Hadoop.
-     *
+     * 
      * @param clsName Class name.
-     * @param ctx Context.
+     * @param ctx Context.                
      * @return {@code true} If the class has external dependencies.
      */
     boolean hasExternalDependencies(String clsName, CollectingContext ctx) {
@@ -322,7 +322,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
         rdr.accept(ctx.clsVisitor, 0);
 
-        if (ctx.found()) // We already know that we have dependencies, no need to check parent.
+        if (ctx.found) // We already know that we have dependencies, no need to check parent.
             return true;
 
         // Here we are known to not have any dependencies but possibly we have a parent which has them.
@@ -522,20 +522,11 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** Field visitor. */
         private FieldVisitor fldVisitor;
-
+        
         /** Class visitor. */
         private ClassVisitor clsVisitor;
 
         /**
-         * Answers if the model travers should be finished.
-         *
-         * @return If it is done.
-         */
-        boolean found() {
-            return found;
-        }
-
-        /**
          * Processes a method descriptor
          * @param methDesc The method desc String.
          */
@@ -543,14 +534,14 @@ public class HadoopClassLoader extends URLClassLoader {
             // Process method return type:
             onType(Type.getReturnType(methDesc));
 
-            if (found())
+            if (found)
                 return;
 
             // Process method argument types:
             for (Type t: Type.getArgumentTypes(methDesc)) {
                 onType(t);
 
-                if (found())
+                if (found)
                     return;
             }
         }
@@ -639,7 +630,7 @@ public class HadoopClassLoader extends URLClassLoader {
                 onType(t);
             }
         }
-    }
+    }    
 
     /**
      * Annotation visitor.
@@ -650,7 +641,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /**
          * Annotation visitor.
-         *
+         * 
          * @param ctx The collector.
          */
         CollectingAnnotationVisitor(CollectingContext ctx) {
@@ -661,7 +652,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotation(String name, String desc) {
-            if (ctx.found())
+            if (ctx.found)
                 return null;
 
             ctx.onType(desc);
@@ -671,7 +662,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public void visitEnum(String name, String desc, String val) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onType(desc);
@@ -679,12 +670,12 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitArray(String name) {
-            return ctx.found() ? null : this;
+            return ctx.found ? null : this;
         }
 
         /** {@inheritDoc} */
         @Override public void visit(String name, Object val) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             if (val instanceof Type)
@@ -719,12 +710,12 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
-            if (ctx.found())
+            if (ctx.found)
                 return null;
 
             ctx.onType(desc);
 
-            return ctx.found() ? null : av;
+            return ctx.found ? null : av;
         }
 
         /** {@inheritDoc} */
@@ -773,19 +764,19 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public void visit(int i, int i2, String name, String signature, String superName, String[] ifaces) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onInternalTypeName(superName);
 
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             if (ifaces != null) {
                 for (String iface : ifaces) {
                     ctx.onInternalTypeName(iface);
 
-                    if (ctx.found())
+                    if (ctx.found)
                         return;
                 }
             }
@@ -793,17 +784,17 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
-            if (ctx.found())
+            if (ctx.found)
                 return null;
 
             ctx.onType(desc);
 
-            return ctx.found() ? null : av;
+            return ctx.found ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public void visitInnerClass(String name, String outerName, String innerName, int i) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onInternalTypeName(name);
@@ -811,18 +802,18 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public FieldVisitor visitField(int i, String name, String desc, String signature, Object val) {
-            if (ctx.found())
+            if (ctx.found)
                 return null;
 
             ctx.onType(desc);
 
-            return ctx.found() ? null : fv;
+            return ctx.found ? null : fv;
         }
 
         /** {@inheritDoc} */
         @Override public MethodVisitor visitMethod(int i, String name, String desc, String signature,
             String[] exceptions) {
-            if (ctx.found())
+            if (ctx.found)
                 return null;
 
             ctx.onMethodsDesc(desc);
@@ -833,7 +824,7 @@ public class HadoopClassLoader extends URLClassLoader {
                     ctx.onInternalTypeName(e);
             }
 
-            return ctx.found() ? null : mv;
+            return ctx.found ? null : mv;
         }
     }
 
@@ -862,37 +853,37 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
-            if (ctx.found())
+            if (ctx.found)
                 return null;
 
             ctx.onType(desc);
 
-            return ctx.found() ? null : av;
+            return ctx.found ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitParameterAnnotation(int i, String desc, boolean b) {
-            if (ctx.found())
+            if (ctx.found)
                 return null;
 
             ctx.onType(desc);
 
-            return ctx.found() ? null : av;
+            return ctx.found ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotationDefault() {
-            return ctx.found() ? null : av;
+            return ctx.found ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public void visitFieldInsn(int opcode, String owner, String name, String desc) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onInternalTypeName(owner);
 
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onType(desc);
@@ -911,7 +902,7 @@ public class HadoopClassLoader extends URLClassLoader {
         /** {@inheritDoc} */
         @Override public void visitLocalVariable(String name, String desc, String signature, Label lb,
             Label lb2, int i) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onType(desc);
@@ -919,12 +910,12 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public void visitMethodInsn(int i, String owner, String name, String desc) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onInternalTypeName(owner);
 
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onMethodsDesc(desc);
@@ -932,7 +923,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public void visitMultiANewArrayInsn(String desc, int dim) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onType(desc);
@@ -940,7 +931,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public void visitTryCatchBlock(Label start, Label end, Label hndl, String typeStr) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onInternalTypeName(typeStr);
@@ -948,7 +939,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public void visitTypeInsn(int opcode, String type) {
-            if (ctx.found())
+            if (ctx.found)
                 return;
 
             ctx.onInternalTypeName(type);


[7/9] ignite git commit: IGNITE-2308: Cleanup.

Posted by vo...@apache.org.
IGNITE-2308: Cleanup.


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

Branch: refs/heads/ignite-2308
Commit: db69ec7776c33d74c796837a1b694b28ac9efe9b
Parents: 0de6ed9
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jan 4 10:06:15 2016 +0400
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jan 4 10:06:15 2016 +0400

----------------------------------------------------------------------
 .../hadoop/HadoopClassLoaderTest.java           | 48 ++++++++++----------
 .../hadoop/cls/CircularDependencyHadoop.java    | 14 ------
 .../hadoop/cls/CircularDependencyNoHadoop.java  |  9 ----
 .../hadoop/cls/DependencyNoHadoop.java          |  9 ----
 .../processors/hadoop/cls/HadoopCasting.java    | 23 ----------
 .../hadoop/cls/HadoopClassAnnotation.java       | 10 ----
 .../hadoop/cls/HadoopConstructorInvocation.java | 13 ------
 .../HadoopDeclaredCheckedExceptionInMethod.java | 13 ------
 .../HadoopDeclaredRuntimeExceptionInMethod.java | 13 ------
 .../processors/hadoop/cls/HadoopExtends.java    | 10 ----
 .../processors/hadoop/cls/HadoopField.java      | 13 ------
 .../processors/hadoop/cls/HadoopImplements.java | 19 --------
 .../hadoop/cls/HadoopInitializer.java           | 14 ------
 .../processors/hadoop/cls/HadoopInnerClass.java | 15 ------
 .../hadoop/cls/HadoopLocalVariableType.java     | 20 --------
 .../hadoop/cls/HadoopMethodAnnotation.java      | 12 -----
 .../hadoop/cls/HadoopMethodInvocation.java      | 13 ------
 .../hadoop/cls/HadoopMethodParameter.java       | 13 ------
 .../hadoop/cls/HadoopMethodReturnType.java      | 13 ------
 .../processors/hadoop/cls/HadoopOuterClass.java | 20 --------
 .../hadoop/cls/HadoopParameterAnnotation.java   | 11 -----
 .../hadoop/cls/HadoopStaticField.java           | 11 -----
 .../hadoop/cls/HadoopStaticInitializer.java     | 16 -------
 .../processors/hadoop/cls/NoHadoop.java         |  7 ---
 .../hadoop/deps/CircularDependencyHadoop.java   | 31 +++++++++++++
 .../hadoop/deps/CircularDependencyNoHadoop.java | 26 +++++++++++
 .../hadoop/deps/DependencyNoHadoop.java         | 26 +++++++++++
 .../processors/hadoop/deps/HadoopCasting.java   | 40 ++++++++++++++++
 .../hadoop/deps/HadoopClassAnnotation.java      | 27 +++++++++++
 .../deps/HadoopConstructorInvocation.java       | 30 ++++++++++++
 .../HadoopDeclaredCheckedExceptionInMethod.java | 30 ++++++++++++
 .../HadoopDeclaredRuntimeExceptionInMethod.java | 30 ++++++++++++
 .../processors/hadoop/deps/HadoopExtends.java   | 27 +++++++++++
 .../processors/hadoop/deps/HadoopField.java     | 30 ++++++++++++
 .../hadoop/deps/HadoopImplements.java           | 36 +++++++++++++++
 .../hadoop/deps/HadoopInitializer.java          | 31 +++++++++++++
 .../hadoop/deps/HadoopInnerClass.java           | 30 ++++++++++++
 .../hadoop/deps/HadoopLocalVariableType.java    | 37 +++++++++++++++
 .../hadoop/deps/HadoopMethodAnnotation.java     | 29 ++++++++++++
 .../hadoop/deps/HadoopMethodInvocation.java     | 30 ++++++++++++
 .../hadoop/deps/HadoopMethodParameter.java      | 30 ++++++++++++
 .../hadoop/deps/HadoopMethodReturnType.java     | 30 ++++++++++++
 .../hadoop/deps/HadoopOuterClass.java           | 37 +++++++++++++++
 .../hadoop/deps/HadoopParameterAnnotation.java  | 28 ++++++++++++
 .../hadoop/deps/HadoopStaticField.java          | 28 ++++++++++++
 .../hadoop/deps/HadoopStaticInitializer.java    | 33 ++++++++++++++
 .../processors/hadoop/deps/NoHadoop.java        | 24 ++++++++++
 47 files changed, 723 insertions(+), 336 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
index 88f43b8..d0964e8 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
@@ -17,34 +17,32 @@
 
 package org.apache.ignite.internal.processors.hadoop;
 
-import java.util.HashSet;
 import javax.security.auth.AuthPermission;
 import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.ignite.internal.processors.hadoop.cls.CircularDependencyHadoop;
-import org.apache.ignite.internal.processors.hadoop.cls.CircularDependencyNoHadoop;
-import org.apache.ignite.internal.processors.hadoop.cls.DependencyNoHadoop;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopCasting;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopClassAnnotation;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopConstructorInvocation;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopDeclaredCheckedExceptionInMethod;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopDeclaredRuntimeExceptionInMethod;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopExtends;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopField;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopImplements;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopInitializer;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopInnerClass;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopLocalVariableType;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopMethodAnnotation;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopMethodInvocation;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopMethodParameter;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopMethodReturnType;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopOuterClass;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopParameterAnnotation;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopStaticField;
-import org.apache.ignite.internal.processors.hadoop.cls.HadoopStaticInitializer;
-import org.apache.ignite.internal.processors.hadoop.cls.NoHadoop;
+import org.apache.ignite.internal.processors.hadoop.deps.CircularDependencyHadoop;
+import org.apache.ignite.internal.processors.hadoop.deps.CircularDependencyNoHadoop;
+import org.apache.ignite.internal.processors.hadoop.deps.DependencyNoHadoop;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopCasting;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopClassAnnotation;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopConstructorInvocation;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopDeclaredCheckedExceptionInMethod;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopDeclaredRuntimeExceptionInMethod;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopExtends;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopField;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopImplements;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopInitializer;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopInnerClass;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopLocalVariableType;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopMethodAnnotation;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopMethodInvocation;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopMethodParameter;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopMethodReturnType;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopOuterClass;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopParameterAnnotation;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopStaticField;
+import org.apache.ignite.internal.processors.hadoop.deps.HadoopStaticInitializer;
+import org.apache.ignite.internal.processors.hadoop.deps.NoHadoop;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyHadoop.java
deleted file mode 100644
index 02df9be..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyHadoop.java
+++ /dev/null
@@ -1,14 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.mapreduce.Job;
-
-/**
- * Class has a direct Hadoop dependency and a circular dependency on another class.
- */
-public class CircularDependencyHadoop {
-    /** */
-    Job[][] jobs = new Job[4][4];
-
-    /** */
-    private CircularDependencyNoHadoop y;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyNoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyNoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyNoHadoop.java
deleted file mode 100644
index 0f9b543..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/CircularDependencyNoHadoop.java
+++ /dev/null
@@ -1,9 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-/**
- * Does not have direct Hadoop dependency, but has a circular
- */
-public class CircularDependencyNoHadoop {
-    /** */
-    private CircularDependencyHadoop x;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/DependencyNoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/DependencyNoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/DependencyNoHadoop.java
deleted file mode 100644
index f13f1ce..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/DependencyNoHadoop.java
+++ /dev/null
@@ -1,9 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-/**
- * Has a unidirected dependency on Hadoop-dependent class.
- */
-public class DependencyNoHadoop {
-    /** */
-    HadoopField x;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopCasting.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopCasting.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopCasting.java
deleted file mode 100644
index 39a58f6..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopCasting.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Class contains casting to a Hadoop type.
- */
-public abstract class HadoopCasting <T> {
-    /** */
-    public abstract T create();
-
-    /** */
-    public void consume(T t) {
-        // noop
-    }
-
-    /** */
-    void test(HadoopCasting<FileSystem> c) {
-        FileSystem fs = c.create();
-
-        c.consume(fs);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopClassAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopClassAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopClassAnnotation.java
deleted file mode 100644
index ff18f82..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopClassAnnotation.java
+++ /dev/null
@@ -1,10 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * Class has Hadoop annotation.
- */
-@InterfaceAudience.Public
-public class HadoopClassAnnotation {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopConstructorInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopConstructorInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopConstructorInvocation.java
deleted file mode 100644
index c7ac886..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopConstructorInvocation.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Invokes a Hadoop type constructor.
- */
-public class HadoopConstructorInvocation {
-    /** */
-    private void foo() {
-        Object x = new Configuration();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredCheckedExceptionInMethod.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredCheckedExceptionInMethod.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredCheckedExceptionInMethod.java
deleted file mode 100644
index b9890f1..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredCheckedExceptionInMethod.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.fs.ChecksumException;
-
-/**
- * Method declares a checked Hadoop Exception.
- */
-public class HadoopDeclaredCheckedExceptionInMethod {
-    /** */
-    void foo() throws ChecksumException {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredRuntimeExceptionInMethod.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredRuntimeExceptionInMethod.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredRuntimeExceptionInMethod.java
deleted file mode 100644
index 9c8c000..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopDeclaredRuntimeExceptionInMethod.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-
-/**
- * Method declares a runtime Hadoop Exception.
- */
-public class HadoopDeclaredRuntimeExceptionInMethod {
-    /** */
-    void foo() throws HadoopIllegalArgumentException {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopExtends.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopExtends.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopExtends.java
deleted file mode 100644
index f0a082e..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopExtends.java
+++ /dev/null
@@ -1,10 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.fs.LocalFileSystem;
-
-/**
- * Class extends a Hadoop class.
- */
-public class HadoopExtends extends LocalFileSystem {
-    // noop
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopField.java
deleted file mode 100644
index ccdeb0e..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopField.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Has a Hadoop field.
- */
-public class HadoopField {
-    /**
-     *
-     */
-    private Configuration conf;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopImplements.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopImplements.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopImplements.java
deleted file mode 100644
index a143e85..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopImplements.java
+++ /dev/null
@@ -1,19 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Implements a Hadoop interface.
- */
-public class HadoopImplements implements Configurable {
-    /** {@inheritDoc} */
-    @Override public void setConf(Configuration conf) {
-        // noop
-    }
-
-    /** {@inheritDoc} */
-    @Override public Configuration getConf() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInitializer.java
deleted file mode 100644
index f153e57..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInitializer.java
+++ /dev/null
@@ -1,14 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-/**
- * Has a field initialized with an expression invoking Hadoop method.
- */
-public class HadoopInitializer {
-    /** */
-    private final Object x = org.apache.hadoop.fs.FileSystem.getDefaultUri(null);
-
-    /** */
-    HadoopInitializer() throws Exception {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInnerClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInnerClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInnerClass.java
deleted file mode 100644
index 15d0a4a..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopInnerClass.java
+++ /dev/null
@@ -1,15 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.AbstractFileSystem;
-
-/**
- * Has a *static* inner class depending on Hadoop.
- */
-public class HadoopInnerClass {
-    /** */
-    private static abstract class Foo implements Configurable {
-        // nothing
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopLocalVariableType.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopLocalVariableType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopLocalVariableType.java
deleted file mode 100644
index a82909c..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopLocalVariableType.java
+++ /dev/null
@@ -1,20 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Has a local variable of Hadoop type.
- */
-public class HadoopLocalVariableType {
-    /** */
-    void foo() {
-        Configuration c = null;
-
-        moo(c);
-    }
-
-    /** */
-    void moo(Object x) {
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodAnnotation.java
deleted file mode 100644
index b62677b..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodAnnotation.java
+++ /dev/null
@@ -1,12 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-/**
- * Method has a Hadoop annotation.
- */
-public class HadoopMethodAnnotation {
-    /** */
-    @org.apache.hadoop.classification.InterfaceStability.Unstable
-    void foo() {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodInvocation.java
deleted file mode 100644
index 85a1e49..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodInvocation.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Method contains a Hadoop type method invocation.
- */
-public class HadoopMethodInvocation {
-    /** */
-    void foo(FileSystem fs) {
-        fs.getChildFileSystems();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodParameter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodParameter.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodParameter.java
deleted file mode 100644
index 427b771..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodParameter.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Contains a formal parameter of Hadoop type.
- */
-public class HadoopMethodParameter {
-    /** */
-    protected void paramaterMethod(Configuration c) {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodReturnType.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodReturnType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodReturnType.java
deleted file mode 100644
index afb883d..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopMethodReturnType.java
+++ /dev/null
@@ -1,13 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Contains a method return value of Hadoop type.
- */
-public class HadoopMethodReturnType {
-    /** */
-    FileSystem fsMethod() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopOuterClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopOuterClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopOuterClass.java
deleted file mode 100644
index 8a453d7..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopOuterClass.java
+++ /dev/null
@@ -1,20 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Outer class depends on Hadoop, but Inner *static* one does not.
- */
-public class HadoopOuterClass {
-    /** */
-    Configuration c;
-
-    /** */
-    public static class InnerNoHadoop {
-        /** */
-        int x;
-
-        /** */
-        void foo() {}
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopParameterAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopParameterAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopParameterAnnotation.java
deleted file mode 100644
index 03da8e2..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopParameterAnnotation.java
+++ /dev/null
@@ -1,11 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-/**
- * Has a paramater annotated with a Hadoop annotation.
- */
-public class HadoopParameterAnnotation {
-    /** */
-    void foo(@org.apache.hadoop.classification.InterfaceStability.Stable Object annotatedParam) {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticField.java
deleted file mode 100644
index fd11093..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticField.java
+++ /dev/null
@@ -1,11 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Has a static field of Hadoop type.
- */
-public class HadoopStaticField {
-    /** */
-    static FileSystem fs;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticInitializer.java
deleted file mode 100644
index 022493b..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/HadoopStaticInitializer.java
+++ /dev/null
@@ -1,16 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-import java.util.List;
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Uses Hadoop type in a static initializer.
- */
-public class HadoopStaticInitializer {
-    /** */
-    static final List x;
-
-    static {
-        x = FileSystem.getAllStatistics();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/NoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/NoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/NoHadoop.java
deleted file mode 100644
index 8bd13f9..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/cls/NoHadoop.java
+++ /dev/null
@@ -1,7 +0,0 @@
-package org.apache.ignite.internal.processors.hadoop.cls;
-
-/**
- * Class that does not anyhow depend on Hadoop.
- */
-public class NoHadoop {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyHadoop.java
new file mode 100644
index 0000000..24db626
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyHadoop.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.hadoop.deps;
+
+import org.apache.hadoop.mapreduce.Job;
+
+/**
+ * Class has a direct Hadoop dependency and a circular dependency on another class.
+ */
+public class CircularDependencyHadoop {
+    /** */
+    Job[][] jobs = new Job[4][4];
+
+    /** */
+    private CircularDependencyNoHadoop y;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyNoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyNoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyNoHadoop.java
new file mode 100644
index 0000000..4b019e4
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyNoHadoop.java
@@ -0,0 +1,26 @@
+/*
+ * 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.hadoop.deps;
+
+/**
+ * Does not have direct Hadoop dependency, but has a circular
+ */
+public class CircularDependencyNoHadoop {
+    /** */
+    private CircularDependencyHadoop x;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/DependencyNoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/DependencyNoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/DependencyNoHadoop.java
new file mode 100644
index 0000000..25a29a1
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/DependencyNoHadoop.java
@@ -0,0 +1,26 @@
+/*
+ * 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.hadoop.deps;
+
+/**
+ * Has a unidirected dependency on Hadoop-dependent class.
+ */
+public class DependencyNoHadoop {
+    /** */
+    HadoopField x;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopCasting.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopCasting.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopCasting.java
new file mode 100644
index 0000000..3a5de89
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopCasting.java
@@ -0,0 +1,40 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Class contains casting to a Hadoop type.
+ */
+public abstract class HadoopCasting <T> {
+    /** */
+    public abstract T create();
+
+    /** */
+    public void consume(T t) {
+        // noop
+    }
+
+    /** */
+    void test(HadoopCasting<FileSystem> c) {
+        FileSystem fs = c.create();
+
+        c.consume(fs);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopClassAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopClassAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopClassAnnotation.java
new file mode 100644
index 0000000..cb934d2
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopClassAnnotation.java
@@ -0,0 +1,27 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Class has Hadoop annotation.
+ */
+@InterfaceAudience.Public
+public class HadoopClassAnnotation {
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopConstructorInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopConstructorInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopConstructorInvocation.java
new file mode 100644
index 0000000..5d641b1
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopConstructorInvocation.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.processors.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Invokes a Hadoop type constructor.
+ */
+public class HadoopConstructorInvocation {
+    /** */
+    private void foo() {
+        Object x = new Configuration();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredCheckedExceptionInMethod.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredCheckedExceptionInMethod.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredCheckedExceptionInMethod.java
new file mode 100644
index 0000000..91e6410
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredCheckedExceptionInMethod.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.processors.hadoop.deps;
+
+import org.apache.hadoop.fs.ChecksumException;
+
+/**
+ * Method declares a checked Hadoop Exception.
+ */
+public class HadoopDeclaredCheckedExceptionInMethod {
+    /** */
+    void foo() throws ChecksumException {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredRuntimeExceptionInMethod.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredRuntimeExceptionInMethod.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredRuntimeExceptionInMethod.java
new file mode 100644
index 0000000..bef47d3
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredRuntimeExceptionInMethod.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.processors.hadoop.deps;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+
+/**
+ * Method declares a runtime Hadoop Exception.
+ */
+public class HadoopDeclaredRuntimeExceptionInMethod {
+    /** */
+    void foo() throws HadoopIllegalArgumentException {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopExtends.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopExtends.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopExtends.java
new file mode 100644
index 0000000..e7324f8
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopExtends.java
@@ -0,0 +1,27 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.fs.LocalFileSystem;
+
+/**
+ * Class extends a Hadoop class.
+ */
+public class HadoopExtends extends LocalFileSystem {
+    // noop
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopField.java
new file mode 100644
index 0000000..21e3f38
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopField.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.processors.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Has a Hadoop field.
+ */
+public class HadoopField {
+    /**
+     *
+     */
+    private Configuration conf;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopImplements.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopImplements.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopImplements.java
new file mode 100644
index 0000000..d6f3c1b
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopImplements.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.internal.processors.hadoop.deps;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Implements a Hadoop interface.
+ */
+public class HadoopImplements implements Configurable {
+    /** {@inheritDoc} */
+    @Override public void setConf(Configuration conf) {
+        // noop
+    }
+
+    /** {@inheritDoc} */
+    @Override public Configuration getConf() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInitializer.java
new file mode 100644
index 0000000..7a7f809
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInitializer.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.hadoop.deps;
+
+/**
+ * Has a field initialized with an expression invoking Hadoop method.
+ */
+public class HadoopInitializer {
+    /** */
+    private final Object x = org.apache.hadoop.fs.FileSystem.getDefaultUri(null);
+
+    /** */
+    HadoopInitializer() throws Exception {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInnerClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInnerClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInnerClass.java
new file mode 100644
index 0000000..3fa3e41
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInnerClass.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.processors.hadoop.deps;
+
+import org.apache.hadoop.conf.Configurable;
+
+/**
+ * Has a *static* inner class depending on Hadoop.
+ */
+public class HadoopInnerClass {
+    /** */
+    private static abstract class Foo implements Configurable {
+        // nothing
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopLocalVariableType.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopLocalVariableType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopLocalVariableType.java
new file mode 100644
index 0000000..de2354e
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopLocalVariableType.java
@@ -0,0 +1,37 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Has a local variable of Hadoop type.
+ */
+public class HadoopLocalVariableType {
+    /** */
+    void foo() {
+        Configuration c = null;
+
+        moo(c);
+    }
+
+    /** */
+    void moo(Object x) {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodAnnotation.java
new file mode 100644
index 0000000..a013021
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodAnnotation.java
@@ -0,0 +1,29 @@
+/*
+ * 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.hadoop.deps;
+
+/**
+ * Method has a Hadoop annotation.
+ */
+public class HadoopMethodAnnotation {
+    /** */
+    @org.apache.hadoop.classification.InterfaceStability.Unstable
+    void foo() {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodInvocation.java
new file mode 100644
index 0000000..b4d52b6
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodInvocation.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.processors.hadoop.deps;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Method contains a Hadoop type method invocation.
+ */
+public class HadoopMethodInvocation {
+    /** */
+    void foo(FileSystem fs) {
+        fs.getChildFileSystems();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodParameter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodParameter.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodParameter.java
new file mode 100644
index 0000000..c9621aa
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodParameter.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.processors.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Contains a formal parameter of Hadoop type.
+ */
+public class HadoopMethodParameter {
+    /** */
+    protected void paramaterMethod(Configuration c) {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodReturnType.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodReturnType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodReturnType.java
new file mode 100644
index 0000000..b2a1a3f
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodReturnType.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.processors.hadoop.deps;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Contains a method return value of Hadoop type.
+ */
+public class HadoopMethodReturnType {
+    /** */
+    FileSystem fsMethod() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopOuterClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopOuterClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopOuterClass.java
new file mode 100644
index 0000000..142198c
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopOuterClass.java
@@ -0,0 +1,37 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Outer class depends on Hadoop, but Inner *static* one does not.
+ */
+public class HadoopOuterClass {
+    /** */
+    Configuration c;
+
+    /** */
+    public static class InnerNoHadoop {
+        /** */
+        int x;
+
+        /** */
+        void foo() {}
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopParameterAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopParameterAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopParameterAnnotation.java
new file mode 100644
index 0000000..02bb099
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopParameterAnnotation.java
@@ -0,0 +1,28 @@
+/*
+ * 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.hadoop.deps;
+
+/**
+ * Has a paramater annotated with a Hadoop annotation.
+ */
+public class HadoopParameterAnnotation {
+    /** */
+    void foo(@org.apache.hadoop.classification.InterfaceStability.Stable Object annotatedParam) {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticField.java
new file mode 100644
index 0000000..eff1c4e
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticField.java
@@ -0,0 +1,28 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Has a static field of Hadoop type.
+ */
+public class HadoopStaticField {
+    /** */
+    static FileSystem fs;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticInitializer.java
new file mode 100644
index 0000000..109b971
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticInitializer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.hadoop.deps;
+
+import java.util.List;
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Uses Hadoop type in a static initializer.
+ */
+public class HadoopStaticInitializer {
+    /** */
+    static final List x;
+
+    static {
+        x = FileSystem.getAllStatistics();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69ec77/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/NoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/NoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/NoHadoop.java
new file mode 100644
index 0000000..aff3981
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/NoHadoop.java
@@ -0,0 +1,24 @@
+/*
+ * 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.hadoop.deps;
+
+/**
+ * Class that does not anyhow depend on Hadoop.
+ */
+public class NoHadoop {
+}


[8/9] ignite git commit: IGNITE-2308: Finalization.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithParameterAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithParameterAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithParameterAnnotation.java
new file mode 100644
index 0000000..9d3414e
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithParameterAnnotation.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.hadoop.deps;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Has a paramater annotated with a Hadoop annotation.
+ */
+@SuppressWarnings("unused")
+public class WithParameterAnnotation {
+    /** */
+    void foo(@InterfaceStability.Stable Object annotatedParam) {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticField.java
new file mode 100644
index 0000000..301b912
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticField.java
@@ -0,0 +1,29 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Has a static field of Hadoop type.
+ */
+@SuppressWarnings("unused")
+public class WithStaticField {
+    /** */
+    static FileSystem fs;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticInitializer.java
new file mode 100644
index 0000000..e0fc2f3
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithStaticInitializer.java
@@ -0,0 +1,34 @@
+/*
+ * 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.hadoop.deps;
+
+import java.util.List;
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Uses Hadoop type in a static initializer.
+ */
+@SuppressWarnings("unused")
+public class WithStaticInitializer {
+    /** */
+    static final List x;
+
+    static {
+        x = FileSystem.getAllStatistics();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/Without.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/Without.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/Without.java
new file mode 100644
index 0000000..ab84740
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/Without.java
@@ -0,0 +1,25 @@
+/*
+ * 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.hadoop.deps;
+
+/**
+ * Class that does not anyhow depend on Hadoop.
+ */
+public class Without {
+    // No-op.
+}


[9/9] ignite git commit: IGNITE-2308: Finalization.

Posted by vo...@apache.org.
IGNITE-2308: Finalization.


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

Branch: refs/heads/ignite-2308
Commit: 42a6a8c2757d776e13e0a1a86a47afc8717646f2
Parents: db69ec7
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jan 4 10:25:17 2016 +0400
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jan 4 10:25:17 2016 +0400

----------------------------------------------------------------------
 .../processors/hadoop/HadoopClassLoader.java    |   5 +-
 .../hadoop/HadoopClassLoaderTest.java           | 119 +++++++++----------
 .../hadoop/deps/CircularDependencyHadoop.java   |  31 -----
 .../hadoop/deps/CircularDependencyNoHadoop.java |  26 ----
 .../hadoop/deps/CircularWIthHadoop.java         |  32 +++++
 .../hadoop/deps/CircularWithoutHadoop.java      |  27 +++++
 .../hadoop/deps/DependencyNoHadoop.java         |  26 ----
 .../processors/hadoop/deps/HadoopCasting.java   |  40 -------
 .../hadoop/deps/HadoopClassAnnotation.java      |  27 -----
 .../deps/HadoopConstructorInvocation.java       |  30 -----
 .../HadoopDeclaredCheckedExceptionInMethod.java |  30 -----
 .../HadoopDeclaredRuntimeExceptionInMethod.java |  30 -----
 .../processors/hadoop/deps/HadoopExtends.java   |  27 -----
 .../processors/hadoop/deps/HadoopField.java     |  30 -----
 .../hadoop/deps/HadoopImplements.java           |  36 ------
 .../hadoop/deps/HadoopInitializer.java          |  31 -----
 .../hadoop/deps/HadoopInnerClass.java           |  30 -----
 .../hadoop/deps/HadoopLocalVariableType.java    |  37 ------
 .../hadoop/deps/HadoopMethodAnnotation.java     |  29 -----
 .../hadoop/deps/HadoopMethodInvocation.java     |  30 -----
 .../hadoop/deps/HadoopMethodParameter.java      |  30 -----
 .../hadoop/deps/HadoopMethodReturnType.java     |  30 -----
 .../hadoop/deps/HadoopOuterClass.java           |  37 ------
 .../hadoop/deps/HadoopParameterAnnotation.java  |  28 -----
 .../hadoop/deps/HadoopStaticField.java          |  28 -----
 .../hadoop/deps/HadoopStaticInitializer.java    |  33 -----
 .../processors/hadoop/deps/NoHadoop.java        |  24 ----
 .../processors/hadoop/deps/WithCast.java        |  41 +++++++
 .../hadoop/deps/WithClassAnnotation.java        |  28 +++++
 .../hadoop/deps/WithConstructorInvocation.java  |  31 +++++
 .../processors/hadoop/deps/WithExtends.java     |  27 +++++
 .../processors/hadoop/deps/WithField.java       |  29 +++++
 .../processors/hadoop/deps/WithImplements.java  |  36 ++++++
 .../hadoop/deps/WithIndirectField.java          |  27 +++++
 .../processors/hadoop/deps/WithInitializer.java |  33 +++++
 .../processors/hadoop/deps/WithInnerClass.java  |  31 +++++
 .../hadoop/deps/WithLocalVariable.java          |  38 ++++++
 .../hadoop/deps/WithMethodAnnotation.java       |  32 +++++
 .../hadoop/deps/WithMethodArgument.java         |  31 +++++
 .../hadoop/deps/WithMethodCheckedException.java |  31 +++++
 .../hadoop/deps/WithMethodInvocation.java       |  31 +++++
 .../hadoop/deps/WithMethodReturnType.java       |  31 +++++
 .../hadoop/deps/WithMethodRuntimeException.java |  31 +++++
 .../processors/hadoop/deps/WithOuterClass.java  |  38 ++++++
 .../hadoop/deps/WithParameterAnnotation.java    |  31 +++++
 .../processors/hadoop/deps/WithStaticField.java |  29 +++++
 .../hadoop/deps/WithStaticInitializer.java      |  34 ++++++
 .../processors/hadoop/deps/Without.java         |  25 ++++
 48 files changed, 779 insertions(+), 769 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index 55d87ea..735133f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -434,10 +434,7 @@ public class HadoopClassLoader extends URLClassLoader {
      * @return HADOOP_HOME Variable.
      */
     @Nullable public static String hadoopHome() {
-        return "C:\\Personal\\programs\\hadoop-2.6.0";
-
-        // TODO: FIX!
-        //return getEnv("HADOOP_PREFIX", getEnv("HADOOP_HOME", null));
+        return getEnv("HADOOP_PREFIX", getEnv("HADOOP_HOME", null));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
index d0964e8..55fac2c 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
@@ -20,32 +20,32 @@ package org.apache.ignite.internal.processors.hadoop;
 import javax.security.auth.AuthPermission;
 import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.ignite.internal.processors.hadoop.deps.CircularDependencyHadoop;
-import org.apache.ignite.internal.processors.hadoop.deps.CircularDependencyNoHadoop;
-import org.apache.ignite.internal.processors.hadoop.deps.DependencyNoHadoop;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopCasting;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopClassAnnotation;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopConstructorInvocation;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopDeclaredCheckedExceptionInMethod;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopDeclaredRuntimeExceptionInMethod;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopExtends;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopField;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopImplements;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopInitializer;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopInnerClass;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopLocalVariableType;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopMethodAnnotation;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopMethodInvocation;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopMethodParameter;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopMethodReturnType;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopOuterClass;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopParameterAnnotation;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopStaticField;
-import org.apache.ignite.internal.processors.hadoop.deps.HadoopStaticInitializer;
-import org.apache.ignite.internal.processors.hadoop.deps.NoHadoop;
+import org.apache.ignite.internal.processors.hadoop.deps.CircularWIthHadoop;
+import org.apache.ignite.internal.processors.hadoop.deps.CircularWithoutHadoop;
+import org.apache.ignite.internal.processors.hadoop.deps.WithIndirectField;
+import org.apache.ignite.internal.processors.hadoop.deps.WithCast;
+import org.apache.ignite.internal.processors.hadoop.deps.WithClassAnnotation;
+import org.apache.ignite.internal.processors.hadoop.deps.WithConstructorInvocation;
+import org.apache.ignite.internal.processors.hadoop.deps.WithMethodCheckedException;
+import org.apache.ignite.internal.processors.hadoop.deps.WithMethodRuntimeException;
+import org.apache.ignite.internal.processors.hadoop.deps.WithExtends;
+import org.apache.ignite.internal.processors.hadoop.deps.WithField;
+import org.apache.ignite.internal.processors.hadoop.deps.WithImplements;
+import org.apache.ignite.internal.processors.hadoop.deps.WithInitializer;
+import org.apache.ignite.internal.processors.hadoop.deps.WithInnerClass;
+import org.apache.ignite.internal.processors.hadoop.deps.WithLocalVariable;
+import org.apache.ignite.internal.processors.hadoop.deps.WithMethodAnnotation;
+import org.apache.ignite.internal.processors.hadoop.deps.WithMethodInvocation;
+import org.apache.ignite.internal.processors.hadoop.deps.WithMethodArgument;
+import org.apache.ignite.internal.processors.hadoop.deps.WithMethodReturnType;
+import org.apache.ignite.internal.processors.hadoop.deps.WithOuterClass;
+import org.apache.ignite.internal.processors.hadoop.deps.WithParameterAnnotation;
+import org.apache.ignite.internal.processors.hadoop.deps.WithStaticField;
+import org.apache.ignite.internal.processors.hadoop.deps.WithStaticInitializer;
+import org.apache.ignite.internal.processors.hadoop.deps.Without;
 
 /**
- *
+ * Tests for Hadoop classloader.
  */
 public class HadoopClassLoaderTest extends TestCase {
     /** */
@@ -55,51 +55,43 @@ public class HadoopClassLoaderTest extends TestCase {
      * @throws Exception If failed.
      */
     public void testClassLoading() throws Exception {
-        assertNotSame(CircularDependencyHadoop.class, ldr.loadClass(CircularDependencyHadoop.class.getName()));
-        assertNotSame(CircularDependencyNoHadoop.class, ldr.loadClass(CircularDependencyNoHadoop.class.getName()));
+        assertNotSame(CircularWIthHadoop.class, ldr.loadClass(CircularWIthHadoop.class.getName()));
+        assertNotSame(CircularWithoutHadoop.class, ldr.loadClass(CircularWithoutHadoop.class.getName()));
 
-        assertSame(NoHadoop.class, ldr.loadClass(NoHadoop.class.getName()));
+        assertSame(Without.class, ldr.loadClass(Without.class.getName()));
     }
 
     /**
-     *
+     * Test dependency search.
      */
     public void testDependencySearch() {
-        // Various positive cases of Hadoop classes dependency:
+        // Positive cases:
         final Class[] positiveClasses = {
-            // Hadoop class itself:
             Configuration.class,
-            // Class for that org.apache.ignite.internal.processors.hadoop.HadoopClassLoader.isHadoopIgfs returns true:
             HadoopUtils.class,
-
-            HadoopStaticField.class,
-            HadoopCasting.class,
-            HadoopClassAnnotation.class,
-            HadoopConstructorInvocation.class,
-            HadoopDeclaredCheckedExceptionInMethod.class,
-            HadoopDeclaredRuntimeExceptionInMethod.class,
-            HadoopExtends.class,
-            HadoopField.class,
-            HadoopImplements.class,
-            HadoopInitializer.class,
-
-            // TODO: actually the 2 below classes do not depend on Hadoop, should not be detected as such.
-            // TODO: but for now they are, so this behavior is asserted in test:
-            HadoopInnerClass.class,
-            HadoopOuterClass.InnerNoHadoop.class,
-
-            HadoopLocalVariableType.class,
-            HadoopMethodAnnotation.class,
-            HadoopMethodInvocation.class,
-            HadoopMethodParameter.class,
-            HadoopMethodReturnType.class,
-            HadoopParameterAnnotation.class,
-            HadoopStaticField.class,
-            HadoopStaticInitializer.class,
-
-            DependencyNoHadoop.class,
-            CircularDependencyHadoop.class,
-            CircularDependencyNoHadoop.class,
+            WithStaticField.class,
+            WithCast.class,
+            WithClassAnnotation.class,
+            WithConstructorInvocation.class,
+            WithMethodCheckedException.class,
+            WithMethodRuntimeException.class,
+            WithExtends.class,
+            WithField.class,
+            WithImplements.class,
+            WithInitializer.class,
+            WithInnerClass.class,
+            WithOuterClass.InnerNoHadoop.class,
+            WithLocalVariable.class,
+            WithMethodAnnotation.class,
+            WithMethodInvocation.class,
+            WithMethodArgument.class,
+            WithMethodReturnType.class,
+            WithParameterAnnotation.class,
+            WithStaticField.class,
+            WithStaticInitializer.class,
+            WithIndirectField.class,
+            CircularWIthHadoop.class,
+            CircularWithoutHadoop.class,
         };
 
         for (Class c: positiveClasses)
@@ -107,15 +99,12 @@ public class HadoopClassLoaderTest extends TestCase {
 
         // Negative cases:
         final Class[] negativeClasses = {
-            // java.lang.*:
             Object.class,
-            // javax.*:
             AuthPermission.class,
-            NoHadoop.class,
+            Without.class,
         };
 
         for (Class c: negativeClasses)
-            assertFalse(c.getName(),
-                ldr.hasExternalDependencies(c.getName()));
+            assertFalse(c.getName(), ldr.hasExternalDependencies(c.getName()));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyHadoop.java
deleted file mode 100644
index 24db626..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyHadoop.java
+++ /dev/null
@@ -1,31 +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.hadoop.deps;
-
-import org.apache.hadoop.mapreduce.Job;
-
-/**
- * Class has a direct Hadoop dependency and a circular dependency on another class.
- */
-public class CircularDependencyHadoop {
-    /** */
-    Job[][] jobs = new Job[4][4];
-
-    /** */
-    private CircularDependencyNoHadoop y;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyNoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyNoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyNoHadoop.java
deleted file mode 100644
index 4b019e4..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularDependencyNoHadoop.java
+++ /dev/null
@@ -1,26 +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.hadoop.deps;
-
-/**
- * Does not have direct Hadoop dependency, but has a circular
- */
-public class CircularDependencyNoHadoop {
-    /** */
-    private CircularDependencyHadoop x;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWIthHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWIthHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWIthHadoop.java
new file mode 100644
index 0000000..c3aa7d9
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWIthHadoop.java
@@ -0,0 +1,32 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.mapreduce.Job;
+
+/**
+ * Class has a direct Hadoop dependency and a circular dependency on another class.
+ */
+@SuppressWarnings("unused")
+public class CircularWIthHadoop {
+    /** */
+    private Job[][] jobs = new Job[4][4];
+
+    /** */
+    private CircularWithoutHadoop y;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWithoutHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWithoutHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWithoutHadoop.java
new file mode 100644
index 0000000..93d659c
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/CircularWithoutHadoop.java
@@ -0,0 +1,27 @@
+/*
+ * 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.hadoop.deps;
+
+/**
+ * Does not have direct Hadoop dependency, but has a circular
+ */
+@SuppressWarnings("unused")
+public class CircularWithoutHadoop {
+    /** */
+    private CircularWIthHadoop x;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/DependencyNoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/DependencyNoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/DependencyNoHadoop.java
deleted file mode 100644
index 25a29a1..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/DependencyNoHadoop.java
+++ /dev/null
@@ -1,26 +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.hadoop.deps;
-
-/**
- * Has a unidirected dependency on Hadoop-dependent class.
- */
-public class DependencyNoHadoop {
-    /** */
-    HadoopField x;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopCasting.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopCasting.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopCasting.java
deleted file mode 100644
index 3a5de89..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopCasting.java
+++ /dev/null
@@ -1,40 +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.hadoop.deps;
-
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Class contains casting to a Hadoop type.
- */
-public abstract class HadoopCasting <T> {
-    /** */
-    public abstract T create();
-
-    /** */
-    public void consume(T t) {
-        // noop
-    }
-
-    /** */
-    void test(HadoopCasting<FileSystem> c) {
-        FileSystem fs = c.create();
-
-        c.consume(fs);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopClassAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopClassAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopClassAnnotation.java
deleted file mode 100644
index cb934d2..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopClassAnnotation.java
+++ /dev/null
@@ -1,27 +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.hadoop.deps;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * Class has Hadoop annotation.
- */
-@InterfaceAudience.Public
-public class HadoopClassAnnotation {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopConstructorInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopConstructorInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopConstructorInvocation.java
deleted file mode 100644
index 5d641b1..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopConstructorInvocation.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.hadoop.deps;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Invokes a Hadoop type constructor.
- */
-public class HadoopConstructorInvocation {
-    /** */
-    private void foo() {
-        Object x = new Configuration();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredCheckedExceptionInMethod.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredCheckedExceptionInMethod.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredCheckedExceptionInMethod.java
deleted file mode 100644
index 91e6410..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredCheckedExceptionInMethod.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.hadoop.deps;
-
-import org.apache.hadoop.fs.ChecksumException;
-
-/**
- * Method declares a checked Hadoop Exception.
- */
-public class HadoopDeclaredCheckedExceptionInMethod {
-    /** */
-    void foo() throws ChecksumException {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredRuntimeExceptionInMethod.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredRuntimeExceptionInMethod.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredRuntimeExceptionInMethod.java
deleted file mode 100644
index bef47d3..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopDeclaredRuntimeExceptionInMethod.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.hadoop.deps;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-
-/**
- * Method declares a runtime Hadoop Exception.
- */
-public class HadoopDeclaredRuntimeExceptionInMethod {
-    /** */
-    void foo() throws HadoopIllegalArgumentException {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopExtends.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopExtends.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopExtends.java
deleted file mode 100644
index e7324f8..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopExtends.java
+++ /dev/null
@@ -1,27 +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.hadoop.deps;
-
-import org.apache.hadoop.fs.LocalFileSystem;
-
-/**
- * Class extends a Hadoop class.
- */
-public class HadoopExtends extends LocalFileSystem {
-    // noop
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopField.java
deleted file mode 100644
index 21e3f38..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopField.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.hadoop.deps;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Has a Hadoop field.
- */
-public class HadoopField {
-    /**
-     *
-     */
-    private Configuration conf;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopImplements.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopImplements.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopImplements.java
deleted file mode 100644
index d6f3c1b..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopImplements.java
+++ /dev/null
@@ -1,36 +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.hadoop.deps;
-
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Implements a Hadoop interface.
- */
-public class HadoopImplements implements Configurable {
-    /** {@inheritDoc} */
-    @Override public void setConf(Configuration conf) {
-        // noop
-    }
-
-    /** {@inheritDoc} */
-    @Override public Configuration getConf() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInitializer.java
deleted file mode 100644
index 7a7f809..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInitializer.java
+++ /dev/null
@@ -1,31 +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.hadoop.deps;
-
-/**
- * Has a field initialized with an expression invoking Hadoop method.
- */
-public class HadoopInitializer {
-    /** */
-    private final Object x = org.apache.hadoop.fs.FileSystem.getDefaultUri(null);
-
-    /** */
-    HadoopInitializer() throws Exception {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInnerClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInnerClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInnerClass.java
deleted file mode 100644
index 3fa3e41..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopInnerClass.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.hadoop.deps;
-
-import org.apache.hadoop.conf.Configurable;
-
-/**
- * Has a *static* inner class depending on Hadoop.
- */
-public class HadoopInnerClass {
-    /** */
-    private static abstract class Foo implements Configurable {
-        // nothing
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopLocalVariableType.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopLocalVariableType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopLocalVariableType.java
deleted file mode 100644
index de2354e..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopLocalVariableType.java
+++ /dev/null
@@ -1,37 +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.hadoop.deps;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Has a local variable of Hadoop type.
- */
-public class HadoopLocalVariableType {
-    /** */
-    void foo() {
-        Configuration c = null;
-
-        moo(c);
-    }
-
-    /** */
-    void moo(Object x) {
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodAnnotation.java
deleted file mode 100644
index a013021..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodAnnotation.java
+++ /dev/null
@@ -1,29 +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.hadoop.deps;
-
-/**
- * Method has a Hadoop annotation.
- */
-public class HadoopMethodAnnotation {
-    /** */
-    @org.apache.hadoop.classification.InterfaceStability.Unstable
-    void foo() {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodInvocation.java
deleted file mode 100644
index b4d52b6..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodInvocation.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.hadoop.deps;
-
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Method contains a Hadoop type method invocation.
- */
-public class HadoopMethodInvocation {
-    /** */
-    void foo(FileSystem fs) {
-        fs.getChildFileSystems();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodParameter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodParameter.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodParameter.java
deleted file mode 100644
index c9621aa..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodParameter.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.hadoop.deps;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Contains a formal parameter of Hadoop type.
- */
-public class HadoopMethodParameter {
-    /** */
-    protected void paramaterMethod(Configuration c) {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodReturnType.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodReturnType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodReturnType.java
deleted file mode 100644
index b2a1a3f..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopMethodReturnType.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.hadoop.deps;
-
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Contains a method return value of Hadoop type.
- */
-public class HadoopMethodReturnType {
-    /** */
-    FileSystem fsMethod() {
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopOuterClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopOuterClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopOuterClass.java
deleted file mode 100644
index 142198c..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopOuterClass.java
+++ /dev/null
@@ -1,37 +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.hadoop.deps;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * Outer class depends on Hadoop, but Inner *static* one does not.
- */
-public class HadoopOuterClass {
-    /** */
-    Configuration c;
-
-    /** */
-    public static class InnerNoHadoop {
-        /** */
-        int x;
-
-        /** */
-        void foo() {}
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopParameterAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopParameterAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopParameterAnnotation.java
deleted file mode 100644
index 02bb099..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopParameterAnnotation.java
+++ /dev/null
@@ -1,28 +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.hadoop.deps;
-
-/**
- * Has a paramater annotated with a Hadoop annotation.
- */
-public class HadoopParameterAnnotation {
-    /** */
-    void foo(@org.apache.hadoop.classification.InterfaceStability.Stable Object annotatedParam) {
-        // noop
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticField.java
deleted file mode 100644
index eff1c4e..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticField.java
+++ /dev/null
@@ -1,28 +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.hadoop.deps;
-
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Has a static field of Hadoop type.
- */
-public class HadoopStaticField {
-    /** */
-    static FileSystem fs;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticInitializer.java
deleted file mode 100644
index 109b971..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/HadoopStaticInitializer.java
+++ /dev/null
@@ -1,33 +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.hadoop.deps;
-
-import java.util.List;
-import org.apache.hadoop.fs.FileSystem;
-
-/**
- * Uses Hadoop type in a static initializer.
- */
-public class HadoopStaticInitializer {
-    /** */
-    static final List x;
-
-    static {
-        x = FileSystem.getAllStatistics();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/NoHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/NoHadoop.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/NoHadoop.java
deleted file mode 100644
index aff3981..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/NoHadoop.java
+++ /dev/null
@@ -1,24 +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.hadoop.deps;
-
-/**
- * Class that does not anyhow depend on Hadoop.
- */
-public class NoHadoop {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithCast.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithCast.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithCast.java
new file mode 100644
index 0000000..5b1e8e0
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithCast.java
@@ -0,0 +1,41 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Class contains casting to a Hadoop type.
+ */
+@SuppressWarnings("unused")
+public abstract class WithCast<T> {
+    /** */
+    public abstract T create();
+
+    /** */
+    public void consume(T t) {
+        // noop
+    }
+
+    /** */
+    void test(WithCast<FileSystem> c) {
+        FileSystem fs = c.create();
+
+        c.consume(fs);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithClassAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithClassAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithClassAnnotation.java
new file mode 100644
index 0000000..a9ecae0
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithClassAnnotation.java
@@ -0,0 +1,28 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Class has Hadoop annotation.
+ */
+@SuppressWarnings("unused")
+@InterfaceAudience.Public
+public class WithClassAnnotation {
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithConstructorInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithConstructorInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithConstructorInvocation.java
new file mode 100644
index 0000000..98c8991
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithConstructorInvocation.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.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Invokes a Hadoop type constructor.
+ */
+@SuppressWarnings("unused")
+public class WithConstructorInvocation {
+    /** */
+    private void foo() {
+        Object x = new Configuration();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithExtends.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithExtends.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithExtends.java
new file mode 100644
index 0000000..80c99e1
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithExtends.java
@@ -0,0 +1,27 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.fs.LocalFileSystem;
+
+/**
+ * Class extends a Hadoop class.
+ */
+public class WithExtends extends LocalFileSystem {
+    // noop
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithField.java
new file mode 100644
index 0000000..dd979db
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithField.java
@@ -0,0 +1,29 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Has a Hadoop field.
+ */
+@SuppressWarnings("unused")
+public class WithField {
+    /** */
+    private Configuration conf;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithImplements.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithImplements.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithImplements.java
new file mode 100644
index 0000000..c2d8e5b
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithImplements.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.internal.processors.hadoop.deps;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Implements a Hadoop interface.
+ */
+public class WithImplements implements Configurable {
+    /** {@inheritDoc} */
+    @Override public void setConf(Configuration conf) {
+        // noop
+    }
+
+    /** {@inheritDoc} */
+    @Override public Configuration getConf() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithIndirectField.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithIndirectField.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithIndirectField.java
new file mode 100644
index 0000000..ce078f1
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithIndirectField.java
@@ -0,0 +1,27 @@
+/*
+ * 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.hadoop.deps;
+
+/**
+ * Has a unidirected dependency on Hadoop-dependent class.
+ */
+@SuppressWarnings("unused")
+public class WithIndirectField {
+    /** */
+    WithField x;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInitializer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInitializer.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInitializer.java
new file mode 100644
index 0000000..360986c
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInitializer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.hadoop.deps;
+
+/**
+ * Has a field initialized with an expression invoking Hadoop method.
+ */
+
+@SuppressWarnings({"ConstantConditions", "unused"})
+public class WithInitializer {
+    /** */
+    private final Object x = org.apache.hadoop.fs.FileSystem.getDefaultUri(null);
+
+    /** */
+    WithInitializer() throws Exception {
+        // noop
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInnerClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInnerClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInnerClass.java
new file mode 100644
index 0000000..4a5a49c
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithInnerClass.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.hadoop.deps;
+
+import org.apache.hadoop.conf.Configurable;
+
+/**
+ * Has a *static* inner class depending on Hadoop.
+ */
+@SuppressWarnings("unused")
+public class WithInnerClass {
+    /** */
+    private static abstract class Foo implements Configurable {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithLocalVariable.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithLocalVariable.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithLocalVariable.java
new file mode 100644
index 0000000..ea4a5de
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithLocalVariable.java
@@ -0,0 +1,38 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Has a local variable of Hadoop type.
+ */
+@SuppressWarnings({"unused", "ConstantConditions"})
+public class WithLocalVariable {
+    /** */
+    void foo() {
+        Configuration c = null;
+
+        moo(c);
+    }
+
+    /** */
+    void moo(Object x) {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodAnnotation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodAnnotation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodAnnotation.java
new file mode 100644
index 0000000..ff9fbe0
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodAnnotation.java
@@ -0,0 +1,32 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Method has a Hadoop annotation.
+ */
+@SuppressWarnings("unused")
+public class WithMethodAnnotation {
+    /** */
+    @InterfaceStability.Unstable
+    void foo() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodArgument.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodArgument.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodArgument.java
new file mode 100644
index 0000000..7f639e4
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodArgument.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.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Contains a formal parameter of Hadoop type.
+ */
+@SuppressWarnings("unused")
+public class WithMethodArgument {
+    /** */
+    protected void paramaterMethod(Configuration c) {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodCheckedException.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodCheckedException.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodCheckedException.java
new file mode 100644
index 0000000..8fd12ae
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodCheckedException.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.hadoop.deps;
+
+import org.apache.hadoop.fs.ChecksumException;
+
+/**
+ * Method declares a checked Hadoop Exception.
+ */
+@SuppressWarnings("unused")
+public class WithMethodCheckedException {
+    /** */
+    void foo() throws ChecksumException {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodInvocation.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodInvocation.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodInvocation.java
new file mode 100644
index 0000000..de8b306
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodInvocation.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.hadoop.deps;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Method contains a Hadoop type method invocation.
+ */
+@SuppressWarnings("unused")
+public class WithMethodInvocation {
+    /** */
+    void foo(FileSystem fs) {
+        fs.getChildFileSystems();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodReturnType.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodReturnType.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodReturnType.java
new file mode 100644
index 0000000..0e0ea72
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodReturnType.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.hadoop.deps;
+
+import org.apache.hadoop.fs.FileSystem;
+
+/**
+ * Contains a method return value of Hadoop type.
+ */
+@SuppressWarnings("unused")
+public class WithMethodReturnType {
+    /** */
+    FileSystem fsMethod() {
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodRuntimeException.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodRuntimeException.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodRuntimeException.java
new file mode 100644
index 0000000..dcd471c
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithMethodRuntimeException.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.hadoop.deps;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+
+/**
+ * Method declares a runtime Hadoop Exception.
+ */
+@SuppressWarnings("unused")
+public class WithMethodRuntimeException {
+    /** */
+    void foo() throws HadoopIllegalArgumentException {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/42a6a8c2/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithOuterClass.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithOuterClass.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithOuterClass.java
new file mode 100644
index 0000000..cae1da7
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/deps/WithOuterClass.java
@@ -0,0 +1,38 @@
+/*
+ * 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.hadoop.deps;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Outer class depends on Hadoop, but Inner *static* one does not.
+ */
+@SuppressWarnings("unused")
+public class WithOuterClass {
+    /** */
+    Configuration c;
+
+    /** */
+    public static class InnerNoHadoop {
+        /** */
+        int x;
+
+        /** */
+        void foo() {}
+    }
+}


[5/9] ignite git commit: IGNITE-2308: Refactoring.

Posted by vo...@apache.org.
IGNITE-2308: Refactoring.


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

Branch: refs/heads/ignite-2308
Commit: e6f4455c47ae9f832a7e96095ab8ab1ccb07b00a
Parents: dda6b27
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jan 4 10:01:21 2016 +0400
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jan 4 10:01:21 2016 +0400

----------------------------------------------------------------------
 .../processors/hadoop/HadoopClassLoader.java    | 229 ++++++++++---------
 .../hadoop/HadoopClassLoaderTest.java           |   5 +-
 2 files changed, 122 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e6f4455c/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index a2c9df4..010a6bd 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -30,7 +30,7 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.v2.HadoopDaemon;
 import org.apache.ignite.internal.processors.hadoop.v2.HadoopNativeCodeLoader;
@@ -166,7 +166,7 @@ public class HadoopClassLoader extends URLClassLoader {
                 Boolean hasDeps = cache.get(name);
 
                 if (hasDeps == null) {
-                    hasDeps = hasExternalDependencies(name, new HashSet<String>());
+                    hasDeps = hasExternalDependencies(name);
 
                     cache.put(name, hasDeps);
                 }
@@ -273,10 +273,30 @@ public class HadoopClassLoader extends URLClassLoader {
     }
 
     /**
+     * Check whether class has external dependencies on Hadoop.
+     *
+     * @param clsName Class name.
+     * @return {@code True} if class has external dependencies.
+     */
+    boolean hasExternalDependencies(String clsName) {
+        CollectingContext ctx = new CollectingContext();
+
+        ctx.annVisitor = new CollectingAnnotationVisitor(ctx);
+        ctx.mthdVisitor = new CollectingMethodVisitor(ctx, ctx.annVisitor);
+        ctx.fldVisitor = new CollectingFieldVisitor(ctx, ctx.annVisitor);
+        ctx.clsVisitor = new CollectingClassVisitor(ctx, ctx.annVisitor, ctx.mthdVisitor, ctx.fldVisitor);
+
+        return hasExternalDependencies(clsName, ctx);
+    }
+
+    /**
+     * Check whether class has external dependencies on Hadoop.
+     *
      * @param clsName Class name.
+     * @param ctx Context.
      * @return {@code true} If the class has external dependencies.
      */
-    boolean hasExternalDependencies(String clsName, Set<String> visited) {
+    boolean hasExternalDependencies(String clsName, CollectingContext ctx) {
         if (isHadoop(clsName)) // Hadoop must not be in classpath but Idea sucks, so filtering explicitly as external.
             return true;
 
@@ -298,20 +318,11 @@ public class HadoopClassLoader extends URLClassLoader {
             throw new RuntimeException("Failed to read class: " + clsName, e);
         }
 
-        visited.add(clsName);
-
-        final AtomicBoolean hasDeps = new AtomicBoolean();
-
-        Collector c = new Collector(hasDeps, visited);
+        ctx.visited.add(clsName);
 
-        AnnotationVisitor annVisitor = new CollectingAnnotationVisitor(c);
-        MethodVisitor mthdVisitor = new CollectingMethodVisitor(c, annVisitor);
-        FieldVisitor fldVisitor = new CollectingFieldVisitor(c, annVisitor);
-        ClassVisitor clsVisitor = new CollectingClassVisitor(c, annVisitor, mthdVisitor, fldVisitor);
+        rdr.accept(ctx.clsVisitor, 0);
 
-        rdr.accept(clsVisitor, 0);
-
-        if (hasDeps.get()) // We already know that we have dependencies, no need to check parent.
+        if (ctx.found()) // We already know that we have dependencies, no need to check parent.
             return true;
 
         // Here we are known to not have any dependencies but possibly we have a parent which has them.
@@ -322,13 +333,13 @@ public class HadoopClassLoader extends URLClassLoader {
 
         String parentCls = clsName.substring(0, idx);
 
-        if (visited.contains(parentCls))
+        if (ctx.visited.contains(parentCls))
             return false;
 
         Boolean res = cache.get(parentCls);
 
         if (res == null)
-            res = hasExternalDependencies(parentCls, visited);
+            res = hasExternalDependencies(parentCls, ctx);
 
         return res;
     }
@@ -423,7 +434,10 @@ public class HadoopClassLoader extends URLClassLoader {
      * @return HADOOP_HOME Variable.
      */
     @Nullable public static String hadoopHome() {
-        return getEnv("HADOOP_PREFIX", getEnv("HADOOP_HOME", null));
+        return "C:\\Personal\\programs\\hadoop-2.6.0";
+
+        // TODO: FIX!
+        //return getEnv("HADOOP_PREFIX", getEnv("HADOOP_HOME", null));
     }
 
     /**
@@ -491,38 +505,34 @@ public class HadoopClassLoader extends URLClassLoader {
     }
 
     /**
-     * Implement business logic of the dependency analysis.
-     * Keeps the necessary state.
+     * Context for dependencies collection.
      */
-    private class Collector {
-        /** Attribute gets 'true' if the positive answer is found. */
-        private final AtomicBoolean hasDeps;
+    private class CollectingContext {
+        /** Visited classes. */
+        private final Set<String> visited = new HashSet<>();
 
-        /** Collection of visited class names to prevent infinite loops in case of
-         * circular dependencies. */
-        private final Set<String> visited;
+        /** Whether dependency found. */
+        private boolean found;
 
-        /**
-         * Constructor.
-         *
-         * @param hasDeps has dependencies initial value.
-         * @param visited visited set initial value.
-         */
-        Collector(AtomicBoolean hasDeps, Set<String> visited) {
-            assert hasDeps != null;
-            assert visited != null;
+        /** Annotation visitor. */
+        private AnnotationVisitor annVisitor;
 
-            this.hasDeps = hasDeps;
-            this.visited = visited;
-        }
+        /** Method visitor. */
+        private MethodVisitor mthdVisitor;
+
+        /** Field visitor. */
+        private FieldVisitor fldVisitor;
+
+        /** Class visitor. */
+        private ClassVisitor clsVisitor;
 
         /**
          * Answers if the model travers should be finished.
          *
          * @return If it is done.
          */
-        boolean isDone() {
-            return hasDeps.get();
+        boolean found() {
+            return found;
         }
 
         /**
@@ -533,14 +543,14 @@ public class HadoopClassLoader extends URLClassLoader {
             // Process method return type:
             onType(Type.getReturnType(methDesc));
 
-            if (isDone())
+            if (found())
                 return;
 
             // Process method argument types:
             for (Type t: Type.getArgumentTypes(methDesc)) {
                 onType(t);
 
-                if (isDone())
+                if (found())
                     return;
             }
         }
@@ -563,8 +573,8 @@ public class HadoopClassLoader extends URLClassLoader {
 
             Boolean res = cache.get(depCls);
 
-            if (res == Boolean.TRUE || (res == null && hasExternalDependencies(depCls, visited)))
-                hasDeps.set(true);
+            if (res == Boolean.TRUE || (res == null && hasExternalDependencies(depCls, this)))
+                found = true;
         }
 
         /**
@@ -636,48 +646,49 @@ public class HadoopClassLoader extends URLClassLoader {
      */
     private static class CollectingAnnotationVisitor extends AnnotationVisitor {
         /** */
-        final Collector c;
+        final CollectingContext ctx;
 
         /**
+         * Annotation visitor.
          *
-         * @param c The collector.
+         * @param ctx The collector.
          */
-        CollectingAnnotationVisitor(Collector c) {
+        CollectingAnnotationVisitor(CollectingContext ctx) {
             super(Opcodes.ASM4);
 
-            this.c = c;
+            this.ctx = ctx;
         }
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotation(String name, String desc) {
-            if (c.isDone())
+            if (ctx.found())
                 return null;
 
-            c.onType(desc);
+            ctx.onType(desc);
 
             return this;
         }
 
         /** {@inheritDoc} */
         @Override public void visitEnum(String name, String desc, String val) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onType(desc);
+            ctx.onType(desc);
         }
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitArray(String name) {
-            return c.isDone() ? null : this;
+            return ctx.found() ? null : this;
         }
 
         /** {@inheritDoc} */
         @Override public void visit(String name, Object val) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
             if (val instanceof Type)
-                c.onType((Type)val);
+                ctx.onType((Type)val);
         }
 
         /** {@inheritDoc} */
@@ -691,7 +702,7 @@ public class HadoopClassLoader extends URLClassLoader {
      */
     private static class CollectingFieldVisitor extends FieldVisitor {
         /** Collector. */
-        private final Collector c;
+        private final CollectingContext ctx;
 
         /** Annotation visitor. */
         private final AnnotationVisitor av;
@@ -699,21 +710,21 @@ public class HadoopClassLoader extends URLClassLoader {
         /**
          * Constructor.
          */
-        CollectingFieldVisitor(Collector c, AnnotationVisitor av) {
+        CollectingFieldVisitor(CollectingContext ctx, AnnotationVisitor av) {
             super(Opcodes.ASM4);
 
-            this.c = c;
+            this.ctx = ctx;
             this.av = av;
         }
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
-            if (c.isDone())
+            if (ctx.found())
                 return null;
 
-            c.onType(desc);
+            ctx.onType(desc);
 
-            return c.isDone() ? null : av;
+            return ctx.found() ? null : av;
         }
 
         /** {@inheritDoc} */
@@ -732,7 +743,7 @@ public class HadoopClassLoader extends URLClassLoader {
      */
     private static class CollectingClassVisitor extends ClassVisitor {
         /** Collector. */
-        private final Collector c;
+        private final CollectingContext ctx;
 
         /** Annotation visitor. */
         private final AnnotationVisitor av;
@@ -746,15 +757,15 @@ public class HadoopClassLoader extends URLClassLoader {
         /**
          * Constructor.
          *
-         * @param c Collector.
+         * @param ctx Collector.
          * @param av Annotation visitor.
          * @param mv Method visitor.
          * @param fv Field visitor.
          */
-        CollectingClassVisitor(Collector c, AnnotationVisitor av, MethodVisitor mv, FieldVisitor fv) {
+        CollectingClassVisitor(CollectingContext ctx, AnnotationVisitor av, MethodVisitor mv, FieldVisitor fv) {
             super(Opcodes.ASM4);
 
-            this.c = c;
+            this.ctx = ctx;
             this.av = av;
             this.mv = mv;
             this.fv = fv;
@@ -762,19 +773,19 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public void visit(int i, int i2, String name, String signature, String superName, String[] ifaces) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onInternalTypeName(superName);
+            ctx.onInternalTypeName(superName);
 
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
             if (ifaces != null) {
                 for (String iface : ifaces) {
-                    c.onInternalTypeName(iface);
+                    ctx.onInternalTypeName(iface);
 
-                    if (c.isDone())
+                    if (ctx.found())
                         return;
                 }
             }
@@ -782,47 +793,47 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
-            if (c.isDone())
+            if (ctx.found())
                 return null;
 
-            c.onType(desc);
+            ctx.onType(desc);
 
-            return c.isDone() ? null : av;
+            return ctx.found() ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public void visitInnerClass(String name, String outerName, String innerName, int i) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onInternalTypeName(name);
+            ctx.onInternalTypeName(name);
         }
 
         /** {@inheritDoc} */
         @Override public FieldVisitor visitField(int i, String name, String desc, String signature, Object val) {
-            if (c.isDone())
+            if (ctx.found())
                 return null;
 
-            c.onType(desc);
+            ctx.onType(desc);
 
-            return c.isDone() ? null : fv;
+            return ctx.found() ? null : fv;
         }
 
         /** {@inheritDoc} */
         @Override public MethodVisitor visitMethod(int i, String name, String desc, String signature,
             String[] exceptions) {
-            if (c.isDone())
+            if (ctx.found())
                 return null;
 
-            c.onMethodsDesc(desc);
+            ctx.onMethodsDesc(desc);
 
             // Process declared method exceptions:
             if (exceptions != null) {
                 for (String e : exceptions)
-                    c.onInternalTypeName(e);
+                    ctx.onInternalTypeName(e);
             }
 
-            return c.isDone() ? null : mv;
+            return ctx.found() ? null : mv;
         }
     }
 
@@ -831,7 +842,7 @@ public class HadoopClassLoader extends URLClassLoader {
      */
     private static class CollectingMethodVisitor extends MethodVisitor {
         /** Collector. */
-        private final Collector c;
+        private final CollectingContext ctx;
 
         /** Annotation visitor. */
         private final AnnotationVisitor av;
@@ -839,52 +850,52 @@ public class HadoopClassLoader extends URLClassLoader {
         /**
          * Constructor.
          *
-         * @param c Collector.
+         * @param ctx Collector.
          * @param av Annotation visitor.
          */
-        private CollectingMethodVisitor(Collector c, AnnotationVisitor av) {
+        private CollectingMethodVisitor(CollectingContext ctx, AnnotationVisitor av) {
             super(Opcodes.ASM4);
 
-            this.c = c;
+            this.ctx = ctx;
             this.av = av;
         }
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
-            if (c.isDone())
+            if (ctx.found())
                 return null;
 
-            c.onType(desc);
+            ctx.onType(desc);
 
-            return c.isDone() ? null : av;
+            return ctx.found() ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitParameterAnnotation(int i, String desc, boolean b) {
-            if (c.isDone())
+            if (ctx.found())
                 return null;
 
-            c.onType(desc);
+            ctx.onType(desc);
 
-            return c.isDone() ? null : av;
+            return ctx.found() ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotationDefault() {
-            return c.isDone() ? null : av;
+            return ctx.found() ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public void visitFieldInsn(int opcode, String owner, String name, String desc) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onInternalTypeName(owner);
+            ctx.onInternalTypeName(owner);
 
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onType(desc);
+            ctx.onType(desc);
         }
 
         /** {@inheritDoc} */
@@ -900,47 +911,47 @@ public class HadoopClassLoader extends URLClassLoader {
         /** {@inheritDoc} */
         @Override public void visitLocalVariable(String name, String desc, String signature, Label lb,
             Label lb2, int i) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onType(desc);
+            ctx.onType(desc);
         }
 
         /** {@inheritDoc} */
         @Override public void visitMethodInsn(int i, String owner, String name, String desc) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onInternalTypeName(owner);
+            ctx.onInternalTypeName(owner);
 
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onMethodsDesc(desc);
+            ctx.onMethodsDesc(desc);
         }
 
         /** {@inheritDoc} */
         @Override public void visitMultiANewArrayInsn(String desc, int dim) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onType(desc);
+            ctx.onType(desc);
         }
 
         /** {@inheritDoc} */
         @Override public void visitTryCatchBlock(Label start, Label end, Label hndl, String typeStr) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onInternalTypeName(typeStr);
+            ctx.onInternalTypeName(typeStr);
         }
 
         /** {@inheritDoc} */
         @Override public void visitTypeInsn(int opcode, String type) {
-            if (c.isDone())
+            if (ctx.found())
                 return;
 
-            c.onInternalTypeName(type);
+            ctx.onInternalTypeName(type);
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6f4455c/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
index e878d9a..88f43b8 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
@@ -105,8 +105,7 @@ public class HadoopClassLoaderTest extends TestCase {
         };
 
         for (Class c: positiveClasses)
-            assertTrue(c.getName(),
-                ldr.hasExternalDependencies(c.getName(), new HashSet<String>()));
+            assertTrue(c.getName(), ldr.hasExternalDependencies(c.getName()));
 
         // Negative cases:
         final Class[] negativeClasses = {
@@ -119,6 +118,6 @@ public class HadoopClassLoaderTest extends TestCase {
 
         for (Class c: negativeClasses)
             assertFalse(c.getName(),
-                ldr.hasExternalDependencies(c.getName(), new HashSet<String>()));
+                ldr.hasExternalDependencies(c.getName()));
     }
 }
\ No newline at end of file


[4/9] ignite git commit: IGNITE-2308: Fixing styling.

Posted by vo...@apache.org.
IGNITE-2308: Fixing styling.


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

Branch: refs/heads/ignite-2308
Commit: dda6b27c86a3e8998bba4a37ff8fd0bf455b4c56
Parents: 67c1e0f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jan 4 09:47:12 2016 +0400
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jan 4 09:47:12 2016 +0400

----------------------------------------------------------------------
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |   1 -
 .../processors/hadoop/HadoopClassLoader.java    | 468 +++++++++----------
 2 files changed, 215 insertions(+), 254 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dda6b27c/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index a72452b..5dce67f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -328,7 +328,6 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             if (initSecondary) {
                 Map<String, String> props = paths.properties();
-                //Object payload0 = paths.getPayload(getClass().getClassLoader());
 
                 String secUri = props.get(SECONDARY_FS_URI);
                 String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);

http://git-wip-us.apache.org/repos/asf/ignite/blob/dda6b27c/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index 89a55e4..a2c9df4 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -128,14 +128,14 @@ public class HadoopClassLoader extends URLClassLoader {
      * @return {@code true} if we need to check this class.
      */
     private static boolean isHadoopIgfs(String cls) {
-        String ignitePackPrefix = "org.apache.ignite";
+        String ignitePkgPrefix = "org.apache.ignite";
 
-        int len = ignitePackPrefix.length();
+        int len = ignitePkgPrefix.length();
 
-        return cls.startsWith(ignitePackPrefix)
-            && (cls.indexOf("igfs.", len) != -1
-                || cls.indexOf(".fs.", len) != -1
-                || cls.indexOf("hadoop.", len) != -1);
+        return cls.startsWith(ignitePkgPrefix) && (
+            cls.indexOf("igfs.", len) != -1 ||
+            cls.indexOf(".fs.", len) != -1 ||
+            cls.indexOf("hadoop.", len) != -1);
     }
 
     /**
@@ -276,7 +276,7 @@ public class HadoopClassLoader extends URLClassLoader {
      * @param clsName Class name.
      * @return {@code true} If the class has external dependencies.
      */
-    boolean hasExternalDependencies(final String clsName, final Set<String> visited) {
+    boolean hasExternalDependencies(String clsName, Set<String> visited) {
         if (isHadoop(clsName)) // Hadoop must not be in classpath but Idea sucks, so filtering explicitly as external.
             return true;
 
@@ -304,13 +304,10 @@ public class HadoopClassLoader extends URLClassLoader {
 
         Collector c = new Collector(hasDeps, visited);
 
-        AnnotationVisitor annotationVisitor = new CollectingAnnotationVisitor(c);
-
-        MethodVisitor methVisitor = new CollectingMethodVisitor(c, annotationVisitor);
-
-        FieldVisitor fieldVisitor = new CollectingFieldVisitor(c, annotationVisitor);
-
-        ClassVisitor clsVisitor = new CollectingClassVisitor(c, annotationVisitor, methVisitor, fieldVisitor);
+        AnnotationVisitor annVisitor = new CollectingAnnotationVisitor(c);
+        MethodVisitor mthdVisitor = new CollectingMethodVisitor(c, annVisitor);
+        FieldVisitor fldVisitor = new CollectingFieldVisitor(c, annVisitor);
+        ClassVisitor clsVisitor = new CollectingClassVisitor(c, annVisitor, mthdVisitor, fldVisitor);
 
         rdr.accept(clsVisitor, 0);
 
@@ -337,27 +334,186 @@ public class HadoopClassLoader extends URLClassLoader {
     }
 
     /**
+     * @param name Class name.
+     * @return {@code true} If this is a valid class name.
+     */
+    private static boolean validateClassName(String name) {
+        int len = name.length();
+
+        if (len <= 1)
+            return false;
+
+        if (!Character.isJavaIdentifierStart(name.charAt(0)))
+            return false;
+
+        boolean hasDot = false;
+
+        for (int i = 1; i < len; i++) {
+            char c = name.charAt(i);
+
+            if (c == '.')
+                hasDot = true;
+            else if (!Character.isJavaIdentifierPart(c))
+                return false;
+        }
+
+        return hasDot;
+    }
+
+    /**
+     * @param name Variable name.
+     * @param dflt Default.
+     * @return Value.
+     */
+    private static String getEnv(String name, String dflt) {
+        String res = System.getProperty(name);
+
+        if (F.isEmpty(res))
+            res = System.getenv(name);
+
+        return F.isEmpty(res) ? dflt : res;
+    }
+
+    /**
+     * @param res Result.
+     * @param dir Directory.
+     * @param startsWith Starts with prefix.
+     * @throws MalformedURLException If failed.
+     */
+    private static void addUrls(Collection<URL> res, File dir, final String startsWith) throws Exception {
+        File[] files = dir.listFiles(new FilenameFilter() {
+            @Override public boolean accept(File dir, String name) {
+                return startsWith == null || name.startsWith(startsWith);
+            }
+        });
+
+        if (files == null)
+            throw new IOException("Path is not a directory: " + dir);
+
+        for (File file : files)
+            res.add(file.toURI().toURL());
+    }
+
+    /**
+     * @param urls URLs.
+     * @return URLs.
+     */
+    private static URL[] addHadoopUrls(URL[] urls) {
+        Collection<URL> hadoopJars;
+
+        try {
+            hadoopJars = hadoopUrls();
+        }
+        catch (IgniteCheckedException e) {
+            throw new RuntimeException(e);
+        }
+
+        ArrayList<URL> list = new ArrayList<>(hadoopJars.size() + appJars.size() + (urls == null ? 0 : urls.length));
+
+        list.addAll(appJars);
+        list.addAll(hadoopJars);
+
+        if (!F.isEmpty(urls))
+            list.addAll(F.asList(urls));
+
+        return list.toArray(new URL[list.size()]);
+    }
+
+    /**
+     * @return HADOOP_HOME Variable.
+     */
+    @Nullable public static String hadoopHome() {
+        return getEnv("HADOOP_PREFIX", getEnv("HADOOP_HOME", null));
+    }
+
+    /**
+     * @return Collection of jar URLs.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static Collection<URL> hadoopUrls() throws IgniteCheckedException {
+        Collection<URL> hadoopUrls = hadoopJars;
+
+        if (hadoopUrls != null)
+            return hadoopUrls;
+
+        synchronized (HadoopClassLoader.class) {
+            hadoopUrls = hadoopJars;
+
+            if (hadoopUrls != null)
+                return hadoopUrls;
+
+            hadoopUrls = new ArrayList<>();
+
+            String hadoopPrefix = hadoopHome();
+
+            if (F.isEmpty(hadoopPrefix))
+                throw new IgniteCheckedException("Failed resolve Hadoop installation location. Either HADOOP_PREFIX or " +
+                    "HADOOP_HOME environment variables must be set.");
+
+            String commonHome = getEnv("HADOOP_COMMON_HOME", hadoopPrefix + "/share/hadoop/common");
+            String hdfsHome = getEnv("HADOOP_HDFS_HOME", hadoopPrefix + "/share/hadoop/hdfs");
+            String mapredHome = getEnv("HADOOP_MAPRED_HOME", hadoopPrefix + "/share/hadoop/mapreduce");
+
+            try {
+                addUrls(hadoopUrls, new File(commonHome + "/lib"), null);
+                addUrls(hadoopUrls, new File(hdfsHome + "/lib"), null);
+                addUrls(hadoopUrls, new File(mapredHome + "/lib"), null);
+
+                addUrls(hadoopUrls, new File(hdfsHome), "hadoop-hdfs-");
+
+                addUrls(hadoopUrls, new File(commonHome), "hadoop-common-");
+                addUrls(hadoopUrls, new File(commonHome), "hadoop-auth-");
+                addUrls(hadoopUrls, new File(commonHome + "/lib"), "hadoop-auth-");
+
+                addUrls(hadoopUrls, new File(mapredHome), "hadoop-mapreduce-client-common");
+                addUrls(hadoopUrls, new File(mapredHome), "hadoop-mapreduce-client-core");
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException(e);
+            }
+
+            hadoopJars = hadoopUrls;
+
+            return hadoopUrls;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopClassLoader.class, this);
+    }
+
+    /**
+     * Getter for name field.
+     */
+    public String name() {
+        return name;
+    }
+
+    /**
      * Implement business logic of the dependency analysis.
      * Keeps the necessary state.
      */
     private class Collector {
         /** Attribute gets 'true' if the positive answer is found. */
-        final AtomicBoolean hasDeps;
+        private final AtomicBoolean hasDeps;
 
         /** Collection of visited class names to prevent infinite loops in case of
          * circular dependencies. */
-        final Set<String> visited;
+        private final Set<String> visited;
 
         /**
          * Constructor.
          *
          * @param hasDeps has dependencies initial value.
-         * @param visitedSet visited set initial value.
+         * @param visited visited set initial value.
          */
-        Collector(AtomicBoolean hasDeps, Set<String> visitedSet) {
-            this.hasDeps = hasDeps;
+        Collector(AtomicBoolean hasDeps, Set<String> visited) {
+            assert hasDeps != null;
+            assert visited != null;
 
-            this.visited = visitedSet;
+            this.hasDeps = hasDeps;
+            this.visited = visited;
         }
 
         /**
@@ -407,8 +563,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
             Boolean res = cache.get(depCls);
 
-            if (res == Boolean.TRUE
-                || (res == null && hasExternalDependencies(depCls, visited)))
+            if (res == Boolean.TRUE || (res == null && hasExternalDependencies(depCls, visited)))
                 hasDeps.set(true);
         }
 
@@ -479,7 +634,7 @@ public class HadoopClassLoader extends URLClassLoader {
     /**
      * Annotation visitor.
      */
-    static class CollectingAnnotationVisitor extends AnnotationVisitor {
+    private static class CollectingAnnotationVisitor extends AnnotationVisitor {
         /** */
         final Collector c;
 
@@ -513,29 +668,32 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitArray(String name) {
-            return this;
+            return c.isDone() ? null : this;
         }
 
         /** {@inheritDoc} */
         @Override public void visit(String name, Object val) {
+            if (c.isDone())
+                return;
+
             if (val instanceof Type)
                 c.onType((Type)val);
         }
 
         /** {@inheritDoc} */
         @Override public void visitEnd() {
-            // noop
+            // No-op.
         }
     }
 
     /**
      * Field visitor.
      */
-    static class CollectingFieldVisitor extends FieldVisitor {
-        /** */
+    private static class CollectingFieldVisitor extends FieldVisitor {
+        /** Collector. */
         private final Collector c;
 
-        /** */
+        /** Annotation visitor. */
         private final AnnotationVisitor av;
 
         /**
@@ -545,7 +703,6 @@ public class HadoopClassLoader extends URLClassLoader {
             super(Opcodes.ASM4);
 
             this.c = c;
-
             this.av = av;
         }
 
@@ -556,38 +713,43 @@ public class HadoopClassLoader extends URLClassLoader {
 
             c.onType(desc);
 
-            return av;
+            return c.isDone() ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public void visitAttribute(Attribute attr) {
-            // noop
+            // No-op.
         }
 
         /** {@inheritDoc} */
         @Override public void visitEnd() {
-            // noop
+            // No-op.
         }
     }
 
     /**
      * Class visitor.
      */
-    static class CollectingClassVisitor extends ClassVisitor {
-        /** */
+    private static class CollectingClassVisitor extends ClassVisitor {
+        /** Collector. */
         private final Collector c;
 
-        /** */
+        /** Annotation visitor. */
         private final AnnotationVisitor av;
 
-        /** */
+        /** Method visitor. */
         private final MethodVisitor mv;
 
-        /** */
+        /** Field visitor. */
         private final FieldVisitor fv;
 
         /**
          * Constructor.
+         *
+         * @param c Collector.
+         * @param av Annotation visitor.
+         * @param mv Method visitor.
+         * @param fv Field visitor.
          */
         CollectingClassVisitor(Collector c, AnnotationVisitor av, MethodVisitor mv, FieldVisitor fv) {
             super(Opcodes.ASM4);
@@ -599,8 +761,7 @@ public class HadoopClassLoader extends URLClassLoader {
         }
 
         /** {@inheritDoc} */
-        @Override public void visit(int i, int i2, String name, String signature, String superName,
-            String[] ifaces) {
+        @Override public void visit(int i, int i2, String name, String signature, String superName, String[] ifaces) {
             if (c.isDone())
                 return;
 
@@ -626,7 +787,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
             c.onType(desc);
 
-            return av;
+            return c.isDone() ? null : av;
         }
 
         /** {@inheritDoc} */
@@ -644,7 +805,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
             c.onType(desc);
 
-            return fv;
+            return c.isDone() ? null : fv;
         }
 
         /** {@inheritDoc} */
@@ -661,27 +822,27 @@ public class HadoopClassLoader extends URLClassLoader {
                     c.onInternalTypeName(e);
             }
 
-            return mv;
+            return c.isDone() ? null : mv;
         }
     }
 
     /**
      * Method visitor.
      */
-    static class CollectingMethodVisitor extends MethodVisitor {
-        /** */
+    private static class CollectingMethodVisitor extends MethodVisitor {
+        /** Collector. */
         private final Collector c;
 
-        /** */
+        /** Annotation visitor. */
         private final AnnotationVisitor av;
 
         /**
          * Constructor.
          *
-         * @param c The collector.
-         * @param av The annotation visitor.
+         * @param c Collector.
+         * @param av Annotation visitor.
          */
-        CollectingMethodVisitor(Collector c, AnnotationVisitor av) {
+        private CollectingMethodVisitor(Collector c, AnnotationVisitor av) {
             super(Opcodes.ASM4);
 
             this.c = c;
@@ -695,7 +856,7 @@ public class HadoopClassLoader extends URLClassLoader {
 
             c.onType(desc);
 
-            return av;
+            return c.isDone() ? null : av;
         }
 
         /** {@inheritDoc} */
@@ -705,12 +866,12 @@ public class HadoopClassLoader extends URLClassLoader {
 
             c.onType(desc);
 
-            return av;
+            return c.isDone() ? null : av;
         }
 
         /** {@inheritDoc} */
         @Override public AnnotationVisitor visitAnnotationDefault() {
-            return av;
+            return c.isDone() ? null : av;
         }
 
         /** {@inheritDoc} */
@@ -728,54 +889,12 @@ public class HadoopClassLoader extends URLClassLoader {
 
         /** {@inheritDoc} */
         @Override public void visitInvokeDynamicInsn(String name, String desc, Handle bsm, Object... bsmArgs) {
-            if (c.isDone())
-                return;
-
-            c.onMethodsDesc(desc);
-
-            if (c.isDone())
-                return;
-
-            if (bsmArgs != null) {
-                for (Object b: bsmArgs) {
-                    if (b instanceof Type)
-                        c.onType((Type) b);
-
-                    if (c.isDone())
-                        return;
-                }
-            }
+            // No-op.
         }
 
         /** {@inheritDoc} */
-        @Override public void visitFrame(int type, int nLoc, Object[] locTypes, int nStack,
-        Object[] stackTypes) {
-            // TODO: not sure we need that to detect dependencies. This seems to be superfluous.
-            // Optimizations:
-            // 1) Do not consider frames that are exactly equal to the previous;
-            // 2) Return if local and stack arrays are empty ('nLock' and 'nStack' correspond to the number
-            // of non-null array elements).
-            if (type == Opcodes.F_SAME || (nLoc == 0 && nStack == 0))
-                return;
-
-            if (c.isDone())
-                return;
-
-            for (Object o : locTypes) {
-                if (o instanceof String)
-                    c.onInternalTypeName((String) o);
-
-                if (c.isDone())
-                    return;
-            }
-
-            for (Object o : stackTypes) {
-                if (o instanceof String)
-                    c.onInternalTypeName((String) o);
-
-                if (c.isDone())
-                    return;
-            }
+        @Override public void visitFrame(int type, int nLoc, Object[] locTypes, int nStack, Object[] stackTypes) {
+            // No-op.
         }
 
         /** {@inheritDoc} */
@@ -824,161 +943,4 @@ public class HadoopClassLoader extends URLClassLoader {
             c.onInternalTypeName(type);
         }
     }
-
-    /**
-     * @param name Class name.
-     * @return {@code true} If this is a valid class name.
-     */
-    private static boolean validateClassName(String name) {
-        int len = name.length();
-
-        if (len <= 1)
-            return false;
-
-        if (!Character.isJavaIdentifierStart(name.charAt(0)))
-            return false;
-
-        boolean hasDot = false;
-
-        for (int i = 1; i < len; i++) {
-            char c = name.charAt(i);
-
-            if (c == '.')
-                hasDot = true;
-            else if (!Character.isJavaIdentifierPart(c))
-                return false;
-        }
-
-        return hasDot;
-    }
-
-    /**
-     * @param name Variable name.
-     * @param dflt Default.
-     * @return Value.
-     */
-    private static String getEnv(String name, String dflt) {
-        String res = System.getProperty(name);
-
-        if (F.isEmpty(res))
-            res = System.getenv(name);
-
-        return F.isEmpty(res) ? dflt : res;
-    }
-
-    /**
-     * @param res Result.
-     * @param dir Directory.
-     * @param startsWith Starts with prefix.
-     * @throws MalformedURLException If failed.
-     */
-    private static void addUrls(Collection<URL> res, File dir, final String startsWith) throws Exception {
-        File[] files = dir.listFiles(new FilenameFilter() {
-            @Override public boolean accept(File dir, String name) {
-                return startsWith == null || name.startsWith(startsWith);
-            }
-        });
-
-        if (files == null)
-            throw new IOException("Path is not a directory: " + dir);
-
-        for (File file : files)
-            res.add(file.toURI().toURL());
-    }
-
-    /**
-     * @param urls URLs.
-     * @return URLs.
-     */
-    private static URL[] addHadoopUrls(URL[] urls) {
-        Collection<URL> hadoopJars;
-
-        try {
-            hadoopJars = hadoopUrls();
-        }
-        catch (IgniteCheckedException e) {
-            throw new RuntimeException(e);
-        }
-
-        ArrayList<URL> list = new ArrayList<>(hadoopJars.size() + appJars.size() + (urls == null ? 0 : urls.length));
-
-        list.addAll(appJars);
-        list.addAll(hadoopJars);
-
-        if (!F.isEmpty(urls))
-            list.addAll(F.asList(urls));
-
-        return list.toArray(new URL[list.size()]);
-    }
-
-    /**
-     * @return HADOOP_HOME Variable.
-     */
-    @Nullable public static String hadoopHome() {
-        return getEnv("HADOOP_PREFIX", getEnv("HADOOP_HOME", null));
-    }
-
-    /**
-     * @return Collection of jar URLs.
-     * @throws IgniteCheckedException If failed.
-     */
-    public static Collection<URL> hadoopUrls() throws IgniteCheckedException {
-        Collection<URL> hadoopUrls = hadoopJars;
-
-        if (hadoopUrls != null)
-            return hadoopUrls;
-
-        synchronized (HadoopClassLoader.class) {
-            hadoopUrls = hadoopJars;
-
-            if (hadoopUrls != null)
-                return hadoopUrls;
-
-            hadoopUrls = new ArrayList<>();
-
-            String hadoopPrefix = hadoopHome();
-
-            if (F.isEmpty(hadoopPrefix))
-                throw new IgniteCheckedException("Failed resolve Hadoop installation location. Either HADOOP_PREFIX or " +
-                    "HADOOP_HOME environment variables must be set.");
-
-            String commonHome = getEnv("HADOOP_COMMON_HOME", hadoopPrefix + "/share/hadoop/common");
-            String hdfsHome = getEnv("HADOOP_HDFS_HOME", hadoopPrefix + "/share/hadoop/hdfs");
-            String mapredHome = getEnv("HADOOP_MAPRED_HOME", hadoopPrefix + "/share/hadoop/mapreduce");
-
-            try {
-                addUrls(hadoopUrls, new File(commonHome + "/lib"), null);
-                addUrls(hadoopUrls, new File(hdfsHome + "/lib"), null);
-                addUrls(hadoopUrls, new File(mapredHome + "/lib"), null);
-
-                addUrls(hadoopUrls, new File(hdfsHome), "hadoop-hdfs-");
-
-                addUrls(hadoopUrls, new File(commonHome), "hadoop-common-");
-                addUrls(hadoopUrls, new File(commonHome), "hadoop-auth-");
-                addUrls(hadoopUrls, new File(commonHome + "/lib"), "hadoop-auth-");
-
-                addUrls(hadoopUrls, new File(mapredHome), "hadoop-mapreduce-client-common");
-                addUrls(hadoopUrls, new File(mapredHome), "hadoop-mapreduce-client-core");
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException(e);
-            }
-
-            hadoopJars = hadoopUrls;
-
-            return hadoopUrls;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(HadoopClassLoader.class, this);
-    }
-
-    /**
-     * Getter for name field.
-     */
-    public String name() {
-        return name;
-    }
 }
\ No newline at end of file


[3/9] ignite git commit: Merge branch 'ignite-2308' of https://github.com/iveselovskiy/ignite into ignite-2308

Posted by vo...@apache.org.
Merge branch 'ignite-2308' of https://github.com/iveselovskiy/ignite into ignite-2308


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

Branch: refs/heads/ignite-2308
Commit: 67c1e0f740dcb389977ae8c0065d8b1d738648a4
Parents: ddbe2d5 683fea8
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Jan 4 09:22:03 2016 +0400
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jan 4 09:22:03 2016 +0400

----------------------------------------------------------------------
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |   1 +
 .../processors/hadoop/HadoopClassLoader.java    | 591 +++++++++++++++----
 .../hadoop/HadoopClassLoaderTest.java           | 109 +++-
 .../hadoop/cls/CircularDependencyHadoop.java    |  14 +
 .../hadoop/cls/CircularDependencyNoHadoop.java  |   9 +
 .../hadoop/cls/DependencyNoHadoop.java          |   9 +
 .../processors/hadoop/cls/HadoopCasting.java    |  23 +
 .../hadoop/cls/HadoopClassAnnotation.java       |  10 +
 .../hadoop/cls/HadoopConstructorInvocation.java |  13 +
 .../HadoopDeclaredCheckedExceptionInMethod.java |  13 +
 .../HadoopDeclaredRuntimeExceptionInMethod.java |  13 +
 .../processors/hadoop/cls/HadoopExtends.java    |  10 +
 .../processors/hadoop/cls/HadoopField.java      |  13 +
 .../processors/hadoop/cls/HadoopImplements.java |  19 +
 .../hadoop/cls/HadoopInitializer.java           |  14 +
 .../processors/hadoop/cls/HadoopInnerClass.java |  15 +
 .../hadoop/cls/HadoopLocalVariableType.java     |  20 +
 .../hadoop/cls/HadoopMethodAnnotation.java      |  12 +
 .../hadoop/cls/HadoopMethodInvocation.java      |  13 +
 .../hadoop/cls/HadoopMethodParameter.java       |  13 +
 .../hadoop/cls/HadoopMethodReturnType.java      |  13 +
 .../processors/hadoop/cls/HadoopOuterClass.java |  20 +
 .../hadoop/cls/HadoopParameterAnnotation.java   |  11 +
 .../hadoop/cls/HadoopStaticField.java           |  11 +
 .../hadoop/cls/HadoopStaticInitializer.java     |  16 +
 .../processors/hadoop/cls/NoHadoop.java         |   7 +
 .../testsuites/IgniteHadoopTestSuite.java       |   3 +
 27 files changed, 875 insertions(+), 140 deletions(-)
----------------------------------------------------------------------