You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ka...@apache.org on 2013/03/26 09:15:53 UTC

git commit: SQOOP-946: Sqoop2: Integration: Enhance the POC test case (Jarek Jarcec Cecho via Kate Ting)

Updated Branches:
  refs/heads/sqoop2 27271af13 -> 3fd8bdffc


SQOOP-946: Sqoop2: Integration: Enhance the POC test case
(Jarek Jarcec Cecho via Kate Ting)


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

Branch: refs/heads/sqoop2
Commit: 3fd8bdffc9510cc9c5bbd159e9036891171496a8
Parents: 27271af
Author: Kate Ting <ka...@apache.org>
Authored: Tue Mar 26 04:15:05 2013 -0400
Committer: Kate Ting <ka...@apache.org>
Committed: Tue Mar 26 04:15:05 2013 -0400

----------------------------------------------------------------------
 .../apache/sqoop/integration/TomcatTestCase.java   |   31 ++++-
 .../integration/connector/ConnectorTestCase.java   |   92 +++++++++++++-
 .../connector/jdbc/generic/TableImportTest.java    |   99 ++++++++-------
 3 files changed, 164 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/3fd8bdff/test/src/test/java/org/apache/sqoop/integration/TomcatTestCase.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/TomcatTestCase.java b/test/src/test/java/org/apache/sqoop/integration/TomcatTestCase.java
index 5631144..6eb3184 100644
--- a/test/src/test/java/org/apache/sqoop/integration/TomcatTestCase.java
+++ b/test/src/test/java/org/apache/sqoop/integration/TomcatTestCase.java
@@ -23,6 +23,8 @@ import org.apache.sqoop.client.SqoopClient;
 import org.apache.sqoop.test.minicluster.TomcatSqoopMiniCluster;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
