Skip to content

Commit

Permalink
feat: support FlightSQL in 3.0 (#24678)
Browse files Browse the repository at this point in the history
* feat: support FlightSQL by serving gRPC requests on same port as HTTP

This commit adds support for FlightSQL queries via gRPC to the influxdb3 service. It does so by ensuring the QueryExecutor implements the QueryNamespaceProvider trait, and the underlying QueryDatabase implements QueryNamespace. Satisfying those requirements allows the construction of a FlightServiceServer from the service_grpc_flight crate.

The FlightServiceServer is a gRPC server that can be served via tonic at the API surface; however, enabling this required some tower::Service wrangling. The influxdb3_server/src/server.rs module was introduced to house this code. The objective is to serve both gRPC (via the newly introduced tonic server) and standard REST HTTP requests (via the existing HTTP server) on the same port.

This is accomplished by the HybridService which can handle either gRPC or non-gRPC HTTP requests. The HybridService is wrapped in a HybridMakeService which allows us to serve it via hyper::Server on a single bind address.

End-to-end tests were added in influxdb3/tests/flight.rs. These cover some basic FlightSQL cases. A common.rs module was added that introduces some fixtures to aid in end-to-end tests in influxdb3.
  • Loading branch information
hiltontj authored Feb 26, 2024
1 parent 75afbbd commit 298055e
Show file tree
Hide file tree
Showing 10 changed files with 604 additions and 80 deletions.
16 changes: 14 additions & 2 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

12 changes: 12 additions & 0 deletions influxdb3/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -71,4 +71,16 @@ jemalloc_replacing_malloc = ["tikv-jemalloc-sys", "tikv-jemalloc-ctl"]
clippy = []

[dev-dependencies]
arrow_util = { path = "../arrow_util" }
influxdb_iox_client = { path = "../influxdb_iox_client" }

# Crates.io dependencies in alphabetical order:
arrow = { workspace = true }
arrow-array = "49.0.0"
arrow-flight = "49.0.0"
assert_cmd = "2.0.14"
futures = "0.3.28"
hyper = "0.14"
reqwest = { version = "0.11.24", default-features = false, features = ["rustls-tls"] }
tonic.workspace = true
tower = "0.4.13"
99 changes: 99 additions & 0 deletions influxdb3/tests/common.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
use std::{
net::{SocketAddr, SocketAddrV4, TcpListener},
process::{Child, Command, Stdio},
time::Duration,
};

use assert_cmd::cargo::CommandCargoExt;
use influxdb_iox_client::flightsql::FlightSqlClient;

/// A running instance of the `influxdb3 serve` process
pub struct TestServer {
bind_addr: SocketAddr,
server_process: Child,
http_client: reqwest::Client,
}

impl TestServer {
/// Spawn a new [`TestServer`]
///
/// This will run the `influxdb3 serve` command, and bind its HTTP
/// address to a random port on localhost.
pub async fn spawn() -> Self {
let bind_addr = get_local_bind_addr();
let mut command = Command::cargo_bin("influxdb3").expect("create the influxdb3 command");
let command = command
.arg("serve")
.args(["--http-bind", &bind_addr.to_string()])
// TODO - other configuration can be passed through
.stdout(Stdio::null())
.stderr(Stdio::null());

let server_process = command.spawn().expect("spawn the influxdb3 server process");

let server = Self {
bind_addr,
server_process,
http_client: reqwest::Client::new(),
};

server.wait_until_ready().await;
server
}

/// Get the URL of the running service for use with an HTTP client
pub fn client_addr(&self) -> String {
format!("http://{addr}", addr = self.bind_addr)
}

/// Get a [`FlightSqlClient`] for making requests to the running service over gRPC
pub async fn flight_client(&self, database: &str) -> FlightSqlClient {
let channel = tonic::transport::Channel::from_shared(self.client_addr())
.expect("create tonic channel")
.connect()
.await
.expect("connect to gRPC client");
let mut client = FlightSqlClient::new(channel);
client.add_header("database", database).unwrap();
client.add_header("iox-debug", "true").unwrap();
client
}

fn kill(&mut self) {
self.server_process.kill().expect("kill the server process");
}

async fn wait_until_ready(&self) {
while self
.http_client
.get(format!("{base}/health", base = self.client_addr()))
.send()
.await
.is_err()
{
tokio::time::sleep(Duration::from_millis(10)).await;
}
}
}

impl Drop for TestServer {
fn drop(&mut self) {
self.kill();
}
}

/// Get an available bind address on localhost
///
/// This binds a [`TcpListener`] to 127.0.0.1:0, which will randomly
/// select an available port, and produces the resulting local address.
/// The [`TcpListener`] is dropped at the end of the function, thus
/// freeing the port for use by the caller.
fn get_local_bind_addr() -> SocketAddr {
let ip = std::net::Ipv4Addr::new(127, 0, 0, 1);
let port = 0;
let addr = SocketAddrV4::new(ip, port);
TcpListener::bind(addr)
.expect("bind to a socket address")
.local_addr()
.expect("get local address")
}
151 changes: 151 additions & 0 deletions influxdb3/tests/flight.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
use arrow::record_batch::RecordBatch;
use arrow_flight::{decode::FlightRecordBatchStream, sql::SqlInfo};
use arrow_util::assert_batches_sorted_eq;
use futures::TryStreamExt;

use crate::common::TestServer;

mod common;

#[tokio::test]
async fn flight() {
let server = TestServer::spawn().await;

// use the influxdb3_client to write in some data
write_lp_to_db(
&server,
"foo",
"cpu,host=s1,region=us-east usage=0.9 1\n\
cpu,host=s1,region=us-east usage=0.89 2\n\
cpu,host=s1,region=us-east usage=0.85 3",
)
.await;

let mut client = server.flight_client("foo").await;

// Ad-hoc Query:
{
let response = client.query("SELECT * FROM cpu").await.unwrap();

let batches = collect_stream(response).await;
assert_batches_sorted_eq!(
[
"+------+---------+--------------------------------+-------+",
"| host | region | time | usage |",
"+------+---------+--------------------------------+-------+",
"| s1 | us-east | 1970-01-01T00:00:00.000000001Z | 0.9 |",
"| s1 | us-east | 1970-01-01T00:00:00.000000002Z | 0.89 |",
"| s1 | us-east | 1970-01-01T00:00:00.000000003Z | 0.85 |",
"+------+---------+--------------------------------+-------+",
],
&batches
);
}

// Ad-hoc Query error:
{
let error = client
.query("SELECT * FROM invalid_table")
.await
.unwrap_err();

assert!(error
.to_string()
.contains("table 'public.iox.invalid_table' not found"));
}

// Prepared query:
{
let handle = client.prepare("SELECT * FROM cpu".into()).await.unwrap();
let stream = client.execute(handle).await.unwrap();

let batches = collect_stream(stream).await;
assert_batches_sorted_eq!(
[
"+------+---------+--------------------------------+-------+",
"| host | region | time | usage |",
"+------+---------+--------------------------------+-------+",
"| s1 | us-east | 1970-01-01T00:00:00.000000001Z | 0.9 |",
"| s1 | us-east | 1970-01-01T00:00:00.000000002Z | 0.89 |",
"| s1 | us-east | 1970-01-01T00:00:00.000000003Z | 0.85 |",
"+------+---------+--------------------------------+-------+",
],
&batches
);
}

// Get SQL Infos:
{
let infos = vec![SqlInfo::FlightSqlServerName as u32];
let stream = client.get_sql_info(infos).await.unwrap();
let batches = collect_stream(stream).await;
assert_batches_sorted_eq!(
[
"+-----------+-----------------------------+",
"| info_name | value |",
"+-----------+-----------------------------+",
"| 0 | {string_value=InfluxDB IOx} |",
"+-----------+-----------------------------+",
],
&batches
);
}

// Get Tables
{
type OptStr = std::option::Option<&'static str>;
let stream = client
.get_tables(OptStr::None, OptStr::None, OptStr::None, vec![], false)
.await
.unwrap();
let batches = collect_stream(stream).await;

assert_batches_sorted_eq!(
[
"+--------------+--------------------+-------------+------------+",
"| catalog_name | db_schema_name | table_name | table_type |",
"+--------------+--------------------+-------------+------------+",
"| public | information_schema | columns | VIEW |",
"| public | information_schema | df_settings | VIEW |",
"| public | information_schema | tables | VIEW |",
"| public | information_schema | views | VIEW |",
"| public | iox | cpu | BASE TABLE |",
"+--------------+--------------------+-------------+------------+",
],
&batches
);
}

// Get Catalogs
{
let stream = client.get_catalogs().await.unwrap();
let batches = collect_stream(stream).await;
assert_batches_sorted_eq!(
[
"+--------------+",
"| catalog_name |",
"+--------------+",
"| public |",
"+--------------+",
],
&batches
);
}
}

async fn write_lp_to_db(server: &TestServer, database: &str, lp: &'static str) {
let client = influxdb3_client::Client::new(server.client_addr()).unwrap();
client
.api_v3_write_lp(database)
.body(lp)
.send()
.await
.unwrap();
}

async fn collect_stream(stream: FlightRecordBatchStream) -> Vec<RecordBatch> {
stream
.try_collect()
.await
.expect("gather record batch stream")
}
16 changes: 9 additions & 7 deletions influxdb3_server/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -30,27 +30,29 @@ trace_http = { path = "../trace_http" }
tracker = { path = "../tracker" }

arrow = { workspace = true, features = ["prettyprint"] }
arrow-flight.workspace = true
arrow-json = "49.0.0"
arrow-schema = "49.0.0"
arrow-csv = "49.0.0"
async-trait = "0.1"
chrono = "0.4"
datafusion = { workspace = true }
async-trait = "0.1"
flate2 = "1.0.27"
futures = "0.3.28"
hex = "0.4.3"
hyper = "0.14"
parking_lot = "0.11.1"
pin-project-lite = "0.2"
thiserror = "1.0"
tokio = { version = "1", features = ["rt-multi-thread", "macros", "time"] }
tokio-util = { version = "0.7.9" }
tonic = { workspace = true }
serde = { version = "1.0.197", features = ["derive"] }
serde_json = "1.0.114"
serde_urlencoded = "0.7.0"
sha2 = "0.10.8"
tower = "0.4.13"
flate2 = "1.0.27"
workspace-hack = { version = "0.1", path = "../workspace-hack" }
arrow-json = "49.0.0"
arrow-schema = "49.0.0"
arrow-csv = "49.0.0"
sha2 = "0.10.8"
hex = "0.4.3"

[dev-dependencies]
parquet.workspace = true
Expand Down
14 changes: 14 additions & 0 deletions influxdb3_server/src/grpc.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
use std::sync::Arc;

use arrow_flight::flight_service_server::{
FlightService as Flight, FlightServiceServer as FlightServer,
};
use authz::Authorizer;
use iox_query::QueryNamespaceProvider;

pub(crate) fn make_flight_server<Q: QueryNamespaceProvider>(
server: Arc<Q>,
authz: Option<Arc<dyn Authorizer>>,
) -> FlightServer<impl Flight> {
service_grpc_flight::make_server(server, authz)
}
Loading

0 comments on commit 298055e

Please sign in to comment.