@@ -68,6 +68,7 @@ use risingwave_common_estimate_size::EstimateSize;
6868use risingwave_pb:: connector_service:: SinkMetadata ;
6969use risingwave_pb:: connector_service:: sink_metadata:: Metadata :: Serialized ;
7070use risingwave_pb:: connector_service:: sink_metadata:: SerializedMetadata ;
71+ use risingwave_pb:: stream_plan:: PbSinkSchemaChange ;
7172use serde:: { Deserialize , Serialize } ;
7273use serde_json:: from_value;
7374use serde_with:: { DisplayFromStr , serde_as} ;
@@ -1886,20 +1887,32 @@ impl SinglePhaseCommitCoordinator for IcebergSinkCommitter {
18861887 & mut self ,
18871888 epoch : u64 ,
18881889 metadata : Vec < SinkMetadata > ,
1889- add_columns : Option < Vec < Field > > ,
1890+ schema_change : Option < PbSinkSchemaChange > ,
18901891 ) -> Result < ( ) > {
18911892 tracing:: info!( "Starting iceberg direct commit in epoch {epoch}" ) ;
18921893
18931894 // Commit data if present
1894- if let Some ( ( write_results, snapshot_id) ) = self . pre_commit_inner ( epoch, metadata, None ) ? {
1895+ if let Some ( ( write_results, snapshot_id) ) = self . pre_commit_inner ( epoch, metadata) ? {
18951896 self . commit_datafile ( epoch, write_results, snapshot_id)
18961897 . await ?;
18971898 }
18981899
18991900 // Commit schema change if present
1900- if let Some ( add_columns) = add_columns {
1901+ if let Some ( schema_change) = schema_change {
1902+ let risingwave_pb:: stream_plan:: sink_schema_change:: Op :: AddColumns ( add_columns) =
1903+ schema_change. op . unwrap ( )
1904+ else {
1905+ return Err ( SinkError :: Iceberg ( anyhow ! (
1906+ "Only AddColumns schema change is supported for Iceberg sink"
1907+ ) ) ) ;
1908+ } ;
1909+ let add_fields = add_columns
1910+ . fields
1911+ . into_iter ( )
1912+ . map ( |pb_field| Field :: from_prost ( & pb_field) )
1913+ . collect_vec ( ) ;
19011914 tracing:: info!( ?epoch, "Committing schema change" ) ;
1902- self . commit_schema_change ( add_columns ) . await ?;
1915+ self . commit_schema_change ( add_fields ) . await ?;
19031916 }
19041917
19051918 Ok ( ( ) )
@@ -1921,27 +1934,26 @@ impl TwoPhaseCommitCoordinator for IcebergSinkCommitter {
19211934 & mut self ,
19221935 epoch : u64 ,
19231936 metadata : Vec < SinkMetadata > ,
1924- add_columns : Option < Vec < Field > > ,
1937+ schema_change : Option < PbSinkSchemaChange > ,
19251938 ) -> Result < Vec < u8 > > {
19261939 tracing:: info!( "Starting iceberg pre commit in epoch {epoch}" ) ;
19271940
19281941 // TwoPhaseCommitCoordinator does not support schema change yet
1929- if let Some ( add_columns ) = & add_columns {
1942+ if let Some ( schema_change ) = & schema_change {
19301943 return Err ( SinkError :: Iceberg ( anyhow ! (
19311944 "TwoPhaseCommitCoordinator for Iceberg sink does not support schema change yet, \
1932- but got add_columns : {:?}",
1933- add_columns . iter ( ) . map ( |c| & c . name ) . collect :: < Vec <_>> ( )
1945+ but got schema_change : {:?}",
1946+ schema_change
19341947 ) ) ) ;
19351948 }
19361949
1937- let ( write_results, snapshot_id) =
1938- match self . pre_commit_inner ( epoch, metadata, add_columns) ? {
1939- Some ( ( write_results, snapshot_id) ) => ( write_results, snapshot_id) ,
1940- None => {
1941- tracing:: debug!( ?epoch, "no data to commit" ) ;
1942- return Ok ( vec ! [ ] ) ;
1943- }
1944- } ;
1950+ let ( write_results, snapshot_id) = match self . pre_commit_inner ( epoch, metadata) ? {
1951+ Some ( ( write_results, snapshot_id) ) => ( write_results, snapshot_id) ,
1952+ None => {
1953+ tracing:: debug!( ?epoch, "no data to commit" ) ;
1954+ return Ok ( vec ! [ ] ) ;
1955+ }
1956+ } ;
19451957
19461958 let mut write_results_bytes = Vec :: new ( ) ;
19471959 for each_parallelism_write_result in write_results {
@@ -1957,7 +1969,12 @@ impl TwoPhaseCommitCoordinator for IcebergSinkCommitter {
19571969 Ok ( pre_commit_metadata_bytes)
19581970 }
19591971
1960- async fn commit ( & mut self , epoch : u64 , commit_metadata : Vec < u8 > ) -> Result < ( ) > {
1972+ async fn commit (
1973+ & mut self ,
1974+ epoch : u64 ,
1975+ commit_metadata : Vec < u8 > ,
1976+ _schema_change : Option < PbSinkSchemaChange > ,
1977+ ) -> Result < ( ) > {
19611978 tracing:: info!( "Starting iceberg commit in epoch {epoch}" ) ;
19621979 if commit_metadata. is_empty ( ) {
19631980 tracing:: debug!( ?epoch, "no data to commit" ) ;
@@ -2008,16 +2025,7 @@ impl IcebergSinkCommitter {
20082025 & mut self ,
20092026 _epoch : u64 ,
20102027 metadata : Vec < SinkMetadata > ,
2011- add_columns : Option < Vec < Field > > ,
20122028 ) -> Result < Option < ( Vec < IcebergCommitResult > , i64 ) > > {
2013- if let Some ( add_columns) = add_columns {
2014- return Err ( anyhow ! (
2015- "Iceberg sink not support add columns, but got: {:?}" ,
2016- add_columns
2017- )
2018- . into ( ) ) ;
2019- }
2020-
20212029 let write_results: Vec < IcebergCommitResult > = metadata
20222030 . iter ( )
20232031 . map ( IcebergCommitResult :: try_from)
0 commit comments