@@ -23,8 +23,8 @@ use maplit::{convert_args, hashmap};
23
23
use pgwire:: pg_response:: { PgResponse , StatementType } ;
24
24
use risingwave_common:: bail_not_implemented;
25
25
use risingwave_common:: catalog:: {
26
- is_column_ids_dedup, ColumnCatalog , ColumnDesc , Schema , TableId , INITIAL_SOURCE_VERSION_ID ,
27
- KAFKA_TIMESTAMP_COLUMN_NAME ,
26
+ is_column_ids_dedup, ColumnCatalog , ColumnDesc , ColumnId , Schema , TableId ,
27
+ INITIAL_SOURCE_VERSION_ID , KAFKA_TIMESTAMP_COLUMN_NAME ,
28
28
} ;
29
29
use risingwave_common:: types:: DataType ;
30
30
use risingwave_connector:: parser:: additional_columns:: {
@@ -37,6 +37,7 @@ use risingwave_connector::parser::{
37
37
use risingwave_connector:: schema:: schema_registry:: {
38
38
name_strategy_from_str, SchemaRegistryAuth , SCHEMA_REGISTRY_PASSWORD , SCHEMA_REGISTRY_USERNAME ,
39
39
} ;
40
+ use risingwave_connector:: sink:: iceberg:: IcebergConfig ;
40
41
use risingwave_connector:: source:: cdc:: external:: CdcTableType ;
41
42
use risingwave_connector:: source:: cdc:: {
42
43
CDC_SHARING_MODE_KEY , CDC_SNAPSHOT_BACKFILL , CDC_SNAPSHOT_MODE_KEY , CDC_TRANSACTIONAL_KEY ,
@@ -67,7 +68,6 @@ use thiserror_ext::AsReport;
67
68
use super :: RwPgResponse ;
68
69
use crate :: binder:: Binder ;
69
70
use crate :: catalog:: source_catalog:: SourceCatalog ;
70
- use crate :: catalog:: ColumnId ;
71
71
use crate :: error:: ErrorCode :: { self , InvalidInputSyntax , NotSupported , ProtocolError } ;
72
72
use crate :: error:: { Result , RwError } ;
73
73
use crate :: expr:: Expr ;
@@ -76,7 +76,8 @@ use crate::handler::create_table::{
76
76
ensure_table_constraints_supported, ColumnIdGenerator ,
77
77
} ;
78
78
use crate :: handler:: util:: {
79
- connector_need_pk, get_connector, is_cdc_connector, is_kafka_connector, SourceSchemaCompatExt ,
79
+ connector_need_pk, get_connector, is_cdc_connector, is_iceberg_connector, is_kafka_connector,
80
+ SourceSchemaCompatExt ,
80
81
} ;
81
82
use crate :: handler:: HandlerArgs ;
82
83
use crate :: optimizer:: plan_node:: generic:: SourceNodeKind ;
@@ -333,7 +334,6 @@ pub(crate) async fn bind_columns_from_source(
333
334
334
335
let columns = match ( & source_schema. format , & source_schema. row_encode ) {
335
336
( Format :: Native , Encode :: Native )
336
- | ( Format :: None , Encode :: None )
337
337
| ( Format :: Plain , Encode :: Bytes )
338
338
| ( Format :: DebeziumMongo , Encode :: Json ) => None ,
339
339
( Format :: Plain , Encode :: Protobuf ) => {
@@ -444,6 +444,17 @@ pub(crate) async fn bind_columns_from_source(
444
444
)
445
445
. await ?
446
446
}
447
+ ( Format :: None , Encode :: None ) => {
448
+ if is_iceberg_connector ( with_properties) {
449
+ Some (
450
+ extract_iceberg_columns ( with_properties)
451
+ . await
452
+ . map_err ( |err| ProtocolError ( err. to_report_string ( ) ) ) ?,
453
+ )
454
+ } else {
455
+ None
456
+ }
457
+ }
447
458
( format, encoding) => {
448
459
return Err ( RwError :: from ( ProtocolError ( format ! (
449
460
"Unknown combination {:?} {:?}" ,
@@ -1158,6 +1169,46 @@ pub(super) fn check_nexmark_schema(
1158
1169
Ok ( ( ) )
1159
1170
}
1160
1171
1172
+ pub async fn extract_iceberg_columns (
1173
+ with_properties : & HashMap < String , String > ,
1174
+ ) -> anyhow:: Result < Vec < ColumnCatalog > > {
1175
+ let props = ConnectorProperties :: extract ( with_properties. clone ( ) , true ) ?;
1176
+ if let ConnectorProperties :: Iceberg ( properties) = props {
1177
+ let iceberg_config: IcebergConfig = properties. to_iceberg_config ( ) ;
1178
+ let table = iceberg_config. load_table ( ) . await ?;
1179
+ let iceberg_schema: arrow_schema:: Schema = table
1180
+ . current_table_metadata ( )
1181
+ . current_schema ( ) ?
1182
+ . clone ( )
1183
+ . try_into ( ) ?;
1184
+
1185
+ let columns = iceberg_schema
1186
+ . fields ( )
1187
+ . iter ( )
1188
+ . enumerate ( )
1189
+ . map ( |( i, field) | {
1190
+ let data_type = field. data_type ( ) . clone ( ) ;
1191
+ let column_desc = ColumnDesc :: named (
1192
+ field. name ( ) ,
1193
+ ColumnId :: new ( ( i as u32 ) . try_into ( ) . unwrap ( ) ) ,
1194
+ data_type. into ( ) ,
1195
+ ) ;
1196
+ ColumnCatalog {
1197
+ column_desc,
1198
+ is_hidden : false ,
1199
+ }
1200
+ } )
1201
+ . collect ( ) ;
1202
+
1203
+ Ok ( columns)
1204
+ } else {
1205
+ Err ( anyhow ! ( format!(
1206
+ "Invalid properties for iceberg source: {:?}" ,
1207
+ props
1208
+ ) ) )
1209
+ }
1210
+ }
1211
+
1161
1212
pub async fn check_iceberg_source (
1162
1213
props : & HashMap < String , String > ,
1163
1214
columns : & [ ColumnCatalog ] ,
0 commit comments