forked from apache/horaedb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
Copy pathmod.rs
310 lines (262 loc) · 9.19 KB
/
mod.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
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0.
//! Grpc services
use std::{
net::{AddrParseError, SocketAddr},
stringify,
sync::Arc,
time::Duration,
};
use analytic_engine::setup::OpenedWals;
use ceresdbproto::{
meta_event::meta_event_service_server::MetaEventServiceServer,
remote_engine::remote_engine_service_server::RemoteEngineServiceServer,
storage::storage_service_server::StorageServiceServer,
};
use cluster::ClusterRef;
use common_types::{column_schema, record_batch::RecordBatch};
use futures::FutureExt;
use generic_error::GenericError;
use log::{info, warn};
use macros::define_result;
use proxy::{
forward,
instance::InstanceRef,
schema_config_provider::{self},
Proxy,
};
use query_engine::executor::Executor as QueryExecutor;
use runtime::{JoinHandle, Runtime};
use snafu::{Backtrace, OptionExt, ResultExt, Snafu};
use table_engine::engine::EngineRuntimes;
use tokio::sync::oneshot::{self, Sender};
use tonic::transport::Server;
use crate::grpc::{
meta_event_service::MetaServiceImpl,
remote_engine_service::{
dedup_requests::RequestNotifiers, error, RemoteEngineServiceImpl, RequestKey,
},
storage_service::StorageServiceImpl,
};
mod meta_event_service;
mod metrics;
mod remote_engine_service;
mod storage_service;
#[derive(Debug, Snafu)]
pub enum Error {
#[snafu(display("Internal error, message:{}, cause:{}", msg, source))]
Internal { msg: String, source: GenericError },
#[snafu(display(
"Failed to keep grpc service, err:{}.\nBacktrace:\n{}",
source,
backtrace
))]
FailServe {
source: tonic::transport::Error,
backtrace: Backtrace,
},
#[snafu(display(
"Failed to parse rpc service addr, err:{}.\nBacktrace:\n{}",
source,
backtrace
))]
InvalidRpcServeAddr {
source: AddrParseError,
backtrace: Backtrace,
},
#[snafu(display("Missing meta client config.\nBacktrace:\n{}", backtrace))]
MissingMetaClientConfig { backtrace: Backtrace },
#[snafu(display("Missing grpc environment.\nBacktrace:\n{}", backtrace))]
MissingEnv { backtrace: Backtrace },
#[snafu(display("Missing runtimes.\nBacktrace:\n{}", backtrace))]
MissingRuntimes { backtrace: Backtrace },
#[snafu(display("Missing instance.\nBacktrace:\n{}", backtrace))]
MissingInstance { backtrace: Backtrace },
#[snafu(display("Missing wals.\nBacktrace:\n{}", backtrace))]
MissingWals { backtrace: Backtrace },
#[snafu(display("Missing timeout.\nBacktrace:\n{}", backtrace))]
MissingTimeout { backtrace: Backtrace },
#[snafu(display("Missing proxy.\nBacktrace:\n{}", backtrace))]
MissingProxy { backtrace: Backtrace },
#[snafu(display("Catalog name is not utf8.\nBacktrace:\n{}", backtrace))]
ParseCatalogName {
source: std::string::FromUtf8Error,
backtrace: Backtrace,
},
#[snafu(display("Schema name is not utf8.\nBacktrace:\n{}", backtrace))]
ParseSchemaName {
source: std::string::FromUtf8Error,
backtrace: Backtrace,
},
#[snafu(display("Fail to build forwarder, err:{}", source))]
BuildForwarder { source: forward::Error },
#[snafu(display(
"Fail to build column schema from column:{}, err:{}",
column_name,
source
))]
BuildColumnSchema {
column_name: String,
source: column_schema::Error,
},
#[snafu(display("Invalid column schema, column:{}, err:{}", column_name, source))]
InvalidColumnSchema {
column_name: String,
source: column_schema::Error,
},
#[snafu(display("Invalid argument: {}", msg))]
InvalidArgument { msg: String },
#[snafu(display("Get schema config failed, err:{}", source))]
GetSchemaConfig {
source: schema_config_provider::Error,
},
}
define_result!(Error);
/// Rpc services manages all grpc services of the server.
pub struct RpcServices<Q: QueryExecutor + 'static> {
serve_addr: SocketAddr,
rpc_server: StorageServiceServer<StorageServiceImpl<Q>>,
meta_rpc_server: Option<MetaEventServiceServer<MetaServiceImpl<Q>>>,
remote_engine_server: RemoteEngineServiceServer<RemoteEngineServiceImpl<Q>>,
runtime: Arc<Runtime>,
stop_tx: Option<Sender<()>>,
join_handle: Option<JoinHandle<()>>,
}
impl<Q: QueryExecutor + 'static> RpcServices<Q> {
pub async fn start(&mut self) -> Result<()> {
let rpc_server = self.rpc_server.clone();
let meta_rpc_server = self.meta_rpc_server.clone();
let remote_engine_server = self.remote_engine_server.clone();
let serve_addr = self.serve_addr;
let (stop_tx, stop_rx) = oneshot::channel();
let join_handle = self.runtime.spawn(async move {
info!("Grpc server tries to listen on {}", serve_addr);
let mut router = Server::builder().add_service(rpc_server);
if let Some(s) = meta_rpc_server {
info!("Grpc server serves meta rpc service");
router = router.add_service(s);
};
info!("Grpc server serves remote engine rpc service");
router = router.add_service(remote_engine_server);
router
.serve_with_shutdown(serve_addr, stop_rx.map(drop))
.await
.unwrap_or_else(|e| {
panic!("Grpc server listens failed, err:{e:?}");
});
});
self.join_handle = Some(join_handle);
self.stop_tx = Some(stop_tx);
Ok(())
}
pub async fn shutdown(&mut self) {
if let Some(stop_tx) = self.stop_tx.take() {
let res = stop_tx.send(());
warn!("Send stop signal, send_res:{:?}", res);
}
if let Some(join_handle) = self.join_handle.take() {
let join_res = join_handle.await;
warn!("Finish join with serve task, join_res:{:?}", join_res);
}
}
}
pub struct Builder<Q> {
endpoint: String,
timeout: Option<Duration>,
runtimes: Option<Arc<EngineRuntimes>>,
instance: Option<InstanceRef<Q>>,
cluster: Option<ClusterRef>,
opened_wals: Option<OpenedWals>,
proxy: Option<Arc<Proxy<Q>>>,
request_notifiers: Option<Arc<RequestNotifiers<RequestKey, RecordBatch, error::Error>>>,
}
impl<Q> Builder<Q> {
pub fn new() -> Self {
Self {
endpoint: "0.0.0.0:8381".to_string(),
timeout: None,
runtimes: None,
instance: None,
cluster: None,
opened_wals: None,
proxy: None,
request_notifiers: None,
}
}
pub fn endpoint(mut self, endpoint: String) -> Self {
self.endpoint = endpoint;
self
}
pub fn runtimes(mut self, runtimes: Arc<EngineRuntimes>) -> Self {
self.runtimes = Some(runtimes);
self
}
pub fn instance(mut self, instance: InstanceRef<Q>) -> Self {
self.instance = Some(instance);
self
}
// Cluster is an optional field for building [RpcServices].
pub fn cluster(mut self, cluster: Option<ClusterRef>) -> Self {
self.cluster = cluster;
self
}
pub fn opened_wals(mut self, opened_wals: OpenedWals) -> Self {
self.opened_wals = Some(opened_wals);
self
}
pub fn timeout(mut self, timeout: Option<Duration>) -> Self {
self.timeout = timeout;
self
}
pub fn proxy(mut self, proxy: Arc<Proxy<Q>>) -> Self {
self.proxy = Some(proxy);
self
}
pub fn dedup_requests(mut self, dedup_requests: bool) -> Self {
if dedup_requests {
self.request_notifiers = Some(Arc::new(RequestNotifiers::default()));
}
self
}
}
impl<Q: QueryExecutor + 'static> Builder<Q> {
pub fn build(self) -> Result<RpcServices<Q>> {
let runtimes = self.runtimes.context(MissingRuntimes)?;
let instance = self.instance.context(MissingInstance)?;
let opened_wals = self.opened_wals.context(MissingWals)?;
let proxy = self.proxy.context(MissingProxy)?;
let meta_rpc_server = self.cluster.map(|v| {
let builder = meta_event_service::Builder {
cluster: v,
instance: instance.clone(),
runtime: runtimes.default_runtime.clone(),
opened_wals,
};
MetaEventServiceServer::new(builder.build())
});
let remote_engine_server = {
let service = RemoteEngineServiceImpl {
instance,
runtimes: runtimes.clone(),
request_notifiers: self.request_notifiers,
};
RemoteEngineServiceServer::new(service)
};
let runtime = runtimes.default_runtime.clone();
let storage_service = StorageServiceImpl {
proxy,
runtimes,
timeout: self.timeout,
};
let rpc_server = StorageServiceServer::new(storage_service);
let serve_addr = self.endpoint.parse().context(InvalidRpcServeAddr)?;
Ok(RpcServices {
serve_addr,
rpc_server,
meta_rpc_server,
remote_engine_server,
runtime,
stop_tx: None,
join_handle: None,
})
}
}