risingwave_stream/executor/join/
row.rsuse risingwave_common::row::{self, CompactedRow, OwnedRow, Row, RowExt};
use risingwave_common::types::{DataType, ScalarImpl};
use risingwave_common_estimate_size::EstimateSize;
use crate::executor::StreamExecutorResult;
#[derive(Clone, Debug)]
pub struct JoinRow<R: Row> {
pub row: R,
pub degree: DegreeType,
}
impl<R: Row> JoinRow<R> {
pub fn new(row: R, degree: DegreeType) -> Self {
Self { row, degree }
}
pub fn is_zero_degree(&self) -> bool {
self.degree == 0
}
pub fn to_table_rows<'a>(
&'a self,
state_order_key_indices: &'a [usize],
) -> (&'a R, impl Row + 'a) {
let order_key = (&self.row).project(state_order_key_indices);
let degree = build_degree_row(order_key, self.degree);
(&self.row, degree)
}
pub fn encode(&self) -> EncodedJoinRow {
EncodedJoinRow {
compacted_row: (&self.row).into(),
degree: self.degree,
}
}
}
pub type DegreeType = u64;
fn build_degree_row(order_key: impl Row, degree: DegreeType) -> impl Row {
order_key.chain(row::once(Some(ScalarImpl::Int64(degree as i64))))
}
#[derive(Clone, Debug, EstimateSize)]
pub struct EncodedJoinRow {
pub compacted_row: CompactedRow,
pub degree: DegreeType,
}
impl EncodedJoinRow {
pub fn decode(&self, data_types: &[DataType]) -> StreamExecutorResult<JoinRow<OwnedRow>> {
Ok(JoinRow {
row: self.decode_row(data_types)?,
degree: self.degree,
})
}
fn decode_row(&self, data_types: &[DataType]) -> StreamExecutorResult<OwnedRow> {
let row = self.compacted_row.deserialize(data_types)?;
Ok(row)
}
}