Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-19726

Faild to insert null timestamp value to mysql using spark jdbc

Rank to TopRank to BottomAttach filesAttach ScreenshotBulk Copy AttachmentsBulk Move AttachmentsVotersWatch issueWatchersCreate sub-taskConvert to sub-taskLinkCloneLabelsUpdate Comment AuthorReplace String in CommentUpdate Comment VisibilityDelete Comments
    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.0.0, 2.0.1, 2.0.2, 2.1.0
    • 2.3.0
    • SQL
    • None

    Description

      1. create a table in mysql

      CREATE TABLE `timestamp_test` (
        `id` bigint(23) DEFAULT NULL,
        `time_stamp` datetime NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP
      ) ENGINE=InnoDB DEFAULT CHARSET=utf8
      

      2. insert one row using spark

      CREATE OR REPLACE TEMPORARY VIEW jdbcTable
      USING org.apache.spark.sql.jdbc
      OPTIONS (
        url 'jdbc:mysql://xxx.xxx.xxx.xxx:3306/default?characterEncoding=utf8&useServerPrepStmts=false&rewriteBatchedStatements=true',
        dbtable 'timestamp_test',
        driver 'com.mysql.jdbc.Driver',
        user 'root',
        password 'root'
      );
      
      insert into jdbcTable values (1, null);
      

      the insert statement failed with exceptions:

      Error: org.apache.spark.SparkException: Job aborted due to stage failure: Task 599 in stage 1.0 failed 4 times, most recent failure: Lost task 599.3 in stage 1.0 (TID 1202, A03-R07-I12-135.JD.LOCAL): java.sql.BatchUpdateException: Data truncation: Incorrect datetime value: '1970-01-01 08:00:00' for column 'time_stamp' at row 1
      	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
      	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
      	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
      	at java.lang.reflect.Constructor.newInstance(Constructor.java:422)
      	at com.mysql.jdbc.Util.handleNewInstance(Util.java:404)
      	at com.mysql.jdbc.Util.getInstance(Util.java:387)
      	at com.mysql.jdbc.SQLError.createBatchUpdateException(SQLError.java:1154)
      	at com.mysql.jdbc.PreparedStatement.executeBatchedInserts(PreparedStatement.java:1582)
      	at com.mysql.jdbc.PreparedStatement.executeBatchInternal(PreparedStatement.java:1248)
      	at com.mysql.jdbc.StatementImpl.executeBatch(StatementImpl.java:959)
      	at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$.savePartition(JdbcUtils.scala:227)
      	at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$saveTable$1.apply(JdbcUtils.scala:300)
      	at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$saveTable$1.apply(JdbcUtils.scala:299)
      	at org.apache.spark.rdd.RDD$$anonfun$foreachPartition$1$$anonfun$apply$28.apply(RDD.scala:902)
      	at org.apache.spark.rdd.RDD$$anonfun$foreachPartition$1$$anonfun$apply$28.apply(RDD.scala:902)
      	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1899)
      	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1899)
      	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
      	at org.apache.spark.scheduler.Task.run(Task.scala:86)
      	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      	at java.lang.Thread.run(Thread.java:745)
      Caused by: com.mysql.jdbc.MysqlDataTruncation: Data truncation: Incorrect datetime value: '1970-01-01 08:00:00' for column 'time_stamp' at row 1
      	at com.mysql.jdbc.MysqlIO.checkErrorPacket(MysqlIO.java:3876)
      	at com.mysql.jdbc.MysqlIO.checkErrorPacket(MysqlIO.java:3814)
      	at com.mysql.jdbc.MysqlIO.sendCommand(MysqlIO.java:2478)
      	at com.mysql.jdbc.MysqlIO.sqlQueryDirect(MysqlIO.java:2625)
      	at com.mysql.jdbc.ConnectionImpl.execSQL(ConnectionImpl.java:2551)
      	at com.mysql.jdbc.PreparedStatement.executeInternal(PreparedStatement.java:1861)
      	at com.mysql.jdbc.PreparedStatement.executeUpdateInternal(PreparedStatement.java:2073)
      	at com.mysql.jdbc.PreparedStatement.executeUpdateInternal(PreparedStatement.java:2009)
      	at com.mysql.jdbc.PreparedStatement.executeLargeUpdate(PreparedStatement.java:5094)
      	at com.mysql.jdbc.PreparedStatement.executeBatchedInserts(PreparedStatement.java:1543)
      	... 15 more
      

      Attachments

        Activity

          This comment will be Viewable by All Users Viewable by All Users
          Cancel

          People

            wangshuang wangshuangshuang
            yuananf AnfengYuan
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Slack

                Issue deployment