Uploaded image for project: 'Sqoop (Retired)'
  1. Sqoop (Retired)
  2. SQOOP-3003

Sqoop import fails to query with split-by/boundary-query using Oracle Date/Timestamp

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Patch Available
    • Major
    • Resolution: Unresolved
    • None
    • None
    • connectors/oracle
    • None

    Description

      Given the following example sqoop import command intended to import data from an Oracle test_table, split-by a timestamp_column using a boundary query (e.g. one-day range) with sqoop parallelism of eight:

      sqoop import --connect jdbc:oracle:... --username <username> --password <pwd> --target-dir /tmp/sqoop/test -m 8 --null-string '' --append --query "SELECT primary_key, TO_CHAR(timestamp_column) FROM test_table  WHERE  primary_key != 0 AND \$CONDITIONS" --split-by "timestamp_column" --boundary-query "SELECT TO_TIMESTAMP('1970-01-01', 'yyyy-mm-dd')+numtodsinterval(1472083200,'second'), TO_TIMESTAMP('1970-01-01', 'yyyy-mm-dd')+numtodsinterval(1472169600,'second') FROM DUAL"
      

      The following exception is thrown by each map tasks:

      Caused by: java.sql.SQLDataException: ORA-01843: not a valid month
      
      	at oracle.jdbc.driver.T4CTTIoer.processError(T4CTTIoer.java:447)
      	at oracle.jdbc.driver.T4CTTIoer.processError(T4CTTIoer.java:396)
      	at oracle.jdbc.driver.T4C8Oall.processError(T4C8Oall.java:951)
      	at oracle.jdbc.driver.T4CTTIfun.receive(T4CTTIfun.java:513)
      	at oracle.jdbc.driver.T4CTTIfun.doRPC(T4CTTIfun.java:227)
      	at oracle.jdbc.driver.T4C8Oall.doOALL(T4C8Oall.java:531)
      	at oracle.jdbc.driver.T4CPreparedStatement.doOall8(T4CPreparedStatement.java:208)
      	at oracle.jdbc.driver.T4CPreparedStatement.executeForDescribe(T4CPreparedStatement.java:886)
      	at oracle.jdbc.driver.OracleStatement.executeMaybeDescribe(OracleStatement.java:1175)
      	at oracle.jdbc.driver.OracleStatement.doExecuteWithTimeout(OracleStatement.java:1296)
      	at oracle.jdbc.driver.OraclePreparedStatement.executeInternal(OraclePreparedStatement.java:3613)
      	at oracle.jdbc.driver.OraclePreparedStatement.executeQuery(OraclePreparedStatement.java:3657)
      	at oracle.jdbc.driver.OraclePreparedStatementWrapper.executeQuery(OraclePreparedStatementWrapper.java:1495)
      	at org.apache.sqoop.mapreduce.db.DBRecordReader.executeQuery(DBRecordReader.java:111)
      	at org.apache.sqoop.mapreduce.db.DBRecordReader.nextKeyValue(DBRecordReader.java:235)
      	... 12 more
      

      Inspecting the source code, the issue appears to be attributed to OracleManager failing to set the correct input format (should be OracleDataDrivenDBInputFormat, but appears to be getting set to DataDrivenDBInputFormat, resulting in DateSplitter being applied instead of OracleDateSplitter). OracleManager appears to apply the correct input format when using the --table option in sqoop import, but doesn't apply a similar override when using the --query option, resulting in the input format being defaulted to DataDriveDBInputFormat.

      This defect was tested using 1.4.6-cdh5.5.2-release and 1.4.6-cdh5.6.0-release, and the affected code issue appears to still be applicable as of the latest in trunk.

      Attachments

        1. SQOOP-3003.patch
          0.8 kB
          Nate Clevenger

        Activity

          People

            Unassigned Unassigned
            nateclevenger Nate Clevenger
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: