-
Notifications
You must be signed in to change notification settings - Fork 198
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Store sessions so users can register tables and query them through flight #269
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -40,6 +40,8 @@ pub enum Action { | |
stage_id: usize, | ||
partition_id: usize, | ||
path: String, | ||
host: String, | ||
port: u16, | ||
}, | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
|
||
//! Implementation of the Apache Arrow Flight protocol that wraps an executor. | ||
|
||
use std::convert::TryFrom; | ||
use std::fs::File; | ||
use std::pin::Pin; | ||
|
||
|
@@ -35,14 +36,15 @@ use datafusion::arrow::{ | |
record_batch::RecordBatch, | ||
}; | ||
use futures::{Stream, StreamExt}; | ||
use log::{debug, warn}; | ||
use log::{debug, info, warn}; | ||
use std::io::{Read, Seek}; | ||
use tokio::sync::mpsc::channel; | ||
use tokio::{ | ||
sync::mpsc::{Receiver, Sender}, | ||
task, | ||
}; | ||
use tokio_stream::wrappers::ReceiverStream; | ||
use tonic::metadata::MetadataValue; | ||
use tonic::{Request, Response, Status, Streaming}; | ||
|
||
type FlightDataSender = Sender<Result<FlightData, Status>>; | ||
|
@@ -135,7 +137,23 @@ impl FlightService for BallistaFlightService { | |
&self, | ||
_request: Request<Streaming<HandshakeRequest>>, | ||
) -> Result<Response<Self::HandshakeStream>, Status> { | ||
Err(Status::unimplemented("handshake")) | ||
let token = uuid::Uuid::new_v4(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. When working properly (i.e. supporting alternate endpoints) the Flight SQL JDBC driver will try to talk to the executor with the same credentials as it used with the scheduler. Let's just accept any credentials for now, so that when the issue is fixed in the |
||
info!("do_handshake token={}", token); | ||
|
||
let result = HandshakeResponse { | ||
protocol_version: 0, | ||
payload: token.as_bytes().to_vec(), | ||
}; | ||
let result = Ok(result); | ||
let output = futures::stream::iter(vec![result]); | ||
let str = format!("Bearer {}", token); | ||
let mut resp: Response< | ||
Pin<Box<dyn Stream<Item = Result<_, Status>> + Sync + Send>>, | ||
> = Response::new(Box::pin(output)); | ||
let md = MetadataValue::try_from(str) | ||
.map_err(|_| Status::invalid_argument("authorization not parsable"))?; | ||
resp.metadata_mut().insert("authorization", md); | ||
Ok(resp) | ||
} | ||
|
||
async fn list_flights( | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add host & port to flights so that when the scheduler gets a flight request, it can go find the source. Architecturally, I see this as a good canonicalization of the source of Flight data, but I'm happy to discuss here!