[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

2018-01-09 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/20163#discussion_r160450067
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
+// Pyrolite will unpickle a Python datetime.date to a 
java.util.Calendar
+case (c: Calendar, DateType) => 
DateTimeUtils.fromJavaCalendarForDate(c)
--- End diff --

My 2 cents:
* I am +0 for 2. I think having type coercion for str type but not for 
other types can be confusing to users. Realistically, I think any udf users 
would hit the `null` result for mismatch type anyway (I have hit it so many 
times..) and will learn that `null` means type mismatch. Even we make the 
behavior for str a bit friendly, they will likely to hit the issue with other 
types anyway. I don't think "default returnType for udf is str" is a strong 
reason for "having special type coercion for str", they seem orthogonal. 
**I prefer that we keep the type mismatch behavior consistent for str types 
vs other types (return null) and document this more clearly.**
* I find returning `null` for type mismatch is an unintuitive behavior of 
row-at-a-time udf and prefer not to replicate it in pandas udf.




---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

2018-01-09 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/20163#discussion_r160364055
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
+// Pyrolite will unpickle a Python datetime.date to a 
java.util.Calendar
+case (c: Calendar, DateType) => 
DateTimeUtils.fromJavaCalendarForDate(c)
--- End diff --

Yeah, 2. should work for `StringType`.

I'd also like to add some documents like 1. for users to be careful about 
the return type. I've found that `udf`s return `null` and `pandas_udf`s throw 
some exception in most case when the return type is mismatching.
Of course we can try to make the behavior differences between `udf` and 
`pandas_udf` closer as possible in the future, but I think it is the best 
effort basis for the mismatching return type.


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r160358011
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
+// Pyrolite will unpickle a Python datetime.date to a 
java.util.Calendar
+case (c: Calendar, DateType) => 
DateTimeUtils.fromJavaCalendarForDate(c)
--- End diff --

WDYT about ^ @ueshin?


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r160355531
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
+// Pyrolite will unpickle a Python datetime.date to a 
java.util.Calendar
+case (c: Calendar, DateType) => 
DateTimeUtils.fromJavaCalendarForDate(c)
--- End diff --

Right. Let's go ahead for 2. then. I am fine if it's done as an exception 
for practical purpose. Maybe we could add an if `isinstance(.., basestring)` 
and return directly as a shortcut. I haven't checked the perf diff but I think 
we can do it easily via profile as I mentioned above.


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r160349750
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
+// Pyrolite will unpickle a Python datetime.date to a 
java.util.Calendar
+case (c: Calendar, DateType) => 
DateTimeUtils.fromJavaCalendarForDate(c)
--- End diff --

Yea it's consistent with other un-convertible cases, but `StringType` is 
the default return type, I'm afraid many users many hit this and get confused.


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r160345387
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
+// Pyrolite will unpickle a Python datetime.date to a 
java.util.Calendar
+case (c: Calendar, DateType) => 
DateTimeUtils.fromJavaCalendarForDate(c)
--- End diff --

How about we return `null` in this case? Other cases seems also returning 
`null` if it fails to be converted:

```
>>> from pyspark.sql.functions import udf
>>> f = udf(lambda x: x, "double")
>>> spark.range(1).select(f("id")).show()
++
|(id)|
++
|null|
++
```

Seems we can do it like:

```scala

case StringType => (obj: Any) => nullSafeConvert(obj) {
  case c: Calendar => null
  case _ => UTF8String.fromString(obj.toString)
}
```


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r160094816
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
+// Pyrolite will unpickle a Python datetime.date to a 
java.util.Calendar
+case (c: Calendar, DateType) => 
DateTimeUtils.fromJavaCalendarForDate(c)
--- End diff --

I think he did this in Python side because here we don't know if `Calendar` 
is from `datetime.date` or `datetime.datetime`.


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r160086824
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
+// Pyrolite will unpickle a Python datetime.date to a 
java.util.Calendar
+case (c: Calendar, DateType) => 
DateTimeUtils.fromJavaCalendarForDate(c)
--- End diff --

seems what we need is a `case (c: Calendar, StringType) => ...`


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r160086772
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
+// Pyrolite will unpickle a Python datetime.date to a 
java.util.Calendar
+case (c: Calendar, DateType) => 
DateTimeUtils.fromJavaCalendarForDate(c)
--- End diff --

so we will never hit this?


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

2018-01-05 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/20163#discussion_r160017637
  
--- Diff: python/pyspark/sql/udf.py ---
@@ -26,6 +26,28 @@
 
 
 def _wrap_function(sc, func, returnType):
+def coerce_to_str(v):
+import datetime
+if type(v) == datetime.date or type(v) == datetime.datetime:
+return str(v)
+else:
+return v
+
+# Pyrolite will unpickle both Python datetime.date and 
datetime.datetime objects
+# into java.util.Calendar objects, so the type information on the 
Python side is lost.
+# This is problematic when Spark SQL needs to cast such objects into 
Spark SQL string type,
+# because the format of the string should be different, depending on 
the type of the input
+# object. So for those two specific types we eagerly convert them to 
string here, where the
+# Python type information is still intact.
+if returnType == StringType():
--- End diff --

I have a question, why we need to handle this type conversion? If we expect 
correct string format, isn't it more reasonable to convert the date/datetime to 
strings in the udf, instead of adding this conversion implicitly?


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

2018-01-05 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/20163#discussion_r160017370
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
 ---
@@ -193,6 +193,24 @@ object DateTimeUtils {
 millisToDays(date.getTime)
   }
 
+  /**
+   * Returns the number of days since epoch from java.util.Calendar
+   */
+  def fromJavaCalendarForDate(cal: Calendar): SQLDate = {
+val ms = cal.getTimeInMillis
+cal.getTimeZone match {
+  case null => millisToDays(ms)
+  case tz => millisToDays(ms, tz)
+}
+  }
+
+  /**
+   * Returns SQLTimestamp from java.util.Calendar (microseconds since 
epoch)
--- End diff --

(Matching the comment of fromJavaCalendarForDate.)
nit: Returns the number of microseconds since epoch from java.util.Calendar.




---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r159805825
  
--- Diff: python/pyspark/sql/udf.py ---
@@ -26,6 +26,28 @@
 
 
 def _wrap_function(sc, func, returnType):
+def coerce_to_str(v):
+import datetime
+if type(v) == datetime.date or type(v) == datetime.datetime:
+return str(v)
--- End diff --

I think it's weird that we have a cast here alone ... Can't we register a 
custom Pyrolite unpickler? Does it make the things more complicated?


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r159804507
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
--- End diff --

Of course, separate change obviously.


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r159804356
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala
 ---
@@ -120,10 +121,18 @@ object EvaluatePython {
 case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c, 
dt.precision, dt.scale)
 
 case (c: Int, DateType) => c
--- End diff --

BTW, as a side note, I think we can make the converter for the type and 
then reuse it.


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

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

https://github.com/apache/spark/pull/20163#discussion_r159796028
  
--- Diff: python/pyspark/sql/udf.py ---
@@ -26,6 +26,28 @@
 
 
 def _wrap_function(sc, func, returnType):
+def coerce_to_str(v):
+import datetime
+if type(v) == datetime.date or type(v) == datetime.datetime:
+return str(v)
+else:
+return v
+
+# Pyrolite will unpickle both Python datetime.date and 
datetime.datetime objects
+# into java.util.Calendar objects, so the type information on the 
Python side is lost.
+# This is problematic when Spark SQL needs to cast such objects into 
Spark SQL string type,
+# because the format of the string should be different, depending on 
the type of the input
+# object. So for those two specific types we eagerly convert them to 
string here, where the
+# Python type information is still intact.
+if returnType == StringType():
--- End diff --

This is to handle when a python udf returns `date` or `datetime` but mark 
the return type as string?


---

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



[GitHub] spark pull request #20163: [SPARK-22966][PySpark] Spark SQL should handle Py...

2018-01-04 Thread rednaxelafx
GitHub user rednaxelafx opened a pull request:

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

[SPARK-22966][PySpark] Spark SQL should handle Python UDFs that return a 
datetime.date or datetime.datetime

## What changes were proposed in this pull request?

Perform appropriate conversions for results coming from Python UDFs that 
return `datetime.date` or `datetime.datetime`.

Before this PR, Pyrolite would unpickle both `datetime.date` and 
`datetime.datetime` into a `java.util.Calendar`, which Spark SQL doesn't 
understand, which then leads to incorrect results. An example of such incorrect 
result is:

```
>>> py_date = udf(datetime.date)
>>> spark.range(1).select(py_date(lit(2017), lit(10), 
lit(30))).show(truncate=False)

+--+
|date(2017, 10, 30) 








   |

+--+

|java.util.GregorianCalendar[time=?,areFieldsSet=false,areAllFieldsSet=false,lenient=true,zone=sun.util.calendar.ZoneInfo[id="America/Los_Angeles",offset=-2880,dstSavings=360,useDaylight=true,transitions=185,lastRule=java.util.SimpleTimeZone[id=America/Los_Angeles,offset=-2880,dstSavings=360,useDaylight=true,startYear=0,startMode=3,startMonth=2,startDay=8,startDayOfWeek=1,startTime=720,startTimeMode=0,endMode=3,endMonth=10,endDay=1,endDayOfWeek=1,endTime=720,endTimeMode=0]],firstDayOfWeek=1,minimalDaysInFirstWeek=1,ERA=?,YEAR=2017,MONTH=9,WEEK_OF_YEAR=?,WEEK_OF_MONTH=?,DAY_OF_MONTH=30,DAY_OF_YEAR=?,DAY_OF_WEEK=?,DAY_OF_WEEK_IN_MONTH=?,AM_PM=0,HOUR=0,HOUR_OF_DAY=0,MINUTE=0,SECOND=0,MILLISECOND=?,ZONE_OFFSET=?,DST_OFFSET=?]|

+--+
```
After this PR, the same query above would give correct results:
```
>>> spark.range(1).select(py_date(lit(2017), lit(10), 
lit(30))).show(truncate=False)
+--+
|date(2017, 10, 30)|
+--+
|2017-10-30|
+--+
```

An explicit non-goal of this PR is to change the behavior of timezone 
awareness or