-
Notifications
You must be signed in to change notification settings - Fork 454
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
add pyspark stream integration test with new schema evolution id beha…
…vior Signed-off-by: Liam Murphy <[email protected]>
- Loading branch information
1 parent
0c90760
commit e715999
Showing
2 changed files
with
93 additions
and
1 deletion.
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
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -9,7 +9,7 @@ | |
from deltalake import DeltaTable, write_deltalake | ||
from deltalake.exceptions import DeltaProtocolError | ||
|
||
from .utils import assert_spark_read_equal, get_spark | ||
from .utils import assert_spark_read_equal, get_spark, run_stream_with_checkpoint | ||
|
||
try: | ||
import delta | ||
|
@@ -198,3 +198,78 @@ def test_spark_read_repair_run(tmp_path): | |
) | ||
|
||
assert latest_operation_metrics["operationMetrics"] is not None | ||
|
||
@pytest.mark.pyspark | ||
@pytest.mark.integration | ||
def test_spark_stream_schema_evolution(tmp_path: pathlib.Path): | ||
"""https://github.com/delta-io/delta-rs/issues/3274""" | ||
""" | ||
This test ensures that Spark can still read from tables following | ||
a schema evolution write, since old behavior was to generate a new table ID | ||
between schema evolution runs, which caused Spark to error thinking the table had changed. | ||
""" | ||
|
||
data_first_write = pa.array( | ||
[ | ||
{"name": "Alice", "age": 30, "details": {"email": "[email protected]"}}, | ||
{"name": "Bob", "age": 25, "details": {"email": "[email protected]"}}, | ||
] | ||
) | ||
|
||
data_second_write = pa.array( | ||
[ | ||
{ | ||
"name": "Charlie", | ||
"age": 35, | ||
"details": {"address": "123 Main St", "email": "[email protected]"}, | ||
}, | ||
{ | ||
"name": "Diana", | ||
"age": 28, | ||
"details": {"address": "456 Elm St", "email": "[email protected]"}, | ||
}, | ||
] | ||
) | ||
|
||
schema_first_write = pa.schema( | ||
[ | ||
("name", pa.string()), | ||
("age", pa.int64()), | ||
("details", pa.struct([("email", pa.string())])), | ||
] | ||
) | ||
|
||
schema_second_write = pa.schema( | ||
[ | ||
("name", pa.string()), | ||
("age", pa.int64()), | ||
( | ||
"details", | ||
pa.struct( | ||
[ | ||
("address", pa.string()), | ||
("email", pa.string()), | ||
] | ||
), | ||
), | ||
] | ||
) | ||
table_first_write = pa.Table.from_pylist( | ||
data_first_write, schema=schema_first_write | ||
) | ||
table_second_write = pa.Table.from_pylist( | ||
data_second_write, schema=schema_second_write | ||
) | ||
|
||
write_deltalake(tmp_path, table_first_write, mode="append", engine="rust") | ||
|
||
run_stream_with_checkpoint(tmp_path.as_posix()) | ||
|
||
write_deltalake( | ||
tmp_path, table_second_write, mode="append", engine="rust", schema_mode="merge" | ||
) | ||
|
||
run_stream_with_checkpoint(tmp_path.as_posix()) | ||
|
||
# For this test we don't care about the data, we'll just let any | ||
# exceptions trickle up to report as a failure |
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