@@ -997,3 +997,135 @@ def mock_commit(
997997 result = table .scan ().to_arrow ()
998998 assert len (result ) == 3
999999 assert result ["id" ].to_pylist () == [4 , 5 , 6 ]
1000+
1001+
1002+ class TestUpdateSpecRetry :
1003+ def test_update_spec_retried_on_conflict (self , catalog : SqlCatalog , schema : Schema ) -> None :
1004+ """Test that UpdateSpec operations are retried on CommitFailedException."""
1005+ from pyiceberg .transforms import BucketTransform
1006+
1007+ table = catalog .create_table (
1008+ "default.test_spec_retry" ,
1009+ schema = schema ,
1010+ properties = {
1011+ TableProperties .COMMIT_NUM_RETRIES : "3" ,
1012+ TableProperties .COMMIT_MIN_RETRY_WAIT_MS : "1" ,
1013+ TableProperties .COMMIT_MAX_RETRY_WAIT_MS : "10" ,
1014+ },
1015+ )
1016+
1017+ original_commit = catalog .commit_table
1018+ commit_count = 0
1019+
1020+ def mock_commit (
1021+ tbl : Table , requirements : tuple [TableRequirement , ...], updates : tuple [TableUpdate , ...]
1022+ ) -> CommitTableResponse :
1023+ nonlocal commit_count
1024+ commit_count += 1
1025+ if commit_count == 1 :
1026+ raise CommitFailedException ("Simulated spec conflict" )
1027+ return original_commit (tbl , requirements , updates )
1028+
1029+ with patch .object (catalog , "commit_table" , side_effect = mock_commit ):
1030+ with table .update_spec () as update_spec :
1031+ update_spec .add_field (
1032+ source_column_name = "id" , transform = BucketTransform (16 ), partition_field_name = "id_bucket"
1033+ )
1034+
1035+ assert commit_count == 2
1036+
1037+ def test_update_spec_resolves_conflict_on_retry (self , catalog : SqlCatalog , schema : Schema ) -> None :
1038+ """Test that spec update can resolve conflicts via retry"""
1039+ from pyiceberg .transforms import BucketTransform
1040+
1041+ table = catalog .create_table (
1042+ "default.test_spec_conflict_resolved" ,
1043+ schema = schema ,
1044+ properties = {
1045+ TableProperties .COMMIT_NUM_RETRIES : "5" ,
1046+ TableProperties .COMMIT_MIN_RETRY_WAIT_MS : "1" ,
1047+ TableProperties .COMMIT_MAX_RETRY_WAIT_MS : "10" ,
1048+ },
1049+ )
1050+
1051+ with table .update_spec () as update_spec :
1052+ update_spec .add_field (source_column_name = "id" , transform = BucketTransform (16 ), partition_field_name = "id_bucket" )
1053+
1054+ table2 = catalog .load_table ("default.test_spec_conflict_resolved" )
1055+ with table2 .update_spec () as update_spec2 :
1056+ update_spec2 .add_identity ("id" )
1057+
1058+ assert table .spec ().spec_id == 1
1059+ assert table2 .spec ().spec_id == 2
1060+
1061+ original_commit = catalog .commit_table
1062+ commit_count = 0
1063+
1064+ def mock_commit (
1065+ tbl : Table , requirements : tuple [TableRequirement , ...], updates : tuple [TableUpdate , ...]
1066+ ) -> CommitTableResponse :
1067+ nonlocal commit_count
1068+ commit_count += 1
1069+ return original_commit (tbl , requirements , updates )
1070+
1071+ with patch .object (catalog , "commit_table" , side_effect = mock_commit ):
1072+ # Retry resolves conflicts caused by mismatch spec_id
1073+ with table .update_spec () as update_spec :
1074+ update_spec .add_field (source_column_name = "id" , transform = BucketTransform (8 ), partition_field_name = "id_bucket_new" )
1075+
1076+ assert commit_count == 2
1077+
1078+ def test_transaction_with_spec_change_and_append_retries (
1079+ self , catalog : SqlCatalog , schema : Schema , arrow_table : pa .Table
1080+ ) -> None :
1081+ """Test that a transaction with spec change and append handles retry correctly."""
1082+ table = catalog .create_table (
1083+ "default.test_transaction_spec_and_append" ,
1084+ schema = schema ,
1085+ properties = {
1086+ TableProperties .COMMIT_NUM_RETRIES : "3" ,
1087+ TableProperties .COMMIT_MIN_RETRY_WAIT_MS : "1" ,
1088+ TableProperties .COMMIT_MAX_RETRY_WAIT_MS : "10" ,
1089+ },
1090+ )
1091+
1092+ original_commit = catalog .commit_table
1093+ commit_count = 0
1094+ captured_updates : list [tuple [TableUpdate , ...]] = []
1095+
1096+ def mock_commit (
1097+ tbl : Table , requirements : tuple [TableRequirement , ...], updates : tuple [TableUpdate , ...]
1098+ ) -> CommitTableResponse :
1099+ nonlocal commit_count
1100+ commit_count += 1
1101+ captured_updates .append (updates )
1102+ if commit_count == 1 :
1103+ raise CommitFailedException ("Simulated conflict" )
1104+ return original_commit (tbl , requirements , updates )
1105+
1106+ with patch .object (catalog , "commit_table" , side_effect = mock_commit ):
1107+ with table .transaction () as txn :
1108+ with txn .update_spec () as update_spec :
1109+ update_spec .add_identity ("id" )
1110+ txn .append (arrow_table )
1111+
1112+ assert commit_count == 2
1113+
1114+ first_attempt_update_types = [type (u ).__name__ for u in captured_updates [0 ]]
1115+ assert "AddPartitionSpecUpdate" in first_attempt_update_types
1116+ assert "AddSnapshotUpdate" in first_attempt_update_types
1117+
1118+ retry_attempt_update_types = [type (u ).__name__ for u in captured_updates [1 ]]
1119+ assert "AddPartitionSpecUpdate" in retry_attempt_update_types
1120+ assert "AddSnapshotUpdate" in retry_attempt_update_types
1121+
1122+ assert len (table .scan ().to_arrow ()) == 3
1123+
1124+ from pyiceberg .transforms import IdentityTransform
1125+
1126+ assert table .spec ().spec_id == 1
1127+ assert len (table .spec ().fields ) == 1
1128+ partition_field = table .spec ().fields [0 ]
1129+ assert partition_field .name == "id"
1130+ assert partition_field .source_id == 1 # "id" column's field_id
1131+ assert isinstance (partition_field .transform , IdentityTransform )
0 commit comments