You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "alamb (via GitHub)" <gi...@apache.org> on 2023/02/01 12:07:19 UTC

[GitHub] [arrow-datafusion] alamb commented on a diff in pull request #5138: add example for Flight SQL Server that supports JDBC driver

alamb commented on code in PR #5138:
URL: https://github.com/apache/arrow-datafusion/pull/5138#discussion_r1093132785


##########
datafusion-examples/examples/flight_sql_server.rs:
##########
@@ -0,0 +1,551 @@
+// 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::ipc::writer::IpcWriteOptions;
+use arrow::record_batch::RecordBatch;
+use arrow_flight::flight_descriptor::DescriptorType;
+use arrow_flight::flight_service_server::{FlightService, FlightServiceServer};
+use arrow_flight::sql::server::FlightSqlService;
+use arrow_flight::sql::{
+    ActionClosePreparedStatementRequest, ActionCreatePreparedStatementRequest,
+    ActionCreatePreparedStatementResult, Any, CommandGetCatalogs,
+    CommandGetCrossReference, CommandGetDbSchemas, CommandGetExportedKeys,
+    CommandGetImportedKeys, CommandGetPrimaryKeys, CommandGetSqlInfo,
+    CommandGetTableTypes, CommandGetTables, CommandPreparedStatementQuery,
+    CommandPreparedStatementUpdate, CommandStatementQuery, CommandStatementUpdate,
+    ProstMessageExt, SqlInfo, TicketStatementQuery,
+};
+use arrow_flight::utils::batches_to_flight_data;
+use arrow_flight::{
+    Action, FlightData, FlightDescriptor, FlightEndpoint, FlightInfo, IpcMessage,
+    Location, SchemaAsIpc, Ticket,
+};
+use arrow_schema::Schema;
+use dashmap::DashMap;
+use datafusion::logical_expr::LogicalPlan;
+use datafusion::prelude::{DataFrame, ParquetReadOptions, SessionConfig, SessionContext};
+use futures::{stream, Stream};
+use log::info;
+use mimalloc::MiMalloc;
+use prost::Message;
+use std::pin::Pin;
+use std::sync::Arc;
+use tonic::transport::Server;
+use tonic::{Request, Response, Status, Streaming};
+use uuid::Uuid;
+
+#[global_allocator]
+static GLOBAL: MiMalloc = MiMalloc;
+
+macro_rules! status {
+    ($desc:expr, $err:expr) => {
+        Status::internal(format!("{}: {} at {}:{}", $desc, $err, file!(), line!()))
+    };
+}
+
+/// This example shows how to wrap DataFusion with `FlightSqlService` to support connecting
+/// to a standalone DataFusion-based server with a JDBC client, using the open source "JDBC Driver
+/// for Arrow Flight SQL".
+///
+/// To install the JDBC driver in DBeaver for example, see these instructions:
+/// https://docs.dremio.com/software/client-applications/dbeaver/
+/// When configuring the driver, specify property "UseEncryption" = false
+///
+/// JDBC connection string: "jdbc:arrow-flight-sql://127.0.0.1:50051/"
+///
+/// Based heavily on Ballista's implementation: https://github.com/apache/arrow-ballista/blob/main/ballista/scheduler/src/flight_sql.rs
+/// and the example in arrow-rs: https://github.com/apache/arrow-rs/blob/master/arrow-flight/examples/flight_sql_server.rs
+///
+#[tokio::main]
+async fn main() -> Result<(), Box<dyn std::error::Error>> {
+    env_logger::init();
+    let addr = "0.0.0.0:50051".parse()?;
+    let service = FlightSqlServiceImpl {
+        contexts: Default::default(),
+        statements: Default::default(),
+        results: Default::default(),
+    };
+    info!("Listening on {addr:?}");
+    let svc = FlightServiceServer::new(service);
+
+    Server::builder().add_service(svc).serve(addr).await?;
+
+    Ok(())
+}
+
+pub struct FlightSqlServiceImpl {
+    contexts: Arc<DashMap<String, Arc<SessionContext>>>,
+    statements: Arc<DashMap<String, LogicalPlan>>,
+    results: Arc<DashMap<String, Vec<RecordBatch>>>,
+}
+
+impl FlightSqlServiceImpl {
+    fn get_ctx(&self, handle: &str) -> Result<Arc<SessionContext>, Status> {
+        if let Some(context) = self.contexts.get(handle) {
+            Ok(context.clone())
+        } else {
+            Err(Status::internal(format!(
+                "Context handle not found: {handle}"
+            )))?
+        }
+    }
+
+    fn get_plan(&self, handle: &str) -> Result<LogicalPlan, Status> {
+        if let Some(plan) = self.statements.get(handle) {
+            Ok(plan.clone())
+        } else {
+            Err(Status::internal(format!(
+                "Statement handle not found: {handle}"
+            )))?
+        }
+    }
+
+    fn get_result(&self, handle: &str) -> Result<Vec<RecordBatch>, Status> {
+        if let Some(result) = self.results.get(handle) {
+            Ok(result.clone())
+        } else {
+            Err(Status::internal(format!(
+                "Request handle not found: {handle}"
+            )))?
+        }
+    }
+
+    fn remove_plan(&self, handle: &str) -> Result<(), Status> {
+        self.statements.remove(&handle.to_string());
+        Ok(())
+    }
+
+    fn remove_result(&self, handle: &str) -> Result<(), Status> {
+        self.results.remove(&handle.to_string());
+        Ok(())
+    }
+}
+
+#[tonic::async_trait]
+impl FlightSqlService for FlightSqlServiceImpl {
+    type FlightService = FlightSqlServiceImpl;
+
+    async fn do_get_fallback(
+        &self,
+        _request: Request<Ticket>,
+        message: Any,
+    ) -> Result<Response<<Self as FlightService>::DoGetStream>, Status> {
+        if !message.is::<FetchResults>() {
+            Err(Status::unimplemented(format!(
+                "do_get: The defined request is invalid: {}",
+                message.type_url
+            )))?
+        }
+
+        let fr: FetchResults = message
+            .unpack()
+            .map_err(|e| Status::internal(format!("{e:?}")))?
+            .ok_or_else(|| Status::internal("Expected FetchResults but got None!"))?;
+
+        let handle = fr.handle;
+
+        info!("getting results for {handle}");
+        let result = self.get_result(&handle)?;
+        // if we get an empty result, create an empty schema
+        let (schema, batches) = match result.get(0) {
+            None => (Schema::empty(), vec![]),
+            Some(batch) => ((*batch.schema()).clone(), result.clone()),
+        };
+
+        let flight_data = batches_to_flight_data(schema, batches)
+            .map_err(|e| status!("Could not convert batches", e))?
+            .into_iter()
+            .map(Ok);
+
+        let stream: Pin<Box<dyn Stream<Item = Result<FlightData, Status>> + Send>> =
+            Box::pin(stream::iter(flight_data));

Review Comment:
   I think you might be able to use the relatively newly added `FlightDataEncoderBuilder` here to create the FlightData:
   
   https://docs.rs/arrow-flight/32.0.0/arrow_flight/encode/struct.FlightDataEncoderBuilder.html
   
   Pretty much like
   
   ```rust
   // Get an input stream of Result<RecordBatch, FlightError>
   let input_stream = futures::stream::iter(batches);
   
   // Build a stream of `Result<FlightData>` (e.g. to return for do_get)
   let flight_data_stream = FlightDataEncoderBuilder::new()
     .with_schema(shcema)
    .build(input_stream);
   
   tonic::Response::new(flight_data_stream);
   ```
   



-- 
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