@@ -17,6 +17,7 @@ use std::collections::HashMap;
17
17
use anyhow:: anyhow;
18
18
use async_trait:: async_trait;
19
19
use itertools:: Itertools ;
20
+ use prost:: Message ;
20
21
use risingwave_common:: array:: StreamChunk ;
21
22
#[ cfg( test) ]
22
23
use risingwave_common:: catalog:: Field ;
@@ -25,12 +26,16 @@ use risingwave_common::catalog::Schema;
25
26
use risingwave_common:: types:: DataType ;
26
27
use risingwave_common:: util:: addr:: HostAddr ;
27
28
use risingwave_pb:: connector_service:: sink_stream_request:: write_batch:: json_payload:: RowOp ;
28
- use risingwave_pb:: connector_service:: sink_stream_request:: write_batch:: { JsonPayload , Payload } ;
29
+ use risingwave_pb:: connector_service:: sink_stream_request:: write_batch:: {
30
+ JsonPayload , Payload , StreamChunkPayload ,
31
+ } ;
29
32
use risingwave_pb:: connector_service:: sink_stream_request:: {
30
33
Request as SinkRequest , StartEpoch , SyncBatch , WriteBatch ,
31
34
} ;
32
35
use risingwave_pb:: connector_service:: table_schema:: Column ;
33
- use risingwave_pb:: connector_service:: { SinkResponse , SinkStreamRequest , TableSchema } ;
36
+ use risingwave_pb:: connector_service:: {
37
+ SinkPayloadFormat , SinkResponse , SinkStreamRequest , TableSchema ,
38
+ } ;
34
39
use risingwave_rpc_client:: ConnectorClient ;
35
40
use tokio:: sync:: mpsc:: { UnboundedReceiver , UnboundedSender } ;
36
41
use tokio_stream:: StreamExt ;
@@ -105,6 +110,7 @@ pub struct RemoteSink<const APPEND_ONLY: bool> {
105
110
_client : Option < ConnectorClient > ,
106
111
request_sender : Option < UnboundedSender < SinkStreamRequest > > ,
107
112
response_stream : ResponseStreamImpl ,
113
+ payload_format : SinkPayloadFormat ,
108
114
}
109
115
110
116
impl < const APPEND_ONLY : bool > RemoteSink < APPEND_ONLY > {
@@ -141,6 +147,7 @@ impl<const APPEND_ONLY: bool> RemoteSink<APPEND_ONLY> {
141
147
config. connector_type . clone ( ) ,
142
148
config. properties . clone ( ) ,
143
149
table_schema,
150
+ connector_params. sink_payload_format ,
144
151
)
145
152
. await
146
153
. map_err ( SinkError :: from) ?;
@@ -155,6 +162,7 @@ impl<const APPEND_ONLY: bool> RemoteSink<APPEND_ONLY> {
155
162
_client : Some ( client) ,
156
163
request_sender : Some ( request_sender) ,
157
164
response_stream : ResponseStreamImpl :: Grpc ( response) ,
165
+ payload_format : connector_params. sink_payload_format ,
158
166
} )
159
167
}
160
168
@@ -240,24 +248,38 @@ impl<const APPEND_ONLY: bool> RemoteSink<APPEND_ONLY> {
240
248
_client : None ,
241
249
request_sender : Some ( request_sender) ,
242
250
response_stream : ResponseStreamImpl :: Receiver ( response_receiver) ,
251
+ payload_format : SinkPayloadFormat :: Json ,
243
252
}
244
253
}
245
254
}
246
255
247
256
#[ async_trait]
248
257
impl < const APPEND_ONLY : bool > Sink for RemoteSink < APPEND_ONLY > {
249
258
async fn write_batch ( & mut self , chunk : StreamChunk ) -> Result < ( ) > {
250
- let mut row_ops = vec ! [ ] ;
251
- for ( op, row_ref) in chunk. rows ( ) {
252
- let map = record_to_json ( row_ref, & self . schema . fields ) ?;
253
- let row_op = RowOp {
254
- op_type : op. to_protobuf ( ) as i32 ,
255
- line : serde_json:: to_string ( & map)
256
- . map_err ( |e| SinkError :: Remote ( format ! ( "{:?}" , e) ) ) ?,
257
- } ;
258
-
259
- row_ops. push ( row_op) ;
260
- }
259
+ let payload = match self . payload_format {
260
+ SinkPayloadFormat :: Json => {
261
+ let mut row_ops = vec ! [ ] ;
262
+ for ( op, row_ref) in chunk. rows ( ) {
263
+ let map = record_to_json ( row_ref, & self . schema . fields ) ?;
264
+ let row_op = RowOp {
265
+ op_type : op. to_protobuf ( ) as i32 ,
266
+ line : serde_json:: to_string ( & map)
267
+ . map_err ( |e| SinkError :: Remote ( format ! ( "{:?}" , e) ) ) ?,
268
+ } ;
269
+
270
+ row_ops. push ( row_op) ;
271
+ }
272
+ Payload :: JsonPayload ( JsonPayload { row_ops } )
273
+ }
274
+ SinkPayloadFormat :: StreamChunk => {
275
+ let prost_stream_chunk = chunk. to_protobuf ( ) ;
276
+ let binary_data = Message :: encode_to_vec ( & prost_stream_chunk) ;
277
+ Payload :: StreamChunkPayload ( StreamChunkPayload { binary_data } )
278
+ }
279
+ SinkPayloadFormat :: FormatUnspecified => {
280
+ unreachable ! ( "should specify sink payload format" )
281
+ }
282
+ } ;
261
283
262
284
let epoch = self . epoch . ok_or_else ( || {
263
285
SinkError :: Remote ( "epoch has not been initialize, call `begin_epoch`" . to_string ( ) )
@@ -268,7 +290,7 @@ impl<const APPEND_ONLY: bool> Sink for RemoteSink<APPEND_ONLY> {
268
290
request : Some ( SinkRequest :: Write ( WriteBatch {
269
291
epoch,
270
292
batch_id,
271
- payload : Some ( Payload :: JsonPayload ( JsonPayload { row_ops } ) ) ,
293
+ payload : Some ( payload ) ,
272
294
} ) ) ,
273
295
} )
274
296
. map_err ( |e| SinkError :: Remote ( e. to_string ( ) ) ) ?;
@@ -451,6 +473,7 @@ mod test {
451
473
assert_eq ! ( row_2. line, "{\" id\" :3,\" name\" :\" Clare\" }" ) ;
452
474
assert_eq ! ( row_2. op_type, data:: Op :: Insert as i32 ) ;
453
475
}
476
+ _ => unreachable ! ( "should be json payload" ) ,
454
477
}
455
478
}
456
479
_ => panic ! ( "test failed: failed to construct write request" ) ,
@@ -512,6 +535,7 @@ mod test {
512
535
assert_eq ! ( row_2. line, "{\" id\" :6,\" name\" :\" Frank\" }" ) ;
513
536
assert_eq ! ( row_2. op_type, data:: Op :: Insert as i32 ) ;
514
537
}
538
+ _ => unreachable ! ( "should be json payload" ) ,
515
539
}
516
540
}
517
541
_ => panic ! ( "test failed: failed to construct write request" ) ,
0 commit comments