You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by ve...@apache.org on 2015/12/17 05:50:13 UTC

[1/3] sqoop git commit: SQOOP-2745: Using datetime column as a splitter for Oracle no longer works (Jarek Jarcec Cecho via Venkat Ranganathan)

Repository: sqoop
Updated Branches:
  refs/heads/trunk e077f9978 -> f19e2a523


SQOOP-2745: Using datetime column as a splitter for Oracle no longer works
  (Jarek Jarcec Cecho via Venkat Ranganathan)


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

Branch: refs/heads/trunk
Commit: 9c7638d74180cc607ef509b9dd9e6a45ff60c041
Parents: e077f99
Author: Venkat Ranganathan <ve...@hortonworks.com>
Authored: Wed Dec 16 20:47:59 2015 -0800
Committer: Venkat Ranganathan <ve...@hortonworks.com>
Committed: Wed Dec 16 20:47:59 2015 -0800

----------------------------------------------------------------------
 .../db/OracleDataDrivenDBInputFormat.java       |   4 +-
 .../com/cloudera/sqoop/ThirdPartyTests.java     |   2 +
 .../manager/oracle/OracleSplitterTest.java      | 147 +++++++++++++++++++
 3 files changed, 151 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/9c7638d7/src/java/org/apache/sqoop/mapreduce/db/OracleDataDrivenDBInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/sqoop/mapreduce/db/OracleDataDrivenDBInputFormat.java b/src/java/org/apache/sqoop/mapreduce/db/OracleDataDrivenDBInputFormat.java
index 15c5898..8b5103b 100644
--- a/src/java/org/apache/sqoop/mapreduce/db/OracleDataDrivenDBInputFormat.java
+++ b/src/java/org/apache/sqoop/mapreduce/db/OracleDataDrivenDBInputFormat.java
@@ -45,7 +45,7 @@ public class OracleDataDrivenDBInputFormat<T extends DBWritable>
    * into InputSplits.
    */
   @Override
