You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ja...@apache.org on 2015/12/29 09:47:33 UTC

sqoop git commit: SQOOP-2638: Sqoop2: Add integration tests for connector classpath isolation

Repository: sqoop
Updated Branches:
  refs/heads/sqoop2 dedaee6e6 -> a03adec29


SQOOP-2638: Sqoop2: Add integration tests for connector classpath isolation

(Dian Fu via Jarek Jarcec Cecho)


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

Branch: refs/heads/sqoop2
Commit: a03adec29039ba082250006972b7169ff1ac1a8e
Parents: dedaee6
Author: Jarek Jarcec Cecho <ja...@apache.org>
Authored: Tue Dec 29 09:46:43 2015 +0100
Committer: Jarek Jarcec Cecho <ja...@apache.org>
Committed: Tue Dec 29 09:46:43 2015 +0100

----------------------------------------------------------------------
 .../testcases/ConnectorClasspathTestCase.java   | 235 +++++++++++++++++++
 .../connectorloading/ClasspathTest.java         | 209 +++--------------
 .../ConnectorClasspathIsolationTest.java        | 181 ++++++++++++++
 .../from/TestClasspathIsolation.java            |  23 ++
 .../from/TestExtractor.java                     |  41 ++++
 .../from/TestFromConnector.java                 |  90 +++++++
 .../from/TestFromDestroyer.java                 |  30 +++
 .../from/TestFromInitializer.java               |  30 +++
 .../from/TestFromJobConfiguration.java          |  28 +++
 .../from/TestFromLinkConfiguration.java         |  28 +++
 .../from/TestPartition.java                     |  36 +++
 .../from/TestPartitioner.java                   |  41 ++++
 .../from/sqoopconnector.properties              |  18 ++
 .../to/TestClasspathIsolation.java              |  23 ++
 .../to/TestLoader.java                          |  48 ++++
 .../to/TestToConnector.java                     |  88 +++++++
 .../to/TestToDestroyer.java                     |  30 +++
 .../to/TestToInitializer.java                   |  30 +++
 .../to/TestToJobConfiguration.java              |  28 +++
 .../to/TestToLinkConfiguration.java             |  28 +++
 .../to/sqoopconnector.properties                |  18 ++
 21 files changed, 1107 insertions(+), 176 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/main/java/org/apache/sqoop/test/testcases/ConnectorClasspathTestCase.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/testcases/ConnectorClasspathTestCase.java b/test/src/main/java/org/apache/sqoop/test/testcases/ConnectorClasspathTestCase.java
