[GitHub] spark pull request #18664: [SPARK-21375][PYSPARK][SQL][WIP] Add Date and Tim...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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