risingwave_rpc_client/
frontend_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
// 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 std::time::Duration;

use async_trait::async_trait;
use risingwave_common::config::MAX_CONNECTION_WINDOW_SIZE;
use risingwave_common::monitor::{EndpointExt, TcpConfig};
use risingwave_common::util::addr::HostAddr;
use risingwave_pb::frontend_service::frontend_service_client::FrontendServiceClient;
use risingwave_pb::frontend_service::{GetTableReplacePlanRequest, GetTableReplacePlanResponse};
use tokio_retry::strategy::{jitter, ExponentialBackoff};
use tonic::transport::Endpoint;
use tonic::Response;

use crate::error::Result;
use crate::tracing::{Channel, TracingInjectedChannelExt};
use crate::{RpcClient, RpcClientPool};

const DEFAULT_RETRY_INTERVAL: u64 = 50;
const DEFAULT_RETRY_MAX_DELAY: Duration = Duration::from_secs(5);
const DEFAULT_RETRY_MAX_ATTEMPTS: usize = 10;

#[derive(Clone)]
struct FrontendClient(FrontendServiceClient<Channel>);

impl FrontendClient {
    async fn new(host_addr: HostAddr) -> Result<Self> {
        let channel = Endpoint::from_shared(format!("http://{}", &host_addr))?
            .initial_connection_window_size(MAX_CONNECTION_WINDOW_SIZE)
            .connect_timeout(Duration::from_secs(5))
            .monitored_connect(
                "grpc-frontend-client",
                TcpConfig {
                    tcp_nodelay: true,
                    ..Default::default()
                },
            )
            .await?
            .tracing_injected();

        Ok(Self(
            FrontendServiceClient::new(channel).max_decoding_message_size(usize::MAX),
        ))
    }
}

// similar to the stream_client used in the Meta node
pub type FrontendClientPool = RpcClientPool<FrontendRetryClient>;
pub type FrontendClientPoolRef = Arc<FrontendClientPool>;

#[async_trait]
impl RpcClient for FrontendRetryClient {
    async fn new_client(host_addr: HostAddr) -> Result<Self> {
        Self::new(host_addr).await
    }
}

#[derive(Clone)]
pub struct FrontendRetryClient {
    client: FrontendClient,
}

impl FrontendRetryClient {
    async fn new(host_addr: HostAddr) -> Result<Self> {
        let client = FrontendClient::new(host_addr).await?;
        Ok(Self { client })
    }

    #[inline(always)]
    fn get_retry_strategy() -> impl Iterator<Item = Duration> {
        ExponentialBackoff::from_millis(DEFAULT_RETRY_INTERVAL)
            .max_delay(DEFAULT_RETRY_MAX_DELAY)
            .take(DEFAULT_RETRY_MAX_ATTEMPTS)
            .map(jitter)
    }

    fn should_retry(status: &tonic::Status) -> bool {
        if status.code() == tonic::Code::Unavailable
            || status.code() == tonic::Code::Unknown
            || status.code() == tonic::Code::Unauthenticated
            || status.code() == tonic::Code::Aborted
        {
            return true;
        }
        false
    }

    pub async fn get_table_replace_plan(
        &self,
        request: GetTableReplacePlanRequest,
    ) -> std::result::Result<Response<GetTableReplacePlanResponse>, tonic::Status> {
        tokio_retry::RetryIf::spawn(
            Self::get_retry_strategy(),
            || async {
                self.client
                    .to_owned()
                    .0
                    .get_table_replace_plan(request.clone())
                    .await
            },
            Self::should_retry,
        )
        .await
    }
}