risingwave_batch/task/
channel.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::sync::Arc;

use risingwave_common::array::DataChunk;
use risingwave_pb::batch_plan::exchange_info::DistributionMode as ShuffleDistributionMode;
use risingwave_pb::batch_plan::ExchangeInfo;

use crate::error::{BatchError, Result, SharedResult};
use crate::task::broadcast_channel::{new_broadcast_channel, BroadcastReceiver, BroadcastSender};
use crate::task::consistent_hash_shuffle_channel::{
    new_consistent_shuffle_channel, ConsistentHashShuffleReceiver, ConsistentHashShuffleSender,
};
use crate::task::data_chunk_in_channel::DataChunkInChannel;
use crate::task::fifo_channel::{new_fifo_channel, FifoReceiver, FifoSender};
use crate::task::hash_shuffle_channel::{
    new_hash_shuffle_channel, HashShuffleReceiver, HashShuffleSender,
};

pub(super) trait ChanSender: Send {
    /// This function will block until there's enough resource to process the chunk.
    /// Currently, it will only be called from single thread.
    /// `None` is sent as a mark of the ending of channel.
    async fn send(&mut self, chunk: DataChunk) -> Result<()>;

    /// Close this data channel.
    ///
    /// If finished correctly, we should pass `None`, otherwise we should pass `BatchError`. In
    /// either case we should stop sending more data.
    async fn close(self, error: Option<Arc<BatchError>>) -> Result<()>;
}

#[derive(Debug, Clone)]
pub enum ChanSenderImpl {
    HashShuffle(HashShuffleSender),
    ConsistentHashShuffle(ConsistentHashShuffleSender),
    Fifo(FifoSender),
    Broadcast(BroadcastSender),
}

impl ChanSenderImpl {
    pub(super) async fn send(&mut self, chunk: DataChunk) -> Result<()> {
        match self {
            Self::HashShuffle(sender) => sender.send(chunk).await,
            Self::ConsistentHashShuffle(sender) => sender.send(chunk).await,
            Self::Fifo(sender) => sender.send(chunk).await,
            Self::Broadcast(sender) => sender.send(chunk).await,
        }
    }

    pub(super) async fn close(self, error: Option<Arc<BatchError>>) -> Result<()> {
        match self {
            Self::HashShuffle(sender) => sender.close(error).await,
            Self::ConsistentHashShuffle(sender) => sender.close(error).await,
            Self::Fifo(sender) => sender.close(error).await,
            Self::Broadcast(sender) => sender.close(error).await,
        }
    }
}

pub(super) trait ChanReceiver: Send {
    /// Returns `None` if there's no more data to read.
    /// Otherwise it will wait until there's data.
    async fn recv(&mut self) -> SharedResult<Option<DataChunkInChannel>>;
}

pub enum ChanReceiverImpl {
    HashShuffle(HashShuffleReceiver),
    ConsistentHashShuffle(ConsistentHashShuffleReceiver),
    Fifo(FifoReceiver),
    Broadcast(BroadcastReceiver),
}

impl ChanReceiverImpl {
    pub(super) async fn recv(&mut self) -> SharedResult<Option<DataChunkInChannel>> {
        match self {
            Self::HashShuffle(receiver) => receiver.recv().await,
            Self::ConsistentHashShuffle(receiver) => receiver.recv().await,
            Self::Broadcast(receiver) => receiver.recv().await,
            Self::Fifo(receiver) => receiver.recv().await,
        }
    }
}

/// Output-channel is a synchronous, bounded single-producer-multiple-consumer queue.
/// The producer is the local task executor, the consumer is
/// [`ExchangeService`](risingwave_pb::task_service::exchange_service_server::ExchangeService).
/// The implementation depends on the shuffling strategy.
pub fn create_output_channel(
    shuffle: &ExchangeInfo,
    output_channel_size: usize,
) -> Result<(ChanSenderImpl, Vec<ChanReceiverImpl>)> {
    match shuffle.get_mode()? {
        ShuffleDistributionMode::Single => Ok(new_fifo_channel(output_channel_size)),
        ShuffleDistributionMode::Hash => Ok(new_hash_shuffle_channel(shuffle, output_channel_size)),
        ShuffleDistributionMode::ConsistentHash => {
            Ok(new_consistent_shuffle_channel(shuffle, output_channel_size))
        }
        ShuffleDistributionMode::Broadcast => {
            Ok(new_broadcast_channel(shuffle, output_channel_size))
        }
        ShuffleDistributionMode::Unspecified => unreachable!(),
    }
}