risingwave_rpc_client/
sink_coordinate_client.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
117
118
119
120
121
122
// 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::future::Future;

use anyhow::anyhow;
use futures::{Stream, TryStreamExt};
use risingwave_common::bitmap::Bitmap;
use risingwave_pb::connector_service::coordinate_request::{
    CommitRequest, StartCoordinationRequest, UpdateVnodeBitmapRequest,
};
use risingwave_pb::connector_service::{
    coordinate_request, coordinate_response, CoordinateRequest, CoordinateResponse, PbSinkParam,
    SinkMetadata,
};
use tokio::sync::mpsc::Receiver;
use tokio_stream::wrappers::ReceiverStream;
use tonic::{Response, Status};

use crate::error::RpcError;
use crate::{BidiStreamHandle, SinkCoordinationRpcClient};

pub type CoordinatorStreamHandle = BidiStreamHandle<CoordinateRequest, CoordinateResponse>;

impl CoordinatorStreamHandle {
    pub async fn new(
        mut client: SinkCoordinationRpcClient,
        param: PbSinkParam,
        vnode_bitmap: Bitmap,
    ) -> Result<Self, RpcError> {
        Self::new_with_init_stream(param, vnode_bitmap, |rx| async move {
            client.coordinate(ReceiverStream::new(rx)).await
        })
        .await
    }

    pub async fn new_with_init_stream<F, St, Fut>(
        param: PbSinkParam,
        vnode_bitmap: Bitmap,
        init_stream: F,
    ) -> Result<Self, RpcError>
    where
        F: FnOnce(Receiver<CoordinateRequest>) -> Fut + Send,
        St: Stream<Item = Result<CoordinateResponse, Status>> + Send + Unpin + 'static,
        Fut: Future<Output = Result<Response<St>, Status>> + Send,
    {
        let (stream_handle, first_response) = BidiStreamHandle::initialize(
            CoordinateRequest {
                msg: Some(coordinate_request::Msg::StartRequest(
                    StartCoordinationRequest {
                        vnode_bitmap: Some(vnode_bitmap.to_protobuf()),
                        param: Some(param),
                    },
                )),
            },
            move |rx| async move {
                init_stream(rx)
                    .await
                    .map(|response| {
                        response
                            .into_inner()
                            .map_err(RpcError::from_connector_status)
                    })
                    .map_err(RpcError::from_connector_status)
            },
        )
        .await?;
        match first_response {
            CoordinateResponse {
                msg: Some(coordinate_response::Msg::StartResponse(_)),
            } => Ok(stream_handle),
            msg => Err(anyhow!("should get start response but get {:?}", msg).into()),
        }
    }

    pub async fn commit(&mut self, epoch: u64, metadata: SinkMetadata) -> anyhow::Result<()> {
        self.send_request(CoordinateRequest {
            msg: Some(coordinate_request::Msg::CommitRequest(CommitRequest {
                epoch,
                metadata: Some(metadata),
            })),
        })
        .await?;
        match self.next_response().await? {
            CoordinateResponse {
                msg: Some(coordinate_response::Msg::CommitResponse(_)),
            } => Ok(()),
            msg => Err(anyhow!("should get commit response but get {:?}", msg)),
        }
    }

    pub async fn update_vnode_bitmap(&mut self, vnode_bitmap: &Bitmap) -> anyhow::Result<()> {
        self.send_request(CoordinateRequest {
            msg: Some(coordinate_request::Msg::UpdateVnodeRequest(
                UpdateVnodeBitmapRequest {
                    vnode_bitmap: Some(vnode_bitmap.to_protobuf()),
                },
            )),
        })
        .await?;
        Ok(())
    }

    pub async fn stop(&mut self) -> anyhow::Result<()> {
        self.send_request(CoordinateRequest {
            msg: Some(coordinate_request::Msg::Stop(true)),
        })
        .await?;
        Ok(())
    }
}