You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ch...@apache.org on 2013/03/02 01:07:21 UTC

git commit: SQOOP-908: Create MySQL and PostgreSQL database providers

Updated Branches:
  refs/heads/sqoop2 2642b66cf -> f64dba2e0


SQOOP-908: Create MySQL and PostgreSQL database providers

(Jarcec Cecho via Cheolsoo Park)


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

Branch: refs/heads/sqoop2
Commit: f64dba2e01e61e92ae6fd3bdcb6e794f198f41ae
Parents: 2642b66
Author: Cheolsoo Park <ch...@apache.org>
Authored: Fri Mar 1 16:06:20 2013 -0800
Committer: Cheolsoo Park <ch...@apache.org>
Committed: Fri Mar 1 16:06:20 2013 -0800

----------------------------------------------------------------------
 pom.xml                                            |   12 ++
 test/pom.xml                                       |   10 ++
 .../org/apache/sqoop/test/db/DatabaseProvider.java |    8 ++
 .../sqoop/test/db/DatabaseProviderFactory.java     |   39 +++++++
 .../org/apache/sqoop/test/db/DerbyProvider.java    |    7 +-
 .../org/apache/sqoop/test/db/MySQLProvider.java    |   83 +++++++++++++++
 .../apache/sqoop/test/db/PostgreSQLProvider.java   |   83 +++++++++++++++
 .../test/minicluster/TomcatSqoopMiniCluster.java   |    2 +
 .../apache/sqoop/integration/TomcatTestCase.java   |    8 +-
 .../integration/connector/ConnectorTestCase.java   |   10 ++-
 .../connector/jdbc/generic/TableImportTest.java    |   11 ++-
 11 files changed, 262 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b569473..0620d08 100644
--- a/pom.xml
+++ b/pom.xml
@@ -106,6 +106,8 @@ limitations under the License.
     <tomcat.major.version>6</tomcat.major.version>
     <tomcat.minor.version>0.36</tomcat.minor.version>
     <tomcat.version>${tomcat.major.version}.${tomcat.minor.version}</tomcat.version>
+    <jdbc.mysql.version>5.1.23</jdbc.mysql.version>
+    <jdbc.postgresql.version>9.1-901.jdbc4</jdbc.postgresql.version>
   </properties>
 
   <dependencies>
@@ -350,6 +352,16 @@ limitations under the License.
         <artifactId>cargo-core-container-tomcat</artifactId>
         <version>${cargo.version}</version>
       </dependency>
