Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
63 commits
Select commit Hold shift + click to select a range
81f4866
Implemented is_variant_null expression
harshmotw-db Apr 11, 2024
3673ef7
Merge branch 'apache:master' into master
harshmotw-db Apr 11, 2024
7acb773
minor change
harshmotw-db Apr 11, 2024
efe4ba1
Intermediate step in mapping from_json to parse_json
harshmotw-db Apr 11, 2024
e1e561d
Regenerated golden files
harshmotw-db Apr 12, 2024
c2329f2
Fixed some comments left by Chenhao (one still remaining)
harshmotw-db Apr 12, 2024
cb973a9
Merge branch 'apache:master' into master
harshmotw-db Apr 12, 2024
f82c11a
Fixed major bug in the StaticInvoke call in IsVariantNull
harshmotw-db Apr 12, 2024
85afe57
Added is_variant_null tests in VariantEndToEndSuite where it is worki…
harshmotw-db Apr 12, 2024
506dd20
Adding one more test to re trigger CI/CD tests. Currently, there are …
harshmotw-db Apr 12, 2024
c0607f0
regenerated golden files again
harshmotw-db Apr 12, 2024
af0ddc2
Added support for variant in from_json
harshmotw-db Apr 15, 2024
c2dd13a
Added unit tests to check if from_json is working as intended when sc…
harshmotw-db Apr 15, 2024
763a478
Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expr…
harshmotw-db Apr 15, 2024
82b3a27
Merge branch 'from_json_variant' of https://github.com/harshmotw-db/s…
harshmotw-db Apr 15, 2024
7767e55
Merge branch 'master' into master
harshmotw-db Apr 15, 2024
b32b78d
Merge branch 'master' into master
harshmotw-db Apr 16, 2024
6d8a588
temporary commit
harshmotw-db Apr 16, 2024
db4766d
change to help with merging
harshmotw-db Apr 16, 2024
a495b8d
change to help with merging
harshmotw-db Apr 16, 2024
2de1bec
Merge branch 'master' into from_json_variant
harshmotw-db Apr 16, 2024
1340ace
Fixed error message in error-conditions.json to incorporate variant type
harshmotw-db Apr 16, 2024
8e09ce0
Added intermediate support for variant in pyspark
harshmotw-db Apr 17, 2024
4a6b522
Added remaining scalar types to the Python Variant library
harshmotw-db Apr 18, 2024
678f02a
Removed unnecessary changes
harshmotw-db Apr 18, 2024
9a8ac8d
style improvements
harshmotw-db Apr 18, 2024
3c40bf2
Removed unnecessary changes
harshmotw-db Apr 18, 2024
74c8476
Removed unnecessary changes
harshmotw-db Apr 18, 2024
70c1336
Removed unnecessary changes
harshmotw-db Apr 18, 2024
2bfa77c
Removed unnecessary changes
harshmotw-db Apr 18, 2024
4f5b86e
Removed unnecessary changes
harshmotw-db Apr 18, 2024
5a3aed8
Removed unnecessary zone id in _to_python
harshmotw-db Apr 18, 2024
6db38ce
Removed unnecessary changes
harshmotw-db Apr 18, 2024
533c01a
Removed unnecessary changes
harshmotw-db Apr 18, 2024
f0742cf
Merge branch 'master' of https://github.com/harshmotw-db/spark into p…
harshmotw-db Apr 18, 2024
f53ee04
Resolved merge conflicts
harshmotw-db Apr 18, 2024
9ecbdc7
Merge branch 'master' of https://github.com/harshmotw-db/spark into p…
harshmotw-db Apr 18, 2024
69c53e8
Removed unnecessary changes
harshmotw-db Apr 18, 2024
402f380
Temporary commit
harshmotw-db Apr 18, 2024
428b354
Added changes recommended by Gene
harshmotw-db Apr 18, 2024
a5073da
Remove unnecessary changes
harshmotw-db Apr 18, 2024
5967553
Fixed Hyukjin's comment
harshmotw-db Apr 19, 2024
0afcfae
Fixed Python linting
harshmotw-db Apr 19, 2024
6920915
Merge branch 'python_scalar_variant' of https://github.com/harshmotw-…
harshmotw-db Apr 19, 2024
351f2d2
Made linting change
harshmotw-db Apr 19, 2024
b7d24ce
Replaced pytz with zoneinfo
harshmotw-db Apr 19, 2024
16e7b63
removed unnecessary change
harshmotw-db Apr 19, 2024
85ab57f
Fixed linting and a comment
harshmotw-db Apr 19, 2024
e3f494f
Fixed lint comment
harshmotw-db Apr 19, 2024
524a4f4
Fixed documentation message
harshmotw-db Apr 19, 2024
d70c2db
Fixed toJson function
harshmotw-db Apr 19, 2024
8cce917
Fixed minor error in _to_json
harshmotw-db Apr 19, 2024
473a0f2
Added quotes around date and timestamp in toJson
harshmotw-db Apr 19, 2024
6f06be3
minor changes
harshmotw-db Apr 19, 2024
bd4e0a9
minor changes
harshmotw-db Apr 19, 2024
764e633
Merge branch 'apache:master' into master
harshmotw-db Apr 19, 2024
a3ceee9
Implemented try_parse_json
harshmotw-db Apr 19, 2024
06c9a65
More python linting
harshmotw-db Apr 19, 2024
2e3863e
Added change for document generation
harshmotw-db Apr 19, 2024
731f6b5
Merge branch 'apache:master' into master
harshmotw-db Apr 20, 2024
ee98221
Merge branch 'master' of https://github.com/harshmotw-db/spark
harshmotw-db Apr 20, 2024
0b00d2a
Merge branch 'master' into python_scalar_variant
harshmotw-db Apr 21, 2024
5780ab3
regenerated golden files
harshmotw-db Apr 21, 2024
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions python/docs/source/reference/pyspark.sql/variant_val.rst
Original file line number Diff line number Diff line change
Expand Up @@ -25,3 +25,4 @@ VariantVal
:toctree: api/

