risingwave_stream/executor/join/
row.rs1use risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowExt};
16use risingwave_common::types::{DataType, ScalarImpl};
17use risingwave_common_estimate_size::EstimateSize;
18
19use crate::executor::StreamExecutorResult;
20
21#[derive(Clone, Debug)]
23pub struct JoinRow<R: Row> {
24 pub row: R,
25 pub degree: DegreeType,
26}
27
28impl<R: Row> JoinRow<R> {
29 pub fn new(row: R, degree: DegreeType) -> Self {
30 Self { row, degree }
31 }
32
33 pub fn is_zero_degree(&self) -> bool {
34 self.degree == 0
35 }
36
37 pub fn to_table_rows<'a>(
42 &'a self,
43 state_order_key_indices: &'a [usize],
44 ) -> (&'a R, impl Row + 'a) {
45 let order_key = (&self.row).project(state_order_key_indices);
46 let degree = build_degree_row(order_key, self.degree);
47 (&self.row, degree)
48 }
49
50 pub fn encode(&self) -> EncodedJoinRow {
51 EncodedJoinRow {
52 compacted_row: (&self.row).into(),
53 degree: self.degree,
54 }
55 }
56}
57
58pub type DegreeType = u64;
59
60fn build_degree_row(order_key: impl Row, degree: DegreeType) -> impl Row {
61 order_key.chain(row::once(Some(ScalarImpl::Int64(degree as i64))))
62}
63
64#[derive(Clone, Debug, EstimateSize)]
65pub struct EncodedJoinRow {
66 pub compacted_row: CompactedRow,
67 pub degree: DegreeType,
68}
69
70impl EncodedJoinRow {
71 pub fn decode(&self, data_types: &[DataType]) -> StreamExecutorResult<JoinRow<OwnedRow>> {
72 Ok(JoinRow {
73 row: self.decode_row(data_types)?,
74 degree: self.degree,
75 })
76 }
77
78 fn decode_row(&self, data_types: &[DataType]) -> StreamExecutorResult<OwnedRow> {
79 let row = self.compacted_row.deserialize(data_types)?;
80 Ok(row)
81 }
82}