+      <dependency>
+        <groupId>mysql</groupId>
+        <artifactId>mysql-connector-java</artifactId>
+        <version>${jdbc.mysql.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>postgresql</groupId>
+        <artifactId>postgresql</artifactId>
+        <version>${jdbc.postgresql.version}</version>
+      </dependency>
    </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index fe47ff7..26aa556 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -83,6 +83,16 @@ limitations under the License.
       <artifactId>derbyclient</artifactId>
     </dependency>
 
+    <dependency>
+      <groupId>mysql</groupId>
+      <artifactId>mysql-connector-java</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>postgresql</groupId>
+      <artifactId>postgresql</artifactId>
+    </dependency>
+
   </dependencies>
 
   <!-- Add classifier name to the JAR name -->

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/test/src/main/java/org/apache/sqoop/test/db/DatabaseProvider.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/db/DatabaseProvider.java b/test/src/main/java/org/apache/sqoop/test/db/DatabaseProvider.java
index 364ff61..dce1043 100644
--- a/test/src/main/java/org/apache/sqoop/test/db/DatabaseProvider.java
+++ b/test/src/main/java/org/apache/sqoop/test/db/DatabaseProvider.java
@@ -104,10 +104,18 @@ abstract public class DatabaseProvider {
     return "NULL";
   }
 
+  public String getJdbcDriver() {
+    return null;
+  }
+
   /**
    * Start the handler.
    */
   public void start() {
+    if(getJdbcDriver() != null) {
+      loadClass(getJdbcDriver());
+    }
+
     // Create connection to the database server
     try {
       setConnection(DriverManager.getConnection(getConnectionUrl(), getConnectionUsername(), getConnectionPassword()));

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/test/src/main/java/org/apache/sqoop/test/db/DatabaseProviderFactory.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/db/DatabaseProviderFactory.java b/test/src/main/java/org/apache/sqoop/test/db/DatabaseProviderFactory.java
new file mode 100644
index 0000000..bc430f1
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/db/DatabaseProviderFactory.java
@@ -0,0 +1,39 @@
+/**
+ * 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.db;
+
+import java.util.Properties;
+
+/**
+ * Create database provider.
+ */
+public class DatabaseProviderFactory {
+
+  public static final String PROVIDER_PROPERTY = "sqoop.provider.class";
+
+  public static DatabaseProvider getProvider(Properties properties) throws ClassNotFoundException, IllegalAccessException, InstantiationException {
+    String className = properties.getProperty(PROVIDER_PROPERTY);
+    if(className == null) {
+      return new DerbyProvider();
+    }
+
+    Class klass = Class.forName(className);
+    return (DatabaseProvider)klass.newInstance();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/test/src/main/java/org/apache/sqoop/test/db/DerbyProvider.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/db/DerbyProvider.java b/test/src/main/java/org/apache/sqoop/test/db/DerbyProvider.java
index 402fab9..aa6b897 100644
--- a/test/src/main/java/org/apache/sqoop/test/db/DerbyProvider.java
+++ b/test/src/main/java/org/apache/sqoop/test/db/DerbyProvider.java
@@ -46,8 +46,6 @@ public class DerbyProvider extends DatabaseProvider {
       throw new RuntimeException("Can't derby server", e);
     }
 
-    // Load JDBC driver and create connection
-    loadClass(DRIVER);
     super.start();
   }
 
@@ -83,6 +81,11 @@ public class DerbyProvider extends DatabaseProvider {
   }
 
   @Override
+  public String getJdbcDriver() {
+    return DRIVER;
+  }
+
+  @Override
   public String getConnectionUrl() {
     return "jdbc:derby://localhost:1527/memory:sqoop;create=true";
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/test/src/main/java/org/apache/sqoop/test/db/MySQLProvider.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/db/MySQLProvider.java b/test/src/main/java/org/apache/sqoop/test/db/MySQLProvider.java
new file mode 100644
index 0000000..4f21935
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/db/MySQLProvider.java
@@ -0,0 +1,83 @@
+/**
+ * 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.db;
+
+/**
+ * MySQL Provider that will connect to remote MySQL server.
+ *
+ * JDBC can be configured via system properties. Default value is server running
+ * on the same box (localhost) that is access via sqoop/sqoop credentials.
+ */
+public class MySQLProvider extends DatabaseProvider {
+
+  public static final String DRIVER = "com.mysql.jdbc.Driver";
+
+  private static final String CONNECTION = System.getProperties().getProperty(
+    "sqoop.provider.mysql.jdbc",
+    "jdbc:mysql://localhost/test"
+  );
+
+  private static final String USERNAME = System.getProperties().getProperty(
+    "sqoop.provider.mysql.username",
+    "sqoop"
+  );
+
+  private static final String PASSWORD = System.getProperties().getProperty(
+    "sqoop.provider.mysql.password",
+    "sqoop"
+  );
+
+  @Override
+  public String getConnectionUrl() {
+    return CONNECTION;
+  }
+
+  @Override
+  public String getConnectionUsername() {
+    return USERNAME;
+  }
+
+  @Override
+  public String getConnectionPassword() {
+    return PASSWORD;
+  }
+
+  @Override
+  public String escapeColumnName(String columnName) {
+    return escape(columnName);
+  }
+
+  @Override
+  public String escapeTableName(String tableName) {
+    return escape(tableName);
+  }
+
+  @Override
+  public String escapeValueString(String value) {
+    return "\"" + value + "\"";
+  }
+
+  @Override
+  public String getJdbcDriver() {
+    return DRIVER;
+  }
+
+  public String escape(String entity) {
+    return "`" + entity + "`";
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/test/src/main/java/org/apache/sqoop/test/db/PostgreSQLProvider.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/sqoop/test/db/PostgreSQLProvider.java b/test/src/main/java/org/apache/sqoop/test/db/PostgreSQLProvider.java
new file mode 100644
index 0000000..3259768
--- /dev/null
+++ b/test/src/main/java/org/apache/sqoop/test/db/PostgreSQLProvider.java
@@ -0,0 +1,83 @@
+/**
+ * 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.db;
+
+/**
+ * PostgreSQL Provider that will connect to remote PostgreSQL server.
+ *
+ * JDBC can be configured via system properties. Default value is server running
+ * on the same box (localhost) that is access via sqoop/sqoop credentials.
+ */
+public class PostgreSQLProvider extends DatabaseProvider {
+
+  public static final String DRIVER = "org.postgresql.Driver";
+
+  private static final String CONNECTION = System.getProperties().getProperty(
+    "sqoop.provider.postgresql.jdbc",
+    "jdbc:postgresql://localhost/test"
+  );
+
+  private static final String USERNAME = System.getProperties().getProperty(
+    "sqoop.provider.postgresql.username",
+    "sqoop"
+  );
+
+  private static final String PASSWORD = System.getProperties().getProperty(
+    "sqoop.provider.postgresql.password",
+    "sqoop"
+  );
+
+  @Override
+  public String getConnectionUrl() {
+    return CONNECTION;
+  }
+
+  @Override
+  public String getConnectionUsername() {
+    return USERNAME;
+  }
+
+  @Override
+  public String getConnectionPassword() {
+    return PASSWORD;
+  }
+
+  @Override
+  public String escapeColumnName(String columnName) {
+    return escape(columnName);
+  }
+
+  @Override
+  public String escapeTableName(String tableName) {
+    return escape(tableName);
+  }
+
+  @Override
+  public String escapeValueString(String value) {
+    return "'" + value + "'";
+  }
+
+  @Override
+  public String getJdbcDriver() {
+    return DRIVER;
+  }
+
+  public String escape(String entity) {
+    return "\"" + entity + "\"";
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/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 ed6c596..567ddbc 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
@@ -93,6 +93,8 @@ public class TomcatSqoopMiniCluster extends SqoopMiniCluster {
          jar.contains("jackson-") || // Jackson
          jar.contains("derby")    || // Derby drivers
          jar.contains("avro-")    || // Avro
+         jar.contains("mysql")    || // MySQL JDBC driver
+         jar.contains("postgre")  || // PostgreSQL JDBC driver
          jar.contains("google")      // Google libraries (guava, ...)
        ) {
         extraClassPath.add(jar);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/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 7c8a978..8b46179 100644
--- a/test/src/test/java/org/apache/sqoop/integration/TomcatTestCase.java
+++ b/test/src/test/java/org/apache/sqoop/integration/TomcatTestCase.java
@@ -63,7 +63,7 @@ abstract public class TomcatTestCase {
 
   @Before
   public void startServer() throws Exception {
-    cluster = new TomcatSqoopMiniCluster(TMP_PATH);
+    cluster = new TomcatSqoopMiniCluster(getTemporaryPath());
     cluster.start();
   }
 
@@ -72,6 +72,10 @@ abstract public class TomcatTestCase {
     cluster.stop();
   }
 
+  public String getTemporaryPath() {
+    return TMP_PATH;
+  }
+
   /**
    * Return testing server URL
    *
@@ -87,7 +91,7 @@ abstract public class TomcatTestCase {
    * @return
    */
   public String getMapreduceDirectory() {
-    return cluster.getTemporaryPath() + "/mapreduce-job-io";
+    return getTemporaryPath() + "/mapreduce-job-io";
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/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 595810f..14f7dc4 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
@@ -17,9 +17,10 @@
  */
 package org.apache.sqoop.integration.connector;
 
+import org.apache.log4j.Logger;
 import org.apache.sqoop.integration.TomcatTestCase;
 import org.apache.sqoop.test.db.DatabaseProvider;
-import org.apache.sqoop.test.db.DerbyProvider;
+import org.apache.sqoop.test.db.DatabaseProviderFactory;
 import org.junit.After;
 import org.junit.Before;
 
@@ -30,11 +31,14 @@ import org.junit.Before;
  */
 abstract public class ConnectorTestCase extends TomcatTestCase {
 
+  private static final Logger LOG = Logger.getLogger(ConnectorTestCase.class);
+
   protected DatabaseProvider provider;
 
   @Before
-  public void startProvider() {
-    provider = new DerbyProvider();
+  public void startProvider() throws Exception {
+    provider = DatabaseProviderFactory.getProvider(System.getProperties());
+    LOG.info("Starting database provider: " + provider.getClass().getName());
     provider.start();
   }
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f64dba2e/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 3a8b1b5..20588d3 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
@@ -36,12 +36,13 @@ import org.apache.sqoop.model.MFramework;
 import org.apache.sqoop.model.MJob;
 import org.apache.sqoop.model.MStringInput;
 import org.apache.sqoop.model.MSubmission;
-import org.apache.sqoop.test.db.DerbyProvider;
 import org.apache.sqoop.validation.Status;
+import org.apache.sqoop.validation.Validation;
 import org.junit.Test;
 
+import java.util.Map;
+
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
@@ -82,8 +83,10 @@ public class TableImportTest extends ConnectorTestCase {
                                              framework.getConnectionForms());
 
     // Connector values
-    ((MStringInput) (connection.getConnectorPart().getForms().get(0).getInputs().get(0))).setValue(DerbyProvider.DRIVER);
+    ((MStringInput) (connection.getConnectorPart().getForms().get(0).getInputs().get(0))).setValue(provider.getJdbcDriver());
     ((MStringInput) (connection.getConnectorPart().getForms().get(0).getInputs().get(1))).setValue(provider.getConnectionUrl());
+    ((MStringInput) (connection.getConnectorPart().getForms().get(0).getInputs().get(2))).setValue(provider.getConnectionUsername());
+    ((MStringInput) (connection.getConnectorPart().getForms().get(0).getInputs().get(3))).setValue(provider.getConnectionPassword());
     // Framework values
     // No need to set anything
 
@@ -124,7 +127,7 @@ public class TableImportTest extends ConnectorTestCase {
 
     // Wait until the job finish
     do {
-      Thread.sleep(1000);
+      Thread.sleep(5000);
       submission = submissionRequest.read(getServerUrl(), "" + job.getPersistenceId()).getSubmission();
     } while(submission.getStatus().isRunning());