-  protected DBSplitter getSplitter(int sqlDataType) {
+  protected DBSplitter getSplitter(int sqlDataType, long splitLimit) {
     switch (sqlDataType) {
     case Types.DATE:
     case Types.TIME:
@@ -53,7 +53,7 @@ public class OracleDataDrivenDBInputFormat<T extends DBWritable>
       return new OracleDateSplitter();
 
     default:
-      return super.getSplitter(sqlDataType);
+      return super.getSplitter(sqlDataType, splitLimit);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/9c7638d7/src/test/com/cloudera/sqoop/ThirdPartyTests.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/ThirdPartyTests.java b/src/test/com/cloudera/sqoop/ThirdPartyTests.java
index 06b4ae6..50f3192 100644
--- a/src/test/com/cloudera/sqoop/ThirdPartyTests.java
+++ b/src/test/com/cloudera/sqoop/ThirdPartyTests.java
@@ -57,6 +57,7 @@ import org.apache.sqoop.manager.netezza.NetezzaExportManualTest;
 import org.apache.sqoop.manager.netezza.NetezzaImportManualTest;
 import org.apache.sqoop.manager.oracle.OracleCallExportTest;
 import org.apache.sqoop.manager.oracle.OracleIncrementalImportTest;
+import org.apache.sqoop.manager.oracle.OracleSplitterTest;
 import org.apache.sqoop.manager.sqlserver.SQLServerDatatypeExportDelimitedFileManualTest;
 import org.apache.sqoop.manager.sqlserver.SQLServerDatatypeExportSequenceFileManualTest;
 import org.apache.sqoop.manager.sqlserver.SQLServerDatatypeImportDelimitedFileManualTest;
@@ -96,6 +97,7 @@ public final class ThirdPartyTests extends TestCase {
     suite.addTestSuite(OracleManagerTest.class);
     suite.addTestSuite(OracleCompatTest.class);
     suite.addTestSuite(OracleIncrementalImportTest.class);
+    suite.addTestSuite(OracleSplitterTest.class);
 
     // SQL Server
     suite.addTestSuite(SQLServerDatatypeExportDelimitedFileManualTest.class);

http://git-wip-us.apache.org/repos/asf/sqoop/blob/9c7638d7/src/test/org/apache/sqoop/manager/oracle/OracleSplitterTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/OracleSplitterTest.java b/src/test/org/apache/sqoop/manager/oracle/OracleSplitterTest.java
new file mode 100644
index 0000000..fc5f56b
--- /dev/null
+++ b/src/test/org/apache/sqoop/manager/oracle/OracleSplitterTest.java
@@ -0,0 +1,147 @@
+/**
+ * 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.manager.oracle;
+
+import com.cloudera.sqoop.SqoopOptions;
+import com.cloudera.sqoop.manager.OracleUtils;
+import com.cloudera.sqoop.testutil.CommonArgs;
+import com.cloudera.sqoop.testutil.ImportJobTestCase;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Writer;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Test various custom splitters for Oracle.
+ */
+public class OracleSplitterTest extends ImportJobTestCase {
+
+  public static final Log LOG = LogFactory.getLog(
+      OracleSplitterTest.class.getName());
+
+  @Override
+  protected boolean useHsqldbTestServer() {
+    return false;
+  }
+
+  @Override
+  protected String getConnectString() {
+    return OracleUtils.CONNECT_STRING;
+  }
+
+  @Override
+  protected SqoopOptions getSqoopOptions(Configuration conf) {
+    SqoopOptions opts = new SqoopOptions(conf);
+    OracleUtils.setOracleAuth(opts);
+    return opts;
+  }
+
+  @Override
+  protected void dropTableIfExists(String table) throws SQLException {
+    OracleUtils.dropTable(table, getManager());
+  }
+
+  /** the names of the tables we're creating. */
+  private List<String> tableNames;
+
+  @Override
+  public void tearDown() {
+    // Clean up the database on our way out.
+    for (String tableName : tableNames) {
+      try {
+        dropTableIfExists(tableName);
+      } catch (SQLException e) {
+        LOG.warn("Error trying to drop table '" + tableName
+                 + "' on tearDown: " + e);
+      }
+    }
+    super.tearDown();
+  }
+
+  protected String [] getArgv(String tableName, String connPropsFileName, String splitByColumn) {
+    ArrayList<String> args = new ArrayList<String>();
+
+    CommonArgs.addHadoopFlags(args);
+
+    args.add("--connect");
+    args.add(getConnectString());
+    args.add("--target-dir");
+    args.add(getWarehouseDir());
+    args.add("--num-mappers");
+    args.add("2");
+    args.add("--split-by");
+    args.add(splitByColumn);
+    args.add("--table");
+    args.add(tableName);
+    args.add("--connection-param-file");
+    args.add(connPropsFileName);
+
+    return args.toArray(new String[0]);
+  }
+
+  public void testTimestampSplitter() throws IOException {
+    tableNames = new ArrayList<String>();
+    String [] types = { "INT", "VARCHAR(10)", "TIMESTAMP", };
+    String [] vals = {
+      "1", "'old_data'", "TO_TIMESTAMP('1999-01-01 11:11:11', 'YYYY-MM-DD HH24:MI:SS')",
+      "2", "'new_data'", "TO_TIMESTAMP('2000-11-11 23:23:23', 'YYYY-MM-DD HH24:MI:SS')",
+    };
+    String tableName = getTableName();
+    tableNames.add(tableName);
+    createTableWithColTypes(types, vals);
+    // Some version of Oracle's jdbc drivers automatically convert date to
+    // timestamp. Since we don't want this to happen for this test,
+    // we must explicitly use a property file to control this behavior.
+    String connPropsFileName = "connection.properties";
+    FileUtils.writeStringToFile(new File(connPropsFileName), "oracle.jdbc.mapDateToTimestamp=false");
+    String[] args = getArgv(tableName, connPropsFileName, getColName(2));
+    runImport(args);
+
+    File file;
+    List<String> lines;
+
+    // First row should be in the first file
+    file = new File(this.getWarehouseDir(), "part-m-00000");
+    lines = FileUtils.readLines(file, "UTF-8");
+    assertEquals(1, lines.size());
+    assertEquals("1,old_data,1999-01-01 11:11:11.0", lines.get(0));
+
+    // With second line in the second file
+    file = new File(this.getWarehouseDir(), "part-m-00001");
+    lines = FileUtils.readLines(file, "UTF-8");
+    assertEquals(1, lines.size());
+    assertEquals("2,new_data,2000-11-11 23:23:23.0", lines.get(0));
+  }
+}


[3/3] sqoop git commit: SQOOP-2747: Allow customizing test username and password for Oracle tests (Jarek Jarcec Cecho via Venkat Ranganathan)

Posted by ve...@apache.org.
SQOOP-2747: Allow customizing test username and password for Oracle tests
  (Jarek Jarcec Cecho via Venkat Ranganathan)


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

Branch: refs/heads/trunk
Commit: f19e2a523579db8c28a96febfd3cf35a5d58adc6
Parents: e4f12a9
Author: Venkat Ranganathan <ve...@hortonworks.com>
Authored: Wed Dec 16 20:49:31 2015 -0800
Committer: Venkat Ranganathan <ve...@hortonworks.com>
Committed: Wed Dec 16 20:49:31 2015 -0800

----------------------------------------------------------------------
 build.xml                                            | 11 +++++++++--
 src/test/com/cloudera/sqoop/manager/OracleUtils.java |  8 +++-----
 2 files changed, 12 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/f19e2a52/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index 148cc6d..97e5502 100644
--- a/build.xml
+++ b/build.xml
@@ -350,8 +350,9 @@
       value="${name}/[conf]/[artifact]-[revision](-[classifier]).[ext]"/>
 
   <!--test related properties -->
-  <property name="sqoop.test.oracle.connectstring"
-            value="jdbc:oracle:thin:@//localhost/xe"/>
+  <property name="sqoop.test.oracle.connectstring" value="jdbc:oracle:thin:@//localhost/xe"/>
+  <property name="sqoop.test.oracle.username" value="SQOOPTEST"/>
+  <property name="sqoop.test.oracle.password" value="12345"/>
 
   <property name="sqoop.test.mysql.connectstring.host_url"
             value="jdbc:mysql://localhost/"/>
@@ -954,6 +955,12 @@
       <sysproperty key="sqoop.test.oracle.connectstring"
                    value="${sqoop.test.oracle.connectstring}"/>
 
+      <sysproperty key="sqoop.test.oracle.username"
+                   value="${sqoop.test.oracle.username}"/>
+
+      <sysproperty key="sqoop.test.oracle.password"
+                   value="${sqoop.test.oracle.password}"/>
+
       <sysproperty key="sqoop.test.mysql.connectstring.host_url"
                    value="${sqoop.test.mysql.connectstring.host_url}"/>
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/f19e2a52/src/test/com/cloudera/sqoop/manager/OracleUtils.java
----------------------------------------------------------------------
diff --git a/src/test/com/cloudera/sqoop/manager/OracleUtils.java b/src/test/com/cloudera/sqoop/manager/OracleUtils.java
index d9e31b8..655aa61 100644
--- a/src/test/com/cloudera/sqoop/manager/OracleUtils.java
+++ b/src/test/com/cloudera/sqoop/manager/OracleUtils.java
@@ -37,11 +37,9 @@ public final class OracleUtils {
   // Express edition hardcoded name.
   public static final String ORACLE_DATABASE_NAME = "xe";
 
-  public static final String CONNECT_STRING =
-      System.getProperty("sqoop.test.oracle.connectstring",
-      "jdbc:oracle:thin:@//localhost/" + ORACLE_DATABASE_NAME);
-  public static final String ORACLE_USER_NAME = "SQOOPTEST";
-  public static final String ORACLE_USER_PASS = "12345";
+  public static final String CONNECT_STRING = System.getProperty("sqoop.test.oracle.connectstring", "jdbc:oracle:thin:@//localhost/" + ORACLE_DATABASE_NAME);
+  public static final String ORACLE_USER_NAME = System.getProperty("sqoop.test.oracle.username", "SQOOPTEST");
+  public static final String ORACLE_USER_PASS = System.getProperty("sqoop.test.oracle.password", "12345");
 
   public static final String ORACLE_SECONDARY_USER_NAME = "SQOOPTEST2";
   public static final String ORACLE_SECONDARY_USER_PASS = "ABCDEF";


[2/3] sqoop git commit: SQOOP-2746: Add test case for Oracle incremental import using Timestamp (Jarek Jarcec Cecho via Venkat Ranganathan)

Posted by ve...@apache.org.
SQOOP-2746: Add test case for Oracle incremental import using Timestamp
  (Jarek Jarcec Cecho via Venkat Ranganathan)


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

Branch: refs/heads/trunk
Commit: e4f12a99bfd01d25666e32c9a753b4ff42aef583
Parents: 9c7638d
Author: Venkat Ranganathan <ve...@hortonworks.com>
Authored: Wed Dec 16 20:48:46 2015 -0800
Committer: Venkat Ranganathan <ve...@hortonworks.com>
Committed: Wed Dec 16 20:48:46 2015 -0800

----------------------------------------------------------------------
 .../oracle/OracleIncrementalImportTest.java     | 29 ++++++++++++++++++++
 1 file changed, 29 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/e4f12a99/src/test/org/apache/sqoop/manager/oracle/OracleIncrementalImportTest.java
----------------------------------------------------------------------
diff --git a/src/test/org/apache/sqoop/manager/oracle/OracleIncrementalImportTest.java b/src/test/org/apache/sqoop/manager/oracle/OracleIncrementalImportTest.java
index 3bbb1b1..e596646 100644
--- a/src/test/org/apache/sqoop/manager/oracle/OracleIncrementalImportTest.java
+++ b/src/test/org/apache/sqoop/manager/oracle/OracleIncrementalImportTest.java
@@ -108,6 +108,8 @@ public class OracleIncrementalImportTest extends ImportJobTestCase {
     args.add(getWarehouseDir());
     args.add("--num-mappers");
     args.add("1");
+    args.add("--split-by");
+    args.add(checkColumnName);
     args.add("--table");
     args.add(tableName);
     args.add("--incremental");
@@ -154,6 +156,33 @@ public class OracleIncrementalImportTest extends ImportJobTestCase {
                  expectedVal, output);
   }
 
+  public void testIncrementalImportWithLastModifiedTimestamp() throws IOException {
+    tableNames = new ArrayList<String>();
+    String [] types = { "INT", "VARCHAR(10)", "TIMESTAMP", };
+    String [] vals = {
+        "1", "'old_data'",
+        "TO_TIMESTAMP('1999-01-01 11:11:11', 'YYYY-MM-DD HH24:MI:SS')",
+        "2", "'new_data'",
+        "TO_TIMESTAMP('2000-11-11 23:23:23', 'YYYY-MM-DD HH24:MI:SS')", };
+    String tableName = getTableName();
+    tableNames.add(tableName);
+    createTableWithColTypes(types, vals);
+    // Some version of Oracle's jdbc drivers automatically convert date to
+    // timestamp. Since we don't want this to happen for this test,
+    // we must explicitly use a property file to control this behavior.
+    String connPropsFileName = "connection.properties";
+    createFileWithContent(connPropsFileName, "oracle.jdbc.mapDateToTimestamp=false");
+    String[] args = getArgv(tableName, connPropsFileName, getColName(2));
+    runImport(args);
+
+    Path warehousePath = new Path(this.getWarehouseDir());
+    Path filePath = new Path(warehousePath, "part-m-00000");
+    String output = readLineFromPath(filePath);
+    String expectedVal = "2,new_data,2000-11-11 23:23:23.0";
+    assertEquals("Incremental import result expected a different string",
+                 expectedVal, output);
+  }
+
   private void createFileWithContent(String connPropsFileName,
       String fileContent) throws IOException {
     File file = new File(connPropsFileName);