+import org.junit.rules.TestName;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -45,17 +47,29 @@ abstract public class TomcatTestCase {
 
   private static final Logger LOG = Logger.getLogger(TomcatTestCase.class);
 
+  @Rule public TestName name = new TestName();
+
   /**
-   * Temporary path that will be used for this test.
+   * Temporary base path that will be used for tests.
    *
-   * By default we will take look for sqoop.integration.tmpdir property that is
+   * By default we will take a look for sqoop.integration.tmpdir property that is
    * filled up by maven. If the test is not started from maven (IDE) we will
    * pick up configured java.io.tmpdir value. The last results is /tmp/ directory
    * in case that no property is set.
    */
-  private final String TMP_PATH =
-    System.getProperty("sqoop.integration.tmpdir", System.getProperty("java.io.tmpdir", "/tmp"))
-      + "/sqoop-cargo-tests/" + getClass().getName() + "/";
+  private static final String TMP_PATH_BASE =
+    System.getProperty("sqoop.integration.tmpdir", System.getProperty("java.io.tmpdir", "/tmp")) + "/sqoop-cargo-tests/";
+
+  /**
+   * Temporary directory that will be used by the test.
+   *
+   * We will take TMP_PATH_BASE and append two subdirectories. First will be named
+   * after fully qualified class name of current test class, second directory will
+   * be named after current test method name. For example:
+   *
+   * TMP_PATH_BASE/org.apache.sqoop.TestClass/testMethod/
+   */
+  private String tmpPath;
 
   /**
    * Tomcat based Sqoop mini cluster
@@ -69,9 +83,14 @@ abstract public class TomcatTestCase {
 
   @Before
   public void startServer() throws Exception {
+    // Set up the temporary path
+    tmpPath = TMP_PATH_BASE + getClass().getName() + "/" + name.getMethodName() + "/";
+
+    // Set up and start server
     cluster = new TomcatSqoopMiniCluster(getTemporaryPath());
     cluster.start();
 
+    // Initialize Sqoop Client API
     client = new SqoopClient(getServerUrl());
   }
 
@@ -90,7 +109,7 @@ abstract public class TomcatTestCase {
   }
 
   public String getTemporaryPath() {
-    return TMP_PATH;
+    return tmpPath;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/sqoop/blob/3fd8bdff/test/src/test/java/org/apache/sqoop/integration/connector/ConnectorTestCase.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/connector/ConnectorTestCase.java b/test/src/test/java/org/apache/sqoop/integration/connector/ConnectorTestCase.java
index 14f7dc4..837a1aa 100644
--- a/test/src/test/java/org/apache/sqoop/integration/connector/ConnectorTestCase.java
+++ b/test/src/test/java/org/apache/sqoop/integration/connector/ConnectorTestCase.java
@@ -18,11 +18,21 @@
 package org.apache.sqoop.integration.connector;
 
 import org.apache.log4j.Logger;
+import org.apache.sqoop.framework.configuration.OutputFormat;
+import org.apache.sqoop.framework.configuration.StorageType;
 import org.apache.sqoop.integration.TomcatTestCase;
+import org.apache.sqoop.model.MConnection;
+import org.apache.sqoop.model.MFormList;
+import org.apache.sqoop.model.MJob;
+import org.apache.sqoop.model.MPersistableEntity;
 import org.apache.sqoop.test.db.DatabaseProvider;
 import org.apache.sqoop.test.db.DatabaseProviderFactory;
-import org.junit.After;
-import org.junit.Before;
+import org.apache.sqoop.validation.Status;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotSame;
 
 /**
  * Base test case for connector testing.
@@ -33,17 +43,18 @@ abstract public class ConnectorTestCase extends TomcatTestCase {
 
   private static final Logger LOG = Logger.getLogger(ConnectorTestCase.class);
 
-  protected DatabaseProvider provider;
+  protected static DatabaseProvider provider;
 
-  @Before
-  public void startProvider() throws Exception {
+  @BeforeClass
+  public static void startProvider() throws Exception {
     provider = DatabaseProviderFactory.getProvider(System.getProperties());
     LOG.info("Starting database provider: " + provider.getClass().getName());
     provider.start();
   }
 
-  @After
-  public void stopProvider() {
+  @AfterClass
+  public static void stopProvider() {
+    LOG.info("Stopping database provider: " + provider.getClass().getName());
     provider.stop();
   }
 
@@ -62,4 +73,71 @@ abstract public class ConnectorTestCase extends TomcatTestCase {
   protected void insertRow(Object ...values) {
     provider.insertRow(getTableName(), values);
   }
+
+  /**
+   * Fill connection form based on currently active provider.
+   *
+   * @param connection MConnection object to fill
+   */
+  protected void fillConnectionForm(MConnection connection) {
+    MFormList forms = connection.getConnectorPart();
+    forms.getStringInput("connection.jdbcDriver").setValue(provider.getJdbcDriver());
+    forms.getStringInput("connection.connectionString").setValue(provider.getConnectionUrl());
+    forms.getStringInput("connection.username").setValue(provider.getConnectionUsername());
+    forms.getStringInput("connection.password").setValue(provider.getConnectionPassword());
+  }
+
+  /**
+   * Fill output form with specific storage and output type. Mapreduce output directory
+   * will be set to default test value.
+   *
+   * @param job MJOb object to fill
+   * @param storage Storage type that should be set
+   * @param output Output type that should be set
+   */
+  protected void fillOutputForm(MJob job, StorageType storage, OutputFormat output) {
+    MFormList forms = job.getFrameworkPart();
+    forms.getEnumInput("output.storageType").setValue(storage);
+    forms.getEnumInput("output.outputFormat").setValue(output);
+    forms.getStringInput("output.outputDirectory").setValue(getMapreduceDirectory());
+  }
+
+  /**
+   * Create table cities.
+   */
+  protected void createAndLoadTableCities() {
+    createTable("id",
+      "id", "int",
+      "country", "varchar(50)",
+      "city", "varchar(50)"
+    );
+    insertRow(1, "USA", "San Francisco");
+    insertRow(2, "USA", "Sunnyvale");
+    insertRow(3, "Czech Republic", "Brno");
+    insertRow(4, "USA", "Palo Alto");
+  }
+
+  /**
+   * Create connection.
+   *
+   * With asserts to make sure that it was created correctly.
+   *
+   * @param connection
+   */
+  protected void createConnection(MConnection connection) {
+    assertEquals(Status.FINE, getClient().createConnection(connection));
+    assertNotSame(MPersistableEntity.PERSISTANCE_ID_DEFAULT, connection.getPersistenceId());
+  }
+
+ /**
+   * Create job.
+   *
+   * With asserts to make sure that it was created correctly.
+   *
+   * @param job
+   */
+ protected void createJob(MJob job) {
+    assertEquals(Status.FINE, getClient().createJob(job));
+    assertNotSame(MPersistableEntity.PERSISTANCE_ID_DEFAULT, job.getPersistenceId());
+  }
 }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/3fd8bdff/test/src/test/java/org/apache/sqoop/integration/connector/jdbc/generic/TableImportTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/sqoop/integration/connector/jdbc/generic/TableImportTest.java b/test/src/test/java/org/apache/sqoop/integration/connector/jdbc/generic/TableImportTest.java
index 05a799c..adcfbaf 100644
--- a/test/src/test/java/org/apache/sqoop/integration/connector/jdbc/generic/TableImportTest.java
+++ b/test/src/test/java/org/apache/sqoop/integration/connector/jdbc/generic/TableImportTest.java
@@ -24,78 +24,87 @@ import org.apache.sqoop.integration.connector.ConnectorTestCase;
 import org.apache.sqoop.model.MConnection;
 import org.apache.sqoop.model.MFormList;
 import org.apache.sqoop.model.MJob;
-import org.apache.sqoop.model.MPersistableEntity;
 import org.apache.sqoop.model.MSubmission;
