risingwave_rpc_client/
compactor_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
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
// 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 risingwave_common::monitor::EndpointExt;
use risingwave_common::util::addr::HostAddr;
use risingwave_pb::hummock::hummock_manager_service_client::HummockManagerServiceClient;
use risingwave_pb::hummock::{
    GetNewSstIdsRequest, GetNewSstIdsResponse, ReportCompactionTaskRequest,
    ReportCompactionTaskResponse,
};
use risingwave_pb::meta::system_params_service_client::SystemParamsServiceClient;
use risingwave_pb::meta::{GetSystemParamsRequest, GetSystemParamsResponse};
use risingwave_pb::monitor_service::monitor_service_client::MonitorServiceClient;
use risingwave_pb::monitor_service::{StackTraceRequest, StackTraceResponse};
use tokio::sync::RwLock;
use tokio_retry::strategy::{jitter, ExponentialBackoff};
use tonic::transport::{Channel, Endpoint};

use crate::error::{Result, RpcError};
use crate::retry_rpc;
const ENDPOINT_KEEP_ALIVE_INTERVAL_SEC: u64 = 60;
const ENDPOINT_KEEP_ALIVE_TIMEOUT_SEC: u64 = 60;

const DEFAULT_RETRY_INTERVAL: u64 = 20;
const DEFAULT_RETRY_MAX_DELAY: Duration = Duration::from_secs(5);
const DEFAULT_RETRY_MAX_ATTEMPTS: usize = 3;
#[derive(Clone)]
pub struct CompactorClient {
    pub monitor_client: MonitorServiceClient<Channel>,
}

impl CompactorClient {
    pub async fn new(host_addr: HostAddr) -> Result<Self> {
        let channel = Endpoint::from_shared(format!("http://{}", &host_addr))?
            .connect_timeout(Duration::from_secs(5))
            .monitored_connect("grpc-compactor-client", Default::default())
            .await?;
        Ok(Self {
            monitor_client: MonitorServiceClient::new(channel),
        })
    }

    pub async fn stack_trace(&self) -> Result<StackTraceResponse> {
        Ok(self
            .monitor_client
            .to_owned()
            .stack_trace(StackTraceRequest::default())
            .await
            .map_err(RpcError::from_compactor_status)?
            .into_inner())
    }
}

#[derive(Debug, Clone)]
pub struct GrpcCompactorProxyClientCore {
    hummock_client: HummockManagerServiceClient<Channel>,
    system_params_client: SystemParamsServiceClient<Channel>,
}

impl GrpcCompactorProxyClientCore {
    pub(crate) fn new(channel: Channel) -> Self {
        let hummock_client =
            HummockManagerServiceClient::new(channel.clone()).max_decoding_message_size(usize::MAX);
        let system_params_client = SystemParamsServiceClient::new(channel);

        Self {
            hummock_client,
            system_params_client,
        }
    }
}

/// Client to proxy server. Cloning the instance is lightweight.
///
/// Todo(wcy-fdu): add refresh client interface.
#[derive(Debug, Clone)]
pub struct GrpcCompactorProxyClient {
    pub core: Arc<RwLock<GrpcCompactorProxyClientCore>>,
    endpoint: String,
}

impl GrpcCompactorProxyClient {
    pub async fn new(endpoint: String) -> Self {
        let channel = Self::connect_to_endpoint(endpoint.clone()).await;
        let core = Arc::new(RwLock::new(GrpcCompactorProxyClientCore::new(channel)));
        Self { core, endpoint }
    }

    async fn recreate_core(&self) {
        tracing::info!("GrpcCompactorProxyClient rpc transfer failed, try to reconnect");
        let channel = Self::connect_to_endpoint(self.endpoint.clone()).await;
        let mut core = self.core.write().await;
        *core = GrpcCompactorProxyClientCore::new(channel);
    }

    async fn connect_to_endpoint(endpoint: String) -> Channel {
        let endpoint = Endpoint::from_shared(endpoint).expect("Fail to construct tonic Endpoint");
        endpoint
            .http2_keep_alive_interval(Duration::from_secs(ENDPOINT_KEEP_ALIVE_INTERVAL_SEC))
            .keep_alive_timeout(Duration::from_secs(ENDPOINT_KEEP_ALIVE_TIMEOUT_SEC))
            .connect_timeout(Duration::from_secs(5))
            .monitored_connect("grpc-compactor-proxy-client", Default::default())
            .await
            .expect("Failed to create channel via proxy rpc endpoint.")
    }

    pub async fn get_new_sst_ids(
        &self,
        request: GetNewSstIdsRequest,
    ) -> std::result::Result<tonic::Response<GetNewSstIdsResponse>, tonic::Status> {
        retry_rpc!(self, get_new_sst_ids, request, GetNewSstIdsResponse)
    }

    pub async fn report_compaction_task(
        &self,
        request: ReportCompactionTaskRequest,
    ) -> std::result::Result<tonic::Response<ReportCompactionTaskResponse>, tonic::Status> {
        retry_rpc!(
            self,
            report_compaction_task,
            request,
            ReportCompactionTaskResponse
        )
    }

    pub async fn get_system_params(
        &self,
    ) -> std::result::Result<tonic::Response<GetSystemParamsResponse>, tonic::Status> {
        tokio_retry::RetryIf::spawn(
            Self::get_retry_strategy(),
            || async {
                let mut system_params_client = self.core.read().await.system_params_client.clone();
                let rpc_res = system_params_client
                    .get_system_params(GetSystemParamsRequest {})
                    .await;
                if rpc_res.is_err() {
                    self.recreate_core().await;
                }
                rpc_res
            },
            Self::should_retry,
        )
        .await
    }

    #[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)
    }

    #[inline(always)]
    fn should_retry(status: &tonic::Status) -> bool {
        if status.code() == tonic::Code::Unavailable
            || status.code() == tonic::Code::Unknown
            || (status.code() == tonic::Code::Unauthenticated
                && status.message().contains("invalid auth token"))
        {
            return true;
        }
        false
    }
}

#[macro_export]
macro_rules! retry_rpc {
    ($self:expr, $rpc_call:ident, $request:expr, $response:ty) => {
        tokio_retry::RetryIf::spawn(
            Self::get_retry_strategy(),
            || async {
                let mut hummock_client = $self.core.read().await.hummock_client.clone();
                let rpc_res = hummock_client.$rpc_call($request.clone()).await;
                if rpc_res.is_err() {
                    $self.recreate_core().await;
                }
                rpc_res
            },
            Self::should_retry,
        )
        .await
    };
}