@@ -26,6 +26,7 @@ use risingwave_connector::sink::{
26
26
SINK_FORMAT_APPEND_ONLY , SINK_FORMAT_OPTION , SINK_USER_FORCE_APPEND_ONLY_OPTION ,
27
27
} ;
28
28
use risingwave_pb:: stream_plan:: stream_node:: NodeBody as ProstStreamNode ;
29
+ use tracing:: info;
29
30
30
31
use super :: derive:: { derive_columns, derive_pk} ;
31
32
use super :: utils:: IndicesDisplay ;
@@ -72,10 +73,22 @@ impl StreamSink {
72
73
let required_dist = match input. distribution ( ) {
73
74
Distribution :: Single => RequiredDist :: single ( ) ,
74
75
_ => {
75
- assert_matches ! ( user_distributed_by, RequiredDist :: Any ) ;
76
- RequiredDist :: shard_by_key ( input. schema ( ) . len ( ) , input. logical_pk ( ) )
76
+ match properties. get ( "connector" ) {
77
+ Some ( s) if s == "iceberg" => {
78
+ // iceberg with multiple parallelism will fail easily with concurrent commit
79
+ // on metadata
80
+ // TODO: reset iceberg sink to have multiple parallelism
81
+ info ! ( "setting iceberg sink parallelism to singleton" ) ;
82
+ RequiredDist :: single ( )
83
+ }
84
+ _ => {
85
+ assert_matches ! ( user_distributed_by, RequiredDist :: Any ) ;
86
+ RequiredDist :: shard_by_key ( input. schema ( ) . len ( ) , input. logical_pk ( ) )
87
+ }
88
+ }
77
89
}
78
90
} ;
91
+
79
92
let input = required_dist. enforce_if_not_satisfies ( input, & Order :: any ( ) ) ?;
80
93
let columns = derive_columns ( input. schema ( ) , out_names, & user_cols) ?;
81
94
0 commit comments