-import org.apache.sqoop.validation.Status;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertTrue;
 
 /**
- * Proof of concept implementation of first "real" integration test.
- *
- * Will be improved when client API will be created.
+ * Import simple table with various configurations.
  */
 public class TableImportTest extends ConnectorTestCase {
 
   private static final Logger LOG = Logger.getLogger(TableImportTest.class);
 
-  /**
-   * This test is proof of concept.
-   *
-   * It will be refactored once we will create reasonable client interface.
-   */
   @Test
-  public void testBasicTableImport() throws Exception {
-    createTable("id",
-      "id", "int",
-      "txt", "varchar(50)"
-    );
-    insertRow(1, "San Francisco");
-    insertRow(2, "Sunnyvale");
-    insertRow(3, "Brno");
-
-    // Connection creation and job submission will be refactored once
-    // the client API for embedding Sqoop client will be ready.
+  public void testBasicImport() throws Exception {
+    createAndLoadTableCities();
 
     // Connection creation
     MConnection connection = getClient().newConnection(1L);
+    fillConnectionForm(connection);
+    createConnection(connection);
+
+    // Job creation
+    MJob job = getClient().newJob(connection.getPersistenceId(), MJob.Type.IMPORT);
 
-    MFormList forms;
     // Connector values
-    forms = connection.getConnectorPart();
-    forms.getStringInput("connection.jdbcDriver").setValue(provider.getJdbcDriver());
-    forms.getStringInput("connection.connectionString").setValue(provider.getConnectionUrl());
-    forms.getStringInput("connection.username").setValue(provider.getConnectionUsername());
-    forms.getStringInput("connection.password").setValue(provider.getConnectionPassword());
+    MFormList forms = job.getConnectorPart();
+    forms.getStringInput("table.tableName").setValue(provider.escapeTableName(getTableName()));
+    forms.getStringInput("table.partitionColumn").setValue(provider.escapeColumnName("id"));
     // Framework values
-    // No need to set anything
+    fillOutputForm(job, StorageType.HDFS, OutputFormat.TEXT_FILE);
+    createJob(job);
 
-    assertEquals(Status.FINE, getClient().createConnection(connection));
-    assertNotSame(MPersistableEntity.PERSISTANCE_ID_DEFAULT, connection.getPersistenceId());
+    MSubmission submission = getClient().startSubmission(job.getPersistenceId());
+    assertTrue(submission.getStatus().isRunning());
+
+    // Wait until the job finish - this active waiting will be removed once
+    // Sqoop client API will get blocking support.
+    do {
+      Thread.sleep(5000);
+      submission = getClient().getSubmissionStatus(job.getPersistenceId());
+    } while(submission.getStatus().isRunning());
+
+    // Assert correct output
+    assertMapreduceOutput(
+      "1,'USA','San Francisco'",
+      "2,'USA','Sunnyvale'",
+      "3,'Czech Republic','Brno'",
+      "4,'USA','Palo Alto'"
+    );
+
+    // Clean up testing table
+    dropTable();
+  }
+
+  @Test
+  public void testColumns() throws Exception {
+    createAndLoadTableCities();
+
+    // Connection creation
+    MConnection connection = getClient().newConnection(1L);
+    fillConnectionForm(connection);
+
+    createConnection(connection);
 
     // Job creation
     MJob job = getClient().newJob(connection.getPersistenceId(), MJob.Type.IMPORT);
 
     // Connector values
-    forms = job.getConnectorPart();
+    MFormList forms = job.getConnectorPart();
     forms.getStringInput("table.tableName").setValue(provider.escapeTableName(getTableName()));
     forms.getStringInput("table.partitionColumn").setValue(provider.escapeColumnName("id"));
+    forms.getStringInput("table.columns").setValue(provider.escapeColumnName("id") + "," + provider.escapeColumnName("country"));
     // Framework values
-    forms = job.getFrameworkPart();
-    forms.getEnumInput("output.storageType").setValue(StorageType.HDFS);
-    forms.getEnumInput("output.outputFormat").setValue(OutputFormat.TEXT_FILE);
-    forms.getStringInput("output.outputDirectory").setValue(getMapreduceDirectory());
-
-    assertEquals(Status.FINE, getClient().createJob(job));
-    assertNotSame(MPersistableEntity.PERSISTANCE_ID_DEFAULT, job.getPersistenceId());
+    fillOutputForm(job, StorageType.HDFS, OutputFormat.TEXT_FILE);
+    createJob(job);
 
     MSubmission submission = getClient().startSubmission(job.getPersistenceId());
     assertTrue(submission.getStatus().isRunning());
 
-    // Wait until the job finish
+    // Wait until the job finish - this active waiting will be removed once
+    // Sqoop client API will get blocking support.
     do {
       Thread.sleep(5000);
       submission = getClient().getSubmissionStatus(job.getPersistenceId());
@@ -103,13 +112,13 @@ public class TableImportTest extends ConnectorTestCase {
 
     // Assert correct output
     assertMapreduceOutput(
-      "1,'San Francisco'",
-      "2,'Sunnyvale'",
-      "3,'Brno'"
+      "1,'USA'",
+      "2,'USA'",
+      "3,'Czech Republic'",
+      "4,'USA'"
     );
 
     // Clean up testing table
     dropTable();
   }
-
 }