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


##########
ballista/rust/scheduler/src/flight_sql.rs:
##########
@@ -0,0 +1,460 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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 arrow_flight::flight_descriptor::DescriptorType;
+use arrow_flight::flight_service_server::FlightService;
+use arrow_flight::sql::server::FlightSqlService;
+use arrow_flight::sql::{
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult, CommandGetCatalogs, 
CommandGetCrossReference,
+    CommandGetDbSchemas, CommandGetExportedKeys, CommandGetImportedKeys,
+    CommandGetPrimaryKeys, CommandGetSqlInfo, CommandGetTableTypes, 
CommandGetTables,
+    CommandPreparedStatementQuery, CommandPreparedStatementUpdate, 
CommandStatementQuery,
+    CommandStatementUpdate, SqlInfo, TicketStatementQuery,
+};
+use arrow_flight::{
+    FlightData, FlightDescriptor, FlightEndpoint, FlightInfo, Location, Ticket,
+};
+use log::{debug, error};
+use std::collections::HashMap;
+use std::sync::{Arc, Mutex};
+use std::time::Duration;
+use tonic::{Response, Status, Streaming};
+
+use crate::scheduler_server::event::QueryStageSchedulerEvent;
+use crate::scheduler_server::SchedulerServer;
+use arrow_flight::SchemaAsIpc;
+use ballista_core::config::BallistaConfig;
+use ballista_core::serde::protobuf;
+use ballista_core::serde::protobuf::job_status;
+use ballista_core::serde::protobuf::JobStatus;
+use ballista_core::serde::protobuf::PhysicalPlanNode;
+use datafusion::arrow;
+use datafusion::arrow::datatypes::Schema;
+use datafusion::arrow::ipc::writer::{IpcDataGenerator, IpcWriteOptions};
+use datafusion::logical_expr::LogicalPlan;
+use datafusion_proto::protobuf::LogicalPlanNode;
+use prost::Message;
+use tokio::time::sleep;
+use uuid::Uuid;
+
+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("Error getting partition 
ID".to_string()))?
+                };
+                let authority = if let Some(ref md) = loc.executor_meta {
+                    format!("{}:{}", md.host, md.port)
+                } else {
+                    Err(Status::internal(
+                        "Invalid partition location, missing executor metadata"
+                            .to_string(),
+                    ))?
+                };
+                if let Some(ref stats) = loc.partition_stats {
+                    num_rows += stats.num_rows;
+                    num_bytes += stats.num_bytes;
+                } else {
+                    Err(Status::internal("Error getting stats".to_string()))?

Review Comment:
   There's a lot of untested branches here. I'm still trying to figure out how 
to test properly. If the Rust implementation of the Flight SQL client is far 
enough along, integration tests would be good. Otherwise (and perhaps in 
addition to), unit test would be helpful. But so much of this is interacting 
with other parts of the system, I'm not quite sure how to do it. I'll look over 
`test_poll_work()` as a reference.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to