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/08/22 03:04:15 UTC

sqoop git commit: SQOOP-2364: Sqoop2: Provide test infrastructure base class for server tests

Repository: sqoop
Updated Branches:
  refs/heads/sqoop2 7aca9f4f6 -> d26c5031b


SQOOP-2364: Sqoop2: Provide test infrastructure base class for server tests

(Abraham Elmahrek 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/d26c5031
Tree: http://git-wip-us.apache.org/repos/asf/sqoop/tree/d26c5031
Diff: http://git-wip-us.apache.org/repos/asf/sqoop/diff/d26c5031

Branch: refs/heads/sqoop2
Commit: d26c5031bdb7cfdfff14bbcdec1fe263f570800e
Parents: 7aca9f4
Author: Jarek Jarcec Cecho <ja...@apache.org>
Authored: Fri Aug 21 18:03:54 2015 -0700
Committer: Jarek Jarcec Cecho <ja...@apache.org>
Committed: Fri Aug 21 18:03:54 2015 -0700

----------------------------------------------------------------------
 test/pom.xml                                    |  15 +
 .../test/infrastructure/SqoopTestCase.java      | 283 +++++++++++++++++++
 .../minicluster/InProcessSqoopMiniCluster.java  |   5 +
 .../test/minicluster/SqoopMiniCluster.java      |   5 +
 .../minicluster/TomcatSqoopMiniCluster.java     |   1 +
 .../org/apache/sqoop/test/utils/HdfsUtils.java  |   8 +-
 .../server/InvalidRESTCallsTest.java            |  11 +-
 .../integration/server/ShowJobInOrderTest.java  |   9 +-
 .../SubmissionWithDisabledModelObjectsTest.java |  31 +-
 .../sqoop/integration/server/VersionTest.java   |  10 +-
 .../test/resources/integration-tests-suite.xml  |   7 -
 .../resources/new-integration-tests-suite.xml   |  33 +++
 12 files changed, 391 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index a9502d2..3e11f59 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -202,6 +202,21 @@ limitations under the License.
             </configuration>
           </execution>
           <execution>
+            <id>new-integration-test</id>
+            <goals>
+              <goal>test</goal>
+            </goals>
+            <phase>integration-test</phase>
+            <configuration>
+              <suiteXmlFiles>
+                <suiteXmlFile>src/test/resources/new-integration-tests-suite.xml</suiteXmlFile>
+              </suiteXmlFiles>
+              <properties>
+                <suitename>new-integration-tests</suitename>
+              </properties>
+            </configuration>
+          </execution>
+          <execution>
             <id>upgrade-test</id>
             <goals>
               <goal>test</goal>

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java b/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
index e60caf7..5cae43b 100644
--- a/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
+++ b/test/src/main/java/org/apache/sqoop/test/infrastructure/SqoopTestCase.java
@@ -21,9 +21,26 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.log4j.Logger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.sqoop.client.SqoopClient;
+import org.apache.sqoop.client.SubmissionCallback;
+import org.apache.sqoop.common.test.db.DatabaseProvider;
+import org.apache.sqoop.common.test.db.TableName;
+import org.apache.sqoop.connector.hdfs.configuration.ToFormat;
+import org.apache.sqoop.model.MConfigList;
+import org.apache.sqoop.model.MJob;
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.model.MPersistableEntity;
+import org.apache.sqoop.model.MSubmission;
+import org.apache.sqoop.submission.SubmissionStatus;
+import org.apache.sqoop.test.data.Cities;
+import org.apache.sqoop.test.data.ShortStories;
+import org.apache.sqoop.test.data.UbuntuReleases;
+import org.apache.sqoop.test.infrastructure.providers.DatabaseInfrastructureProvider;
 import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
 import org.apache.sqoop.test.infrastructure.providers.InfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.SqoopInfrastructureProvider;
 import org.apache.sqoop.test.utils.HdfsUtils;
+import org.apache.sqoop.validation.Status;
 import org.testng.ITest;
 import org.testng.ITestContext;
 import org.testng.ITestNGMethod;
@@ -38,6 +55,9 @@ import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotSame;
+
 /**
  * Use Infrastructure annotation to boot up miniclusters.
  * Order is built-in to code. Hadoop comes first, then
@@ -51,10 +71,32 @@ public class SqoopTestCase implements ITest {
   private static final Map<String, InfrastructureProvider> PROVIDERS
       = new HashMap<String, InfrastructureProvider>();
 
+  /**
+   * Default submission callbacks that are printing various status about the submission.
+   */
+  protected static SubmissionCallback DEFAULT_SUBMISSION_CALLBACKS = new SubmissionCallback() {
+    @Override
+    public void submitted(MSubmission submission) {
+      LOG.info("Submission submitted: " + submission);
+    }
+
+    @Override
+    public void updated(MSubmission submission) {
+      LOG.info("Submission updated: " + submission);
+    }
+
+    @Override
+    public void finished(MSubmission submission) {
+      LOG.info("Submission finished: " + submission);
+    }
+  };
+
   private static String suiteName;
 
   private String methodName;
 
+  private SqoopClient client;
+
   @BeforeSuite
   public static void findSuiteName(ITestContext context) {
     suiteName = context.getSuite().getName();
@@ -79,6 +121,10 @@ public class SqoopTestCase implements ITest {
     // Find infrastructure provider classes to be used.
     Set<Class<? extends InfrastructureProvider>> providers = new HashSet<Class<? extends InfrastructureProvider>>();
     for (ITestNGMethod method : context.getSuite().getAllMethods()) {
+      LOG.debug("Looking up dependencies on method ("
+          + method.getConstructorOrMethod().getDeclaringClass().getCanonicalName()
+          + "#" + method.getConstructorOrMethod().getMethod().getName()
+          + ")");
       Infrastructure ann;
 
       // If the method has an infrastructure annotation, process it.
@@ -173,4 +219,241 @@ public class SqoopTestCase implements ITest {
     InfrastructureProvider provider = PROVIDERS.get(providerClass.getCanonicalName());
     return ((T) provider);
   }
+
+  /**
+   * Get the data directory for tests.
+   * @return
+   */
+  public String getMapreduceDirectory() {
+    return HdfsUtils.joinPathFragments(
+        getInfrastructureProvider(HadoopInfrastructureProvider.class).getInstance().getTestDirectory(),
+        getClass().getName(),
+        getTestName());
+  }
+
+  /**
+   * Fill RDBMS Link Configuration with infrastructure provider info.
+   * @param link
+   */
+  public void fillRdbmsLinkConfig(MLink link) {
+    DatabaseProvider provider = getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance();
+
+    MConfigList configs = link.getConnectorLinkConfig();
+    configs.getStringInput("linkConfig.jdbcDriver").setValue(provider.getJdbcDriver());
+    configs.getStringInput("linkConfig.connectionString").setValue(provider.getConnectionUrl());
+    configs.getStringInput("linkConfig.username").setValue(provider.getConnectionUsername());
+    configs.getStringInput("linkConfig.password").setValue(provider.getConnectionPassword());
+  }
+
+  /**
+   * Fill RDBMS FROM Configuration with infrastructure provider info.
+   * @param job
+   * @param partitionColumn
+   */
+  public void fillRdbmsFromConfig(MJob job, String partitionColumn) {
+    DatabaseProvider provider = getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance();
+
+    MConfigList fromConfig = job.getFromJobConfig();
+    fromConfig.getStringInput("fromJobConfig.tableName").setValue(provider.escapeTableName(getTableName().getTableName()));
+    fromConfig.getStringInput("fromJobConfig.partitionColumn").setValue(provider.escapeColumnName(partitionColumn));
+  }
+
+  /**
+   * Fill RDBMS TO Configuration with infrastructure provider info.
+   * @param job
+   */
+  public void fillRdbmsToConfig(MJob job) {
+    DatabaseProvider provider = getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance();
+
+    MConfigList toConfig = job.getToJobConfig();
+    toConfig.getStringInput("toJobConfig.tableName").setValue(provider.escapeTableName(getTableName().getTableName()));
+  }
+
+  /**
+   * Fill HDFS Link Configuration with infrastructure provider info.
+   * @param link
+   */
+  public void fillHdfsLinkConfig(MLink link) {
+    MConfigList configs = link.getConnectorLinkConfig();
+    configs.getStringInput("linkConfig.confDir").setValue(
+        getInfrastructureProvider(SqoopInfrastructureProvider.class).getInstance()
+            .getConfigurationPath());
+  }
+
+  /**
+   * Fill HDFS FROM Configuration with infrastructure provider info.
+   * @param job
+   */
+  public void fillHdfsFromConfig(MJob job) {
+    MConfigList fromConfig = job.getFromJobConfig();
+    fromConfig.getStringInput("fromJobConfig.inputDirectory").setValue(getMapreduceDirectory());
+  }
+
+  /**
+   * Fill HDFS TO Configuration with infrastructure provider info.
+   * @param job
+   * @param output
+   */
+  public void fillHdfsToConfig(MJob job, ToFormat output) {
+    MConfigList toConfig = job.getToJobConfig();
+    toConfig.getEnumInput("toJobConfig.outputFormat").setValue(output);
+    toConfig.getStringInput("toJobConfig.outputDirectory").setValue(getMapreduceDirectory());
+  }
+
+  public String getSqoopServerUrl() {
+    if (getInfrastructureProvider(SqoopInfrastructureProvider.class) == null) {
+      return null;
+    }
+
+    return getInfrastructureProvider(SqoopInfrastructureProvider.class).getInstance()
+        .getServerUrl();
+  }
+
+  /**
+   * Create a sqoop client
+   * @return SqoopClient
+   */
+  public SqoopClient getClient() {
+    if (client == null) {
+      String serverUrl = getSqoopServerUrl();
+
+      if (serverUrl != null) {
+        client = new SqoopClient(serverUrl);
+      }
+    }
+    return client;
+  }
+
+  /**
+   * Create link with asserts to make sure that it was created correctly.
+   *
+   * @param link
+   */
+  public void saveLink(MLink link) {
+    assertEquals(Status.OK, getClient().saveLink(link));
+    assertNotSame(MPersistableEntity.PERSISTANCE_ID_DEFAULT, link.getPersistenceId());
+  }
+
+  /**
+   * Create job with asserts to make sure that it was created correctly.
+   *
+   * @param job
+   */
+  public void saveJob(MJob job) {
+    assertEquals(Status.OK, getClient().saveJob(job));
+    assertNotSame(MPersistableEntity.PERSISTANCE_ID_DEFAULT, job.getPersistenceId());
+  }
+
+  /**
+   * Run job with given jid.
+   *
+   * @param jid Job id
+   * @throws Exception
+   */
+  public void executeJob(long jid) throws Exception {
+    MSubmission finalSubmission = getClient().startJob(jid, DEFAULT_SUBMISSION_CALLBACKS, 100);
+
+    if(finalSubmission.getStatus().isFailure()) {
+      LOG.error("Submission has failed: " + finalSubmission.getError().getErrorSummary());
+      LOG.error("Corresponding error details: " + finalSubmission.getError().getErrorDetails());
+    }
+    assertEquals(SubmissionStatus.SUCCEEDED, finalSubmission.getStatus(), "Submission finished with error: " + finalSubmission.getError().getErrorSummary());
+  }
+
+  /**
+   * Fetch table name to be used by this test.
+   * @return TableName
+   */
+  public TableName getTableName() {
+    return new TableName(getClass().getSimpleName());
+  }
+
+  /**
+   * Create table with table name for this test.
+   * @param primaryKey
+   * @param columns
+   */
+  public void createTable(String primaryKey, String ...columns) {
+    getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance()
+        .createTable(getTableName(), primaryKey, columns);
+  }
+
+  /**
+   * Drop table for this test.
+   */
+  public void dropTable() {
+    getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance()
+        .dropTable(getTableName());
+  }
+
+  /**
+   * Insert row into table for this test.
+   * @param values
+   */
+  public void insertRow(Object ...values) {
+    getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance()
+        .insertRow(getTableName(), values);
+  }
+
+  /**
+   * Insert row into table for this test.
+   * @param escapeValues
+   * @param values
+   */
+  public void insertRow(Boolean escapeValues, Object ...values) {
+    getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance()
+        .insertRow(getTableName(), escapeValues, values);
+  }
+
+  /**
+   * Fetch row count of table for this test.
+   * @return long count
+   */
+  public long rowCount() {
+    return getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance()
+        .rowCount(getTableName());
+  }
+
+  /**
+   * Dump the table for this test.
+   */
+  public void dumpTable() {
+    getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance()
+        .dumpTable(getTableName());
+  }
+
+  /**
+   * Create and load cities data.
+   */
+  public void createAndLoadTableCities() {
+    new Cities(getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance(), getTableName()).createTables().loadBasicData();
+  }
+
+  /**
+   * Create ubuntu releases table.
+   */
+  public void createTableUbuntuReleases() {
+    new UbuntuReleases(getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance(), getTableName()).createTables();
+  }
+
+  /**
+   * Create and load ubuntu releases data.
+   */
+  public void createAndLoadTableUbuntuReleases() {
+    new UbuntuReleases(getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance(), getTableName()).createTables().loadBasicData();
+  }
+
+  /**
+   * Create short stories table.
+   */
+  public void createTableShortStories() {
+    new ShortStories(getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance(), getTableName()).createTables();
+  }
+
+  /**
+   * Create and load short stories data.
+   */
+  public void createAndLoadTableShortStories() {
+    new ShortStories(getInfrastructureProvider(DatabaseInfrastructureProvider.class).getInstance(), getTableName()).createTables().loadBasicData();
+  }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/main/java/org/apache/sqoop/test/minicluster/InProcessSqoopMiniCluster.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/minicluster/InProcessSqoopMiniCluster.java b/test/src/main/java/org/apache/sqoop/test/minicluster/InProcessSqoopMiniCluster.java
index e11e82c..ae1fac1 100644
--- a/test/src/main/java/org/apache/sqoop/test/minicluster/InProcessSqoopMiniCluster.java
+++ b/test/src/main/java/org/apache/sqoop/test/minicluster/InProcessSqoopMiniCluster.java
@@ -52,4 +52,9 @@ public class InProcessSqoopMiniCluster extends SqoopMiniCluster {
     SqoopServer.destroy();
   }
 
+  /** {@inheritDoc} */
+  @Override
+  public String getServerUrl() {
+    return "http://localhost:12000/sqoop";
+  }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java b/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java
index 758eb2f..7440025 100644
--- a/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java
+++ b/test/src/main/java/org/apache/sqoop/test/minicluster/SqoopMiniCluster.java
@@ -101,6 +101,11 @@ public abstract class SqoopMiniCluster {
   abstract public void stop() throws Exception;
 
   /**
+   * @return server URL (e.g. http://localhost:12000/sqoop)
+   */
+  abstract public String getServerUrl();
+
+  /**
    * Prepare temporary directory for starting Sqoop server.
    *
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/main/java/org/apache/sqoop/test/minicluster/TomcatSqoopMiniCluster.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/minicluster/TomcatSqoopMiniCluster.java b/test/src/main/java/org/apache/sqoop/test/minicluster/TomcatSqoopMiniCluster.java
index 18a4b12..83f42b6 100644
--- a/test/src/main/java/org/apache/sqoop/test/minicluster/TomcatSqoopMiniCluster.java
+++ b/test/src/main/java/org/apache/sqoop/test/minicluster/TomcatSqoopMiniCluster.java
@@ -189,6 +189,7 @@ public class TomcatSqoopMiniCluster extends SqoopMiniCluster {
   /**
    * Return server URL.
    */
+  @Override
   public String getServerUrl() {
     // We're not doing any changes, so return default URL
     return "http://localhost:" + port + "/sqoop/";

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/main/java/org/apache/sqoop/test/utils/HdfsUtils.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/utils/HdfsUtils.java b/test/src/main/java/org/apache/sqoop/test/utils/HdfsUtils.java
index 79700a6..7f83b2d 100644
--- a/test/src/main/java/org/apache/sqoop/test/utils/HdfsUtils.java
+++ b/test/src/main/java/org/apache/sqoop/test/utils/HdfsUtils.java
@@ -22,6 +22,11 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.log4j.Logger;
+import org.apache.sqoop.connector.hdfs.configuration.ToFormat;
+import org.apache.sqoop.model.MConfigList;
+import org.apache.sqoop.model.MJob;
+import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.test.hadoop.HadoopRunner;
 
 import java.io.BufferedWriter;
 import java.io.FileNotFoundException;
@@ -67,8 +72,7 @@ public class HdfsUtils {
    * Create HDFS file with given content.
    *
    * @param fs filesystem object
-   * @param directory Directory where the file should be created
-   * @param filename File name
+   * @param path path to file be created
    * @param lines Individual lines that should be written into the file
    * @throws IOException
    */

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/test/java/org/apache/sqoop/integration/server/InvalidRESTCallsTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/server/InvalidRESTCallsTest.java b/test/src/test/java/org/apache/sqoop/integration/server/InvalidRESTCallsTest.java
index f1ef336..0490cf0 100644
--- a/test/src/test/java/org/apache/sqoop/integration/server/InvalidRESTCallsTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/server/InvalidRESTCallsTest.java
@@ -21,6 +21,11 @@ import org.apache.log4j.Logger;
 import com.google.common.collect.Iterables;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.sqoop.test.infrastructure.Infrastructure;
+import org.apache.sqoop.test.infrastructure.SqoopTestCase;
+import org.apache.sqoop.test.infrastructure.providers.DatabaseInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.SqoopInfrastructureProvider;
 import org.apache.sqoop.test.testcases.ConnectorTestCase;
 import org.apache.sqoop.test.utils.ParametrizedUtils;
 import org.testng.ITest;
@@ -35,7 +40,8 @@ import java.nio.charset.Charset;
 
 import static org.testng.Assert.assertEquals;
 
-public class InvalidRESTCallsTest extends ConnectorTestCase implements ITest {
+@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class})
+public class InvalidRESTCallsTest extends SqoopTestCase {
 
   private static final Logger LOG = Logger.getLogger(InvalidRESTCallsTest.class);
 
@@ -120,11 +126,10 @@ public class InvalidRESTCallsTest extends ConnectorTestCase implements ITest {
   public void test() throws Exception {
     LOG.info("Start: " + getTestName());
 
-    URL url = new URL(getServerUrl() +  desc.rest);
+    URL url = new URL(getSqoopServerUrl() +  desc.rest);
     HttpURLConnection connection = new DelegationTokenAuthenticatedURL().openConnection(url, new DelegationTokenAuthenticatedURL.Token());
     connection.setRequestMethod(desc.method);
 
-
     if(desc.data != null) {
       connection.setDoOutput(true);
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java b/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java
index 88cfcb5..6c0622c 100644
--- a/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/server/ShowJobInOrderTest.java
@@ -20,6 +20,10 @@ package org.apache.sqoop.integration.server;
 import org.apache.sqoop.connector.hdfs.configuration.ToFormat;
 import org.apache.sqoop.model.MJob;
 import org.apache.sqoop.model.MLink;
+import org.apache.sqoop.test.infrastructure.Infrastructure;
+import org.apache.sqoop.test.infrastructure.SqoopTestCase;
+import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.SqoopInfrastructureProvider;
 import org.apache.sqoop.test.testcases.ConnectorTestCase;
 import org.testng.annotations.Test;
 
@@ -30,7 +34,8 @@ import static org.testng.Assert.assertEquals;
 /**
  * Ensure that jobs will be shown in order
  */
-public class ShowJobInOrderTest extends ConnectorTestCase {
+@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class})
+public class ShowJobInOrderTest extends SqoopTestCase {
 
   public ShowJobInOrderTest() {
   }
@@ -46,7 +51,7 @@ public class ShowJobInOrderTest extends ConnectorTestCase {
 
     // HDFS link
     MLink hdfsLink = getClient().createLink("hdfs-connector");
-    fillHdfsLink(hdfsLink);
+    fillHdfsLinkConfig(hdfsLink);
     saveLink(hdfsLink);
 
     // Job creation

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java b/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java
index 199b77c..93cc7f6 100644
--- a/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/server/SubmissionWithDisabledModelObjectsTest.java
@@ -18,14 +18,18 @@
 package org.apache.sqoop.integration.server;
 
 import org.apache.sqoop.client.ClientError;
-import org.apache.sqoop.common.Direction;
 import org.apache.sqoop.common.SqoopException;
 import org.apache.sqoop.connector.hdfs.configuration.ToFormat;
 import org.apache.sqoop.error.code.DriverError;
 import org.apache.sqoop.model.MLink;
-import org.apache.sqoop.model.MConfigList;
 import org.apache.sqoop.model.MJob;
-import org.apache.sqoop.test.testcases.ConnectorTestCase;
+import org.apache.sqoop.test.infrastructure.Infrastructure;
+import org.apache.sqoop.test.infrastructure.SqoopTestCase;
+import org.apache.sqoop.test.infrastructure.providers.DatabaseInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.SqoopInfrastructureProvider;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Factory;
 import org.testng.annotations.Test;
@@ -39,7 +43,8 @@ import static org.testng.Assert.fail;
  * Ensure that server will reject starting job when either job itself
  * or corresponding link is disabled.
  */
-public class SubmissionWithDisabledModelObjectsTest extends ConnectorTestCase {
+@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class, DatabaseInfrastructureProvider.class})
+public class SubmissionWithDisabledModelObjectsTest extends SqoopTestCase {
 
   private boolean enabledLink;
   private boolean enabledJob;
@@ -59,10 +64,18 @@ public class SubmissionWithDisabledModelObjectsTest extends ConnectorTestCase {
     };
   }
 
-  @Test
-  public void testWithDisabledObjects() throws Exception {
+  @BeforeMethod
+  public void setupRdbmsTable() {
     createAndLoadTableCities();
+  }
 
+  @AfterMethod
+  public void tearDownRdbmsTable() {
+    dropTable();
+  }
+
+  @Test
+  public void testWithDisabledObjects() throws Exception {
     // RDBMS link
     MLink rdbmsLink = getClient().createLink("generic-jdbc-connector");
     fillRdbmsLinkConfig(rdbmsLink);
@@ -70,7 +83,7 @@ public class SubmissionWithDisabledModelObjectsTest extends ConnectorTestCase {
 
     // HDFS link
     MLink hdfsLink = getClient().createLink("hdfs-connector");
-    fillHdfsLink(hdfsLink);
+    fillHdfsLinkConfig(hdfsLink);
     saveLink(hdfsLink);
 
     // Job creation
@@ -90,7 +103,7 @@ public class SubmissionWithDisabledModelObjectsTest extends ConnectorTestCase {
 
     // Try to execute the job and verify that the it was not executed
     try {
-      executeJob(job);
+      executeJob(job.getPersistenceId());
       fail("Expected exception as the model classes are disabled.");
     } catch(SqoopException ex) {
       // Top level exception should be CLIENT_0001
@@ -109,8 +122,6 @@ public class SubmissionWithDisabledModelObjectsTest extends ConnectorTestCase {
       } else {
         fail("Unexpected expception retrieved from server " + cause);
       }
-    } finally {
-      dropTable();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java b/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java
index bcaf5f5..ad5a2db 100644
--- a/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/server/VersionTest.java
@@ -19,7 +19,10 @@ package org.apache.sqoop.integration.server;
 
 import org.apache.sqoop.client.request.VersionResourceRequest;
 import org.apache.sqoop.common.VersionInfo;
-import org.apache.sqoop.test.testcases.TomcatTestCase;
+import org.apache.sqoop.test.infrastructure.Infrastructure;
+import org.apache.sqoop.test.infrastructure.SqoopTestCase;
+import org.apache.sqoop.test.infrastructure.providers.HadoopInfrastructureProvider;
+import org.apache.sqoop.test.infrastructure.providers.SqoopInfrastructureProvider;
 import org.apache.sqoop.json.VersionBean;
 import org.testng.annotations.Test;
 
@@ -28,12 +31,13 @@ import static org.testng.Assert.assertEquals;
 /**
  * Basic test to check that server is working and returning correct version info.
  */
-public class VersionTest extends TomcatTestCase {
+@Infrastructure(dependencies = {HadoopInfrastructureProvider.class, SqoopInfrastructureProvider.class})
+public class VersionTest extends SqoopTestCase {
 
   @Test
   public void testVersion() {
     VersionResourceRequest versionRequest = new VersionResourceRequest();
-    VersionBean versionBean = versionRequest.read(getServerUrl());
+    VersionBean versionBean = versionRequest.read(getSqoopServerUrl());
 
     assertEquals(versionBean.getBuildVersion(), VersionInfo.getBuildVersion());
     assertEquals(versionBean.getBuildDate(), VersionInfo.getBuildDate());

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/test/resources/integration-tests-suite.xml
----------------------------------------------------------------------
diff --git a/test/src/test/resources/integration-tests-suite.xml b/test/src/test/resources/integration-tests-suite.xml
index c105329..1cf3299 100644
--- a/test/src/test/resources/integration-tests-suite.xml
+++ b/test/src/test/resources/integration-tests-suite.xml
@@ -29,11 +29,4 @@ limitations under the License.
       <package name="org.apache.sqoop.integration.connector.*"/>
     </packages>
   </test>
-
-  <test name="ServerTests">
-    <packages>
-      <package name="org.apache.sqoop.integration.server"/>
-    </packages>
-  </test>
-
 </suite>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/sqoop/blob/d26c5031/test/src/test/resources/new-integration-tests-suite.xml
----------------------------------------------------------------------
diff --git a/test/src/test/resources/new-integration-tests-suite.xml b/test/src/test/resources/new-integration-tests-suite.xml
new file mode 100644
index 0000000..8206e52
--- /dev/null
+++ b/test/src/test/resources/new-integration-tests-suite.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+
+<!DOCTYPE suite SYSTEM "http://testng.org/testng-1.0.dtd" >
+
+<suite name="NewIntegrationTests" verbose="2" parallel="false">
+
+  <listeners>
+    <listener class-name="org.apache.sqoop.test.testng.SqoopTestListener" />
+  </listeners>
+
+  <test name="ServerTests">
+    <packages>
+      <package name="org.apache.sqoop.integration.server"/>
+    </packages>
+  </test>
+
+</suite>