forked from apache/arrow
-
Notifications
You must be signed in to change notification settings - Fork 0
/
flight-server.rs
143 lines (116 loc) · 4.56 KB
/
flight-server.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
use std::pin::Pin;
use futures::Stream;
use tonic::transport::Server;
use tonic::{Request, Response, Status, Streaming};
use datafusion::execution::context::ExecutionContext;
use flight::{
flight_service_server::FlightService, flight_service_server::FlightServiceServer,
Action, ActionType, Criteria, Empty, FlightData, FlightDescriptor, FlightInfo,
HandshakeRequest, HandshakeResponse, PutResult, SchemaResult, Ticket,
};
#[derive(Clone)]
pub struct FlightServiceImpl {}
#[tonic::async_trait]
impl FlightService for FlightServiceImpl {
type HandshakeStream = Pin<
Box<dyn Stream<Item = Result<HandshakeResponse, Status>> + Send + Sync + 'static>,
>;
type ListFlightsStream =
Pin<Box<dyn Stream<Item = Result<FlightInfo, Status>> + Send + Sync + 'static>>;
type DoGetStream =
Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send + Sync + 'static>>;
type DoPutStream =
Pin<Box<dyn Stream<Item = Result<PutResult, Status>> + Send + Sync + 'static>>;
type DoActionStream = Pin<
Box<dyn Stream<Item = Result<flight::Result, Status>> + Send + Sync + 'static>,
>;
type ListActionsStream =
Pin<Box<dyn Stream<Item = Result<ActionType, Status>> + Send + Sync + 'static>>;
async fn do_get(
&self,
request: Request<Ticket>,
) -> Result<Response<Self::DoGetStream>, Status> {
let ticket = request.into_inner();
match String::from_utf8(ticket.ticket.to_vec()) {
Ok(sql) => {
println!("do_get: {}", sql);
// create local execution context
let mut ctx = ExecutionContext::new();
ctx.register_parquet("alltypes_plain", "alltypes_plain.snappy.parquet")
.unwrap();
// create the query plan
let plan = ctx
.create_logical_plan(&sql)
.map_err(|e| to_tonic_err(&e))?;
let plan = ctx.optimize(&plan).map_err(|e| to_tonic_err(&e))?;
let plan = ctx
.create_physical_plan(&plan, 1024 * 1024)
.map_err(|e| to_tonic_err(&e))?;
//TODO make this async
// execute the query
let results = ctx.collect(plan.as_ref()).map_err(|e| to_tonic_err(&e))?;
let flights: Vec<Result<FlightData, Status>> = results
.iter()
.map(|batch| Ok(FlightData::from(batch)))
.collect();
let output = futures::stream::iter(flights);
Ok(Response::new(Box::pin(output) as Self::DoGetStream))
}
Err(e) => Err(Status::invalid_argument(format!("Invalid ticket: {:?}", e))),
}
}
async fn handshake(
&self,
_request: Request<Streaming<HandshakeRequest>>,
) -> Result<Response<Self::HandshakeStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn list_flights(
&self,
_request: Request<Criteria>,
) -> Result<Response<Self::ListFlightsStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn get_flight_info(
&self,
_request: Request<FlightDescriptor>,
) -> Result<Response<FlightInfo>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn get_schema(
&self,
_request: Request<FlightDescriptor>,
) -> Result<Response<SchemaResult>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn do_put(
&self,
_request: Request<Streaming<FlightData>>,
) -> Result<Response<Self::DoPutStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn do_action(
&self,
_request: Request<Action>,
) -> Result<Response<Self::DoActionStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
async fn list_actions(
&self,
_request: Request<Empty>,
) -> Result<Response<Self::ListActionsStream>, Status> {
Err(Status::unimplemented("Not yet implemented"))
}
}
fn to_tonic_err(e: &datafusion::error::ExecutionError) -> Status {
Status::internal(format!("{:?}", e))
}
#[tokio::main]
async fn main() -> Result<(), Box<dyn std::error::Error>> {
let addr = "0.0.0.0:50051".parse()?;
let service = FlightServiceImpl {};
let svc = FlightServiceServer::new(service);
println!("Listening on {:?}", addr);
Server::builder().add_service(svc).serve(addr).await?;
Ok(())
}