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/11 16:27:33 UTC

[06/11] ignite git commit: IGNITE-2308: Fixed HadoopClassLoader dependency resolution. This closes #391.

IGNITE-2308: Fixed HadoopClassLoader dependency resolution. This closes #391.


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

Branch: refs/heads/ignite-2314
Commit: 012ca7308bf125d2c90d68ef7a8bc75aeb84bf53
Parents: ddbe2d5
Author: iveselovskiy <iv...@gridgain.com>
Authored: Mon Jan 4 10:47:28 2016 +0400
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Jan 4 10:47:28 2016 +0400

----------------------------------------------------------------------
 .../processors/hadoop/HadoopClassLoader.java    | 636 ++++++++++++++-----
 .../hadoop/HadoopClassLoaderTest.java           | 101 ++-
 .../hadoop/deps/CircularWIthHadoop.java         |  32 +
 .../hadoop/deps/CircularWithoutHadoop.java      |  27 +
 .../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 +
 .../testsuites/IgniteHadoopTestSuite.java       |   3 +
 26 files changed, 1279 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/012ca730/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..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
@@ -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;
@@ -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 ignitePkgPrefix = "org.apache.ignite";
+
+        int len = ignitePkgPrefix.length();
 
-        return cls.startsWith(ignitePackagePrefix) && (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);
     }
 
     /**
@@ -159,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);
                 }
@@ -266,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(final String clsName, final 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;
 
@@ -291,157 +318,14 @@ public class HadoopClassLoader extends URLClassLoader {
             throw new RuntimeException("Failed to read class: " + clsName, e);
         }
 
-        visited.add(clsName);
-
-        final AtomicBoolean hasDeps = new AtomicBoolean();
-
-        rdr.accept(new ClassVisitor(Opcodes.ASM4) {
-            AnnotationVisitor av = new AnnotationVisitor(Opcodes.ASM4) {
-                // TODO
-            };
-
-            FieldVisitor fv = new FieldVisitor(Opcodes.ASM4) {
-                @Override public AnnotationVisitor visitAnnotation(String desc, boolean b) {
-                    onType(desc);
-
-                    return av;
-                }
-            };
-
-            MethodVisitor mv = new MethodVisitor(Opcodes.ASM4) {
-                @Override public AnnotationVisitor visitAnnotation(String desc, boolean b) {
-                    onType(desc);
-
-                    return av;
-                }
-
-                @Override public AnnotationVisitor visitParameterAnnotation(int i, String desc, boolean b) {
-                    onType(desc);
-
-                    return av;
-                }
-
-                @Override public AnnotationVisitor visitAnnotationDefault() {
-                    return av;
-                }
-
-                @Override public void visitFieldInsn(int i, String owner, String name, String desc) {
-                    onType(owner);
-                    onType(desc);
-                }
-
-                @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);
-                    }
-
-                    for (Object o : stackTypes) {
-                        if (o instanceof String)
-                            onType((String)o);
-                    }
-                }
-
-                @Override public void visitLocalVariable(String name, String desc, String signature, Label lb,
-                    Label lb2, int i) {
-                    onType(desc);
-                }
-
-                @Override public void visitMethodInsn(int i, String owner, String name, String desc) {
-                    onType(owner);
-                }
-
-                @Override public void visitMultiANewArrayInsn(String desc, int dim) {
-                    onType(desc);
-                }
-
-                @Override public void visitTryCatchBlock(Label lb, Label lb2, Label lb3, String e) {
-                    onType(e);
-                }
-            };
-
-            void onClass(String depCls) {
-                assert validateClassName(depCls) : depCls;
-
-                if (depCls.startsWith("java.")) // Filter out platform classes.
-                    return;
-
-                if (visited.contains(depCls))
-                    return;
-
-                Boolean res = cache.get(depCls);
-
-                if (res == Boolean.TRUE || (res == null && hasExternalDependencies(depCls, visited)))
-                    hasDeps.set(true);
-            }
-
-            void onType(String type) {
-                if (type == null)
-                    return;
-
-                int off = 0;
-
-                while (type.charAt(off) == '[')
-                    off++; // Handle arrays.
-
-                if (off != 0)
-                    type = type.substring(off);
-
-                if (type.length() == 1)
-                    return; // Get rid of primitives.
-
-                if (type.charAt(type.length() - 1) == ';') {
-                    assert type.charAt(0) == 'L' : type;
-
-                    type = type.substring(1, type.length() - 1);
-                }
-
-                type = type.replace('/', '.');
-
-                onClass(type);
-            }
-
-            @Override public void visit(int i, int i2, String name, String signature, String superName,
-                String[] ifaces) {
-                onType(superName);
-
-                if (ifaces != null) {
-                    for (String iface : ifaces)
-                        onType(iface);
-                }
-            }
-
-            @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
-                onType(desc);
-
-                return av;
-            }
-
-            @Override public void visitInnerClass(String name, String outerName, String innerName, int i) {
-                onType(name);
-            }
-
-            @Override public FieldVisitor visitField(int i, String name, String desc, String signature, Object val) {
-                onType(desc);
+        ctx.visited.add(clsName);
 
-                return fv;
-            }
+        rdr.accept(ctx.clsVisitor, 0);
 
-            @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;
-            }
-        }, 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 have them.
+        // Here we are known to not have any dependencies but possibly we have a parent which has them.
         int idx = clsName.lastIndexOf('$');
 
         if (idx == -1) // No parent class.
@@ -449,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;
     }
@@ -616,4 +500,446 @@ public class HadoopClassLoader extends URLClassLoader {
     public String name() {
         return name;
     }
+
+    /**
+     * Context for dependencies collection.
+     */
+    private class CollectingContext {
+        /** Visited classes. */
+        private final Set<String> visited = new HashSet<>();
+
+        /** Whether dependency found. */
+        private boolean found;
+
+        /** Annotation visitor. */
+        private AnnotationVisitor annVisitor;
+
+        /** Method visitor. */
+        private MethodVisitor mthdVisitor;
+
+        /** Field visitor. */
+        private FieldVisitor fldVisitor;
+        
+        /** Class visitor. */
+        private ClassVisitor clsVisitor;
+
+        /**
+         * Processes a method descriptor
+         * @param methDesc The method desc String.
+         */
+        void onMethodsDesc(final String methDesc) {
+            // Process method return type:
+            onType(Type.getReturnType(methDesc));
+
+            if (found)
+                return;
+
+            // Process method argument types:
+            for (Type t: Type.getArgumentTypes(methDesc)) {
+                onType(t);
+
+                if (found)
+                    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;
+
+            if (depCls.startsWith("java.") || depCls.startsWith("javax.")) // Filter out platform classes.
+                return;
+
+            if (visited.contains(depCls))
+                return;
+
+            Boolean res = cache.get(depCls);
+
+            if (res == Boolean.TRUE || (res == null && hasExternalDependencies(depCls, this)))
+                found = true;
+        }
+
+        /**
+         * Analyses dependencies of given type.
+         *
+         * @param t The type to process.
+         */
+        void onType(Type t) {
+            if (t == null)
+                return;
+
+            int sort = t.getSort();
+
+            switch (sort) {
+                case Type.ARRAY:
+                    onType(t.getElementType());
+
+                    break;
+
+                case Type.OBJECT:
+                    onClass(t.getClassName());
+
+                    break;
+            }
+        }
+
+        /**
+         * Analyses dependencies of given object type.
+         *
+         * @param objType The object type to process.
+         */
+        void onInternalTypeName(String objType) {
+            if (objType == null)
+                return;
+
+            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.
+            }
+        }
+
+        /**
+         * 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.
+
+                Type t = Type.getType(desc);
+
+                onType(t);
+            }
+        }
+    }    
+
+    /**
+     * Annotation visitor.
+     */
+    private static class CollectingAnnotationVisitor extends AnnotationVisitor {
+        /** */
+        final CollectingContext ctx;
+
+        /**
+         * Annotation visitor.
+         * 
+         * @param ctx The collector.
+         */
+        CollectingAnnotationVisitor(CollectingContext ctx) {
+            super(Opcodes.ASM4);
+
+            this.ctx = ctx;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotation(String name, String desc) {
+            if (ctx.found)
+                return null;
+
+            ctx.onType(desc);
+
+            return this;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitEnum(String name, String desc, String val) {
+            if (ctx.found)
+                return;
+
+            ctx.onType(desc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitArray(String name) {
+            return ctx.found ? null : this;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visit(String name, Object val) {
+            if (ctx.found)
+                return;
+
+            if (val instanceof Type)
+                ctx.onType((Type)val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitEnd() {
+            // No-op.
+        }
+    }
+
+    /**
+     * Field visitor.
+     */
+    private static class CollectingFieldVisitor extends FieldVisitor {
+        /** Collector. */
+        private final CollectingContext ctx;
+
+        /** Annotation visitor. */
+        private final AnnotationVisitor av;
+
+        /**
+         * Constructor.
+         */
+        CollectingFieldVisitor(CollectingContext ctx, AnnotationVisitor av) {
+            super(Opcodes.ASM4);
+
+            this.ctx = ctx;
+            this.av = av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
+            if (ctx.found)
+                return null;
+
+            ctx.onType(desc);
+
+            return ctx.found ? null : av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitAttribute(Attribute attr) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitEnd() {
+            // No-op.
+        }
+    }
+
+    /**
+     * Class visitor.
+     */
+    private static class CollectingClassVisitor extends ClassVisitor {
+        /** Collector. */
+        private final CollectingContext ctx;
+
+        /** Annotation visitor. */
+        private final AnnotationVisitor av;
+
+        /** Method visitor. */
+        private final MethodVisitor mv;
+
+        /** Field visitor. */
+        private final FieldVisitor fv;
+
+        /**
+         * Constructor.
+         *
+         * @param ctx Collector.
+         * @param av Annotation visitor.
+         * @param mv Method visitor.
+         * @param fv Field visitor.
+         */
+        CollectingClassVisitor(CollectingContext ctx, AnnotationVisitor av, MethodVisitor mv, FieldVisitor fv) {
+            super(Opcodes.ASM4);
+
+            this.ctx = ctx;
+            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 (ctx.found)
+                return;
+
+            ctx.onInternalTypeName(superName);
+
+            if (ctx.found)
+                return;
+
+            if (ifaces != null) {
+                for (String iface : ifaces) {
+                    ctx.onInternalTypeName(iface);
+
+                    if (ctx.found)
+                        return;
+                }
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
+            if (ctx.found)
+                return null;
+
+            ctx.onType(desc);
+
+            return ctx.found ? null : av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitInnerClass(String name, String outerName, String innerName, int i) {
+            if (ctx.found)
+                return;
+
+            ctx.onInternalTypeName(name);
+        }
+
+        /** {@inheritDoc} */
+        @Override public FieldVisitor visitField(int i, String name, String desc, String signature, Object val) {
+            if (ctx.found)
+                return null;
+
+            ctx.onType(desc);
+
+            return ctx.found ? null : fv;
+        }
+
+        /** {@inheritDoc} */
+        @Override public MethodVisitor visitMethod(int i, String name, String desc, String signature,
+            String[] exceptions) {
+            if (ctx.found)
+                return null;
+
+            ctx.onMethodsDesc(desc);
+
+            // Process declared method exceptions:
+            if (exceptions != null) {
+                for (String e : exceptions)
+                    ctx.onInternalTypeName(e);
+            }
+
+            return ctx.found ? null : mv;
+        }
+    }
+
+    /**
+     * Method visitor.
+     */
+    private static class CollectingMethodVisitor extends MethodVisitor {
+        /** Collector. */
+        private final CollectingContext ctx;
+
+        /** Annotation visitor. */
+        private final AnnotationVisitor av;
+
+        /**
+         * Constructor.
+         *
+         * @param ctx Collector.
+         * @param av Annotation visitor.
+         */
+        private CollectingMethodVisitor(CollectingContext ctx, AnnotationVisitor av) {
+            super(Opcodes.ASM4);
+
+            this.ctx = ctx;
+            this.av = av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
+            if (ctx.found)
+                return null;
+
+            ctx.onType(desc);
+
+            return ctx.found ? null : av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitParameterAnnotation(int i, String desc, boolean b) {
+            if (ctx.found)
+                return null;
+
+            ctx.onType(desc);
+
+            return ctx.found ? null : av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public AnnotationVisitor visitAnnotationDefault() {
+            return ctx.found ? null : av;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitFieldInsn(int opcode, String owner, String name, String desc) {
+            if (ctx.found)
+                return;
+
+            ctx.onInternalTypeName(owner);
+
+            if (ctx.found)
+                return;
+
+            ctx.onType(desc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitInvokeDynamicInsn(String name, String desc, Handle bsm, Object... bsmArgs) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitFrame(int type, int nLoc, Object[] locTypes, int nStack, Object[] stackTypes) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitLocalVariable(String name, String desc, String signature, Label lb,
+            Label lb2, int i) {
+            if (ctx.found)
+                return;
+
+            ctx.onType(desc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitMethodInsn(int i, String owner, String name, String desc) {
+            if (ctx.found)
+                return;
+
+            ctx.onInternalTypeName(owner);
+
+            if (ctx.found)
+                return;
+
+            ctx.onMethodsDesc(desc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitMultiANewArrayInsn(String desc, int dim) {
+            if (ctx.found)
+                return;
+
+            ctx.onType(desc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitTryCatchBlock(Label start, Label end, Label hndl, String typeStr) {
+            if (ctx.found)
+                return;
+
+            ctx.onInternalTypeName(typeStr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void visitTypeInsn(int opcode, String type) {
+            if (ctx.found)
+                return;
+
+            ctx.onInternalTypeName(type);
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/012ca730/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..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
@@ -17,53 +17,94 @@
 
 package org.apache.ignite.internal.processors.hadoop;
 
+import javax.security.auth.AuthPermission;
 import junit.framework.TestCase;
-import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.conf.Configuration;
+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 {
     /** */
-    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(CircularWIthHadoop.class, ldr.loadClass(CircularWIthHadoop.class.getName()));
+        assertNotSame(CircularWithoutHadoop.class, ldr.loadClass(CircularWithoutHadoop.class.getName()));
 
-//    public void testDependencySearch() {
-//        assertTrue(ldr.hasExternalDependencies(Test1.class.getName(), new HashSet<String>()));
-//        assertTrue(ldr.hasExternalDependencies(Test2.class.getName(), new HashSet<String>()));
-//    }
+        assertSame(Without.class, ldr.loadClass(Without.class.getName()));
+    }
 
     /**
-     *
+     * Test dependency search.
      */
-    private static class Test1 {
-        /** */
-        Test2 t2;
+    public void testDependencySearch() {
+        // Positive cases:
+        final Class[] positiveClasses = {
+            Configuration.class,
+            HadoopUtils.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,
+        };
 
-        /** */
-        Job[][] jobs = new Job[4][4];
-    }
+        for (Class c: positiveClasses)
+            assertTrue(c.getName(), ldr.hasExternalDependencies(c.getName()));
 
-    /**
-     *
-     */
-    private static abstract class Test2 {
-        /** */
-        abstract Test1 t1();
-    }
+        // Negative cases:
+        final Class[] negativeClasses = {
+            Object.class,
+            AuthPermission.class,
+            Without.class,
+        };
 
-    /**
-     *
-     */
-    private static class Test3 {
-        // No-op.
+        for (Class c: negativeClasses)
+            assertFalse(c.getName(), ldr.hasExternalDependencies(c.getName()));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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/012ca730/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() {}
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/012ca730/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/012ca730/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/012ca730/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/012ca730/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.
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/012ca730/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())));