1313// limitations under the License.
1414
1515use std:: assert_matches:: assert_matches;
16- use std:: collections:: hash_map:: Entry ;
17- use std:: collections:: { HashMap , HashSet } ;
16+ use std:: collections:: HashSet ;
1817use std:: marker:: PhantomData ;
1918use std:: ops:: { Bound , Deref , Index } ;
2019
@@ -31,7 +30,6 @@ use risingwave_common::catalog::{
3130use risingwave_common:: hash:: { VirtualNode , VnodeBitmapExt } ;
3231use risingwave_common:: row:: { CompactedRow , OwnedRow , RowExt } ;
3332use risingwave_common:: types:: { DEBEZIUM_UNAVAILABLE_VALUE , DataType , ScalarImpl } ;
34- use risingwave_common:: util:: chunk_coalesce:: DataChunkBuilder ;
3533use risingwave_common:: util:: iter_util:: { ZipEqDebug , ZipEqFast } ;
3634use risingwave_common:: util:: sort_util:: { ColumnOrder , OrderType , cmp_datum} ;
3735use risingwave_common:: util:: value_encoding:: { BasicSerde , ValueRowSerializer } ;
@@ -507,7 +505,7 @@ impl<S: StateStore, SD: ValueRowSerde> MaterializeExecutor<S, SD> {
507505 )
508506 . await ?;
509507
510- match generate_output ( change_buffer, data_types. clone ( ) ) ? {
508+ match change_buffer. into_chunk ( data_types. clone ( ) ) {
511509 Some ( output_chunk) => {
512510 self . state_table . write_chunk ( output_chunk. clone ( ) ) ;
513511 self . state_table . try_flush ( ) . await ?;
@@ -1199,134 +1197,8 @@ fn handle_toast_columns_for_postgres_cdc(
11991197 OwnedRow :: new ( fixed_row_data)
12001198}
12011199
1202- /// Construct output `StreamChunk` from given buffer.
1203- fn generate_output (
1204- change_buffer : ChangeBuffer ,
1205- data_types : Vec < DataType > ,
1206- ) -> StreamExecutorResult < Option < StreamChunk > > {
1207- // construct output chunk
1208- // TODO(st1page): when materialize partial columns(), we should construct some columns in the pk
1209- let mut new_ops: Vec < Op > = vec ! [ ] ;
1210- let mut new_rows: Vec < OwnedRow > = vec ! [ ] ;
1211- for ( _, row_op) in change_buffer. into_parts ( ) {
1212- match row_op {
1213- ChangeBufferKeyOp :: Insert ( value) => {
1214- new_ops. push ( Op :: Insert ) ;
1215- new_rows. push ( value) ;
1216- }
1217- ChangeBufferKeyOp :: Delete ( old_value) => {
1218- new_ops. push ( Op :: Delete ) ;
1219- new_rows. push ( old_value) ;
1220- }
1221- ChangeBufferKeyOp :: Update ( ( old_value, new_value) ) => {
1222- // if old_value == new_value, we don't need to emit updates to downstream.
1223- if old_value != new_value {
1224- new_ops. push ( Op :: UpdateDelete ) ;
1225- new_ops. push ( Op :: UpdateInsert ) ;
1226- new_rows. push ( old_value) ;
1227- new_rows. push ( new_value) ;
1228- }
1229- }
1230- }
1231- }
1232- let mut data_chunk_builder = DataChunkBuilder :: new ( data_types, new_rows. len ( ) + 1 ) ;
1233-
1234- for row in new_rows {
1235- let res = data_chunk_builder. append_one_row ( row) ;
1236- debug_assert ! ( res. is_none( ) ) ;
1237- }
1238-
1239- if let Some ( new_data_chunk) = data_chunk_builder. consume_all ( ) {
1240- let new_stream_chunk = StreamChunk :: new ( new_ops, new_data_chunk. columns ( ) . to_vec ( ) ) ;
1241- Ok ( Some ( new_stream_chunk) )
1242- } else {
1243- Ok ( None )
1244- }
1245- }
1246-
1247- /// `ChangeBuffer` is a buffer to handle chunk into `KeyOp`.
1248- /// TODO(rc): merge with `TopNStaging`.
1249- pub struct ChangeBuffer {
1250- buffer : HashMap < Vec < u8 > , ChangeBufferKeyOp > ,
1251- }
1200+ type ChangeBuffer = crate :: common:: change_buffer:: ChangeBuffer < Vec < u8 > , OwnedRow > ;
12521201
1253- /// `KeyOp` variant for `ChangeBuffer` that stores `OwnedRow` instead of Bytes
1254- enum ChangeBufferKeyOp {
1255- Insert ( OwnedRow ) ,
1256- Delete ( OwnedRow ) ,
1257- /// (`old_value`, `new_value`)
1258- Update ( ( OwnedRow , OwnedRow ) ) ,
1259- }
1260-
1261- impl ChangeBuffer {
1262- fn new ( ) -> Self {
1263- Self {
1264- buffer : HashMap :: new ( ) ,
1265- }
1266- }
1267-
1268- fn insert ( & mut self , pk : Vec < u8 > , value : OwnedRow ) {
1269- let entry = self . buffer . entry ( pk) ;
1270- match entry {
1271- Entry :: Vacant ( e) => {
1272- e. insert ( ChangeBufferKeyOp :: Insert ( value) ) ;
1273- }
1274- Entry :: Occupied ( mut e) => {
1275- if let ChangeBufferKeyOp :: Delete ( old_value) = e. get_mut ( ) {
1276- let old_val = std:: mem:: take ( old_value) ;
1277- e. insert ( ChangeBufferKeyOp :: Update ( ( old_val, value) ) ) ;
1278- } else {
1279- unreachable ! ( ) ;
1280- }
1281- }
1282- }
1283- }
1284-
1285- fn delete ( & mut self , pk : Vec < u8 > , old_value : OwnedRow ) {
1286- let entry: Entry < ' _ , Vec < u8 > , ChangeBufferKeyOp > = self . buffer . entry ( pk) ;
1287- match entry {
1288- Entry :: Vacant ( e) => {
1289- e. insert ( ChangeBufferKeyOp :: Delete ( old_value) ) ;
1290- }
1291- Entry :: Occupied ( mut e) => match e. get_mut ( ) {
1292- ChangeBufferKeyOp :: Insert ( _) => {
1293- e. remove ( ) ;
1294- }
1295- ChangeBufferKeyOp :: Update ( ( prev, _curr) ) => {
1296- let prev = std:: mem:: take ( prev) ;
1297- e. insert ( ChangeBufferKeyOp :: Delete ( prev) ) ;
1298- }
1299- ChangeBufferKeyOp :: Delete ( _) => {
1300- unreachable ! ( ) ;
1301- }
1302- } ,
1303- }
1304- }
1305-
1306- fn update ( & mut self , pk : Vec < u8 > , old_value : OwnedRow , new_value : OwnedRow ) {
1307- let entry = self . buffer . entry ( pk) ;
1308- match entry {
1309- Entry :: Vacant ( e) => {
1310- e. insert ( ChangeBufferKeyOp :: Update ( ( old_value, new_value) ) ) ;
1311- }
1312- Entry :: Occupied ( mut e) => match e. get_mut ( ) {
1313- ChangeBufferKeyOp :: Insert ( _) => {
1314- e. insert ( ChangeBufferKeyOp :: Insert ( new_value) ) ;
1315- }
1316- ChangeBufferKeyOp :: Update ( ( _prev, curr) ) => {
1317- * curr = new_value;
1318- }
1319- ChangeBufferKeyOp :: Delete ( _) => {
1320- unreachable ! ( )
1321- }
1322- } ,
1323- }
1324- }
1325-
1326- fn into_parts ( self ) -> HashMap < Vec < u8 > , ChangeBufferKeyOp > {
1327- self . buffer
1328- }
1329- }
13301202impl < S : StateStore , SD : ValueRowSerde > Execute for MaterializeExecutor < S , SD > {
13311203 fn execute ( self : Box < Self > ) -> BoxedMessageStream {
13321204 self . execute_inner ( ) . boxed ( )
0 commit comments