VariantVal.toPython
VariantVal.toJson
115 changes: 111 additions & 4 deletions python/pyspark/sql/tests/test_types.py
Original file line number Diff line number Diff line change
Expand Up @@ -1427,8 +1427,10 @@ def test_variant_type(self):
("-int4", "-69633", -69633),
("int8", "4295033089", 4295033089),
("-int8", "-4294967297", -4294967297),
("float4", "1.23456789e-30", 1.23456789e-30),
("-float4", "-4.56789e+29", -4.56789e29),
("float4", "3.402e+38", 3.402e38),
("-float4", "-3.402e+38", -3.402e38),
("float8", "1.79769e+308", 1.79769e308),
("-float8", "-1.79769e+308", -1.79769e308),
("dec4", "123.456", Decimal("123.456")),
("-dec4", "-321.654", Decimal("-321.654")),
("dec8", "429.4967297", Decimal("429.4967297")),
Expand All @@ -1447,17 +1449,77 @@ def test_variant_type(self):
F.struct([F.parse_json(F.lit('{"b": "2"}'))]).alias("s"),
F.create_map([F.lit("k"), F.parse_json(F.lit('{"c": true}'))]).alias("m"),
).collect()[0]
variants = [row["v"], row["a"][0], row["s"]["col1"], row["m"]["k"]]

# These data types are not supported by parse_json yet so they are being handled
# separately - Date, Timestamp, TimestampNTZ, Binary, Float (Single Precision)
date_columns = self.spark.sql(
"select cast(Date('2021-01-01')"
+ " as variant) as d0, cast(Date('1800-12-31')"
+ " as variant) as d1"
).collect()[0]
float_columns = self.spark.sql(
"select cast(Float(5.5)" + " as variant) as f0, cast(Float(-5.5) as variant) as f1"
).collect()[0]
binary_columns = self.spark.sql(
"select cast(binary(x'324FA69E')" + " as variant) as b"
).collect()[0]
timetamp_ntz_columns = self.spark.sql(
"select cast(cast('1940-01-01 12:33:01.123'"
+ " as timestamp_ntz) as variant) as tntz0, cast(cast('2522-12-31 05:57:13'"
+ " as timestamp_ntz) as variant) as tntz1, cast(cast('0001-07-15 17:43:26+08:00'"
+ " as timestamp_ntz) as variant) as tntz2"
).collect()[0]
timetamp_columns = self.spark.sql(
"select cast(cast('1940-01-01 12:35:13.123+7:30'"
+ " as timestamp) as variant) as t0, cast(cast('2522-12-31 00:00:00-5:23'"
+ " as timestamp) as variant) as t1, cast(cast('0001-12-31 01:01:01+08:00'"
+ " as timestamp) as variant) as t2"
).collect()[0]

