mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-08 14:22:58 +00:00
feat: trace_id in query context (#2014)
* chore: unify once_cell version * chore: update cargo lock * chore: add gen_trace_id * chore: add trace_id to query_ctx * chore: add debug log * Revert "chore: add debug log" This reverts commit f52ab3bb300f1d73117cd6ebbd8e0162829b1aba. * chore: add frontend node id option * chore: add query ctx to query engine ctx * chore: set trace_id to logical_plan api * chore: add trace_id in grpc entrance * chore: generate trace_id while creating query_ctx * chore: fix typo * chore: extract trace_id from grpc header * chore: extract trace_id from grpc header * chore: fix clippy * chore: add `QueryContextBuilder` * chore: change node_id in fe to string
This commit is contained in:
8
Cargo.lock
generated
8
Cargo.lock
generated
@@ -1954,6 +1954,8 @@ dependencies = [
|
||||
"opentelemetry 0.17.0",
|
||||
"opentelemetry-jaeger",
|
||||
"parking_lot 0.12.1",
|
||||
"rand",
|
||||
"rs-snowflake",
|
||||
"serde",
|
||||
"tokio",
|
||||
"tracing",
|
||||
@@ -7768,6 +7770,12 @@ dependencies = [
|
||||
"serde",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
name = "rs-snowflake"
|
||||
version = "0.6.0"
|
||||
source = "registry+https://github.com/rust-lang/crates.io-index"
|
||||
checksum = "e60ef3b82994702bbe4e134d98aadca4b49ed04440148985678d415c68127666"
|
||||
|
||||
[[package]]
|
||||
name = "rsa"
|
||||
version = "0.6.1"
|
||||
|
||||
@@ -77,6 +77,7 @@ futures-util = "0.3"
|
||||
greptime-proto = { git = "https://github.com/GreptimeTeam/greptime-proto.git", rev = "356694a72f12ad9e15008d4245a0b4fe48f982ad" }
|
||||
itertools = "0.10"
|
||||
lazy_static = "1.4"
|
||||
once_cell = "1.18"
|
||||
opentelemetry-proto = { version = "0.2", features = ["gen-tonic", "metrics"] }
|
||||
parquet = "40.0"
|
||||
paste = "1.0"
|
||||
|
||||
@@ -73,7 +73,7 @@ async fn run() {
|
||||
|
||||
let logical = mock_logical_plan();
|
||||
event!(Level::INFO, "plan size: {:#?}", logical.len());
|
||||
let result = db.logical_plan(logical).await.unwrap();
|
||||
let result = db.logical_plan(logical, None).await.unwrap();
|
||||
|
||||
event!(Level::INFO, "result: {:#?}", result);
|
||||
}
|
||||
|
||||
@@ -139,20 +139,20 @@ impl Database {
|
||||
|
||||
async fn handle(&self, request: Request) -> Result<u32> {
|
||||
let mut client = self.client.make_database_client()?.inner;
|
||||
let request = self.to_rpc_request(request);
|
||||
let request = self.to_rpc_request(request, None);
|
||||
let response = client.handle(request).await?.into_inner();
|
||||
from_grpc_response(response)
|
||||
}
|
||||
|
||||
#[inline]
|
||||
fn to_rpc_request(&self, request: Request) -> GreptimeRequest {
|
||||
fn to_rpc_request(&self, request: Request, trace_id: Option<u64>) -> GreptimeRequest {
|
||||
GreptimeRequest {
|
||||
header: Some(RequestHeader {
|
||||
catalog: self.catalog.clone(),
|
||||
schema: self.schema.clone(),
|
||||
authorization: self.ctx.auth_header.clone(),
|
||||
dbname: self.dbname.clone(),
|
||||
trace_id: None,
|
||||
trace_id,
|
||||
span_id: None,
|
||||
}),
|
||||
request: Some(request),
|
||||
@@ -161,17 +161,27 @@ impl Database {
|
||||
|
||||
pub async fn sql(&self, sql: &str) -> Result<Output> {
|
||||
let _timer = timer!(metrics::METRIC_GRPC_SQL);
|
||||
self.do_get(Request::Query(QueryRequest {
|
||||
query: Some(Query::Sql(sql.to_string())),
|
||||
}))
|
||||
self.do_get(
|
||||
Request::Query(QueryRequest {
|
||||
query: Some(Query::Sql(sql.to_string())),
|
||||
}),
|
||||
None,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn logical_plan(&self, logical_plan: Vec<u8>) -> Result<Output> {
|
||||
pub async fn logical_plan(
|
||||
&self,
|
||||
logical_plan: Vec<u8>,
|
||||
trace_id: Option<u64>,
|
||||
) -> Result<Output> {
|
||||
let _timer = timer!(metrics::METRIC_GRPC_LOGICAL_PLAN);
|
||||
self.do_get(Request::Query(QueryRequest {
|
||||
query: Some(Query::LogicalPlan(logical_plan)),
|
||||
}))
|
||||
self.do_get(
|
||||
Request::Query(QueryRequest {
|
||||
query: Some(Query::LogicalPlan(logical_plan)),
|
||||
}),
|
||||
trace_id,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
@@ -183,69 +193,90 @@ impl Database {
|
||||
step: &str,
|
||||
) -> Result<Output> {
|
||||
let _timer = timer!(metrics::METRIC_GRPC_PROMQL_RANGE_QUERY);
|
||||
self.do_get(Request::Query(QueryRequest {
|
||||
query: Some(Query::PromRangeQuery(PromRangeQuery {
|
||||
query: promql.to_string(),
|
||||
start: start.to_string(),
|
||||
end: end.to_string(),
|
||||
step: step.to_string(),
|
||||
})),
|
||||
}))
|
||||
self.do_get(
|
||||
Request::Query(QueryRequest {
|
||||
query: Some(Query::PromRangeQuery(PromRangeQuery {
|
||||
query: promql.to_string(),
|
||||
start: start.to_string(),
|
||||
end: end.to_string(),
|
||||
step: step.to_string(),
|
||||
})),
|
||||
}),
|
||||
None,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn create(&self, expr: CreateTableExpr) -> Result<Output> {
|
||||
let _timer = timer!(metrics::METRIC_GRPC_CREATE_TABLE);
|
||||
self.do_get(Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::CreateTable(expr)),
|
||||
}))
|
||||
self.do_get(
|
||||
Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::CreateTable(expr)),
|
||||
}),
|
||||
None,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn alter(&self, expr: AlterExpr) -> Result<Output> {
|
||||
let _timer = timer!(metrics::METRIC_GRPC_ALTER);
|
||||
self.do_get(Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::Alter(expr)),
|
||||
}))
|
||||
self.do_get(
|
||||
Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::Alter(expr)),
|
||||
}),
|
||||
None,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn drop_table(&self, expr: DropTableExpr) -> Result<Output> {
|
||||
let _timer = timer!(metrics::METRIC_GRPC_DROP_TABLE);
|
||||
self.do_get(Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::DropTable(expr)),
|
||||
}))
|
||||
self.do_get(
|
||||
Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::DropTable(expr)),
|
||||
}),
|
||||
None,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn flush_table(&self, expr: FlushTableExpr) -> Result<Output> {
|
||||
let _timer = timer!(metrics::METRIC_GRPC_FLUSH_TABLE);
|
||||
self.do_get(Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::FlushTable(expr)),
|
||||
}))
|
||||
self.do_get(
|
||||
Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::FlushTable(expr)),
|
||||
}),
|
||||
None,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn compact_table(&self, expr: CompactTableExpr) -> Result<Output> {
|
||||
let _timer = timer!(metrics::METRIC_GRPC_COMPACT_TABLE);
|
||||
self.do_get(Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::CompactTable(expr)),
|
||||
}))
|
||||
self.do_get(
|
||||
Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::CompactTable(expr)),
|
||||
}),
|
||||
None,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
pub async fn truncate_table(&self, expr: TruncateTableExpr) -> Result<Output> {
|
||||
let _timer = timer!(metrics::METRIC_GRPC_TRUNCATE_TABLE);
|
||||
self.do_get(Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::TruncateTable(expr)),
|
||||
}))
|
||||
self.do_get(
|
||||
Request::Ddl(DdlRequest {
|
||||
expr: Some(DdlExpr::TruncateTable(expr)),
|
||||
}),
|
||||
None,
|
||||
)
|
||||
.await
|
||||
}
|
||||
|
||||
async fn do_get(&self, request: Request) -> Result<Output> {
|
||||
async fn do_get(&self, request: Request, trace_id: Option<u64>) -> Result<Output> {
|
||||
// FIXME(paomian): should be added some labels for metrics
|
||||
let _timer = timer!(metrics::METRIC_GRPC_DO_GET);
|
||||
let request = self.to_rpc_request(request);
|
||||
let request = self.to_rpc_request(request, trace_id);
|
||||
let request = Ticket {
|
||||
ticket: request.encode_to_vec().into(),
|
||||
};
|
||||
|
||||
@@ -183,7 +183,7 @@ impl Repl {
|
||||
.encode(plan)
|
||||
.context(SubstraitEncodeLogicalPlanSnafu)?;
|
||||
|
||||
self.database.logical_plan(plan.to_vec()).await
|
||||
self.database.logical_plan(plan.to_vec(), None).await
|
||||
} else {
|
||||
self.database.sql(&sql).await
|
||||
}
|
||||
|
||||
@@ -16,7 +16,7 @@ datatypes = { path = "../../datatypes" }
|
||||
libc = "0.2"
|
||||
num = "0.4"
|
||||
num-traits = "0.2"
|
||||
once_cell = "1.10"
|
||||
once_cell.workspace = true
|
||||
paste = "1.0"
|
||||
snafu.workspace = true
|
||||
statrs = "0.16"
|
||||
|
||||
@@ -9,7 +9,7 @@ async-trait.workspace = true
|
||||
common-error = { path = "../error" }
|
||||
common-telemetry = { path = "../telemetry" }
|
||||
metrics.workspace = true
|
||||
once_cell = "1.12"
|
||||
once_cell.workspace = true
|
||||
paste.workspace = true
|
||||
snafu.workspace = true
|
||||
tokio.workspace = true
|
||||
|
||||
@@ -15,7 +15,7 @@ console-subscriber = { version = "0.1", optional = true }
|
||||
metrics-exporter-prometheus = { version = "0.11", default-features = false }
|
||||
metrics-util = "0.14"
|
||||
metrics.workspace = true
|
||||
once_cell = "1.10"
|
||||
once_cell.workspace = true
|
||||
opentelemetry = { version = "0.17", default-features = false, features = [
|
||||
"trace",
|
||||
"rt-tokio",
|
||||
@@ -24,6 +24,8 @@ opentelemetry-jaeger = { version = "0.16", features = ["rt-tokio"] }
|
||||
parking_lot = { version = "0.12", features = [
|
||||
"deadlock_detection",
|
||||
], optional = true }
|
||||
rand.workspace = true
|
||||
rs-snowflake = "0.6"
|
||||
serde.workspace = true
|
||||
tokio.workspace = true
|
||||
tracing = "0.1"
|
||||
|
||||
@@ -17,7 +17,44 @@ mod macros;
|
||||
pub mod metric;
|
||||
mod panic_hook;
|
||||
|
||||
use std::collections::hash_map::DefaultHasher;
|
||||
use std::hash::{Hash, Hasher};
|
||||
|
||||
pub use logging::{init_default_ut_logging, init_global_logging};
|
||||
pub use metric::init_default_metrics_recorder;
|
||||
use once_cell::sync::OnceCell;
|
||||
pub use panic_hook::set_panic_hook;
|
||||
use parking_lot::Mutex;
|
||||
use rand::random;
|
||||
use snowflake::SnowflakeIdBucket;
|
||||
pub use {common_error, tracing, tracing_appender, tracing_futures, tracing_subscriber};
|
||||
|
||||
static NODE_ID: OnceCell<u64> = OnceCell::new();
|
||||
static TRACE_BUCKET: OnceCell<Mutex<SnowflakeIdBucket>> = OnceCell::new();
|
||||
|
||||
pub fn gen_trace_id() -> u64 {
|
||||
let mut bucket = TRACE_BUCKET
|
||||
.get_or_init(|| {
|
||||
// if node_id is not initialized, how about random one?
|
||||
let node_id = NODE_ID.get_or_init(|| 0);
|
||||
info!("initializing bucket with node_id: {}", node_id);
|
||||
let bucket = SnowflakeIdBucket::new(1, (*node_id) as i32);
|
||||
Mutex::new(bucket)
|
||||
})
|
||||
.lock();
|
||||
(*bucket).get_id() as u64
|
||||
}
|
||||
|
||||
pub fn init_node_id(node_id: Option<String>) {
|
||||
let node_id = node_id.map(|id| calculate_hash(&id)).unwrap_or(random());
|
||||
match NODE_ID.set(node_id) {
|
||||
Ok(_) => {}
|
||||
Err(_) => warn!("node_id is already initialized"),
|
||||
}
|
||||
}
|
||||
|
||||
fn calculate_hash<T: Hash>(t: &T) -> u64 {
|
||||
let mut s = DefaultHasher::new();
|
||||
t.hash(&mut s);
|
||||
s.finish()
|
||||
}
|
||||
|
||||
@@ -5,6 +5,6 @@ edition.workspace = true
|
||||
license.workspace = true
|
||||
|
||||
[dependencies]
|
||||
once_cell = "1.16"
|
||||
once_cell.workspace = true
|
||||
rand.workspace = true
|
||||
tempfile.workspace = true
|
||||
|
||||
@@ -28,6 +28,7 @@ use crate::service_config::{
|
||||
#[serde(default)]
|
||||
pub struct FrontendOptions {
|
||||
pub mode: Mode,
|
||||
pub node_id: Option<String>,
|
||||
pub heartbeat: HeartbeatOptions,
|
||||
pub http_options: Option<HttpOptions>,
|
||||
pub grpc_options: Option<GrpcOptions>,
|
||||
@@ -46,6 +47,7 @@ impl Default for FrontendOptions {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
mode: Mode::Standalone,
|
||||
node_id: None,
|
||||
heartbeat: HeartbeatOptions::default(),
|
||||
http_options: Some(HttpOptions::default()),
|
||||
grpc_options: Some(GrpcOptions::default()),
|
||||
|
||||
@@ -206,6 +206,8 @@ impl Instance {
|
||||
Arc::new(handlers_executor),
|
||||
));
|
||||
|
||||
common_telemetry::init_node_id(opts.node_id.clone());
|
||||
|
||||
Ok(Instance {
|
||||
catalog_manager,
|
||||
script_executor,
|
||||
|
||||
@@ -60,7 +60,7 @@ impl DatanodeInstance {
|
||||
|
||||
let result = self
|
||||
.db
|
||||
.logical_plan(substrait_plan.to_vec())
|
||||
.logical_plan(substrait_plan.to_vec(), None)
|
||||
.await
|
||||
.context(error::RequestDatanodeSnafu)?;
|
||||
let Output::RecordBatches(record_batches) = result else { unreachable!() };
|
||||
|
||||
@@ -33,7 +33,7 @@ h2 = "0.3"
|
||||
http-body = "0.4"
|
||||
lazy_static.workspace = true
|
||||
metrics.workspace = true
|
||||
once_cell = "1.17"
|
||||
once_cell.workspace = true
|
||||
parking_lot = "0.12"
|
||||
prost.workspace = true
|
||||
rand.workspace = true
|
||||
|
||||
@@ -36,7 +36,7 @@ greptime-proto.workspace = true
|
||||
humantime = "2.1"
|
||||
metrics.workspace = true
|
||||
object-store = { path = "../object-store" }
|
||||
once_cell = "1.10"
|
||||
once_cell.workspace = true
|
||||
partition = { path = "../partition" }
|
||||
promql = { path = "../promql" }
|
||||
promql-parser = "0.1.1"
|
||||
|
||||
@@ -82,7 +82,7 @@ impl DatafusionQueryEngine {
|
||||
plan: LogicalPlan,
|
||||
query_ctx: QueryContextRef,
|
||||
) -> Result<Output> {
|
||||
let mut ctx = QueryEngineContext::new(self.state.session_state());
|
||||
let mut ctx = QueryEngineContext::new(self.state.session_state(), query_ctx.clone());
|
||||
|
||||
// `create_physical_plan` will optimize logical plan internally
|
||||
let physical_plan = self.create_physical_plan(&mut ctx, &plan).await?;
|
||||
@@ -363,10 +363,12 @@ impl QueryExecutor for DatafusionQueryEngine {
|
||||
plan: &Arc<dyn PhysicalPlan>,
|
||||
) -> Result<SendableRecordBatchStream> {
|
||||
let _timer = timer!(metrics::METRIC_EXEC_PLAN_ELAPSED);
|
||||
let task_ctx = ctx.build_task_ctx();
|
||||
|
||||
match plan.output_partitioning().partition_count() {
|
||||
0 => Ok(Box::pin(EmptyRecordBatchStream::new(plan.schema()))),
|
||||
1 => Ok(plan
|
||||
.execute(0, ctx.state().task_ctx())
|
||||
.execute(0, task_ctx)
|
||||
.context(error::ExecutePhysicalPlanSnafu)
|
||||
.map_err(BoxedError::new)
|
||||
.context(QueryExecutionSnafu))?,
|
||||
@@ -377,7 +379,7 @@ impl QueryExecutor for DatafusionQueryEngine {
|
||||
// CoalescePartitionsExec must produce a single partition
|
||||
assert_eq!(1, plan.output_partitioning().partition_count());
|
||||
let df_stream = plan
|
||||
.execute(0, ctx.state().task_ctx())
|
||||
.execute(0, task_ctx)
|
||||
.context(error::DatafusionSnafu {
|
||||
msg: "Failed to execute DataFusion merge exec",
|
||||
})
|
||||
|
||||
@@ -132,18 +132,19 @@ impl MergeScanExec {
|
||||
}
|
||||
}
|
||||
|
||||
pub fn to_stream(&self) -> Result<SendableRecordBatchStream> {
|
||||
pub fn to_stream(&self, context: Arc<TaskContext>) -> Result<SendableRecordBatchStream> {
|
||||
let substrait_plan = self.substrait_plan.to_vec();
|
||||
let peers = self.peers.clone();
|
||||
let clients = self.clients.clone();
|
||||
let table = self.table.clone();
|
||||
let trace_id = context.task_id().and_then(|id| id.parse().ok());
|
||||
|
||||
let stream = try_stream! {
|
||||
for peer in peers {
|
||||
let client = clients.get_client(&peer).await;
|
||||
let database = Database::new(&table.catalog_name, &table.schema_name, client);
|
||||
let output: Output = database
|
||||
.logical_plan(substrait_plan.clone())
|
||||
.logical_plan(substrait_plan.clone(), trace_id)
|
||||
.await
|
||||
.context(RemoteRequestSnafu)
|
||||
.map_err(BoxedError::new)
|
||||
@@ -220,9 +221,11 @@ impl ExecutionPlan for MergeScanExec {
|
||||
fn execute(
|
||||
&self,
|
||||
_partition: usize,
|
||||
_context: Arc<TaskContext>,
|
||||
context: Arc<TaskContext>,
|
||||
) -> Result<DfSendableRecordBatchStream> {
|
||||
Ok(Box::pin(DfRecordBatchStreamAdapter::new(self.to_stream()?)))
|
||||
Ok(Box::pin(DfRecordBatchStreamAdapter::new(
|
||||
self.to_stream(context)?,
|
||||
)))
|
||||
}
|
||||
|
||||
fn statistics(&self) -> Statistics {
|
||||
|
||||
@@ -12,20 +12,42 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use datafusion::execution::context::SessionState;
|
||||
use std::sync::Arc;
|
||||
|
||||
use datafusion::execution::context::{SessionState, TaskContext};
|
||||
use session::context::QueryContextRef;
|
||||
|
||||
#[derive(Debug)]
|
||||
pub struct QueryEngineContext {
|
||||
state: SessionState,
|
||||
query_ctx: QueryContextRef,
|
||||
}
|
||||
|
||||
impl QueryEngineContext {
|
||||
pub fn new(state: SessionState) -> Self {
|
||||
Self { state }
|
||||
pub fn new(state: SessionState, query_ctx: QueryContextRef) -> Self {
|
||||
Self { state, query_ctx }
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn state(&self) -> &SessionState {
|
||||
&self.state
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn query_ctx(&self) -> QueryContextRef {
|
||||
self.query_ctx.clone()
|
||||
}
|
||||
|
||||
pub fn build_task_ctx(&self) -> Arc<TaskContext> {
|
||||
let task_id = self.query_ctx.trace_id().to_string();
|
||||
let state = &self.state;
|
||||
Arc::new(TaskContext::new(
|
||||
Some(task_id),
|
||||
state.session_id().to_string(),
|
||||
state.config().clone(),
|
||||
state.scalar_functions().clone(),
|
||||
state.aggregate_functions().clone(),
|
||||
state.runtime_env().clone(),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -44,7 +44,7 @@ datafusion-physical-expr = { workspace = true, optional = true }
|
||||
datatypes = { path = "../datatypes" }
|
||||
futures.workspace = true
|
||||
futures-util.workspace = true
|
||||
once_cell = "1.17.0"
|
||||
once_cell.workspace = true
|
||||
paste = { workspace = true, optional = true }
|
||||
query = { path = "../query" }
|
||||
# TODO(discord9): This is a forked and tweaked version of RustPython, please update it to newest original RustPython After RustPython support GC
|
||||
|
||||
@@ -53,7 +53,7 @@ metrics.workspace = true
|
||||
metrics-process = { version = "<1.0.10", optional = true }
|
||||
mime_guess = "2.0"
|
||||
num_cpus = "1.13"
|
||||
once_cell = "1.16"
|
||||
once_cell.workspace = true
|
||||
openmetrics-parser = "0.4"
|
||||
opensrv-mysql = "0.4"
|
||||
opentelemetry-proto.workspace = true
|
||||
|
||||
@@ -18,13 +18,14 @@ use std::time::Instant;
|
||||
use api::helper::request_type;
|
||||
use api::v1::auth_header::AuthScheme;
|
||||
use api::v1::{Basic, GreptimeRequest, RequestHeader};
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use common_error::ext::ErrorExt;
|
||||
use common_error::status_code::StatusCode;
|
||||
use common_query::Output;
|
||||
use common_runtime::Runtime;
|
||||
use common_telemetry::logging;
|
||||
use metrics::{histogram, increment_counter};
|
||||
use session::context::{QueryContext, QueryContextRef};
|
||||
use session::context::{QueryContextBuilder, QueryContextRef};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
|
||||
use crate::auth::{Identity, Password, UserProviderRef};
|
||||
@@ -70,7 +71,6 @@ impl GreptimeRequestHandler {
|
||||
let query_ctx = create_query_context(header);
|
||||
|
||||
let _ = self.auth(header, &query_ctx).await?;
|
||||
|
||||
let handler = self.handler.clone();
|
||||
let request_type = request_type(&query);
|
||||
let db = query_ctx.get_db_string();
|
||||
@@ -145,25 +145,35 @@ impl GreptimeRequestHandler {
|
||||
}
|
||||
|
||||
pub(crate) fn create_query_context(header: Option<&RequestHeader>) -> QueryContextRef {
|
||||
let ctx = QueryContext::arc();
|
||||
if let Some(header) = header {
|
||||
// We provide dbname field in newer versions of protos/sdks
|
||||
// parse dbname from header in priority
|
||||
if !header.dbname.is_empty() {
|
||||
let (catalog, schema) =
|
||||
crate::parse_catalog_and_schema_from_client_database_name(&header.dbname);
|
||||
ctx.set_current_catalog(catalog);
|
||||
ctx.set_current_schema(schema);
|
||||
} else {
|
||||
if !header.catalog.is_empty() {
|
||||
ctx.set_current_catalog(&header.catalog);
|
||||
let (catalog, schema) = header
|
||||
.map(|header| {
|
||||
// We provide dbname field in newer versions of protos/sdks
|
||||
// parse dbname from header in priority
|
||||
if !header.dbname.is_empty() {
|
||||
crate::parse_catalog_and_schema_from_client_database_name(&header.dbname)
|
||||
} else {
|
||||
(
|
||||
if !header.catalog.is_empty() {
|
||||
&header.catalog
|
||||
} else {
|
||||
DEFAULT_CATALOG_NAME
|
||||
},
|
||||
if !header.schema.is_empty() {
|
||||
&header.schema
|
||||
} else {
|
||||
DEFAULT_SCHEMA_NAME
|
||||
},
|
||||
)
|
||||
}
|
||||
if !header.schema.is_empty() {
|
||||
ctx.set_current_schema(&header.schema);
|
||||
}
|
||||
}
|
||||
};
|
||||
ctx
|
||||
})
|
||||
.unwrap_or((DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME));
|
||||
|
||||
QueryContextBuilder::new()
|
||||
.catalog(catalog.to_string())
|
||||
.schema(schema.to_string())
|
||||
.try_trace_id(header.and_then(|h: &RequestHeader| h.trace_id))
|
||||
.build()
|
||||
.to_arc()
|
||||
}
|
||||
|
||||
/// Histogram timer for handling gRPC request.
|
||||
|
||||
@@ -32,6 +32,7 @@ pub struct QueryContext {
|
||||
current_schema: ArcSwap<String>,
|
||||
time_zone: ArcSwap<Option<TimeZone>>,
|
||||
sql_dialect: Box<dyn Dialect + Send + Sync>,
|
||||
trace_id: u64,
|
||||
}
|
||||
|
||||
impl Default for QueryContext {
|
||||
@@ -56,30 +57,25 @@ impl QueryContext {
|
||||
Arc::new(QueryContext::new())
|
||||
}
|
||||
|
||||
pub fn to_arc(self) -> QueryContextRef {
|
||||
Arc::new(self)
|
||||
}
|
||||
|
||||
pub fn new() -> Self {
|
||||
Self {
|
||||
current_catalog: ArcSwap::new(Arc::new(DEFAULT_CATALOG_NAME.to_string())),
|
||||
current_schema: ArcSwap::new(Arc::new(DEFAULT_SCHEMA_NAME.to_string())),
|
||||
time_zone: ArcSwap::new(Arc::new(None)),
|
||||
sql_dialect: Box::new(GreptimeDbDialect {}),
|
||||
trace_id: common_telemetry::gen_trace_id(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn with(catalog: &str, schema: &str) -> Self {
|
||||
Self::with_sql_dialect(catalog, schema, Box::new(GreptimeDbDialect {}))
|
||||
}
|
||||
|
||||
pub fn with_sql_dialect(
|
||||
catalog: &str,
|
||||
schema: &str,
|
||||
sql_dialect: Box<dyn Dialect + Send + Sync>,
|
||||
) -> Self {
|
||||
Self {
|
||||
current_catalog: ArcSwap::new(Arc::new(catalog.to_string())),
|
||||
current_schema: ArcSwap::new(Arc::new(schema.to_string())),
|
||||
time_zone: ArcSwap::new(Arc::new(None)),
|
||||
sql_dialect,
|
||||
}
|
||||
QueryContextBuilder::new()
|
||||
.catalog(catalog.to_string())
|
||||
.schema(schema.to_string())
|
||||
.build()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
@@ -132,6 +128,70 @@ impl QueryContext {
|
||||
pub fn set_time_zone(&self, tz: Option<TimeZone>) {
|
||||
let _ = self.time_zone.swap(Arc::new(tz));
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub fn trace_id(&self) -> u64 {
|
||||
self.trace_id
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Default)]
|
||||
pub struct QueryContextBuilder {
|
||||
catalog: Option<String>,
|
||||
schema: Option<String>,
|
||||
time_zone: Option<TimeZone>,
|
||||
sql_dialect: Option<Box<dyn Dialect + Send + Sync>>,
|
||||
trace_id: Option<u64>,
|
||||
}
|
||||
|
||||
impl QueryContextBuilder {
|
||||
pub fn new() -> Self {
|
||||
Self::default()
|
||||
}
|
||||
|
||||
pub fn catalog(mut self, catalog: String) -> Self {
|
||||
self.catalog = Some(catalog);
|
||||
self
|
||||
}
|
||||
|
||||
pub fn schema(mut self, schema: String) -> Self {
|
||||
self.schema = Some(schema);
|
||||
self
|
||||
}
|
||||
|
||||
pub fn time_zone(mut self, tz: TimeZone) -> Self {
|
||||
self.time_zone = Some(tz);
|
||||
self
|
||||
}
|
||||
|
||||
pub fn sql_dialect(mut self, sql_dialect: Box<dyn Dialect + Send + Sync>) -> Self {
|
||||
self.sql_dialect = Some(sql_dialect);
|
||||
self
|
||||
}
|
||||
|
||||
pub fn trace_id(mut self, trace_id: u64) -> Self {
|
||||
self.trace_id = Some(trace_id);
|
||||
self
|
||||
}
|
||||
|
||||
pub fn try_trace_id(mut self, trace_id: Option<u64>) -> Self {
|
||||
self.trace_id = trace_id;
|
||||
self
|
||||
}
|
||||
|
||||
pub fn build(self) -> QueryContext {
|
||||
QueryContext {
|
||||
current_catalog: ArcSwap::new(Arc::new(
|
||||
self.catalog.unwrap_or(DEFAULT_CATALOG_NAME.to_string()),
|
||||
)),
|
||||
current_schema: ArcSwap::new(Arc::new(
|
||||
self.schema.unwrap_or(DEFAULT_SCHEMA_NAME.to_string()),
|
||||
)),
|
||||
time_zone: ArcSwap::new(Arc::new(self.time_zone)),
|
||||
sql_dialect: self.sql_dialect.unwrap_or(Box::new(GreptimeDbDialect {})),
|
||||
trace_id: self.trace_id.unwrap_or(common_telemetry::gen_trace_id()),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub const DEFAULT_USERNAME: &str = "greptime";
|
||||
|
||||
@@ -20,8 +20,9 @@ use std::sync::Arc;
|
||||
use arc_swap::ArcSwap;
|
||||
use common_catalog::build_db_string;
|
||||
use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME};
|
||||
use context::QueryContextBuilder;
|
||||
|
||||
use crate::context::{Channel, ConnInfo, QueryContext, QueryContextRef, UserInfo};
|
||||
use crate::context::{Channel, ConnInfo, QueryContextRef, UserInfo};
|
||||
|
||||
/// Session for persistent connection such as MySQL, PostgreSQL etc.
|
||||
#[derive(Debug)]
|
||||
@@ -46,11 +47,12 @@ impl Session {
|
||||
|
||||
#[inline]
|
||||
pub fn new_query_context(&self) -> QueryContextRef {
|
||||
Arc::new(QueryContext::with_sql_dialect(
|
||||
self.catalog.load().as_ref(),
|
||||
self.schema.load().as_ref(),
|
||||
self.conn_info.channel.dialect(),
|
||||
))
|
||||
QueryContextBuilder::new()
|
||||
.catalog(self.catalog.load().to_string())
|
||||
.schema(self.schema.load().to_string())
|
||||
.sql_dialect(self.conn_info.channel.dialect())
|
||||
.build()
|
||||
.to_arc()
|
||||
}
|
||||
|
||||
#[inline]
|
||||
|
||||
@@ -17,6 +17,6 @@ datatypes = { path = "../datatypes" }
|
||||
hex = "0.4"
|
||||
itertools.workspace = true
|
||||
mito = { path = "../mito" }
|
||||
once_cell = "1.10"
|
||||
once_cell.workspace = true
|
||||
snafu = { version = "0.7", features = ["backtraces"] }
|
||||
sqlparser.workspace = true
|
||||
|
||||
@@ -34,7 +34,7 @@ meta-client = { path = "../src/meta-client" }
|
||||
meta-srv = { path = "../src/meta-srv" }
|
||||
mito = { path = "../src/mito", features = ["test"] }
|
||||
object-store = { path = "../src/object-store" }
|
||||
once_cell = "1.16"
|
||||
once_cell.workspace = true
|
||||
query = { path = "../src/query" }
|
||||
rand.workspace = true
|
||||
rstest = "0.17"
|
||||
|
||||
Reference in New Issue
Block a user