-
Notifications
You must be signed in to change notification settings - Fork 28.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-22395][SQL][PYTHON] Fix the behavior of timestamp values for Pandas to respect session timezone #19607
Closed
Closed
Changes from 9 commits
Commits
Show all changes
29 commits
Select commit
Hold shift + click to select a range
4735e59
Add a conf to make Pandas DataFrame respect session local timezone.
ueshin 1f85150
Fix toPandas() behavior.
ueshin 5c08ecf
Modify pandas UDFs to respect session timezone.
ueshin ee1a1c8
Workaround for old pandas.
ueshin b1436b8
Don't use is_datetime64tz_dtype for old pandas.
ueshin 6872516
Fix one of the failed tests.
ueshin 1f096bf
Modify check_data udf for debug messages.
ueshin 569bb63
Remove unused method.
ueshin ce07f39
Modify a test.
ueshin ba3d6e3
Add debug print, which will be removed later.
ueshin 9101a3a
Fix style.
ueshin ab13baf
Remove debug prints.
ueshin 4adb073
Modify tests to avoid times within DST.
ueshin 1e0f217
Clean up.
ueshin d18cd36
Merge branch 'master' into issues/SPARK-22395
ueshin 292678f
Fix the behavior of createDataFrame from pandas DataFrame.
ueshin f37c067
Merge branch 'master' into issues/SPARK-22395
ueshin 8b1a4d8
Add a test to check the behavior of createDataFrame from pandas DataF…
ueshin e919ed5
Clarify the usage of Row.
ueshin 9c94f90
Merge branch 'master' into issues/SPARK-22395
ueshin 9cfdde2
Add TODOs for nested timestamp fields.
ueshin 8b1a4a1
Remove workarounds for old Pandas but add some error messages saying …
ueshin 3db2bea
Fix tests.
ueshin 3e23653
Use `_exception_message()` to access error messages.
ueshin d741171
Fix a test.
ueshin e240631
Add a description about deprecation of the config.
ueshin f92eae3
Add migration guide.
ueshin 40a9735
Merge branch 'master' into issues/SPARK-22395
ueshin 9200f38
Address comments.
ueshin File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -2560,19 +2560,25 @@ def count_bucketed_cols(names, table="pyspark_bucket"): | |
|
||
@unittest.skipIf(not _have_pandas, "Pandas not installed") | ||
def test_to_pandas(self): | ||
from datetime import datetime, date | ||
import numpy as np | ||
schema = StructType().add("a", IntegerType()).add("b", StringType())\ | ||
.add("c", BooleanType()).add("d", FloatType()) | ||
.add("c", BooleanType()).add("d", FloatType())\ | ||
.add("dt", DateType()).add("ts", TimestampType()) | ||
data = [ | ||
(1, "foo", True, 3.0), (2, "foo", True, 5.0), | ||
(3, "bar", False, -1.0), (4, "bar", False, 6.0), | ||
(1, "foo", True, 3.0, date(1969, 1, 1), datetime(1969, 1, 1, 1, 1, 1)), | ||
(2, "foo", True, 5.0, None, None), | ||
(3, "bar", False, -1.0, date(2012, 3, 3), datetime(2012, 3, 3, 3, 3, 3)), | ||
(4, "bar", False, 6.0, date(2100, 4, 4), datetime(2100, 4, 4, 4, 4, 4)), | ||
] | ||
df = self.spark.createDataFrame(data, schema) | ||
types = df.toPandas().dtypes | ||
self.assertEquals(types[0], np.int32) | ||
self.assertEquals(types[1], np.object) | ||
self.assertEquals(types[2], np.bool) | ||
self.assertEquals(types[3], np.float32) | ||
self.assertEquals(types[4], 'datetime64[ns]') | ||
self.assertEquals(types[5], 'datetime64[ns]') | ||
|
||
@unittest.skipIf(not _have_pandas, "Pandas not installed") | ||
def test_to_pandas_avoid_astype(self): | ||
|
@@ -3136,14 +3142,42 @@ def test_null_conversion(self): | |
null_counts = pdf.isnull().sum().tolist() | ||
self.assertTrue(all([c == 1 for c in null_counts])) | ||
|
||
def test_toPandas_arrow_toggle(self): | ||
df = self.spark.createDataFrame(self.data, schema=self.schema) | ||
def _toPandas_arrow_toggle(self, df): | ||
self.spark.conf.set("spark.sql.execution.arrow.enabled", "false") | ||
pdf = df.toPandas() | ||
self.spark.conf.set("spark.sql.execution.arrow.enabled", "true") | ||
try: | ||
pdf = df.toPandas() | ||
finally: | ||
self.spark.conf.set("spark.sql.execution.arrow.enabled", "true") | ||
pdf_arrow = df.toPandas() | ||
return pdf, pdf_arrow | ||
|
||
def test_toPandas_arrow_toggle(self): | ||
df = self.spark.createDataFrame(self.data, schema=self.schema) | ||
pdf, pdf_arrow = self._toPandas_arrow_toggle(df) | ||
self.assertFramesEqual(pdf_arrow, pdf) | ||
|
||
def test_toPandas_respect_session_timezone(self): | ||
df = self.spark.createDataFrame(self.data, schema=self.schema) | ||
orig_tz = self.spark.conf.get("spark.sql.session.timeZone") | ||
try: | ||
timezone = "America/New_York" | ||
self.spark.conf.set("spark.sql.session.timeZone", timezone) | ||
self.spark.conf.set("spark.sql.execution.pandas.respectSessionTimeZone", "false") | ||
try: | ||
pdf_la, pdf_arrow_la = self._toPandas_arrow_toggle(df) | ||
self.assertFramesEqual(pdf_arrow_la, pdf_la) | ||
finally: | ||
self.spark.conf.set("spark.sql.execution.pandas.respectSessionTimeZone", "true") | ||
pdf_ny, pdf_arrow_ny = self._toPandas_arrow_toggle(df) | ||
self.assertFramesEqual(pdf_arrow_ny, pdf_ny) | ||
|
||
from pyspark.sql.types import _check_dataframe_localize_timestamps | ||
self.assertFalse(pdf_ny.equals(pdf_la)) | ||
self.assertTrue(pdf_ny.equals( | ||
_check_dataframe_localize_timestamps(pdf_la, self.schema, timezone))) | ||
finally: | ||
self.spark.conf.set("spark.sql.session.timeZone", orig_tz) | ||
|
||
def test_pandas_round_trip(self): | ||
import pandas as pd | ||
import numpy as np | ||
|
@@ -3169,6 +3203,27 @@ def test_filtered_frame(self): | |
@unittest.skipIf(not _have_pandas or not _have_arrow, "Pandas or Arrow not installed") | ||
class VectorizedUDFTests(ReusedSQLTestCase): | ||
|
||
@classmethod | ||
def setUpClass(cls): | ||
ReusedSQLTestCase.setUpClass() | ||
|
||
# Synchronize default timezone between Python and Java | ||
cls.tz_prev = os.environ.get("TZ", None) # save current tz if set | ||
tz = "America/Los_Angeles" | ||
os.environ["TZ"] = tz | ||
time.tzset() | ||
|
||
cls.sc.environment["TZ"] = tz | ||
cls.spark.conf.set("spark.sql.session.timeZone", tz) | ||
|
||
@classmethod | ||
def tearDownClass(cls): | ||
del os.environ["TZ"] | ||
if cls.tz_prev is not None: | ||
os.environ["TZ"] = cls.tz_prev | ||
time.tzset() | ||
ReusedSQLTestCase.tearDownClass() | ||
|
||
def test_vectorized_udf_basic(self): | ||
from pyspark.sql.functions import pandas_udf, col | ||
df = self.spark.range(10).select( | ||
|
@@ -3429,22 +3484,29 @@ def test_vectorized_udf_timestamps(self): | |
f_timestamp_copy = pandas_udf(lambda t: t, returnType=TimestampType()) | ||
df = df.withColumn("timestamp_copy", f_timestamp_copy(col("timestamp"))) | ||
|
||
@pandas_udf(returnType=BooleanType()) | ||
@pandas_udf(returnType=StringType()) | ||
def check_data(idx, timestamp, timestamp_copy): | ||
import pandas as pd | ||
msgs = [] | ||
is_equal = timestamp.isnull() # use this array to check values are equal | ||
for i in range(len(idx)): | ||
# Check that timestamps are as expected in the UDF | ||
is_equal[i] = (is_equal[i] and data[idx[i]][1] is None) or \ | ||
timestamp[i].to_pydatetime() == data[idx[i]][1] | ||
return is_equal | ||
|
||
result = df.withColumn("is_equal", check_data(col("idx"), col("timestamp"), | ||
col("timestamp_copy"))).collect() | ||
if (is_equal[i] and data[idx[i]][1] is None) or \ | ||
timestamp[i].to_pydatetime() == data[idx[i]][1]: | ||
msgs.append(None) | ||
else: | ||
msgs.append( | ||
"timestamp values are not equal (timestamp='%s': data[%d][1]='%s')" | ||
% (timestamp[i], idx[i], data[idx[i]][1])) | ||
return pd.Series(msgs) | ||
|
||
result = df.withColumn("check_data", check_data(col("idx"), col("timestamp"), | ||
col("timestamp_copy"))).collect() | ||
# Check that collection values are correct | ||
self.assertEquals(len(data), len(result)) | ||
for i in range(len(result)): | ||
self.assertEquals(data[i][1], result[i][1]) # "timestamp" col | ||
self.assertTrue(result[i][3]) # "is_equal" data in udf was as expected | ||
self.assertIsNone(result[i][3]) # "check_data" col | ||
|
||
def test_vectorized_udf_return_timestamp_tz(self): | ||
from pyspark.sql.functions import pandas_udf, col | ||
|
@@ -3484,6 +3546,47 @@ def check_records_per_batch(x): | |
else: | ||
self.spark.conf.set("spark.sql.execution.arrow.maxRecordsPerBatch", orig_value) | ||
|
||
def test_vectorized_udf_timestamps_respect_session_timezone(self): | ||
from pyspark.sql.functions import pandas_udf, col | ||
from datetime import datetime | ||
import pandas as pd | ||
schema = StructType([ | ||
StructField("idx", LongType(), True), | ||
StructField("timestamp", TimestampType(), True)]) | ||
data = [(1, datetime(1969, 1, 1, 1, 1, 1)), | ||
(2, datetime(2012, 2, 2, 2, 2, 2)), | ||
(3, None), | ||
(4, datetime(2100, 4, 4, 4, 4, 4))] | ||
df = self.spark.createDataFrame(data, schema=schema) | ||
|
||
f_timestamp_copy = pandas_udf(lambda ts: ts, TimestampType()) | ||
internal_value = pandas_udf( | ||
lambda ts: ts.apply(lambda ts: ts.value if ts is not pd.NaT else None), LongType()) | ||
|
||
orig_tz = self.spark.conf.get("spark.sql.session.timeZone") | ||
try: | ||
timezone = "America/New_York" | ||
self.spark.conf.set("spark.sql.session.timeZone", timezone) | ||
self.spark.conf.set("spark.sql.execution.pandas.respectSessionTimeZone", "false") | ||
try: | ||
df_la = df.withColumn("tscopy", f_timestamp_copy(col("timestamp"))) \ | ||
.withColumn("internal_value", internal_value(col("timestamp"))) | ||
result_la = df_la.select(col("idx"), col("internal_value")).collect() | ||
diff = 3 * 60 * 60 * 1000 * 1000 * 1000 | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here too. it took me a while to check where this 3 came from .. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, I'll add some comments. |
||
result_la_corrected = \ | ||
df_la.select(col("idx"), col("tscopy"), col("internal_value") + diff).collect() | ||
finally: | ||
self.spark.conf.set("spark.sql.execution.pandas.respectSessionTimeZone", "true") | ||
|
||
df_ny = df.withColumn("tscopy", f_timestamp_copy(col("timestamp"))) \ | ||
.withColumn("internal_value", internal_value(col("timestamp"))) | ||
result_ny = df_ny.select(col("idx"), col("tscopy"), col("internal_value")).collect() | ||
|
||
self.assertNotEqual(result_ny, result_la) | ||
self.assertEqual(result_ny, result_la_corrected) | ||
finally: | ||
self.spark.conf.set("spark.sql.session.timeZone", orig_tz) | ||
|
||
|
||
@unittest.skipIf(not _have_pandas or not _have_arrow, "Pandas or Arrow not installed") | ||
class GroupbyApplyTests(ReusedSQLTestCase): | ||
|
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If
self._timezone
is not None, then it will be the SESSION_LOCAL_TIMEZONE and Arrow data will already have this timezone set so nothing needs to be done here right?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, maybe I misunderstood the purpose of this conf "spark.sql.execution.pandas.respectSessionTimeZone". If that is true then what is the behavior of Spark?
convert timestamps in Pandas to remove the timezone and localize to SESSION_LOCAL_TIMEZONE
show Pandas timestamps with SESSION_LOCAL_TIMEZONE set as the timezone
It seems this change is doing (1), but what's wrong with doing (2)? I think that would be a lot cleaner
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, current implementation is doing (1). I'm not sure if we should hold the timezone. cc @cloud-fan @gatorsmile