new file mode 100644
index 0000000..6db1db8
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/testcases/ConnectorClasspathTestCase.java
@@ -0,0 +1,235 @@
+/**
+ * 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.sqoop.test.testcases;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.jar.Attributes;
+import java.util.jar.JarEntry;
+import java.util.jar.JarOutputStream;
+import java.util.jar.Manifest;
+
+import javax.tools.JavaCompiler;
+import javax.tools.JavaFileObject;
+import javax.tools.StandardJavaFileManager;
+import javax.tools.StandardLocation;
+import javax.tools.ToolProvider;
+
+import org.apache.commons.collections.ListUtils;
+
+public class ConnectorClasspathTestCase extends ConnectorTestCase {
+
+  static class JarContents {
+    private List<File> sourceFiles;
+    private List<File> properitesFiles;
+    private List<String> dependencyJarFiles;
+
+    public JarContents(List<File> sourceFiles, List<File> properitesFiles, List<String> dependencyJarFiles) {
+      this.sourceFiles = sourceFiles;
+      this.properitesFiles = properitesFiles;
+      this.dependencyJarFiles = dependencyJarFiles;
+    }
+
+    public List<File> getSourceFiles() {
+      return sourceFiles;
+    }
+
+    public List<File> getProperitesFiles() {
+      return properitesFiles;
+    }
+
+    public List<String> getDependencyJarFiles() {
+      return dependencyJarFiles;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  protected Map<String, String> compileTestConnectorAndDependency(String[] connectorSourceFiles,
+      String[] connectorDependencySourceFiles, String[] connectorPropertyFiles, String connectorJarName,
+      String connectorDependencyJarName, boolean dependencyBuiltInsideConnectorJar) throws Exception {
+    JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
+    if (compiler == null) {
+      throw new IllegalStateException(
+        "Cannot find the system Java compiler. "
+          + "Check that your class path includes tools.jar");
+    }
+
+    Path outputDir = Files.createTempDirectory(null);
+
+    Map<String, JarContents> sourceFileToJarMap = new LinkedHashMap<>();
+
+    ClassLoader classLoader = getClass().getClassLoader();
+    List<File> sourceFiles = new ArrayList<>();
+
+    for (String connectorDependencySourceFile : connectorDependencySourceFiles) {
+      File file = new File(classLoader.getResource(connectorDependencySourceFile).getFile());
+      sourceFiles.add(file);
+    }
+    sourceFileToJarMap.put(connectorDependencyJarName, new JarContents(sourceFiles, ListUtils.EMPTY_LIST, ListUtils.EMPTY_LIST));
+
+    sourceFiles = new ArrayList<>();
+    for (String connectorSourceFile : connectorSourceFiles) {
+      File file = new File(classLoader.getResource(connectorSourceFile).getFile());
+      sourceFiles.add(file);
+    }
+
+    List<File> propertiesFiles = new ArrayList<>();
+    for (String connectorPropertyFile : connectorPropertyFiles) {
+      File file = new File(classLoader.getResource(connectorPropertyFile).getFile());
+      propertiesFiles.add(file);
+    }
+
+    List<String> dependencyFiles = new ArrayList<>();
+    if (dependencyBuiltInsideConnectorJar) {
+      dependencyFiles.add(connectorDependencyJarName);
+    }
+    sourceFileToJarMap.put(connectorJarName, new JarContents(sourceFiles, propertiesFiles, dependencyFiles));
+
+    buildJar(outputDir.toString(), sourceFileToJarMap);
+
+    Map<String, String> jarMap = new HashMap<>();
+    jarMap.put(connectorJarName, outputDir.toString() + File.separator + connectorJarName);
+    jarMap.put(connectorDependencyJarName, outputDir.toString() + File.separator + connectorDependencyJarName);
+    return jarMap;
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings("RV_RETURN_VALUE_IGNORED_BAD_PRACTICE")
+  private void buildJar(String outputDir, Map<String, JarContents> sourceFileToJarMap) throws IOException {
+    JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
+    StandardJavaFileManager fileManager = compiler.getStandardFileManager
+      (null, null, null);
+
+    List<File> sourceFiles = new ArrayList<>();
+    for(JarContents jarContents : sourceFileToJarMap.values()) {
+      sourceFiles.addAll(jarContents.sourceFiles);
+    }
+
+    fileManager.setLocation(StandardLocation.CLASS_OUTPUT,
+      Arrays.asList(new File(outputDir)));
+
+    Iterable<? extends JavaFileObject> compilationUnits1 =
+      fileManager.getJavaFileObjectsFromFiles(sourceFiles);
+
+    boolean compiled = compiler.getTask(null, fileManager, null, null, null, compilationUnits1).call();
+    if (!compiled) {
+      throw new RuntimeException("failed to compile");
+    }
+
+    for(Map.Entry<String, JarContents> jarNameAndContents : sourceFileToJarMap.entrySet()) {
+      Manifest manifest = new Manifest();
+      manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION, "1.0");
+      manifest.getMainAttributes().put(Attributes.Name.CLASS_PATH, ".");
+
+      JarOutputStream target = new JarOutputStream(new FileOutputStream(outputDir + File.separator + jarNameAndContents.getKey()), manifest);
+      List<String> classesForJar = new ArrayList<>();
+      for(File sourceFile : jarNameAndContents.getValue().getSourceFiles()) {
+        //split the file on dot to get the filename from FILENAME.java
+        String fileName = sourceFile.getName().split("\\.")[0];
+        classesForJar.add(fileName);
+      }
+
+      File dir = new File(outputDir);
+      File[] directoryListing = dir.listFiles();
+      if (directoryListing != null) {
+        for (File compiledClass : directoryListing) {
+          String classFileName = compiledClass.getName().split("\\$")[0].split("\\.")[0];
+          if (classesForJar.contains(classFileName)){
+            addFileToJar(compiledClass, target);
+          }
+        }
+      }
+
+      for (File propertiesFile : jarNameAndContents.getValue().getProperitesFiles()) {
+        addFileToJar(propertiesFile, target);
+      }
+
+      for (String dependencyJarFileName : jarNameAndContents.getValue().getDependencyJarFiles()) {
+        File dependencyJarFile = new File(dir, dependencyJarFileName);
+        addFileToJar(dependencyJarFile, target);
+      }
+
+      target.close();
+    }
+    //delete non jar files
+    File dir = new File(outputDir);
+    File[] directoryListing = dir.listFiles();
+    if (directoryListing != null) {
+      for (File file : directoryListing) {
+        String extension = file.getName().split("\\.")[1];
+        if (!extension.equals("jar")) {
+          file.delete();
+        }
+      }
+    }
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings("OS_OPEN_STREAM_EXCEPTION_PATH")
+  private void addFileToJar(File source, JarOutputStream target) throws IOException {
+    String entryName;
+    if (source.getName().endsWith(".jar")) {
+      // put dependency jars into directory "lib"
+      entryName = "lib/" + source.getName();
+    } else {
+      entryName = source.getName();
+    }
+    JarEntry entry = new JarEntry(entryName);
+    entry.setTime(source.lastModified());
+    target.putNextEntry(entry);
+    BufferedInputStream in = null;
+    try {
+      in = new BufferedInputStream(new FileInputStream(source));
+
+      long bufferSize = source.length();
+      if (bufferSize < Integer.MIN_VALUE || bufferSize > Integer.MAX_VALUE) {
+        throw new RuntimeException("file to large to be added to jar");
+      }
+
+      byte[] buffer = new byte[(int) bufferSize];
+      while (true) {
+        int count = in.read(buffer);
+        if (count == -1)
+          break;
+        target.write(buffer, 0, count);
+      }
+    } finally {
+      target.closeEntry();
+      if (in != null) {
+        in.close();
+      }
+    }
+  }
+
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings("RV_RETURN_VALUE_IGNORED_BAD_PRACTICE")
+  protected void deleteJars(Map<String, String> jarMap) {
+    for (String jarPath : jarMap.values()) {
+      (new File(jarPath)).delete();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/java/org/apache/sqoop/integration/connectorloading/ClasspathTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/connectorloading/ClasspathTest.java b/test/src/test/java/org/apache/sqoop/integration/connectorloading/ClasspathTest.java
index e0879ed..4bb6aa1 100644
--- a/test/src/test/java/org/apache/sqoop/integration/connectorloading/ClasspathTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/connectorloading/ClasspathTest.java
@@ -18,7 +18,6 @@
 
 package org.apache.sqoop.integration.connectorloading;
 
-import org.apache.commons.collections.ListUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.sqoop.client.SqoopClient;
 import org.apache.sqoop.core.ConfigurationConstants;
@@ -26,39 +25,40 @@ import org.apache.sqoop.model.MDriverConfig;
 import org.apache.sqoop.model.MJob;
 import org.apache.sqoop.model.MLink;
 import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
-import org.apache.sqoop.test.testcases.ConnectorTestCase;
+import org.apache.sqoop.test.testcases.ConnectorClasspathTestCase;
 import org.apache.sqoop.test.utils.HdfsUtils;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
-import javax.tools.JavaCompiler;
-import javax.tools.JavaFileObject;
-import javax.tools.StandardJavaFileManager;
-import javax.tools.StandardLocation;
-import javax.tools.ToolProvider;
-import java.io.BufferedInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.jar.Attributes;
-import java.util.jar.JarEntry;
-import java.util.jar.JarOutputStream;
-import java.util.jar.Manifest;
 
 @Test(groups = "no-real-cluster")
-public class ClasspathTest extends ConnectorTestCase {
+public class ClasspathTest extends ConnectorClasspathTestCase {
 
   private static final String TEST_CONNECTOR_JAR_NAME = "test-connector.jar";
   private static final String TEST_DEPENDENCY_JAR_NAME = "test-dependency.jar";
 
+  private static final String[] CONNECTOR_SOURCE_FILES = {
+    "TestConnector/TestConnector.java",
+    "TestConnector/TestLinkConfiguration.java",
+    "TestConnector/TestLoader.java",
+    "TestConnector/TestToDestroyer.java",
+    "TestConnector/TestToInitializer.java",
+    "TestConnector/TestToJobConfiguration.java"
+  };
+
+  private static final String[] CONNECTOR_DEPENDENCY_SOURCE_FILES = {
+    "TestConnector/TestDependency.java"
+  };
+
+  private static final String[] CONNECTOR_PROPERTY_FILES = {
+    "TestConnector/sqoopconnector.properties"
+  };
+
   private ClassLoader classLoader;
 
   public static class DerbySqoopMiniCluster extends JettySqoopMiniCluster {
@@ -88,24 +88,6 @@ public class ClasspathTest extends ConnectorTestCase {
     }
   }
 
-  class JarContents {
-    private List<File> sourceFiles;
-    private List<File> properitesFiles;
-
-    public JarContents(List<File> sourceFiles, List<File> properitesFiles){
-      this.sourceFiles = sourceFiles;
-      this.properitesFiles = properitesFiles;
-    }
-
-    public List<File> getSourceFiles() {
-      return sourceFiles;
-    }
-
-    public List<File> getProperitesFiles() {
-      return properitesFiles;
-    }
-  }
-
   public void startSqoopMiniCluster(String extraClasspath, String jobExtraClasspath) throws Exception {
     // And use them for new Derby repo instance
     setCluster(new DerbySqoopMiniCluster(HdfsUtils.joinPathFragments(super.getSqoopMiniClusterTemporaryPath(), getTestName()), hadoopCluster.getConfiguration(), extraClasspath, jobExtraClasspath));
@@ -129,7 +111,13 @@ public class ClasspathTest extends ConnectorTestCase {
 
   @Test
   public void testClasspathSqoopProperties() throws Exception {
-    Map<String, String> jarMap = compileTestConnectorAndDependency();
+    Map<String, String> jarMap = compileTestConnectorAndDependency(
+        CONNECTOR_SOURCE_FILES,
+        CONNECTOR_DEPENDENCY_SOURCE_FILES,
+        CONNECTOR_PROPERTY_FILES,
+        TEST_CONNECTOR_JAR_NAME,
+        TEST_DEPENDENCY_JAR_NAME,
+        false);
     startSqoopMiniCluster(jarMap.get(TEST_CONNECTOR_JAR_NAME), jarMap.get
       (TEST_DEPENDENCY_JAR_NAME));
     createAndLoadTableCities();
@@ -148,7 +136,13 @@ public class ClasspathTest extends ConnectorTestCase {
 
   @Test
   public void testClasspathDriverInput() throws Exception{
-    Map<String, String> jarMap = compileTestConnectorAndDependency();
+    Map<String, String> jarMap = compileTestConnectorAndDependency(
+        CONNECTOR_SOURCE_FILES,
+        CONNECTOR_DEPENDENCY_SOURCE_FILES,
+        CONNECTOR_PROPERTY_FILES,
+        TEST_CONNECTOR_JAR_NAME,
+        TEST_DEPENDENCY_JAR_NAME,
+        false);
     startSqoopMiniCluster(jarMap.get(TEST_CONNECTOR_JAR_NAME), null);
     createAndLoadTableCities();
 
@@ -190,143 +184,6 @@ public class ClasspathTest extends ConnectorTestCase {
     return driverConfig;
   }
 
-  private Map<String, String> compileTestConnectorAndDependency() throws Exception {
-    JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
-    if (compiler == null) {
-      throw new IllegalStateException(
-        "Cannot find the system Java compiler. "
-          + "Check that your class path includes tools.jar");
-    }
-
-    Path outputDir = Files.createTempDirectory(null);
-
-    Map<String, JarContents> sourceFileToJarMap = new HashMap<>();
-
-    ClassLoader classLoader = getClass().getClassLoader();
-    List<File> sourceFiles = new ArrayList<>();
-    File file = new File(classLoader.getResource("TestConnector/TestConnector.java").getFile());
-    sourceFiles.add(file);
-    file = new File(classLoader.getResource("TestConnector/TestLinkConfiguration.java").getFile());
-    sourceFiles.add(file);
-    file = new File(classLoader.getResource("TestConnector/TestLoader.java").getFile());
-    sourceFiles.add(file);
-    file = new File(classLoader.getResource("TestConnector/TestToDestroyer.java").getFile());
-    sourceFiles.add(file);
-    file = new File(classLoader.getResource("TestConnector/TestToInitializer.java").getFile());
-    sourceFiles.add(file);
-    file = new File(classLoader.getResource("TestConnector/TestToJobConfiguration.java").getFile());
-    sourceFiles.add(file);
-
-    List<File> propertiesFiles = new ArrayList<>();
-    file = new File(classLoader.getResource("TestConnector/sqoopconnector.properties").getFile());
-    propertiesFiles.add(file);
-    sourceFileToJarMap.put("test-connector.jar", new JarContents(sourceFiles, propertiesFiles));
-
-    sourceFiles = new ArrayList<>();
-    file = new File(classLoader.getResource("TestConnector/TestDependency.java").getFile());
-    sourceFiles.add(file);
-    sourceFileToJarMap.put("test-dependency.jar", new JarContents(sourceFiles, ListUtils.EMPTY_LIST));
-
-    return buildJar(outputDir.toString(), sourceFileToJarMap);
-  }
-
-  private Map<String, String> buildJar(String outputDir, Map<String, JarContents> sourceFileToJarMap) throws Exception {
-    JavaCompiler compiler = ToolProvider.getSystemJavaCompiler();
-    StandardJavaFileManager fileManager = compiler.getStandardFileManager
-      (null, null, null);
-
-    List<File> sourceFiles = new ArrayList<>();
-    for(JarContents jarContents : sourceFileToJarMap.values()) {
-      sourceFiles.addAll(jarContents.sourceFiles);
-    }
-
-    fileManager.setLocation(StandardLocation.CLASS_OUTPUT,
-      Arrays.asList(new File(outputDir.toString())));
-
-    Iterable<? extends JavaFileObject> compilationUnits1 =
-      fileManager.getJavaFileObjectsFromFiles(sourceFiles);
-
-    boolean compiled = compiler.getTask(null, fileManager, null, null, null, compilationUnits1).call();
-    if (!compiled) {
-      throw new RuntimeException("failed to compile");
-    }
-
-    for(Map.Entry<String, JarContents> jarNameAndContents : sourceFileToJarMap.entrySet()) {
-      Manifest manifest = new Manifest();
-      manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION, "1.0");
-      manifest.getMainAttributes().put(Attributes.Name.CLASS_PATH, ".");
-
-
-      JarOutputStream target = new JarOutputStream(new FileOutputStream(outputDir.toString() + File.separator + jarNameAndContents.getKey()), manifest);
-      List<String> classesForJar = new ArrayList<>();
-      for(File sourceFile : jarNameAndContents.getValue().getSourceFiles()) {
-        //split the file on dot to get the filename from FILENAME.java
-        String fileName = sourceFile.getName().split("\\.")[0];
-        classesForJar.add(fileName);
-      }
-
-      File dir = new File(outputDir);
-      File[] directoryListing = dir.listFiles();
-      for (File compiledClass : directoryListing) {
-        String classFileName = compiledClass.getName().split("\\$")[0].split("\\.")[0];
-        if (classesForJar.contains(classFileName)){
-          addFileToJar(compiledClass, target);
-        }
-      }
-
-      for (File propertiesFile : jarNameAndContents.getValue().getProperitesFiles()) {
-        addFileToJar(propertiesFile, target);
-      }
-
-      target.close();
-
-
-    }
-    //delete non jar files
-    File dir = new File(outputDir);
-    File[] directoryListing = dir.listFiles();
-    for (File file : directoryListing) {
-      String extension = file.getName().split("\\.")[1];
-      if (!extension.equals("jar")) {
-        file.delete();
-      }
-    }
-
-    Map<String, String> jarMap = new HashMap<>();
-    jarMap.put(TEST_CONNECTOR_JAR_NAME, outputDir.toString() + File.separator
-      + TEST_CONNECTOR_JAR_NAME);
-    jarMap.put(TEST_DEPENDENCY_JAR_NAME, outputDir.toString() + File.separator + TEST_DEPENDENCY_JAR_NAME);
-    return jarMap;
-  }
-
-  private void addFileToJar(File source, JarOutputStream target) throws Exception {
-    JarEntry entry = new JarEntry(source.getName());
-    entry.setTime(source.lastModified());
-    target.putNextEntry(entry);
-    BufferedInputStream in = new BufferedInputStream(new FileInputStream(source));
-
-    long bufferSize = source.length();
-    if (bufferSize < Integer.MIN_VALUE || bufferSize > Integer.MAX_VALUE) {
-      throw new RuntimeException("file to large to be added to jar");
-    }
-
-    byte[] buffer = new byte[(int) bufferSize];
-    while (true) {
-      int count = in.read(buffer);
-      if (count == -1)
-        break;
-      target.write(buffer, 0, count);
-    }
-    target.closeEntry();
-    if (in != null) in.close();
-  }
-
-  private void deleteJars(Map<String, String> jarMap) throws Exception {
-    for (String jarPath : jarMap.values()) {
-      (new File(jarPath)).delete();
-    }
-  }
-
   @Override
   public void startSqoop() throws Exception {
     // Do nothing so that Sqoop isn't started before Suite.

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/java/org/apache/sqoop/integration/connectorloading/ConnectorClasspathIsolationTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/connectorloading/ConnectorClasspathIsolationTest.java b/test/src/test/java/org/apache/sqoop/integration/connectorloading/ConnectorClasspathIsolationTest.java
new file mode 100644
index 0000000..5b95631
--- /dev/null
+++ b/test/src/test/java/org/apache/sqoop/integration/connectorloading/ConnectorClasspathIsolationTest.java
@@ -0,0 +1,181 @@
+/**
+ * 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.sqoop.integration.connectorloading;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.sqoop.client.SqoopClient;
+import org.apache.sqoop.core.ConfigurationConstants;
+import org.apache.sqoop.model.MDriverConfig;
+import org.apache.sqoop.model.MJob;
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.test.minicluster.JettySqoopMiniCluster;
+import org.apache.sqoop.test.testcases.ConnectorClasspathTestCase;
+import org.apache.sqoop.test.utils.HdfsUtils;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
+import org.testng.annotations.Test;
+
+@Test(groups = "no-real-cluster")
+public class ConnectorClasspathIsolationTest extends ConnectorClasspathTestCase {
+
+  private static final String TEST_FROM_CONNECTOR_JAR_NAME = "test-from-connector.jar";
+  private static final String TEST_TO_CONNECTOR_JAR_NAME = "test-to-connector.jar";
+  private static final String TEST_FROM_DEPENDENCY_JAR_NAME = "test-from-dependency.jar";
+  private static final String TEST_TO_DEPENDENCY_JAR_NAME = "test-to-dependency.jar";
+
+  private static final String[] FROM_CONNECTOR_SOURCE_FILES = {
+    "TestConnectorClasspathIsolation/from/TestFromConnector.java",
+    "TestConnectorClasspathIsolation/from/TestExtractor.java",
+    "TestConnectorClasspathIsolation/from/TestFromDestroyer.java",
+    "TestConnectorClasspathIsolation/from/TestFromInitializer.java",
+    "TestConnectorClasspathIsolation/from/TestFromJobConfiguration.java",
+    "TestConnectorClasspathIsolation/from/TestPartition.java",
+    "TestConnectorClasspathIsolation/from/TestPartitioner.java",
+    "TestConnectorClasspathIsolation/from/TestFromLinkConfiguration.java"
+  };
+
+  private static final String[] FROM_CONNECTOR_DEPENDENCY_SOURCE_FILES = {
+    "TestConnectorClasspathIsolation/from/TestClasspathIsolation.java"
+  };
+
+  private static final String[] FROM_CONNECTOR_PROPERTY_FILES = {
+    "TestConnectorClasspathIsolation/from/sqoopconnector.properties"
+  };
+
+  private static final String[] TO_CONNECTOR_SOURCE_FILES = {
+    "TestConnectorClasspathIsolation/to/TestToConnector.java",
+    "TestConnectorClasspathIsolation/to/TestLoader.java",
+    "TestConnectorClasspathIsolation/to/TestToDestroyer.java",
+    "TestConnectorClasspathIsolation/to/TestToInitializer.java",
+    "TestConnectorClasspathIsolation/to/TestToJobConfiguration.java",
+    "TestConnectorClasspathIsolation/to/TestToLinkConfiguration.java"
+  };
+
+  private static final String[] TO_CONNECTOR_DEPENDENCY_SOURCE_FILES = {
+    "TestConnectorClasspathIsolation/to/TestClasspathIsolation.java"
+  };
+
+  private static final String[] TO_CONNECTOR_PROPERTY_FILES = {
+    "TestConnectorClasspathIsolation/to/sqoopconnector.properties"
+  };
+
+  private ClassLoader classLoader;
+
+  public static class DerbySqoopMiniCluster extends JettySqoopMiniCluster {
+
+    private String extraClasspath;
+
+    public DerbySqoopMiniCluster(String temporaryPath, Configuration configuration, String extraClasspath) throws Exception {
+      super(temporaryPath, configuration);
+      this.extraClasspath = extraClasspath;
+    }
+
+    @Override
+    protected Map<String, String> getClasspathConfiguration() {
+      Map<String, String> properties = new HashMap<>();
+
+      if (extraClasspath != null) {
+        properties.put(ConfigurationConstants.CLASSPATH, extraClasspath);
+      }
+
+      return properties;
+    }
+  }
+
+  public void startSqoopMiniCluster(String extraClasspath) throws Exception {
+    // And use them for new Derby repo instance
+    setCluster(new DerbySqoopMiniCluster(HdfsUtils.joinPathFragments(super.getSqoopMiniClusterTemporaryPath(), getTestName()), hadoopCluster.getConfiguration(), extraClasspath));
+
+    // Start server
+    getCluster().start();
+
+    // Initialize Sqoop Client API
+    setClient(new SqoopClient(getServerUrl()));
+  }
+
+  @BeforeMethod
+  public void captureClasspath() {
+    classLoader = Thread.currentThread().getContextClassLoader();
+  }
+
+  @AfterMethod
+  public void restoreClasspath(){
+    Thread.currentThread().setContextClassLoader(classLoader);
+  }
+
+  @Test
+  public void testConnectorClasspathIsolation() throws Exception {
+    Map<String, String> fromConnectorJarMap = compileTestConnectorAndDependency(
+        FROM_CONNECTOR_SOURCE_FILES,
+        FROM_CONNECTOR_DEPENDENCY_SOURCE_FILES,
+        FROM_CONNECTOR_PROPERTY_FILES,
+        TEST_FROM_CONNECTOR_JAR_NAME,
+        TEST_FROM_DEPENDENCY_JAR_NAME,
+        true);
+    Map<String, String> toConnectorJarMap = compileTestConnectorAndDependency(
+        TO_CONNECTOR_SOURCE_FILES,
+        TO_CONNECTOR_DEPENDENCY_SOURCE_FILES,
+        TO_CONNECTOR_PROPERTY_FILES,
+        TEST_TO_CONNECTOR_JAR_NAME,
+        TEST_TO_DEPENDENCY_JAR_NAME,
+        true);
+    startSqoopMiniCluster(
+        StringUtils.join(Arrays.asList(fromConnectorJarMap.get(TEST_FROM_CONNECTOR_JAR_NAME), toConnectorJarMap.get(TEST_TO_CONNECTOR_JAR_NAME)), ":"));
+
+    MJob job = prepareJob();
+
+    prepareDriverConfig(job);
+
+    saveJob(job);
+
+    executeJob(job);
+
+    stopSqoop();
+    deleteJars(fromConnectorJarMap);
+  }
+
+  private MJob prepareJob() {
+    MLink rdbmsConnection = getClient().createLink("test-from-connector");
+    saveLink(rdbmsConnection);
+
+    MLink testConnection = getClient().createLink("test-to-connector");
+    saveLink(testConnection);
+
+    MJob job = getClient().createJob(rdbmsConnection.getName(), testConnection.getName());
+
+    return job;
+  }
+
+  private MDriverConfig prepareDriverConfig(MJob job) {
+    MDriverConfig driverConfig = job.getDriverConfig();
+    driverConfig.getIntegerInput("throttlingConfig.numExtractors").setValue(3);
+
+    return driverConfig;
+  }
+
+  @Override
+  public void startSqoop() throws Exception {
+    // Do nothing so that Sqoop isn't started before Suite.
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/TestClasspathIsolation.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/TestClasspathIsolation.java b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestClasspathIsolation.java
new file mode 100644
index 0000000..4b174ca
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestClasspathIsolation.java
@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+
+public class TestClasspathIsolation {
+  public TestClasspathIsolation() {}
+
+  public void methodCalledByFromConnector() {}
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/TestExtractor.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/TestExtractor.java b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestExtractor.java
new file mode 100644
index 0000000..cf9a256
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestExtractor.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.
+ */
+
+import org.apache.sqoop.job.etl.Extractor;
+import org.apache.sqoop.job.etl.ExtractorContext;
+
+public class TestExtractor extends Extractor<TestFromLinkConfiguration, TestFromJobConfiguration, TestPartition> {
+
+  private long rowsRead = 0;
+
+  @Override
+  public void extract(ExtractorContext context,
+      TestFromLinkConfiguration linkConfiguration,
+      TestFromJobConfiguration jobConfiguration,
+      TestPartition partition) {
+
+    // This will break if TestClasspathIsolation for to connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByFromConnector();
+  }
+
+  @Override
+  public long getRowsRead() {
+    return rowsRead;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromConnector.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromConnector.java b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromConnector.java
new file mode 100644
index 0000000..c0ae567
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromConnector.java
@@ -0,0 +1,90 @@
+/**
+ * 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.
+ */
+
+import org.apache.sqoop.common.Direction;
+import org.apache.sqoop.connector.ftp.FtpLoader;
+import org.apache.sqoop.connector.ftp.FtpToDestroyer;
+import org.apache.sqoop.connector.ftp.FtpToInitializer;
+import org.apache.sqoop.connector.jdbc.GenericJdbcConnectorConstants;
+import org.apache.sqoop.connector.spi.ConnectorConfigurableUpgrader;
+import org.apache.sqoop.connector.spi.SqoopConnector;
+import org.apache.sqoop.job.etl.From;
+import org.apache.sqoop.job.etl.To;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.ResourceBundle;
+
+/**
+ * A connector that does not do anything
+ */
+public class TestFromConnector extends SqoopConnector {
+
+  private static final From FROM = new From(TestFromInitializer.class,
+      TestPartitioner.class,
+      TestPartition.class,
+      TestExtractor.class,
+      TestFromDestroyer.class);
+
+  @Override
+  public String getVersion() {
+    return "1.0";
+  }
+
+  @Override
+  public ResourceBundle getBundle(Locale locale) {
+    return ResourceBundle.getBundle(
+      GenericJdbcConnectorConstants.RESOURCE_BUNDLE_NAME, locale);
+  }
+
+  @Override
+  public Class getLinkConfigurationClass() {
+    return TestFromLinkConfiguration.class;
+  }
+
+  @Override
+  public Class getJobConfigurationClass(Direction direction) {
+    switch (direction) {
+      case FROM:
+        return TestFromJobConfiguration.class;
+      default:
+        return null;
+    }
+  }
+
+  @Override
+  public From getFrom() {
+    return FROM;
+  }
+
+  @Override
+  public To getTo() {
+    return null;
+  }
+
+  @Override
+  public ConnectorConfigurableUpgrader getConfigurableUpgrader(String oldConnectorVersion) {
+    return null;
+  }
+
+  @Override
+  public List<Direction> getSupportedDirections() {
+    return Arrays.asList(Direction.FROM);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromDestroyer.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromDestroyer.java b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromDestroyer.java
new file mode 100644
index 0000000..1e852d8
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromDestroyer.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.
+ */
+
+import org.apache.sqoop.job.etl.Destroyer;
+import org.apache.sqoop.job.etl.DestroyerContext;
+
+public class TestFromDestroyer extends Destroyer<TestFromLinkConfiguration, TestFromJobConfiguration> {
+  @Override
+  public void destroy(DestroyerContext context, TestFromLinkConfiguration linkConfig,
+                      TestFromJobConfiguration jobConfig) {
+    // This will break if TestClasspathIsolation for to connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByFromConnector();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromInitializer.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromInitializer.java b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromInitializer.java
new file mode 100644
index 0000000..7d8fa9d
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromInitializer.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.
+ */
+
+import org.apache.sqoop.job.etl.Initializer;
+import org.apache.sqoop.job.etl.InitializerContext;
+
+public class TestFromInitializer extends Initializer<TestFromLinkConfiguration, TestFromJobConfiguration> {
+  @Override
+  public void initialize(InitializerContext context, TestFromLinkConfiguration linkConfig,
+      TestFromJobConfiguration jobConfig) {
+    // This will break if TestClasspathIsolation for to connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByFromConnector();
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromJobConfiguration.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromJobConfiguration.java b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromJobConfiguration.java
new file mode 100644
index 0000000..00d34b4
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromJobConfiguration.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.
+ */
+
+import org.apache.sqoop.model.ConfigurationClass;
+
+@ConfigurationClass
+public class TestFromJobConfiguration {
+  public TestFromJobConfiguration() {
+    // This will break if TestClasspathIsolation for to connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByFromConnector();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromLinkConfiguration.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromLinkConfiguration.java b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromLinkConfiguration.java
new file mode 100644
index 0000000..9e543eb
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestFromLinkConfiguration.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.
+ */
+
+import org.apache.sqoop.model.ConfigurationClass;
+
+@ConfigurationClass
+public class TestFromLinkConfiguration {
+  public TestFromLinkConfiguration() {
+    // This will break if TestClasspathIsolation for to connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByFromConnector();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/TestPartition.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/TestPartition.java b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestPartition.java
new file mode 100644
index 0000000..90bd9e4
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestPartition.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.
+ */
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.sqoop.job.etl.Partition;
+
+public class TestPartition extends Partition {
+  public TestPartition() {}
+
+  @Override
+  public void readFields(DataInput in) throws IOException {}
+
+  @Override
+  public void write(DataOutput out) throws IOException {}
+
+  @Override
+  public String toString() {return "";}
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/TestPartitioner.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/TestPartitioner.java b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestPartitioner.java
new file mode 100644
index 0000000..0254ad2
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/TestPartitioner.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.
+ */
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.sqoop.job.etl.Partition;
+import org.apache.sqoop.job.etl.Partitioner;
+import org.apache.sqoop.job.etl.PartitionerContext;
+
+public class TestPartitioner extends Partitioner<TestFromLinkConfiguration, TestFromJobConfiguration> {
+  public TestPartitioner() {
+  }
+
+  @Override
+  public List<Partition> getPartitions(PartitionerContext context,
+      TestFromLinkConfiguration linkConfiguration, TestFromJobConfiguration fromJobConfiguration) {
+    // This will break if TestClasspathIsolation for to connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByFromConnector();
+
+    List<Partition> partitionList = new ArrayList<Partition>();
+    partitionList.add(new TestPartition());
+    return partitionList;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/from/sqoopconnector.properties
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/from/sqoopconnector.properties b/test/src/test/resources/TestConnectorClasspathIsolation/from/sqoopconnector.properties
new file mode 100644
index 0000000..1aa31f8
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/from/sqoopconnector.properties
@@ -0,0 +1,18 @@
+# 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.
+
+# Test Connector Properties
+org.apache.sqoop.connector.class = TestFromConnector
+org.apache.sqoop.connector.name = test-from-connector

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/to/TestClasspathIsolation.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/to/TestClasspathIsolation.java b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestClasspathIsolation.java
new file mode 100644
index 0000000..98023b8
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestClasspathIsolation.java
@@ -0,0 +1,23 @@
+/**
+ * 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.
+ */
+
+public class TestClasspathIsolation {
+  public TestClasspathIsolation() {}
+
+  public void methodCalledByToConnector() {}
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/to/TestLoader.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/to/TestLoader.java b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestLoader.java
new file mode 100644
index 0000000..3218fc0
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestLoader.java
@@ -0,0 +1,48 @@
+/**
+ * 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.
+ */
+
+import org.apache.sqoop.etl.io.DataReader;
+import org.apache.sqoop.job.etl.Loader;
+import org.apache.sqoop.job.etl.LoaderContext;
+
+import java.util.UUID;
+
+public class TestLoader extends Loader<TestToLinkConfiguration, TestToJobConfiguration> {
+
+  private long rowsWritten = 0;
+
+
+  @Override
+  public void load(LoaderContext context, TestToLinkConfiguration linkConfiguration,
+                   TestToJobConfiguration toJobConfig) throws Exception {
+    DataReader reader = context.getDataReader();
+
+    // This will break if TestClasspathIsolation for from connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByToConnector();
+
+    while (reader.readTextRecord() != null){
+      rowsWritten++;
+    }
+  }
+
+  @Override
+  public long getRowsWritten() {
+    return rowsWritten;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToConnector.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToConnector.java b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToConnector.java
new file mode 100644
index 0000000..ec7f23c
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToConnector.java
@@ -0,0 +1,88 @@
+/**
+ * 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.
+ */
+
+import org.apache.sqoop.common.Direction;
+import org.apache.sqoop.connector.ftp.FtpLoader;
+import org.apache.sqoop.connector.ftp.FtpToDestroyer;
+import org.apache.sqoop.connector.ftp.FtpToInitializer;
+import org.apache.sqoop.connector.jdbc.GenericJdbcConnectorConstants;
+import org.apache.sqoop.connector.spi.ConnectorConfigurableUpgrader;
+import org.apache.sqoop.connector.spi.SqoopConnector;
+import org.apache.sqoop.job.etl.From;
+import org.apache.sqoop.job.etl.To;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.ResourceBundle;
+
+/**
+ * A connector that does not do anything
+ */
+public class TestToConnector extends SqoopConnector {
+
+  private static final To TO = new To(TestToInitializer.class,
+    TestLoader.class,
+    TestToDestroyer.class);
+
+  @Override
+  public String getVersion() {
+    return "1.0";
+  }
+
+  @Override
+  public ResourceBundle getBundle(Locale locale) {
+    return ResourceBundle.getBundle(
+      GenericJdbcConnectorConstants.RESOURCE_BUNDLE_NAME, locale);
+  }
+
+  @Override
+  public Class getLinkConfigurationClass() {
+    return TestToLinkConfiguration.class;
+  }
+
+  @Override
+  public Class getJobConfigurationClass(Direction direction) {
+    switch (direction) {
+      case TO:
+        return TestToJobConfiguration.class;
+      default:
+        return null;
+    }
+  }
+
+  @Override
+  public From getFrom() {
+    return null;
+  }
+
+  @Override
+  public To getTo() {
+    return TO;
+  }
+
+  @Override
+  public ConnectorConfigurableUpgrader getConfigurableUpgrader(String oldConnectorVersion) {
+    return null;
+  }
+
+  @Override
+  public List<Direction> getSupportedDirections() {
+    return Arrays.asList(Direction.TO);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToDestroyer.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToDestroyer.java b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToDestroyer.java
new file mode 100644
index 0000000..e2a4d8f
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToDestroyer.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.
+ */
+
+import org.apache.sqoop.job.etl.Destroyer;
+import org.apache.sqoop.job.etl.DestroyerContext;
+
+public class TestToDestroyer extends Destroyer<TestToLinkConfiguration, TestToJobConfiguration> {
+  @Override
+  public void destroy(DestroyerContext context, TestToLinkConfiguration linkConfig,
+                      TestToJobConfiguration jobConfig) {
+    // This will break if TestClasspathIsolation for from connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByToConnector();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToInitializer.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToInitializer.java b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToInitializer.java
new file mode 100644
index 0000000..dd2e336
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToInitializer.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.
+ */
+
+import org.apache.sqoop.job.etl.Initializer;
+import org.apache.sqoop.job.etl.InitializerContext;
+
+public class TestToInitializer extends Initializer<TestToLinkConfiguration, TestToJobConfiguration> {
+  @Override
+  public void initialize(InitializerContext context, TestToLinkConfiguration linkConfig,
+                         TestToJobConfiguration jobConfig) {
+    // This will break if TestClasspathIsolation for from connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByToConnector();
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToJobConfiguration.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToJobConfiguration.java b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToJobConfiguration.java
new file mode 100644
index 0000000..ff1c3ed
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToJobConfiguration.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.
+ */
+
+import org.apache.sqoop.model.ConfigurationClass;
+
+@ConfigurationClass
+public class TestToJobConfiguration {
+  public TestToJobConfiguration() {
+    // This will break if TestClasspathIsolation for from connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByToConnector();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToLinkConfiguration.java
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToLinkConfiguration.java b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToLinkConfiguration.java
new file mode 100644
index 0000000..296b2e6
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/to/TestToLinkConfiguration.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.
+ */
+
+import org.apache.sqoop.model.ConfigurationClass;
+
+@ConfigurationClass
+public class TestToLinkConfiguration {
+  public TestToLinkConfiguration() {
+    // This will break if TestClasspathIsolation for from connector is loaded
+    TestClasspathIsolation testClasspathIsolation = new TestClasspathIsolation();
+    testClasspathIsolation.methodCalledByToConnector();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/a03adec2/test/src/test/resources/TestConnectorClasspathIsolation/to/sqoopconnector.properties
----------------------------------------------------------------------
diff --git a/test/src/test/resources/TestConnectorClasspathIsolation/to/sqoopconnector.properties b/test/src/test/resources/TestConnectorClasspathIsolation/to/sqoopconnector.properties
new file mode 100644
index 0000000..48e60c9
--- /dev/null
+++ b/test/src/test/resources/TestConnectorClasspathIsolation/to/sqoopconnector.properties
@@ -0,0 +1,18 @@
+# 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.
+
+# Test Connector Properties
+org.apache.sqoop.connector.class = TestToConnector
+org.apache.sqoop.connector.name = test-to-connector