variants = [
row["v"],
row["a"][0],
row["s"]["col1"],
row["m"]["k"],
date_columns["d0"],
date_columns["d1"],
float_columns["f0"],
float_columns["f1"],
binary_columns["b"],
timetamp_ntz_columns["tntz0"],
timetamp_ntz_columns["tntz1"],
timetamp_ntz_columns["tntz2"],
timetamp_columns["t0"],
timetamp_columns["t1"],
timetamp_columns["t2"],
]

for v in variants:
self.assertEqual(type(v), VariantVal)

# check str
# check str (to_json)
as_string = str(variants[0])
for key, expected, _ in expected_values:
self.assertTrue('"%s":%s' % (key, expected) in as_string)
self.assertEqual(str(variants[1]), '{"a":1}')
self.assertEqual(str(variants[2]), '{"b":"2"}')
self.assertEqual(str(variants[3]), '{"c":true}')
self.assertEqual(str(variants[4]), '"2021-01-01"')
self.assertEqual(str(variants[5]), '"1800-12-31"')
self.assertEqual(str(variants[6]), "5.5")
self.assertEqual(str(variants[7]), "-5.5")
self.assertEqual(str(variants[8]), '"Mk+mng=="')
self.assertEqual(str(variants[9]), '"1940-01-01 12:33:01.123000"')
self.assertEqual(str(variants[10]), '"2522-12-31 05:57:13"')
self.assertEqual(str(variants[11]), '"0001-07-15 17:43:26"')
self.assertEqual(str(variants[12]), '"1940-01-01 05:05:13.123000+00:00"')
self.assertEqual(str(variants[13]), '"2522-12-31 05:23:00+00:00"')
self.assertEqual(str(variants[14]), '"0001-12-30 17:01:01+00:00"')

# Check to_json on timestamps with custom timezones
self.assertEqual(
variants[12].toJson("America/Los_Angeles"), '"1939-12-31 21:05:13.123000-08:00"'
)

# check toPython
as_python = variants[0].toPython()
Expand All @@ -1466,6 +1528,51 @@ def test_variant_type(self):
self.assertEqual(variants[1].toPython(), {"a": 1})
self.assertEqual(variants[2].toPython(), {"b": "2"})
self.assertEqual(variants[3].toPython(), {"c": True})
self.assertEqual(variants[4].toPython(), datetime.date(2021, 1, 1))
self.assertEqual(variants[5].toPython(), datetime.date(1800, 12, 31))
self.assertEqual(variants[6].toPython(), float(5.5))
self.assertEqual(variants[7].toPython(), float(-5.5))
self.assertEqual(variants[8].toPython(), bytearray(b"2O\xa6\x9e"))
self.assertEqual(variants[9].toPython(), datetime.datetime(1940, 1, 1, 12, 33, 1, 123000))
self.assertEqual(variants[10].toPython(), datetime.datetime(2522, 12, 31, 5, 57, 13))
self.assertEqual(variants[11].toPython(), datetime.datetime(1, 7, 15, 17, 43, 26))
self.assertEqual(
variants[12].toPython(),
datetime.datetime(
1940,
1,
1,
12,
35,
13,
123000,
tzinfo=datetime.timezone(datetime.timedelta(hours=7, minutes=30)),
),
)
self.assertEqual(
variants[13].toPython(),
datetime.datetime(
2522,
12,
31,
3,
3,
31,
tzinfo=datetime.timezone(datetime.timedelta(hours=-2, minutes=-20, seconds=31)),
),
)
self.assertEqual(
variants[14].toPython(),
datetime.datetime(
1,
12,
31,
16,
3,
23,
tzinfo=datetime.timezone(datetime.timedelta(hours=23, minutes=2, seconds=22)),
),
)

# check repr
self.assertEqual(str(variants[0]), str(eval(repr(variants[0]))))
Expand Down
13 changes: 13 additions & 0 deletions python/pyspark/sql/types.py
Original file line number Diff line number Diff line change
Expand Up @@ -1521,6 +1521,19 @@ def toPython(self) -> Any:
"""
return VariantUtils.to_python(self.value, self.metadata)

def toJson(self, zone_id: str = "UTC") -> str:
"""
Convert the VariantVal to a JSON string. The zone ID represents the time zone that the
timestamp should be printed in. It is defaulted to UTC. The list of valid zone IDs can be
found by importing the `zoneinfo` module and running :code:`zoneinfo.available_timezones()`.

Returns
-------
str
A JSON string that represents the Variant.
"""
return VariantUtils.to_json(self.value, self.metadata, zone_id)


_atomic_types: List[Type[DataType]] = [
StringType,
Expand Down
Loading