[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-17 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r145271690
  
--- Diff: python/pyspark/serializers.py ---
@@ -259,11 +261,13 @@ def load_stream(self, stream):
 """
 Deserialize ArrowRecordBatches to an Arrow table and return as a 
list of pandas.Series.
 """
+from pyspark.sql.types import _check_dataframe_localize_timestamps
 import pyarrow as pa
 reader = pa.open_stream(stream)
 for batch in reader:
-table = pa.Table.from_batches([batch])
-yield [c.to_pandas() for c in table.itercolumns()]
+# NOTE: changed from pa.Columns.to_pandas, timezone issue in 
conversion fixed in 0.7.1
+pdf = _check_dataframe_localize_timestamps(batch.to_pandas())
+yield [c for _, c in pdf.iteritems()]
--- End diff --

After running some tests, this change does not significantly degrade 
performance, but there seems to be a small difference.  cc @ueshin 

I ran various columns of random data through a `pandas_udf` repeatedly with 
and without this change.  Test was in local mode with default Spark conf, 
looking at min wall clock time of 10 loops

before change: 2.595558
after change: 2.681813

Do you think the difference here is acceptable for now until arrow is 
upgraded and we can look into again?


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-17 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r145245871
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala 
---
@@ -55,6 +55,12 @@ object ArrowWriter {
   case (DoubleType, vector: NullableFloat8Vector) => new 
DoubleWriter(vector)
   case (StringType, vector: NullableVarCharVector) => new 
StringWriter(vector)
   case (BinaryType, vector: NullableVarBinaryVector) => new 
BinaryWriter(vector)
+  case (DateType, vector: NullableDateDayVector) => new 
DateWriter(vector)
+  case (TimestampType, vector: NullableTimeStampMicroTZVector)
+// TODO: Should be able to access timezone from vector
+if field.getType.isInstanceOf[ArrowType.Timestamp] &&
+  field.getType.asInstanceOf[ArrowType.Timestamp].getTimezone != 
null =>
--- End diff --

It can probably be removed.  My original thinking was to ensure that the 
ArrowWriter was created with a valid timezone id and not a null or empty 
string.  I think it's better handled in `toArrowSchema` which takes an Option 
now.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-17 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r145215343
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,38 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+# Timestamps should be in UTC, JVM Arrow timestamps require a 
timezone to be read
+arrow_type = pa.timestamp('us', tz='UTC')
 else:
 raise TypeError("Unsupported type in conversion to Arrow: " + 
str(dt))
 return arrow_type
 
 
+def _check_dataframe_localize_timestamps(df):
+""" Convert timezone aware timestamps to timezone-naive in local time
+"""
+from pandas.types.common import is_datetime64tz_dtype
+for column, series in df.iteritems():
+# TODO: handle nested timestamps?
+if is_datetime64tz_dtype(series.dtype):
+df[column] = 
series.dt.tz_convert('tzlocal()').dt.tz_localize(None)
+return df
+
+
+def _check_series_convert_timestamps_internal(s):
+""" Convert a tz-naive timestamp in local tz to UTC normalized for 
Spark internal storage
+"""
+from pandas.types.common import is_datetime64_dtype
+# TODO: handle nested timestamps?
+if is_datetime64_dtype(s.dtype):
+return 
s.dt.tz_localize('tzlocal()').dt.tz_convert('UTC').values.astype('datetime64[us]')
--- End diff --

Yes, just filed https://issues.apache.org/jira/browse/ARROW-1680 to address 
this.  I'll add this in a comment and check for a performance regression today.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-16 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r145037361
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala 
---
@@ -55,6 +55,12 @@ object ArrowWriter {
   case (DoubleType, vector: NullableFloat8Vector) => new 
DoubleWriter(vector)
   case (StringType, vector: NullableVarCharVector) => new 
StringWriter(vector)
   case (BinaryType, vector: NullableVarBinaryVector) => new 
BinaryWriter(vector)
+  case (DateType, vector: NullableDateDayVector) => new 
DateWriter(vector)
+  case (TimestampType, vector: NullableTimeStampMicroTZVector)
+// TODO: Should be able to access timezone from vector
+if field.getType.isInstanceOf[ArrowType.Timestamp] &&
+  field.getType.asInstanceOf[ArrowType.Timestamp].getTimezone != 
null =>
--- End diff --

Do we need this `if`?


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-16 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r145036404
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,38 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+# Timestamps should be in UTC, JVM Arrow timestamps require a 
timezone to be read
+arrow_type = pa.timestamp('us', tz='UTC')
 else:
 raise TypeError("Unsupported type in conversion to Arrow: " + 
str(dt))
 return arrow_type
 
 
+def _check_dataframe_localize_timestamps(df):
+""" Convert timezone aware timestamps to timezone-naive in local time
+"""
+from pandas.types.common import is_datetime64tz_dtype
+for column, series in df.iteritems():
+# TODO: handle nested timestamps?
+if is_datetime64tz_dtype(series.dtype):
+df[column] = 
series.dt.tz_convert('tzlocal()').dt.tz_localize(None)
+return df
+
+
+def _check_series_convert_timestamps_internal(s):
+""" Convert a tz-naive timestamp in local tz to UTC normalized for 
Spark internal storage
+"""
+from pandas.types.common import is_datetime64_dtype
+# TODO: handle nested timestamps?
+if is_datetime64_dtype(s.dtype):
+return 
s.dt.tz_localize('tzlocal()').dt.tz_convert('UTC').values.astype('datetime64[us]')
--- End diff --

Could you add a comment about the reason we need to do 
`.astype('datetime64[us]')`?


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-16 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r145024991
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3383,6 +3403,42 @@ def test_vectorized_udf_varargs(self):
 res = df.select(f(col('id')))
 self.assertEquals(df.collect(), res.collect())
 
+def test_vectorized_udf_timestamps(self):
+from pyspark.sql.functions import pandas_udf, col
+from datetime import date, datetime
+schema = StructType([
+StructField("idx", LongType(), True),
+StructField("date", DateType(), True),
+StructField("timestamp", TimestampType(), True)])
+data = [(0, date(1969, 1, 1), datetime(1969, 1, 1, 1, 1, 1)),
+(1, date(2012, 2, 2), datetime(2012, 2, 2, 2, 2, 2)),
+(2, date(2100, 3, 3), datetime(2100, 3, 3, 3, 3, 3)),
+(3, date(2104, 4, 4), datetime(2104, 4, 4, 4, 4, 4))]
+
+df = self.spark.createDataFrame(data, schema=schema)
+
+# Check that a timestamp passed through a pandas_udf will not be 
altered by timezone calc
+identity = pandas_udf(lambda t: t, returnType=TimestampType())
+df = df.withColumn("timestamp_copy", identity(col("timestamp")))
+
+@pandas_udf(returnType=BooleanType())
+def check_data(idx, date, timestamp, timestamp_copy):
+is_equal = timestamp == timestamp_copy
+if is_equal.all():
+for i in xrange(len(is_equal)):
--- End diff --

Let's just use `range`. seems failed in Python 3 due to missing `xrange`.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-16 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144994750
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3086,18 +3086,35 @@ class ArrowTests(ReusedPySparkTestCase):
 
 @classmethod
 def setUpClass(cls):
+from datetime import datetime
 ReusedPySparkTestCase.setUpClass()
+
+# Synchronize default timezone between Python and Java
+tz = "America/Los_Angeles"
+os.environ["TZ"] = tz
--- End diff --

Hmm, I was assuming that `os.environ["TZ"]` wouldn't be set, but I guess it 
could be. I'll fix that.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-16 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144993515
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,47 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+arrow_type = pa.timestamp('us', tz='UTC')
 else:
 raise TypeError("Unsupported type in conversion to Arrow: " + 
str(dt))
 return arrow_type
 
 
+def _check_localize_series_timestamps(s):
+from pandas.types.common import is_datetime64_dtype
+# TODO: handle nested timestamps?
+if is_datetime64_dtype(s.dtype):
+# TODO: pyarrow.Column.to_pandas keeps data in UTC but removes 
timezone
--- End diff --

I confirmed this is not an issue as of pyarrow 0.7.1.  I'll test the 
workaround above from @ueshin and see if there is an obvious performance 
regression.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-12 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144300577
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala 
---
@@ -31,7 +31,8 @@ object ArrowUtils {
 
   // todo: support more types.
 
-  def toArrowType(dt: DataType): ArrowType = dt match {
+  /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for 
TimestampTypes */
+  def toArrowType(dt: DataType, timeZoneId: Option[String]): ArrowType = 
dt match {
--- End diff --

Btw, this is a bit nit pick. So feel free to stick to this if you think 
`Option[String]` is the right thing.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-12 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144300232
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala 
---
@@ -31,7 +31,8 @@ object ArrowUtils {
 
   // todo: support more types.
 
-  def toArrowType(dt: DataType): ArrowType = dt match {
+  /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for 
TimestampTypes */
+  def toArrowType(dt: DataType, timeZoneId: Option[String]): ArrowType = 
dt match {
--- End diff --

Ok. Makes sense. If this arg is required. We should maybe make it just 
`String` instead of `Option[String]`, I find having a required arg to be 
`Option` is not intuitive..


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-12 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144271872
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3086,18 +3086,35 @@ class ArrowTests(ReusedPySparkTestCase):
 
 @classmethod
 def setUpClass(cls):
+from datetime import datetime
 ReusedPySparkTestCase.setUpClass()
+
+# Synchronize default timezone between Python and Java
+tz = "America/Los_Angeles"
+os.environ["TZ"] = tz
--- End diff --

Maybe I am too much worried but shouldn't we keep the original 
`os.environ["TZ"]` and then restore it back in L3115?


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-12 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144277391
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,47 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+arrow_type = pa.timestamp('us', tz='UTC')
--- End diff --

I think It'd be nicer if we have some comments for this explanation.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-12 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144250165
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,47 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+arrow_type = pa.timestamp('us', tz='UTC')
 else:
 raise TypeError("Unsupported type in conversion to Arrow: " + 
str(dt))
 return arrow_type
 
 
+def _check_localize_series_timestamps(s):
+from pandas.types.common import is_datetime64_dtype
+# TODO: handle nested timestamps?
+if is_datetime64_dtype(s.dtype):
+# TODO: pyarrow.Column.to_pandas keeps data in UTC but removes 
timezone
+return 
s.dt.tz_localize('UTC').dt.tz_convert('tzlocal()').dt.tz_localize(None)
+else:
+return s
+
+
+def _check_localize_dataframe_timestamps(df):
+from pandas.types.common import is_datetime64tz_dtype
+for column, series in df.iteritems():
+# TODO: handle nested timestamps?
+if is_datetime64tz_dtype(series.dtype):
+df[column] = 
series.dt.tz_convert('tzlocal()').dt.tz_localize(None)
+return df
+
+
+def _utc_normalize_series_timestamps(s):
+""" Convert a tz-naive timestamp in local tz to UTC normalized
+"""
+# TODO: this should be system local tz or SESSION_LOCAL_TIMEZONE?
+return 
s.dt.tz_localize('tzlocal()').dt.tz_convert('UTC').values.astype('datetime64[us]')
+
+
+def _check_utc_normalize_series_timestamps(s):
+from pandas.types.common import is_datetime64_dtype
+# TODO: handle nested timestamps?
+return _utc_normalize_series_timestamps(s) if 
is_datetime64_dtype(s.dtype) else s
--- End diff --

nit: I'd use the same style for this and 
`_check_localize_series_timestamps`.

```python
if ...:
return ...
else:
return ...
```

or

```python
return ... if ... else ...
```


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-12 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144248880
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,47 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+arrow_type = pa.timestamp('us', tz='UTC')
 else:
 raise TypeError("Unsupported type in conversion to Arrow: " + 
str(dt))
 return arrow_type
 
 
+def _check_localize_series_timestamps(s):
+from pandas.types.common import is_datetime64_dtype
+# TODO: handle nested timestamps?
+if is_datetime64_dtype(s.dtype):
+# TODO: pyarrow.Column.to_pandas keeps data in UTC but removes 
timezone
--- End diff --

I'd suspect it's a bug because we expect the both behave the same way and 
they shouldn't drop the timezone info.
Anyway, how about creating pandas DataFrame first and split into Series.

I mean as follows:

- python/pyspark/serializers.py
```diff
 reader = pa.open_stream(stream)
 for batch in reader:
-table = pa.Table.from_batches([batch])
-yield [_check_localize_series_timestamps(c.to_pandas()) for c 
in table.itercolumns()]
+pdf = batch.to_pandas()
+yield [_check_localize_series_timestamps(c) for _, c in 
pdf.iteritems()]
```

- python/pyspark/sql/types.py
```diff
 def _check_localize_series_timestamps(s):
-from pandas.types.common import is_datetime64_dtype
+from pandas.types.common import is_datetime64tz_dtype
 # TODO: handle nested timestamps?
-if is_datetime64_dtype(s.dtype):
-# TODO: pyarrow.Column.to_pandas keeps data in UTC but removes 
timezone
-return 
s.dt.tz_localize('UTC').dt.tz_convert('tzlocal()').dt.tz_localize(None)
+if is_datetime64tz_dtype(s.dtype):
+return s.dt.tz_convert('tzlocal()').dt.tz_localize(None)
 else:
 return s
```

We need to check if this causes performance regression or not.

Btw, when using this patch, we can use `date(1969, 1, 1)` and 
`datetime(1969, 1, 1, 1, 1, 1)` for the test you marked as TODO in 
`test_vectorized_udf_timestamps`. I'm not exactly sure the reason, though.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144168563
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala 
---
@@ -31,7 +31,8 @@ object ArrowUtils {
 
   // todo: support more types.
 
-  def toArrowType(dt: DataType): ArrowType = dt match {
+  /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for 
TimestampTypes */
+  def toArrowType(dt: DataType, timeZoneId: Option[String]): ArrowType = 
dt match {
--- End diff --

I don't think it's a good idea to have a default value for `timeZoneId`, 
then it makes it very easy to forget it and miss places in the code that really 
should have a timezone.  It will compile and probably pass tests, but might 
fail somewhere down the line.  Any place in Spark should specify a timezone, so 
it's better to force an argument to be used.  Only in the tests, where we are 
sure no TimestampTypes are used would we set it to None.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144168006
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,47 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+arrow_type = pa.timestamp('us', tz='UTC')
 else:
 raise TypeError("Unsupported type in conversion to Arrow: " + 
str(dt))
 return arrow_type
 
 
+def _check_localize_series_timestamps(s):
+from pandas.types.common import is_datetime64_dtype
+# TODO: handle nested timestamps?
+if is_datetime64_dtype(s.dtype):
+# TODO: pyarrow.Column.to_pandas keeps data in UTC but removes 
timezone
+return 
s.dt.tz_localize('UTC').dt.tz_convert('tzlocal()').dt.tz_localize(None)
+else:
+return s
+
+
+def _check_localize_dataframe_timestamps(df):
+from pandas.types.common import is_datetime64tz_dtype
+for column, series in df.iteritems():
+# TODO: handle nested timestamps?
+if is_datetime64tz_dtype(series.dtype):
+df[column] = 
series.dt.tz_convert('tzlocal()').dt.tz_localize(None)
+return df
+
+
+def _utc_normalize_series_timestamps(s):
--- End diff --

I think maybe "convert_to_spark_internal" could be a better name for this


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144167906
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,47 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+arrow_type = pa.timestamp('us', tz='UTC')
 else:
 raise TypeError("Unsupported type in conversion to Arrow: " + 
str(dt))
 return arrow_type
 
 
+def _check_localize_series_timestamps(s):
+from pandas.types.common import is_datetime64_dtype
+# TODO: handle nested timestamps?
+if is_datetime64_dtype(s.dtype):
+# TODO: pyarrow.Column.to_pandas keeps data in UTC but removes 
timezone
+return 
s.dt.tz_localize('UTC').dt.tz_convert('tzlocal()').dt.tz_localize(None)
+else:
+return s
+
+
+def _check_localize_dataframe_timestamps(df):
+from pandas.types.common import is_datetime64tz_dtype
+for column, series in df.iteritems():
+# TODO: handle nested timestamps?
+if is_datetime64tz_dtype(series.dtype):
+df[column] = 
series.dt.tz_convert('tzlocal()').dt.tz_localize(None)
+return df
+
+
+def _utc_normalize_series_timestamps(s):
+""" Convert a tz-naive timestamp in local tz to UTC normalized
+"""
+# TODO: this should be system local tz or SESSION_LOCAL_TIMEZONE?
+return 
s.dt.tz_localize('tzlocal()').dt.tz_convert('UTC').values.astype('datetime64[us]')
--- End diff --

using `values.astype('datetime64[us]')` was the only way I could the right 
unit to send to Spark.  Even though the pyarrow type specifies the unit, if 
there is a timezone also, it will ignore it.  Not sure if this is a bug.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144167503
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,47 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+arrow_type = pa.timestamp('us', tz='UTC')
 else:
 raise TypeError("Unsupported type in conversion to Arrow: " + 
str(dt))
 return arrow_type
 
 
+def _check_localize_series_timestamps(s):
+from pandas.types.common import is_datetime64_dtype
+# TODO: handle nested timestamps?
+if is_datetime64_dtype(s.dtype):
+# TODO: pyarrow.Column.to_pandas keeps data in UTC but removes 
timezone
--- End diff --

It seems like `pyarrow.Column.to_pandas()` produces a different timestamp 
series than `pyarrow.Table.to_pandas()` to get a DataFrame and then accessing 
the timestamp column.  The former keeps timestamps in UTC and removes the 
timezone, so a different conversion was required to get the data right.  Not 
sure if this is an Arrow bug or not, need to look into it more.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144167095
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1619,11 +1619,47 @@ def to_arrow_type(dt):
 arrow_type = pa.decimal(dt.precision, dt.scale)
 elif type(dt) == StringType:
 arrow_type = pa.string()
+elif type(dt) == DateType:
+arrow_type = pa.date32()
+elif type(dt) == TimestampType:
+arrow_type = pa.timestamp('us', tz='UTC')
--- End diff --

I think it is ok to specify 'UTC' here.  In Spark the `ArrowColumnVector` 
requires a timezone to be set because it expects the Arrow 
`NullableTimeStampMicroTZVector` but it doesn't do anything with the tz, so as 
long as the data is UTC time internally, it should be fine for Spark to use.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144166783
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3383,6 +3400,43 @@ def test_vectorized_udf_varargs(self):
 res = df.select(f(col('id')))
 self.assertEquals(df.collect(), res.collect())
 
+def test_vectorized_udf_timestamps(self):
+from pyspark.sql.functions import pandas_udf, col
+from datetime import date, datetime
+schema = StructType([
+StructField("idx", LongType(), True),
+StructField("date", DateType(), True),
+StructField("timestamp", TimestampType(), True)])
+# TODO Fails with time before epoch: (0, date(1969, 1, 1), 
datetime(1969, 1, 1, 1, 1, 1))
+data = [(0, date(1985, 1, 1), datetime(1985, 1, 1, 1, 1, 1)),
+(1, date(2012, 2, 2), datetime(2012, 2, 2, 2, 2, 2)),
+(2, date(2100, 3, 3), datetime(2100, 3, 3, 3, 3, 3)),
+(3, date(2104, 4, 4), datetime(2104, 4, 4, 4, 4, 4))]
+
+df = self.spark.createDataFrame(data, schema=schema)
+
+# Check that a timestamp passed through a pandas_udf will not be 
altered by timezone calc
+identity = pandas_udf(lambda t: t, returnType=TimestampType())
+df = df.withColumn("timestamp_copy", identity(col("timestamp")))
+
+@pandas_udf(returnType=BooleanType())
+def check_data(idx, date, timestamp, timestamp_copy):
+is_equal = timestamp == timestamp_copy
+if is_equal.all():
+for i in xrange(len(is_equal)):
+# TODO Fails with tz offset: date[i].date() == 
data[idx[i]][1] and
--- End diff --

I think the date types are also getting a timezone conversion which cause 
the equals test here to fail, need to look into this more


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r14414
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3383,6 +3400,43 @@ def test_vectorized_udf_varargs(self):
 res = df.select(f(col('id')))
 self.assertEquals(df.collect(), res.collect())
 
+def test_vectorized_udf_timestamps(self):
+from pyspark.sql.functions import pandas_udf, col
+from datetime import date, datetime
+schema = StructType([
+StructField("idx", LongType(), True),
+StructField("date", DateType(), True),
+StructField("timestamp", TimestampType(), True)])
+# TODO Fails with time before epoch: (0, date(1969, 1, 1), 
datetime(1969, 1, 1, 1, 1, 1))
--- End diff --

Somewhere in the tz_convert process, a timestamp before the epoch causes 
this to fail - not sure where yet


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144166470
  
--- Diff: python/pyspark/serializers.py ---
@@ -223,12 +224,13 @@ def _create_batch(series):
 # If a nullable integer series has been promoted to floating point 
with NaNs, need to cast
 # NOTE: this is not necessary with Arrow >= 0.7
 def cast_series(s, t):
-if t is None or s.dtype == t.to_pandas_dtype():
+if t is None or s.dtype == t.to_pandas_dtype() or type(t) == 
pa.TimestampType:
--- End diff --

for timestamps, pyarrow.DataType.to_pandas_dtype will be a "datetime64[us]" 
while s.dtype is "datetime64[ns]" so they are not equal but trying to use 
`astype` will give an error.  So I think this is fine.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144148579
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala 
---
@@ -31,7 +31,8 @@ object ArrowUtils {
 
   // todo: support more types.
 
-  def toArrowType(dt: DataType): ArrowType = dt match {
+  /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for 
TimestampTypes */
+  def toArrowType(dt: DataType, timeZoneId: Option[String]): ArrowType = 
dt match {
--- End diff --

Btw, why don't we set default values for `timeZoneId`? Seems a bit weird 
having to specify `None` for other types.


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144148043
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala 
---
@@ -31,7 +31,8 @@ object ArrowUtils {
 
   // todo: support more types.
 
-  def toArrowType(dt: DataType): ArrowType = dt match {
+  /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for 
TimestampTypes */
+  def toArrowType(dt: DataType, timeZoneId: Option[String]): ArrowType = 
dt match {
--- End diff --

Hmm, I see, it's required only if dt is `TimestampType`. 


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144147353
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala 
---
@@ -31,7 +31,8 @@ object ArrowUtils {
 
   // todo: support more types.
 
-  def toArrowType(dt: DataType): ArrowType = dt match {
+  /** Maps data type from Spark to Arrow. NOTE: timeZoneId required for 
TimestampTypes */
+  def toArrowType(dt: DataType, timeZoneId: Option[String]): ArrowType = 
dt match {
--- End diff --

Why do we use `Option` if this is not allowed to be `None`?


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-11 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r144146290
  
--- Diff: python/pyspark/serializers.py ---
@@ -213,6 +213,7 @@ def __repr__(self):
 
 
 def _create_batch(series):
+from pyspark.sql.types import _check_convert_series_timestamps
--- End diff --

Should this import be at the top?


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-10-10 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r143885245
  
--- Diff: python/pyspark/sql/types.py ---
@@ -1624,6 +1624,40 @@ def to_arrow_type(dt):
 return arrow_type
 
 
+def _localize_series_timestamps(s):
+""" Convert a tz-aware timestamp to local tz-naive
+"""
+return s.dt.tz_localize(None)
+
+
+def _check_localize_series_timestamps(s):
+from pandas.types.common import is_datetime64tz_dtype
+# TODO: handle nested timestamps?
+return _localize_series_timestamps(s) if 
is_datetime64tz_dtype(s.dtype) else s
+
+
+def _check_localize_dataframe_timestamps(df):
+from pandas.types.common import is_datetime64tz_dtype
+for column, series in df.iteritems():
+# TODO: handle nested timestamps?
+if is_datetime64tz_dtype(series.dtype):
+df[column] = _localize_series_timestamps(series)
+return df
+
+
+def _convert_series_timestamps(s):
+""" Convert a tz-naive timestamp in local tz to UTC normalized
+"""
+# TODO: this should be system local tz or SESSION_LOCAL_TIMEZONE?
+return s.dt.tz_convert("UTC")
--- End diff --

this function is called on the returned Series from a `pandas_udf`.  I 
_think_ this will work because internally Spark just wants long values 
normalized to UTC, and will not do anything with the actual timezone id when 
read in with `ArrowColumnVector`.  What do you think?


---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-15 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r133261665
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

@ueshin , the change in #18933 seems to be to just fix this test and I 
think it makes things more confusing.  Also, while that new conf would allow 
`test_toPandas_arrow_toggle` to pass, `test_pandas_round_trip` would still fail 
without having to de-localize the timestamps.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-14 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r133058855
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

Ok, thanks - I'll take a look.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-13 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r132870799
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

I sent a pr #18933 as a WIP for "without-Arrow" version.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-07 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131723483
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

Without Arrow, Spark will provide timestamps that are tz-naive.  We would 
need to construct the Pandas DataFrame, then call `tz_localize` with the 
`SESSION_LOCAL_TIMEZONE` which would then change the internal values to be 
normalized to UTC, as @wesm pointed out above.  I don't think this is a very 
good idea, because it assumes the timestamp was created with the 
`SESSION_LOCAL_TIMEZONE`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131532115
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

without Arrow?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-04 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131506100
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

Which way do you mean, with or without Arrow?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-04 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131503699
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

Actually, I have a question. How does Pandas DataFrame consume the time 
zone we create? Will our output impact them?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-04 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131496971
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

Yes, this test is verify behavior is the same with/without Arrow - that is 
why I made it - and I think we all agree that is the goal.  I'm trying to 
understand if you are suggesting that this new conf, only if enabled, will add 
the session local timezone to the Pandas DataFrame made in 
 `toPandas()` with and without Arrow enabled?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-04 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131487406
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

Let me explain it a little bit more. We are unable to break the backward 
compatibility. When users upgrade to a new version of Spark, we should not 
change our way to read/write data. 

In this specific case, `toPandas()` should respect timezone, as what 
@icexelloss said. I believe we have made such a consensus in this PR. Now, the 
issue is how to fix it without breaking the existing users/applications who 
already rely on the previous/existing Spark behaviors. The solution is to 
introduce a new external configuration for this. Unfortunately, by default, we 
have to turn it off. Maybe in Spark 3.0, we can turn it on by default? 

Second, with/without enabling arrow, Spark should have the exactly same 
external behavior (except performance). This is the rule we have to follow. Do 
you agree?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-04 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131452100
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

@ueshin so you are saying that the timezone would only be added in Python 
when making the Pandas DataFrame?  Then Java would only use an Arrow timestamp 
without a timezone specified and treat it as timezone naive?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-03 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131308903
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

I talked to @gatorsmile and he suggested that we should have another 
configuration to control the behavior of `df.toPandas()` to handle timezone or 
not, the default behavior of which is to not handle it as the same as the 
current behavior.
I'll submit a pr to do it with "without-Arrow" version asap and I'd like 
you to follow the behavior.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-03 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131227296
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

@gatorsmile, can you be a bit explicit about the behavior we want? As we 
discovered that the current behavior of `df.toPandas()` is wrong and 
inconsistent with Spark SQL (because `df,toPandas()` doesn't respect 
`LOCAL_SESSION_TIMEZONE`), do you think we should:
1. be consistent with default `df.toPandas()` even though it is wrong
or:
2. be consistent with Spark SQL

Also this is not huge issue IMHO because we should fix the behavior of 
`df.toPandas()` soon anyway.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r131050585
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

Unfortunately, we can't accept the behavior like this. This will confuse 
our users. 

@ueshin can help you address this.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-02 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r130942914
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

Because of the way Spark handles timestamps, there needs to be a timezone 
in the Arrow data.  So when comparing `toPandas()` both ways, one needs to be 
changed.  I don't think it would be correct to add a timezone to the 
pandas.DataFrame created when `arrow.enable` is false.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-02 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r130940849
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
--- End diff --

It's meant to be enabled for the duration of running `ArrowTests` (except 
for 1 test that disables then enables again).  The `SparkSession` is stopped in 
`tearDownClass` so it will not affect any other tests right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-02 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r130925333
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
 pdf_arrow = df.toPandas()
+# need to remove timezone for comparison
+pdf_arrow["7_timestamp_t"] = \
+pdf_arrow["7_timestamp_t"].apply(lambda ts: 
ts.tz_localize(None))
--- End diff --

This breaks our assumption. No behavior difference should exist when we 
enable/disable `spark.sql.execution.arrow.enable`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-01 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r130795748
  
--- Diff: python/pyspark/sql/tests.py ---
@@ -3036,6 +3052,9 @@ def test_toPandas_arrow_toggle(self):
 pdf = df.toPandas()
 self.spark.conf.set("spark.sql.execution.arrow.enable", "true")
--- End diff --

We do not set it back after converting to `true`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-01 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r13079
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala 
---
@@ -42,6 +43,9 @@ object ArrowUtils {
 case StringType => ArrowType.Utf8.INSTANCE
 case BinaryType => ArrowType.Binary.INSTANCE
 case DecimalType.Fixed(precision, scale) => new 
ArrowType.Decimal(precision, scale)
+case DateType => new ArrowType.Date(DateUnit.DAY)
+case TimestampType =>
+  new ArrowType.Timestamp(TimeUnit.MICROSECOND, 
DateTimeUtils.defaultTimeZone().getID)
--- End diff --

This is wrong, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-08-01 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r130792754
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala ---
@@ -3092,7 +3092,8 @@ class Dataset[T] private[sql](
 val maxRecordsPerBatch = 
sparkSession.sessionState.conf.arrowMaxRecordsPerBatch
 queryExecution.toRdd.mapPartitionsInternal { iter =>
   val context = TaskContext.get()
-  ArrowConverters.toPayloadIterator(iter, schemaCaptured, 
maxRecordsPerBatch, context)
+  ArrowConverters.toPayloadIterator(
+iter, schemaCaptured, maxRecordsPerBatch, context)
--- End diff --

Revert this back?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-07-28 Thread icexelloss
Github user icexelloss commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r130201966
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala 
---
@@ -42,6 +43,9 @@ object ArrowUtils {
 case StringType => ArrowType.Utf8.INSTANCE
 case BinaryType => ArrowType.Binary.INSTANCE
 case DecimalType.Fixed(precision, scale) => new 
ArrowType.Decimal(precision, scale)
+case DateType => new ArrowType.Date(DateUnit.DAY)
+case TimestampType =>
+  new ArrowType.Timestamp(TimeUnit.MICROSECOND, 
DateTimeUtils.defaultTimeZone().getID)
--- End diff --

I do think we should use SQLConf.SESSION_LOCAL_TIMEZONE in this PR. I am 
concerned about potential inconsistent behavior due to system timezone like 
@wesm mentioned.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-07-27 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r130018225
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -792,6 +793,104 @@ class ArrowConvertersSuite extends SharedSQLContext 
with BeforeAndAfterAll {
 collectAndValidate(df, json, "binaryData.json")
   }
 
+  test("date type conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "date",
+ |  "type" : {
+ |"name" : "date",
+ |"unit" : "DAY"
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 32
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 4,
+ |"columns" : [ {
+ |  "name" : "date",
+ |  "count" : 4,
+ |  "VALIDITY" : [ 1, 1, 1, 1 ],
+ |  "DATA" : [ -1, 0, 16533, 382607 ]
+ |} ]
+ |  } ]
+ |}
+   """.stripMargin
+
+val sdf = new SimpleDateFormat("-MM-dd HH:mm:ss.SSS z", Locale.US)
+val d1 = DateTimeUtils.toJavaDate(-1)  // "1969-12-31"
+val d2 = DateTimeUtils.toJavaDate(0)  // "1970-01-01"
+val d3 = new Date(sdf.parse("2015-04-08 13:10:15.000 UTC").getTime)
+val d4 = new Date(sdf.parse("3017-07-18 14:55:00.000 UTC").getTime)
--- End diff --

`d3` and `d4` might be flaky in some timezone.
Should we use `Date.valueOf()`?:

```scala
val d3 = Date.valueOf("2015-04-08")
val d4 = Date.valueOf("3017-07-18")
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-07-18 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r128107798
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -792,6 +793,76 @@ class ArrowConvertersSuite extends SharedSQLContext 
with BeforeAndAfterAll {
 collectAndValidate(df, json, "binaryData.json")
   }
 
+  test("date type conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "date",
+ |  "type" : {
+ |"name" : "date",
+ |"unit" : "DAY"
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 32
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 4,
+ |"columns" : [ {
+ |  "name" : "date",
+ |  "count" : 4,
+ |  "VALIDITY" : [ 1, 1, 1, 1 ],
+ |  "DATA" : [ -1, 0, 16533, 16930 ]
+ |} ]
+ |  } ]
+ |}
+   """.stripMargin
+
+val sdf = new SimpleDateFormat("-MM-dd HH:mm:ss.SSS z", Locale.US)
+val d1 = new Date(-1)  // "1969-12-31 13:10:15.000 UTC"
+val d2 = new Date(0)  // "1970-01-01 13:10:15.000 UTC"
+val d3 = new Date(sdf.parse("2015-04-08 13:10:15.000 UTC").getTime)
+val d4 = new Date(sdf.parse("2016-05-09 12:01:01.000 UTC").getTime)
+
+// Date is created unaware of timezone, but DateTimeUtils force 
defaultTimeZone()
+
assert(DateTimeUtils.toJavaDate(DateTimeUtils.fromJavaDate(d2)).getTime == 
d2.getTime)
--- End diff --

Thanks @ueshin!  I think I got it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-07-17 Thread ueshin
Github user ueshin commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r127879502
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -792,6 +793,76 @@ class ArrowConvertersSuite extends SharedSQLContext 
with BeforeAndAfterAll {
 collectAndValidate(df, json, "binaryData.json")
   }
 
+  test("date type conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "date",
+ |  "type" : {
+ |"name" : "date",
+ |"unit" : "DAY"
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 32
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 4,
+ |"columns" : [ {
+ |  "name" : "date",
+ |  "count" : 4,
+ |  "VALIDITY" : [ 1, 1, 1, 1 ],
+ |  "DATA" : [ -1, 0, 16533, 16930 ]
+ |} ]
+ |  } ]
+ |}
+   """.stripMargin
+
+val sdf = new SimpleDateFormat("-MM-dd HH:mm:ss.SSS z", Locale.US)
+val d1 = new Date(-1)  // "1969-12-31 13:10:15.000 UTC"
+val d2 = new Date(0)  // "1970-01-01 13:10:15.000 UTC"
+val d3 = new Date(sdf.parse("2015-04-08 13:10:15.000 UTC").getTime)
+val d4 = new Date(sdf.parse("2016-05-09 12:01:01.000 UTC").getTime)
+
+// Date is created unaware of timezone, but DateTimeUtils force 
defaultTimeZone()
+
assert(DateTimeUtils.toJavaDate(DateTimeUtils.fromJavaDate(d2)).getTime == 
d2.getTime)
--- End diff --

We handle `DateType` value as days from `1970-01-01` internally.

When converting from/to `Date` to/from internal value, we assume the `Date` 
instance contains the timestamp of `00:00:00` time of the day in 
`TimeZone.getDefault()` timezone, which is the offset of the timezone. e.g. in 
JST (GMT+09:00):

```
scala> TimeZone.setDefault(TimeZone.getTimeZone("JST"))

scala> Date.valueOf("1970-01-01").getTime()
res6: Long = -3240
```

whereas in PST (GMT-08:00):

```
scala> TimeZone.setDefault(TimeZone.getTimeZone("PST"))

scala> Date.valueOf("1970-01-01").getTime()
res8: Long = 2880
```

We use `DateTimeUtils.defaultTimeZone()` to adjust the offset.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-07-17 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r127864419
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -792,6 +793,76 @@ class ArrowConvertersSuite extends SharedSQLContext 
with BeforeAndAfterAll {
 collectAndValidate(df, json, "binaryData.json")
   }
 
+  test("date type conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "date",
+ |  "type" : {
+ |"name" : "date",
+ |"unit" : "DAY"
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 32
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 4,
+ |"columns" : [ {
+ |  "name" : "date",
+ |  "count" : 4,
+ |  "VALIDITY" : [ 1, 1, 1, 1 ],
+ |  "DATA" : [ -1, 0, 16533, 16930 ]
+ |} ]
+ |  } ]
+ |}
+   """.stripMargin
+
+val sdf = new SimpleDateFormat("-MM-dd HH:mm:ss.SSS z", Locale.US)
+val d1 = new Date(-1)  // "1969-12-31 13:10:15.000 UTC"
+val d2 = new Date(0)  // "1970-01-01 13:10:15.000 UTC"
+val d3 = new Date(sdf.parse("2015-04-08 13:10:15.000 UTC").getTime)
+val d4 = new Date(sdf.parse("2016-05-09 12:01:01.000 UTC").getTime)
+
+// Date is created unaware of timezone, but DateTimeUtils force 
defaultTimeZone()
+
assert(DateTimeUtils.toJavaDate(DateTimeUtils.fromJavaDate(d2)).getTime == 
d2.getTime)
--- End diff --

cc @ueshin


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-07-17 Thread BryanCutler
Github user BryanCutler commented on a diff in the pull request:

https://github.com/apache/spark/pull/18664#discussion_r127861741
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala
 ---
@@ -792,6 +793,76 @@ class ArrowConvertersSuite extends SharedSQLContext 
with BeforeAndAfterAll {
 collectAndValidate(df, json, "binaryData.json")
   }
 
+  test("date type conversion") {
+val json =
+  s"""
+ |{
+ |  "schema" : {
+ |"fields" : [ {
+ |  "name" : "date",
+ |  "type" : {
+ |"name" : "date",
+ |"unit" : "DAY"
+ |  },
+ |  "nullable" : true,
+ |  "children" : [ ],
+ |  "typeLayout" : {
+ |"vectors" : [ {
+ |  "type" : "VALIDITY",
+ |  "typeBitWidth" : 1
+ |}, {
+ |  "type" : "DATA",
+ |  "typeBitWidth" : 32
+ |} ]
+ |  }
+ |} ]
+ |  },
+ |  "batches" : [ {
+ |"count" : 4,
+ |"columns" : [ {
+ |  "name" : "date",
+ |  "count" : 4,
+ |  "VALIDITY" : [ 1, 1, 1, 1 ],
+ |  "DATA" : [ -1, 0, 16533, 16930 ]
+ |} ]
+ |  } ]
+ |}
+   """.stripMargin
+
+val sdf = new SimpleDateFormat("-MM-dd HH:mm:ss.SSS z", Locale.US)
+val d1 = new Date(-1)  // "1969-12-31 13:10:15.000 UTC"
+val d2 = new Date(0)  // "1970-01-01 13:10:15.000 UTC"
+val d3 = new Date(sdf.parse("2015-04-08 13:10:15.000 UTC").getTime)
+val d4 = new Date(sdf.parse("2016-05-09 12:01:01.000 UTC").getTime)
+
+// Date is created unaware of timezone, but DateTimeUtils force 
defaultTimeZone()
+
assert(DateTimeUtils.toJavaDate(DateTimeUtils.fromJavaDate(d2)).getTime == 
d2.getTime)
--- End diff --

@holdenk @cloud-fan I'm trying out the DateType conversion and ran into 
this problem.  The Dataset encoder uses `DateTimeUtils.toJavaDate` and 
`fromJavaDate` similar to above, and this forces a `defaultTimeZone()` when 
working with the data.  So a value `new Date(0)` should be the epoch, but in my 
timezone it forces it to be the day before and the test above will not pass.  

What are your thoughts on this, should the conversion to Arrow assume the 
defaultTimeZone()?  is this something that should be fixed first in Spark?  
Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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



[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...

2017-07-17 Thread BryanCutler
GitHub user BryanCutler opened a pull request:

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

[SPARK-21375][PYSPARK][SQL][WIP] Add Date and Timestamp support to 
ArrowConverters for toPandas() Conversion

## What changes were proposed in this pull request?

WIP started with DateType

## How was this patch tested?


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

$ git pull https://github.com/BryanCutler/spark 
arrow-date-timestamp-SPARK-21375

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

https://github.com/apache/spark/pull/18664.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 #18664


commit 5aa8b9e72aee17ffa51f4cb1048f5a3f93a5a380
Author: Bryan Cutler 
Date:   2017-07-13T16:53:31Z

added date type and started test, still some issue with time difference

commit 20313f92758e5639b309ba810945a8415941ef86
Author: Bryan Cutler 
Date:   2017-07-18T00:42:15Z

DateTimeUtils forces defaultTimeZone

commit 69e1e21bf4bebc7bea6bd9322e4300df71a90b18
Author: Bryan Cutler 
Date:   2017-07-18T00:48:47Z

fix style checks




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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