You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/07/25 13:03:09 UTC

[GitHub] [arrow-ballista] avantgardnerio commented on a diff in pull request #93: Add FlightSQL support

avantgardnerio commented on code in PR #93:
URL: https://github.com/apache/arrow-ballista/pull/93#discussion_r928858994


##########
ballista/rust/scheduler/src/flight_sql.rs:
##########
@@ -0,0 +1,402 @@
+use std::collections::HashMap;
+use std::sync::{Arc, Mutex};
+use std::time::Duration;
+use arrow_flight::{FlightData, FlightDescriptor, FlightEndpoint, FlightInfo, Location, Ticket};
+use arrow_flight::flight_descriptor::DescriptorType;
+use arrow_flight::flight_service_server::FlightService;
+use arrow_flight::sql::{ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest, ActionCreatePreparedStatementResult, CommandGetCatalogs, CommandGetCrossReference, CommandGetDbSchemas, CommandGetExportedKeys, CommandGetImportedKeys, CommandGetPrimaryKeys, CommandGetSqlInfo, CommandGetTables, CommandGetTableTypes, CommandPreparedStatementQuery, CommandPreparedStatementUpdate, CommandStatementQuery, CommandStatementUpdate, SqlInfo, TicketStatementQuery};
+use arrow_flight::sql::server::FlightSqlService;
+use log::{debug, error};
+use tonic::{Response, Status, Streaming};
+
+use crate::scheduler_server::SchedulerServer;
+use datafusion_proto::protobuf::LogicalPlanNode;
+use ballista_core::{
+    serde::protobuf::{PhysicalPlanNode},
+};
+use ballista_core::config::BallistaConfig;
+use arrow_flight::SchemaAsIpc;
+use datafusion::arrow;
+use datafusion::arrow::datatypes::Schema;
+use datafusion::arrow::ipc::writer::{IpcDataGenerator, IpcWriteOptions};
+use datafusion::logical_expr::LogicalPlan;
+use tokio::time::sleep;
+use uuid::{Uuid};
+use crate::scheduler_server::event::QueryStageSchedulerEvent;
+use ballista_core::serde::protobuf::job_status;
+use ballista_core::serde::protobuf::JobStatus;
+use ballista_core::serde::protobuf;
+use prost::Message;
+
+#[derive(Clone)]
+pub struct FlightSqlServiceImpl {
+    server: SchedulerServer<LogicalPlanNode, PhysicalPlanNode>,
+    statements: Arc<Mutex<HashMap<Uuid, LogicalPlan>>>,
+}
+
+impl FlightSqlServiceImpl {
+    pub fn new(server: SchedulerServer<LogicalPlanNode, PhysicalPlanNode>) -> Self {
+        Self { server, statements: Arc::new(Mutex::new(HashMap::new())) }
+    }
+}
+
+#[tonic::async_trait]
+impl FlightSqlService for FlightSqlServiceImpl {
+    type FlightService = FlightSqlServiceImpl;
+    // get_flight_info
+    async fn get_flight_info_statement(
+        &self,
+        query: CommandStatementQuery,
+        request: FlightDescriptor,
+    ) -> Result<Response<FlightInfo>, Status> {
+        debug!("Got query:\n{}", query.query);
+
+        // Run query
+        let config_builder = BallistaConfig::builder();
+        let config = config_builder.build()
+            .map_err(|e| Status::internal(format!("Error building config: {}", e)))?;
+        let ctx = self.server
+            .state
+            .session_manager
+            .create_session(&config)
+            .await
+            .map_err(|e| {
+                Status::internal(format!(
+                    "Failed to create SessionContext: {:?}",
+                    e
+                ))
+            })?;
+        let plan = ctx
+            .sql(&query.query.as_str())
+            .await
+            .and_then(|df| df.to_logical_plan())
+            .map_err(|e| Status::internal(format!("Error building plan: {}", e)))?;
+
+        // enqueue job
+        let job_id = self.server.state.task_manager.generate_job_id();
+
+        self.server.state
+            .task_manager
+            .queue_job(&job_id)
+            .await
+            .map_err(|e| {
+                let msg = format!("Failed to queue job {}: {:?}", job_id, e);
+                error!("{}", msg);
+
+                Status::internal(msg)
+            })?;
+
+        let query_stage_event_sender =
+            self.server.query_stage_event_loop.get_sender().map_err(|e| {
+                Status::internal(format!(
+                    "Could not get query stage event sender due to: {}",
+                    e
+                ))
+            })?;
+
+        query_stage_event_sender
+            .post_event(QueryStageSchedulerEvent::JobQueued {
+                job_id: job_id.clone(),
+                session_id: ctx.session_id().clone(),
+                session_ctx: ctx,
+                plan: Box::new(plan.clone()),
+            })
+            .await
+            .map_err(|e| {
+                let msg =
+                    format!("Failed to send JobQueued event for {}: {:?}", job_id, e);
+                error!("{}", msg);
+                Status::internal(msg)
+            })?;
+
+        // let handle = Uuid::new_v4();
+        // let mut statements = self.statements.try_lock()
+        //     .map_err(|e| Status::internal(format!("Error locking statements: {}", e)))?;
+        // statements.insert(handle, plan.clone());
+
+        // poll for job completion
+        let mut num_rows = 0;
+        let mut num_bytes = 0;
+        let fieps = loop {
+            sleep(Duration::from_millis(100)).await;
+            let status = self.server.state.task_manager.get_job_status(&job_id).await
+                .map_err(|e| {
+                    let msg = format!("Error getting status for job {}: {:?}", job_id, e);
+                    error!("{}", msg);
+                    Status::internal(msg)
+                })?;
+            let status: JobStatus = match status {
+                Some(status) => status,
+                None => {
+                    let msg = format!("Error getting status for job {}!", job_id);
+                    error!("{}", msg);
+                    Err(Status::internal(msg))?
+                }
+            };
+            let status: job_status::Status = match status.status {
+                Some(status) => status,
+                None => {
+                    let msg = format!("Error getting status for job {}!", job_id);
+                    error!("{}", msg);
+                    Err(Status::internal(msg))?
+                }
+            };
+            let completed = match status {
+                job_status::Status::Queued(_) => continue,
+                job_status::Status::Running(_) => continue,
+                job_status::Status::Failed(e) => {
+                    Err(Status::internal(format!("Error building plan: {}", e.error)))?
+                }
+                job_status::Status::Completed(comp) => comp
+            };
+            let mut fieps: Vec<_> = vec![];
+            for loc in completed.partition_location.iter() {
+                let fetch = if let Some(ref id) = loc.partition_id {
+                    let fetch = protobuf::FetchPartition {
+                        job_id: id.job_id.clone(),
+                        stage_id: id.stage_id,
+                        partition_id: id.partition_id,
+                        path: loc.path.clone()
+                    };
+                    protobuf::Action {
+                        action_type: Some(protobuf::action::ActionType::FetchPartition(fetch)),
+                        settings: vec![],
+                    }
+                } else {
+                    Err(Status::internal(format!("Error getting partition it")))?
+                };
+                let authority = if let Some(ref md) = loc.executor_meta {
+                    // pub id: ::prost::alloc::string::String,
+                    format!("{}:{}", md.host, md.port)
+                } else {
+                    Err(Status::internal(format!("Error getting location")))?
+                };
+                if let Some(ref stats) = loc.partition_stats {
+                    num_rows += stats.num_rows;
+                    num_bytes += stats.num_bytes;
+                    // pub num_batches: i64,
+                } else {
+                    Err(Status::internal(format!("Error getting stats")))?
+                }
+                let loc = Location { uri: format!("grpc+tcp://{}", authority) };
+                let buf = fetch.encode_to_vec();
+                let ticket = Ticket { ticket: buf };
+                let fiep = FlightEndpoint {
+                    ticket: Some(ticket),
+                    location: vec![loc],
+                };
+                fieps.push(fiep);
+            }
+            break fieps;

Review Comment:
   It's pretty neat. You can actually return a value from a `loop`, and that's what this is doing. I wasn't aware of it either, until this PR. https://doc.rust-lang.org/rust-by-example/flow_control/loop/return.html



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org