[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-02 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/21935


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-02 Thread gengliangwang
Github user gengliangwang commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207159637
  
--- Diff: 
external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala ---
@@ -41,6 +41,7 @@ import org.apache.spark.sql.types._
 class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
   val episodesAvro = testFile("episodes.avro")
   val testAvro = testFile("test.avro")
+  val timestampAvro = testFile("timestamp.avro")
--- End diff --

The schema and data is stated in 
https://github.com/apache/spark/pull/21935/files#diff-9364b0610f92b3cc35a4bc43a80751bfR397
It should be easy to get from test cases.
The other test file episodesAvro also doesn't provide how it is generated.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-02 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207119879
  
--- Diff: 
external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala ---
@@ -41,6 +41,7 @@ import org.apache.spark.sql.types._
 class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
   val episodesAvro = testFile("episodes.avro")
   val testAvro = testFile("test.avro")
+  val timestampAvro = testFile("timestamp.avro")
--- End diff --

at least we should provide how the binary file is generated, or just do 
roundtrip test: Spark write avro files and then read it.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-02 Thread gengliangwang
Github user gengliangwang commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207110511
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -103,31 +108,49 @@ object SchemaConverters {
   catalystType: DataType,
   nullable: Boolean = false,
   recordName: String = "topLevelRecord",
-  prevNameSpace: String = ""): Schema = {
+  prevNameSpace: String = "",
+  outputTimestampType: AvroOutputTimestampType.Value = 
AvroOutputTimestampType.TIMESTAMP_MICROS
--- End diff --

It is also used in `CatalystDataToAvro`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207100964
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala 
---
@@ -86,8 +87,16 @@ class AvroDeserializer(rootAvroType: Schema, 
rootCatalystType: DataType) {
   case (LONG, LongType) => (updater, ordinal, value) =>
 updater.setLong(ordinal, value.asInstanceOf[Long])
 
-  case (LONG, TimestampType) => (updater, ordinal, value) =>
-updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
+  case (LONG, TimestampType) => avroType.getLogicalType match {
+case _: TimestampMillis => (updater, ordinal, value) =>
+  updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
+case _: TimestampMicros => (updater, ordinal, value) =>
+  updater.setLong(ordinal, value.asInstanceOf[Long])
+case null => (updater, ordinal, value) =>
--- End diff --

ditto, add a default case.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207095659
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -103,31 +108,49 @@ object SchemaConverters {
   catalystType: DataType,
   nullable: Boolean = false,
   recordName: String = "topLevelRecord",
-  prevNameSpace: String = ""): Schema = {
+  prevNameSpace: String = "",
+  outputTimestampType: AvroOutputTimestampType.Value = 
AvroOutputTimestampType.TIMESTAMP_MICROS
--- End diff --

do we really need the default value? Seems only one call site excluding the 
recursive ones.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207095392
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala ---
@@ -93,7 +94,13 @@ class AvroSerializer(rootCatalystType: DataType, 
rootAvroType: Schema, nullable:
   case DateType =>
 (getter, ordinal) => getter.getInt(ordinal) * 
DateTimeUtils.MILLIS_PER_DAY
   case TimestampType =>
-(getter, ordinal) => getter.getLong(ordinal) / 1000
+(getter, ordinal) => avroType.getLogicalType match {
+  case _: TimestampMillis => getter.getLong(ordinal) / 1000
+  case _: TimestampMicros => getter.getLong(ordinal)
+  // For backward compatibility, if the Avro type is Long and it 
is not logical type,
+  // output the timestamp value as with millisecond precision.
+  case null => getter.getLong(ordinal) / 1000
--- End diff --

ditto, add a default case.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207095253
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala ---
@@ -93,7 +94,13 @@ class AvroSerializer(rootCatalystType: DataType, 
rootAvroType: Schema, nullable:
   case DateType =>
 (getter, ordinal) => getter.getInt(ordinal) * 
DateTimeUtils.MILLIS_PER_DAY
   case TimestampType =>
-(getter, ordinal) => getter.getLong(ordinal) / 1000
+(getter, ordinal) => avroType.getLogicalType match {
--- End diff --

do not do pattern match per record, we should
```
avroType.getLogicalType match {
  case _: TimestampMillis => (getter, ordinal) => ...
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207095177
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala ---
@@ -79,4 +80,16 @@ class AvroOptions(
   val compression: String = {
 
parameters.get("compression").getOrElse(SQLConf.get.avroCompressionCodec)
   }
+
+  /**
+   * The `outputTimestampType` option sets which Avro timestamp type to 
use when Spark writes
+   * data to Avro files. Currently supported types are `TIMESTAMP_MICROS` 
and `TIMESTAMP_MILLIS`.
+   * TIMESTAMP_MICROS is a logical timestamp type in Avro, which stores 
number of microseconds
+   * from the Unix epoch. TIMESTAMP_MILLIS is also logical, but with 
millisecond precision,
+   * which means Spark has to truncate the microsecond portion of its 
timestamp value.
+   */
+  val outputTimestampType: AvroOutputTimestampType.Value = {
--- End diff --

I'm ok with it, I think parquet should also follow this.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207094996
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala 
---
@@ -86,8 +87,16 @@ class AvroDeserializer(rootAvroType: Schema, 
rootCatalystType: DataType) {
   case (LONG, LongType) => (updater, ordinal, value) =>
 updater.setLong(ordinal, value.asInstanceOf[Long])
 
-  case (LONG, TimestampType) => (updater, ordinal, value) =>
-updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
+  case (LONG, TimestampType) => avroType.getLogicalType match {
+case _: TimestampMillis => (updater, ordinal, value) =>
+  updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
+case _: TimestampMicros => (updater, ordinal, value) =>
+  updater.setLong(ordinal, value.asInstanceOf[Long])
+case null => (updater, ordinal, value) =>
+  // For backward compatibility, if the Avro type is Long and it 
is not logical type,
+  // the value is processed as timestamp type with millisecond 
precision.
+  updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
+  }
--- End diff --

we should add a default case and throw IncompatibleSchemaException, in case 
avro add more logical types for long type in the future.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207089232
  
--- Diff: 
external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala ---
@@ -36,11 +36,12 @@ import org.apache.spark.sql._
 import org.apache.spark.sql.execution.datasources.DataSource
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
-import org.apache.spark.sql.types._
+import org.apache.spark.sql.types.{StructType, _}
--- End diff --

Import looks a bit odd :-)


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207089217
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -103,31 +108,49 @@ object SchemaConverters {
   catalystType: DataType,
   nullable: Boolean = false,
   recordName: String = "topLevelRecord",
-  prevNameSpace: String = ""): Schema = {
+  prevNameSpace: String = "",
+  outputTimestampType: AvroOutputTimestampType.Value = 
AvroOutputTimestampType.TIMESTAMP_MICROS
+): Schema = {
--- End diff --

I believe

```scala
 outputTimestampType: AvroOutputTimestampType.Value = 
AvroOutputTimestampType.TIMESTAMP_MICROS)
: Schema = {
```

is more correct per 
https://github.com/databricks/scala-style-guide#spacing-and-indentation


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207088878
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala ---
@@ -79,4 +80,16 @@ class AvroOptions(
   val compression: String = {
 
parameters.get("compression").getOrElse(SQLConf.get.avroCompressionCodec)
   }
+
+  /**
+   * The `outputTimestampType` option sets which Avro timestamp type to 
use when Spark writes
+   * data to Avro files. Currently supported types are `TIMESTAMP_MICROS` 
and `TIMESTAMP_MILLIS`.
+   * TIMESTAMP_MICROS is a logical timestamp type in Avro, which stores 
number of microseconds
+   * from the Unix epoch. TIMESTAMP_MILLIS is also logical, but with 
millisecond precision,
+   * which means Spark has to truncate the microsecond portion of its 
timestamp value.
+   */
+  val outputTimestampType: AvroOutputTimestampType.Value = {
--- End diff --

Hm, I wouldn't expose this as an option for now - that at least matches to 
Parquet's.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread gengliangwang
Github user gengliangwang commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r207000865
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -103,31 +108,49 @@ object SchemaConverters {
   catalystType: DataType,
   nullable: Boolean = false,
   recordName: String = "topLevelRecord",
-  prevNameSpace: String = ""): Schema = {
+  prevNameSpace: String = "",
+  outputTimestampType: AvroOutputTimestampType.Value = 
AvroOutputTimestampType.TIMESTAMP_MICROS
+): Schema = {
--- End diff --

Not sure if the indent here is correct.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206985120
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -103,31 +108,48 @@ object SchemaConverters {
   catalystType: DataType,
   nullable: Boolean = false,
   recordName: String = "topLevelRecord",
-  prevNameSpace: String = ""): Schema = {
+  prevNameSpace: String = "",
+  outputTimestampType: String = "TIMESTAMP_MICROS"): Schema = {
 val builder = if (nullable) {
   SchemaBuilder.builder().nullable()
 } else {
   SchemaBuilder.builder()
 }
+
 catalystType match {
   case BooleanType => builder.booleanType()
   case ByteType | ShortType | IntegerType => builder.intType()
   case LongType => builder.longType()
   case DateType => builder.longType()
-  case TimestampType => builder.longType()
+  case TimestampType =>
+val timestampType = outputTimestampType match {
+  case "TIMESTAMP_MILLIS" => LogicalTypes.timestampMillis()
--- End diff --

don't hardcode the strings, we can write
```
if (outputTimestampType == 
AvroOutputTimestampType.TIMESTAMP_MICROS.toString) ...
```


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206983995
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -42,7 +43,11 @@ object SchemaConverters {
   case BYTES => SchemaType(BinaryType, nullable = false)
   case DOUBLE => SchemaType(DoubleType, nullable = false)
   case FLOAT => SchemaType(FloatType, nullable = false)
-  case LONG => SchemaType(LongType, nullable = false)
+  case LONG => avroSchema.getLogicalType match {
+case _: TimestampMillis | _: TimestampMicros =>
+  return SchemaType(TimestampType, nullable = false)
--- End diff --

why use `return` here?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206983421
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala ---
@@ -93,7 +94,11 @@ class AvroSerializer(rootCatalystType: DataType, 
rootAvroType: Schema, nullable:
   case DateType =>
 (getter, ordinal) => getter.getInt(ordinal) * 
DateTimeUtils.MILLIS_PER_DAY
   case TimestampType =>
-(getter, ordinal) => getter.getLong(ordinal) / 1000
+(getter, ordinal) => avroType.getLogicalType match {
+  case _: TimestampMillis => getter.getLong(ordinal) / 1000
+  case _: TimestampMicros => getter.getLong(ordinal)
+  case _ => getter.getLong(ordinal)
--- End diff --

ditto


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-08-01 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206983054
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala 
---
@@ -86,8 +87,14 @@ class AvroDeserializer(rootAvroType: Schema, 
rootCatalystType: DataType) {
   case (LONG, LongType) => (updater, ordinal, value) =>
 updater.setLong(ordinal, value.asInstanceOf[Long])
 
-  case (LONG, TimestampType) => (updater, ordinal, value) =>
-updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
+  case (LONG, TimestampType) => avroType.getLogicalType match {
+case _: TimestampMillis => (updater, ordinal, value) =>
+  updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
+case _: TimestampMicros => (updater, ordinal, value) =>
+  updater.setLong(ordinal, value.asInstanceOf[Long])
+case _ => (updater, ordinal, value) =>
+  updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
--- End diff --

Let's add a comment to say it's for backward compatibility reasons. Also we 
should only do it when logical type is null. For other logical types, we should 
fail here.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-07-31 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206748626
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -114,7 +121,10 @@ object SchemaConverters {
   case ByteType | ShortType | IntegerType => builder.intType()
   case LongType => builder.longType()
   case DateType => builder.longType()
-  case TimestampType => builder.longType()
+  case TimestampType =>
+// To be consistent with the previous behavior of writing 
Timestamp type with Avro 1.7,
--- End diff --

For now I think writing out timestamp micros should be good


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-07-31 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206747402
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -35,6 +36,12 @@ object SchemaConverters {
* This function takes an avro schema and returns a sql schema.
*/
   def toSqlType(avroSchema: Schema): SchemaType = {
+avroSchema.getLogicalType match {
--- End diff --

ditto


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-07-31 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206747243
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala 
---
@@ -71,7 +72,15 @@ class AvroDeserializer(rootAvroType: Schema, 
rootCatalystType: DataType) {
   private def newWriter(
   avroType: Schema,
   catalystType: DataType,
-  path: List[String]): (CatalystDataUpdater, Int, Any) => Unit =
+  path: List[String]): (CatalystDataUpdater, Int, Any) => Unit = {
+(avroType.getLogicalType, catalystType) match {
--- End diff --

Can we do this like:

```scala
  case (LONG, TimestampType) => avroType.getLogicalType match {
case _: TimestampMillis => (updater, ordinal, value) =>
  updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
case _: TimestampMicros => (updater, ordinal, value) =>
  updater.setLong(ordinal, value.asInstanceOf[Long])
case _ => (updater, ordinal, value) =>
  updater.setLong(ordinal, value.asInstanceOf[Long] * 1000)
  }
```

? Looks they have Avro long type anyway. Thought it's better to read and 
actually safer and correct. 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-07-31 Thread gengliangwang
Github user gengliangwang commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206743304
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -114,7 +121,10 @@ object SchemaConverters {
   case ByteType | ShortType | IntegerType => builder.intType()
   case LongType => builder.longType()
   case DateType => builder.longType()
-  case TimestampType => builder.longType()
+  case TimestampType =>
+// To be consistent with the previous behavior of writing 
Timestamp type with Avro 1.7,
--- End diff --

Previously we write timestamp as `Long` and divide the value by 
1000(millisecond precision).
Maybe I need to revise the comment.
+1 on the new config.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-07-31 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206742399
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -114,7 +121,10 @@ object SchemaConverters {
   case ByteType | ShortType | IntegerType => builder.intType()
   case LongType => builder.longType()
   case DateType => builder.longType()
-  case TimestampType => builder.longType()
+  case TimestampType =>
+// To be consistent with the previous behavior of writing 
Timestamp type with Avro 1.7,
--- End diff --

also we should follow parquet and have a config 
`spark.sql.avro.outputTimestampType` to control it.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-07-31 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206742249
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -114,7 +121,10 @@ object SchemaConverters {
   case ByteType | ShortType | IntegerType => builder.intType()
   case LongType => builder.longType()
   case DateType => builder.longType()
-  case TimestampType => builder.longType()
+  case TimestampType =>
+// To be consistent with the previous behavior of writing 
Timestamp type with Avro 1.7,
+// the default output Avro Timestamp type is with millisecond 
precision.
+builder.longBuilder().prop(LogicalType.LOGICAL_TYPE_PROP, 
"timestamp-millis").endLong()
--- End diff --

is there a better API for it? hardcoding a string is hacky.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-07-31 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/21935#discussion_r206742158
  
--- Diff: 
external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala 
---
@@ -114,7 +121,10 @@ object SchemaConverters {
   case ByteType | ShortType | IntegerType => builder.intType()
   case LongType => builder.longType()
   case DateType => builder.longType()
-  case TimestampType => builder.longType()
+  case TimestampType =>
+// To be consistent with the previous behavior of writing 
Timestamp type with Avro 1.7,
--- End diff --

the previous behavior is: we can't write out timestamp data, isn't it?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #21935: [SPARK-24773] Avro: support logical timestamp typ...

2018-07-31 Thread gengliangwang
GitHub user gengliangwang opened a pull request:

https://github.com/apache/spark/pull/21935

[SPARK-24773] Avro: support logical timestamp type with different precisions

## What changes were proposed in this pull request?

Support reading Avro logical timestamp type with different precisions

https://avro.apache.org/docs/1.8.2/spec.html#Timestamp+%28millisecond+precision%29

Also support writing Spark Timestamp type to Avro timestamp type with 
millisecond precision, which is consistent with the previous behavior of 
writing Timestamp type with Avro 1.7.

## How was this patch tested?

Unit test 


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/gengliangwang/spark avro_timestamp

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/21935.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #21935


commit 3a53f5569fa47cf0fb0e3eadb0f1c134dc1448b0
Author: Gengliang Wang 
Date:   2018-07-31T18:27:18Z

Avro: support logical timestamp type




---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org