Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id DEC34200D36 for ; Mon, 6 Nov 2017 21:23:05 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id DD5ED160BD5; Mon, 6 Nov 2017 20:23:05 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 0925B160BEC for ; Mon, 6 Nov 2017 21:23:04 +0100 (CET) Received: (qmail 2777 invoked by uid 500); 6 Nov 2017 20:23:04 -0000 Mailing-List: contact issues-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@spark.apache.org Received: (qmail 2768 invoked by uid 99); 6 Nov 2017 20:23:04 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 06 Nov 2017 20:23:04 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 711381807ED for ; Mon, 6 Nov 2017 20:23:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.002 X-Spam-Level: X-Spam-Status: No, score=-100.002 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id nh0gyTQt6ndW for ; Mon, 6 Nov 2017 20:23:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 53CDE5FC6C for ; Mon, 6 Nov 2017 20:23:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 73DC3E0E80 for ; Mon, 6 Nov 2017 20:23:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 252BA23F05 for ; Mon, 6 Nov 2017 20:23:00 +0000 (UTC) Date: Mon, 6 Nov 2017 20:23:00 +0000 (UTC) From: "Saniya Tech (JIRA)" To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (SPARK-22460) Spark De-serialization of Timestamp field is Incorrect MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 06 Nov 2017 20:23:06 -0000 [ https://issues.apache.org/jira/browse/SPARK-22460?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Saniya Tech updated SPARK-22460: -------------------------------- Description: We are trying to serialize Timestamp fields to Avro using spark-avro connector. I can see the Timestamp fields are getting correctly serialized as long (milliseconds since Epoch). I verified that the data is correctly read back from the Avro files. It is when we encode the Dataset as a case class that timestamp field is incorrectly converted to a long value as seconds since Epoch. As can be seen below, this shifts the timestamp many years in the future. Code used to reproduce the issue: {code:java} import java.sql.Timestamp import com.databricks.spark.avro._ import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession} case class TestRecord(name: String, modified: Timestamp) import spark.implicits._ val data = Seq( TestRecord("One", new Timestamp(System.currentTimeMillis())) ) // Serialize: val parameters = Map("recordName" -> "TestRecord", "recordNamespace" -> "com.example.domain") val path = s"s3a://some-bucket/output/" val ds = spark.createDataset(data) ds.write .options(parameters) .mode(SaveMode.Overwrite) .avro(path) // // De-serialize val output = spark.read.avro(path).as[TestRecord] {code} Output from the test: {code:java} scala> data.head res4: TestRecord = TestRecord(One,2017-11-06 20:06:19.419) scala> output.collect().head res5: TestRecord = TestRecord(One,49819-12-16 17:23:39.0) {code} was: We are trying to serialize Timestamp fields to Avro using spark-avro connector. I can see the Timestamp fields are getting correctly serialized as long (milliseconds since Epoch). I verified that the data is correctly read back from the Avro files. It is when we encode the Dataset as a case class that timestamp field is incorrectly converted to as long value as seconds since Epoch. As can be seen below, this shifts the timestamp many years in the future. Code used to reproduce the issue: {code:java} import java.sql.Timestamp import com.databricks.spark.avro._ import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession} case class TestRecord(name: String, modified: Timestamp) import spark.implicits._ val data = Seq( TestRecord("One", new Timestamp(System.currentTimeMillis())) ) // Serialize: val parameters = Map("recordName" -> "TestRecord", "recordNamespace" -> "com.example.domain") val path = s"s3a://some-bucket/output/" val ds = spark.createDataset(data) ds.write .options(parameters) .mode(SaveMode.Overwrite) .avro(path) // // De-serialize val output = spark.read.avro(path).as[TestRecord] {code} Output from the test: {code:java} scala> data.head res4: TestRecord = TestRecord(One,2017-11-06 20:06:19.419) scala> output.collect().head res5: TestRecord = TestRecord(One,49819-12-16 17:23:39.0) {code} > Spark De-serialization of Timestamp field is Incorrect > ------------------------------------------------------ > > Key: SPARK-22460 > URL: https://issues.apache.org/jira/browse/SPARK-22460 > Project: Spark > Issue Type: Bug > Components: Input/Output > Affects Versions: 2.1.1 > Reporter: Saniya Tech > > We are trying to serialize Timestamp fields to Avro using spark-avro connector. I can see the Timestamp fields are getting correctly serialized as long (milliseconds since Epoch). I verified that the data is correctly read back from the Avro files. It is when we encode the Dataset as a case class that timestamp field is incorrectly converted to a long value as seconds since Epoch. As can be seen below, this shifts the timestamp many years in the future. > Code used to reproduce the issue: > {code:java} > import java.sql.Timestamp > import com.databricks.spark.avro._ > import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession} > case class TestRecord(name: String, modified: Timestamp) > import spark.implicits._ > val data = Seq( > TestRecord("One", new Timestamp(System.currentTimeMillis())) > ) > // Serialize: > val parameters = Map("recordName" -> "TestRecord", "recordNamespace" -> "com.example.domain") > val path = s"s3a://some-bucket/output/" > val ds = spark.createDataset(data) > ds.write > .options(parameters) > .mode(SaveMode.Overwrite) > .avro(path) > // > // De-serialize > val output = spark.read.avro(path).as[TestRecord] > {code} > Output from the test: > {code:java} > scala> data.head > res4: TestRecord = TestRecord(One,2017-11-06 20:06:19.419) > scala> output.collect().head > res5: TestRecord = TestRecord(One,49819-12-16 17:23:39.0) > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org For additional commands, e-mail: issues-help@spark.apache.org