From c64f009bfb1fca1b46f3957547344d35d33a6a69 Mon Sep 17 00:00:00 2001 From: Denis Smirnov <sd@picodata.io> Date: Fri, 26 May 2023 17:09:29 +0700 Subject: [PATCH] feat: non-SQL insert Current commit redesigns distributed INSERT command. Previously we dispatched insert SQL command to the storages. If INSERT could be done locally (without building a virtual table on the router) we used SQL "bucket_id(<string>)" function to recalculate buckets on the starage via SQL. This approach had the main disadvantage - it worked only with a "bucket_id" SQL function that had a single argument as a parameter. An attempt to support multiple parameters of different types (tuple columns as we plan to implement for Picodata engine in future) faced serious technical problems. Also, the old implementation had performance issues: - we created temporary spaces on the storage even for a single VALUE insertion - we always dispatched VALUES to the storage to build a virtual table - the resulting SQL was too verbose (as we produced a subquery under INSERT node) It was desided to get rid of the old approach and migrate to non-SQL insertion. It means that a new type of motion was introduled - local segment. It allows us to build a virtual table on the storage in Rust memory, then using space API transform and insert collected tuples directly into the target space within a single transaction. Also we can materailize VALUES (if they contain only constants) on the router and get rid of the redundant network transmission over the network. --- Cargo.lock | 2 +- sbroad-benches/src/engine.rs | 86 +++- .../src/api/calculate_bucket_id.rs | 2 +- sbroad-cartridge/src/api/exec_query.rs | 6 +- sbroad-cartridge/src/cartridge.rs | 27 ++ sbroad-cartridge/src/cartridge/router.rs | 87 ++-- sbroad-cartridge/src/cartridge/storage.rs | 127 ++++-- .../test_app/test/integration/api_test.lua | 21 - .../test/integration/left_outer_join_test.lua | 62 ++- sbroad-core/Cargo.toml | 2 +- sbroad-core/src/backend/sql/tree.rs | 41 +- sbroad-core/src/core.lua | 26 +- sbroad-core/src/errors.rs | 10 + sbroad-core/src/executor.rs | 130 ++---- sbroad-core/src/executor/bucket.rs | 60 ++- sbroad-core/src/executor/bucket/tests.rs | 2 +- sbroad-core/src/executor/engine.rs | 6 +- sbroad-core/src/executor/engine/helpers.rs | 410 +++++++++++++++++- .../src/executor/engine/helpers/vshard.rs | 2 +- sbroad-core/src/executor/engine/mock.rs | 95 +++- sbroad-core/src/executor/hash.rs | 6 +- sbroad-core/src/executor/ir.rs | 59 ++- sbroad-core/src/executor/result.rs | 14 +- sbroad-core/src/executor/result/tests.rs | 2 +- sbroad-core/src/executor/tests.rs | 224 +++------- sbroad-core/src/executor/tests/bucket_id.rs | 4 +- sbroad-core/src/executor/tests/cast.rs | 4 +- sbroad-core/src/executor/tests/concat.rs | 8 +- .../src/executor/tests/empty_motion.rs | 4 +- sbroad-core/src/executor/tests/subtree.rs | 12 +- sbroad-core/src/executor/vtable.rs | 63 ++- sbroad-core/src/frontend/sql.rs | 3 +- sbroad-core/src/frontend/sql/ir/tests.rs | 74 ++-- .../src/frontend/sql/ir/tests/single.rs | 3 +- sbroad-core/src/ir/explain.rs | 75 ++-- sbroad-core/src/ir/explain/tests.rs | 46 +- sbroad-core/src/ir/explain/tests/concat.rs | 4 +- sbroad-core/src/ir/operator.rs | 2 +- sbroad-core/src/ir/relation.rs | 64 ++- sbroad-core/src/ir/relation/tests.rs | 30 +- .../src/ir/transformation/redistribution.rs | 129 +++--- .../transformation/redistribution/groupby.rs | 6 - .../transformation/redistribution/insert.rs | 201 +-------- .../redistribution/tests/segment.rs | 32 +- sbroad-core/src/ir/tree/subtree.rs | 7 +- sbroad-core/src/ir/value.rs | 3 +- sbroad-picodata/src/api.rs | 2 +- sbroad-picodata/src/runtime.rs | 2 + sbroad-picodata/src/runtime/meta/router.rs | 12 +- sbroad-picodata/src/runtime/router.rs | 52 ++- sbroad-picodata/src/runtime/storage.rs | 124 ++++-- 51 files changed, 1547 insertions(+), 928 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a60b775751..ec527bd6d0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1142,7 +1142,7 @@ dependencies = [ "pest_derive", "pretty_assertions", "rand", - "rmp-serde 0.14.4", + "rmp-serde 1.0.0", "sbroad-proc", "serde", "serde_yaml", diff --git a/sbroad-benches/src/engine.rs b/sbroad-benches/src/engine.rs index e356a52445..e1b764a00d 100644 --- a/sbroad-benches/src/engine.rs +++ b/sbroad-benches/src/engine.rs @@ -7,12 +7,16 @@ use sbroad::errors::{Action, Entity, SbroadError}; use sbroad::executor::bucket::Buckets; use sbroad::executor::engine::QueryCache; use sbroad::executor::engine::{ - helpers::{normalize_name_from_sql, sharding_keys_from_map, sharding_keys_from_tuple}, - Metadata, Router, + helpers::{ + normalize_name_from_sql, sharding_keys_from_map, sharding_keys_from_tuple, + vshard::get_random_bucket, + }, + Metadata, Router, Vshard, }; use sbroad::executor::hash::bucket_id_by_tuple; -use sbroad::executor::ir::ExecutionPlan; +use sbroad::executor::ir::{ConnectionType, ExecutionPlan, QueryType}; use sbroad::executor::lru::{Cache, LRUCache, DEFAULT_CAPACITY}; +use sbroad::executor::protocol::Binary; use sbroad::executor::result::ProducerResult; use sbroad::executor::vtable::VirtualTable; use sbroad::frontend::sql::ast::AbstractSyntaxTree; @@ -27,7 +31,7 @@ use sbroad::ir::Plan; pub struct RouterConfigurationMock { functions: HashMap<String, Function>, tables: HashMap<String, Table>, - bucket_count: usize, + bucket_count: u64, sharding_column: String, } @@ -461,8 +465,80 @@ impl Router for RouterRuntimeMock { ) -> Result<Vec<&'rec Value>, SbroadError> { sharding_keys_from_tuple(&*self.metadata()?, &space, rec) } +} + +impl Vshard for RouterRuntimeMock { + fn exec_ir_on_all( + &self, + _required: Binary, + _optional: Binary, + _query_type: QueryType, + _conn_type: ConnectionType, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Query, + Some("exec_ir_on_all function is not implemented for the bench engine".to_string()), + )) + } + + fn bucket_count(&self) -> u64 { + self.metadata.borrow().bucket_count + } + + fn get_random_bucket(&self) -> Buckets { + get_random_bucket(self) + } fn determine_bucket_id(&self, s: &[&Value]) -> u64 { - bucket_id_by_tuple(s, self.metadata.borrow().bucket_count) + bucket_id_by_tuple(s, self.bucket_count()) + } + + fn exec_ir_on_some( + &self, + _sub_plan: ExecutionPlan, + _buckets: &Buckets, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Query, + Some("exec_ir_on_some function is not implemented for the bench engine".to_string()), + )) + } +} + +impl Vshard for &RouterRuntimeMock { + fn exec_ir_on_all( + &self, + _required: Binary, + _optional: Binary, + _query_type: QueryType, + _conn_type: ConnectionType, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Query, + Some("exec_ir_on_all function is not implemented for the bench engine".to_string()), + )) + } + + fn bucket_count(&self) -> u64 { + self.metadata.borrow().bucket_count + } + + fn get_random_bucket(&self) -> Buckets { + get_random_bucket(self) + } + + fn determine_bucket_id(&self, s: &[&Value]) -> u64 { + bucket_id_by_tuple(s, self.bucket_count()) + } + + fn exec_ir_on_some( + &self, + _sub_plan: ExecutionPlan, + _buckets: &Buckets, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Query, + Some("exec_ir_on_some function is not implemented for the bench engine".to_string()), + )) } } diff --git a/sbroad-cartridge/src/api/calculate_bucket_id.rs b/sbroad-cartridge/src/api/calculate_bucket_id.rs index 467c66d6ab..dbf2c8dffa 100644 --- a/sbroad-cartridge/src/api/calculate_bucket_id.rs +++ b/sbroad-cartridge/src/api/calculate_bucket_id.rs @@ -8,7 +8,7 @@ use serde::{de::Deserializer, Deserialize, Serialize}; use crate::api::helper::load_config; use crate::api::COORDINATOR_ENGINE; -use sbroad::executor::engine::Router; +use sbroad::executor::engine::{Router, Vshard}; use sbroad::ir::value::{EncodedValue, Value}; use sbroad::log::tarantool_error; diff --git a/sbroad-cartridge/src/api/exec_query.rs b/sbroad-cartridge/src/api/exec_query.rs index 220c2b5f25..f32d8d7ed5 100644 --- a/sbroad-cartridge/src/api/exec_query.rs +++ b/sbroad-cartridge/src/api/exec_query.rs @@ -49,8 +49,7 @@ pub extern "C" fn dispatch_query(f_ctx: FunctionCtx, args: FunctionArgs) -> c_in )); } }; - let mut query = match Query::new(&*runtime, &lua_params.pattern, lua_params.params) - { + let mut query = match Query::new(&runtime, &lua_params.pattern, lua_params.params) { Ok(q) => q, Err(e) => { error!(Option::from("query dispatch"), &format!("{e:?}")); @@ -128,6 +127,9 @@ pub extern "C" fn execute(f_ctx: FunctionCtx, args: FunctionArgs) -> c_int { if let Some(tuple) = (*result).downcast_ref::<Tuple>() { f_ctx.return_tuple(tuple).unwrap(); 0 + } else if let Some(mp) = (*result).downcast_ref::<Vec<u8>>() { + f_ctx.return_mp(mp.as_slice()).unwrap(); + 0 } else { error!( Option::from("execute"), diff --git a/sbroad-cartridge/src/cartridge.rs b/sbroad-cartridge/src/cartridge.rs index 5525071929..9fe1301ed8 100644 --- a/sbroad-cartridge/src/cartridge.rs +++ b/sbroad-cartridge/src/cartridge.rs @@ -8,8 +8,10 @@ use std::cell::Ref; use opentelemetry::global::{set_text_map_propagator, set_tracer_provider}; use opentelemetry::sdk::propagation::{TextMapCompositePropagator, TraceContextPropagator}; +use sbroad::error; use sbroad::errors::{Action, SbroadError}; use sbroad::otm::update_global_tracer; +use tarantool::tlua::LuaFunction; static SERVICE_NAME: &str = "sbroad"; @@ -73,3 +75,28 @@ pub trait ConfigurationProvider: Sized { /// - Failed to update the configuration. fn update_config(&self, metadata: Self::Configuration) -> Result<(), SbroadError>; } + +fn bucket_count() -> Result<u64, SbroadError> { + let lua = tarantool::lua_state(); + + let bucket_count_fn: LuaFunction<_> = + match lua.eval("return require('vshard').router.bucket_count") { + Ok(v) => v, + Err(e) => { + error!(Option::from("set_bucket_count"), &format!("{e:?}")); + return Err(SbroadError::LuaError(format!( + "Failed lua function load: {e}" + ))); + } + }; + + let bucket_count: u64 = match bucket_count_fn.call() { + Ok(r) => r, + Err(e) => { + error!(Option::from("set_bucket_count"), &format!("{e:?}")); + return Err(SbroadError::LuaError(e.to_string())); + } + }; + + Ok(bucket_count) +} diff --git a/sbroad-cartridge/src/cartridge/router.rs b/sbroad-cartridge/src/cartridge/router.rs index 1d502f81c7..925af6b636 100644 --- a/sbroad-cartridge/src/cartridge/router.rs +++ b/sbroad-cartridge/src/cartridge/router.rs @@ -15,7 +15,7 @@ use std::rc::Rc; use tarantool::tlua::LuaFunction; use crate::cartridge::config::RouterConfiguration; -use crate::cartridge::update_tracing; +use crate::cartridge::{bucket_count, update_tracing}; use sbroad::executor::protocol::Binary; use sbroad::error; @@ -45,7 +45,7 @@ use super::ConfigurationProvider; #[allow(clippy::module_name_repetitions)] pub struct RouterRuntime { metadata: RefCell<RouterConfiguration>, - bucket_count: usize, + bucket_count: u64, ir_cache: RefCell<LRUCache<String, Plan>>, } @@ -258,11 +258,6 @@ impl Router for RouterRuntime { ) -> Result<Vec<&'rec Value>, SbroadError> { sharding_keys_from_tuple(&*self.cached_config()?, &space, rec) } - - /// Calculate bucket for a key. - fn determine_bucket_id(&self, s: &[&Value]) -> u64 { - bucket_id_by_tuple(s, self.bucket_count) - } } impl Statistics for RouterRuntime { @@ -309,55 +304,55 @@ impl RouterRuntime { /// - Failed to detect the correct amount of buckets. pub fn new() -> Result<Self, SbroadError> { let cache: LRUCache<String, Plan> = LRUCache::new(DEFAULT_CAPACITY, None)?; - let mut result = RouterRuntime { + let result = RouterRuntime { metadata: RefCell::new(RouterConfiguration::new()), - bucket_count: 0, + bucket_count: bucket_count()?, ir_cache: RefCell::new(cache), }; - result.set_bucket_count()?; - Ok(result) } +} - /// Function get summary count of bucket from `vshard` - fn set_bucket_count(&mut self) -> Result<(), SbroadError> { - let lua = tarantool::lua_state(); +impl Vshard for RouterRuntime { + fn exec_ir_on_all( + &self, + required: Binary, + optional: Binary, + query_type: QueryType, + conn_type: ConnectionType, + ) -> Result<Box<dyn Any>, SbroadError> { + exec_ir_on_all( + &*self.metadata()?, + required, + optional, + query_type, + conn_type, + ) + } - let bucket_count_fn: LuaFunction<_> = - match lua.eval("return require('vshard').router.bucket_count") { - Ok(v) => v, - Err(e) => { - error!(Option::from("set_bucket_count"), &format!("{e:?}")); - return Err(SbroadError::LuaError(format!( - "Failed lua function load: {e}" - ))); - } - }; + fn bucket_count(&self) -> u64 { + self.bucket_count + } - let bucket_count: u64 = match bucket_count_fn.call() { - Ok(r) => r, - Err(e) => { - error!(Option::from("set_bucket_count"), &format!("{e:?}")); - return Err(SbroadError::LuaError(e.to_string())); - } - }; + fn get_random_bucket(&self) -> Buckets { + get_random_bucket(self) + } - self.bucket_count = match bucket_count.try_into() { - Ok(v) => v, - Err(_) => { - return Err(SbroadError::Invalid( - Entity::Runtime, - Some("invalid bucket count".into()), - )); - } - }; + fn determine_bucket_id(&self, s: &[&Value]) -> u64 { + bucket_id_by_tuple(s, self.bucket_count()) + } - Ok(()) + fn exec_ir_on_some( + &self, + sub_plan: ExecutionPlan, + buckets: &Buckets, + ) -> Result<Box<dyn Any>, SbroadError> { + exec_with_filtered_buckets(self, sub_plan, buckets) } } -impl Vshard for RouterRuntime { +impl Vshard for &RouterRuntime { fn exec_ir_on_all( &self, required: Binary, @@ -375,18 +370,22 @@ impl Vshard for RouterRuntime { } fn bucket_count(&self) -> u64 { - self.bucket_count as u64 + self.bucket_count } fn get_random_bucket(&self) -> Buckets { get_random_bucket(self) } + fn determine_bucket_id(&self, s: &[&Value]) -> u64 { + bucket_id_by_tuple(s, self.bucket_count()) + } + fn exec_ir_on_some( &self, sub_plan: ExecutionPlan, buckets: &Buckets, ) -> Result<Box<dyn Any>, SbroadError> { - exec_with_filtered_buckets(self, sub_plan, buckets) + exec_with_filtered_buckets(*self, sub_plan, buckets) } } diff --git a/sbroad-cartridge/src/cartridge/storage.rs b/sbroad-cartridge/src/cartridge/storage.rs index ad4237d822..63dfd1cfbf 100644 --- a/sbroad-cartridge/src/cartridge/storage.rs +++ b/sbroad-cartridge/src/cartridge/storage.rs @@ -1,26 +1,32 @@ use crate::cartridge::config::StorageConfiguration; -use crate::cartridge::update_tracing; +use crate::cartridge::{bucket_count, update_tracing}; use sbroad::errors::{Action, Entity, SbroadError}; -use sbroad::executor::engine::helpers::compile_encoded_optional; +use sbroad::executor::bucket::Buckets; use sbroad::executor::engine::helpers::storage::runtime::{ prepare, read_prepared, read_unprepared, unprepare, write_prepared, write_unprepared, }; use sbroad::executor::engine::helpers::storage::PreparedStmt; -use sbroad::executor::engine::QueryCache; -use sbroad::executor::ir::QueryType; +use sbroad::executor::engine::helpers::vshard::get_random_bucket; +use sbroad::executor::engine::helpers::{compile_encoded_optional, execute_dml}; +use sbroad::executor::engine::{QueryCache, Vshard}; +use sbroad::executor::hash::bucket_id_by_tuple; +use sbroad::executor::ir::{ConnectionType, ExecutionPlan, QueryType}; use sbroad::executor::lru::{Cache, LRUCache, DEFAULT_CAPACITY}; -use sbroad::executor::protocol::RequiredData; +use sbroad::executor::protocol::{Binary, RequiredData}; +use sbroad::ir::value::Value; use sbroad::{debug, error, warn}; use std::any::Any; use std::cell::{Ref, RefCell}; use std::fmt::Display; use tarantool::tlua::LuaFunction; +use tarantool::tuple::Tuple; use super::ConfigurationProvider; #[allow(clippy::module_name_repetitions)] pub struct StorageRuntime { metadata: RefCell<StorageConfiguration>, + bucket_count: u64, cache: RefCell<LRUCache<String, PreparedStmt>>, } @@ -150,6 +156,44 @@ impl ConfigurationProvider for StorageRuntime { } } +impl Vshard for StorageRuntime { + fn exec_ir_on_all( + &self, + _required: Binary, + _optional: Binary, + _query_type: QueryType, + _conn_type: ConnectionType, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Runtime, + Some("exec_ir_on_all is not supported on the storage".to_string()), + )) + } + + fn bucket_count(&self) -> u64 { + self.bucket_count + } + + fn get_random_bucket(&self) -> Buckets { + get_random_bucket(self) + } + + fn determine_bucket_id(&self, s: &[&Value]) -> u64 { + bucket_id_by_tuple(s, self.bucket_count()) + } + + fn exec_ir_on_some( + &self, + _sub_plan: ExecutionPlan, + _buckets: &Buckets, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Runtime, + Some("exec_ir_on_some is not supported on the storage".to_string()), + )) + } +} + impl StorageRuntime { /// Build a new storage runtime. /// @@ -160,6 +204,7 @@ impl StorageRuntime { LRUCache::new(DEFAULT_CAPACITY, Some(Box::new(unprepare)))?; let result = StorageRuntime { metadata: RefCell::new(StorageConfiguration::new()), + bucket_count: bucket_count()?, cache: RefCell::new(cache), }; @@ -172,55 +217,49 @@ impl StorageRuntime { required: &mut RequiredData, raw_optional: &mut Vec<u8>, ) -> Result<Box<dyn Any>, SbroadError> { - if required.can_be_cached { - return self.execute_cacheable_plan(required, raw_optional); + match required.query_type { + QueryType::DML => self.execute_dml(required, raw_optional), + QueryType::DQL => { + if required.can_be_cached { + self.execute_cacheable_dql(required, raw_optional) + } else { + execute_non_cacheable_dql(required, raw_optional) + } + } } - Self::execute_non_cacheable_plan(required, raw_optional) } #[allow(unused_variables)] - fn execute_non_cacheable_plan( + fn execute_dml( + &self, required: &mut RequiredData, raw_optional: &mut Vec<u8>, ) -> Result<Box<dyn Any>, SbroadError> { - let plan_id = required.plan_id.clone(); - - if required.can_be_cached { + if required.query_type != QueryType::DML { return Err(SbroadError::Invalid( Entity::Plan, - Some("Expected a plan that can not be cached.".to_string()), + Some("Expected a DML plan.".to_string()), )); } - let (pattern_with_params, _tmp_spaces) = compile_encoded_optional(raw_optional)?; - debug!( - Option::from("execute"), - &format!( - "Failed to execute the statement: {}", - pattern_with_params.pattern - ), - ); - let result = if required.query_type == QueryType::DML { - write_unprepared(&pattern_with_params.pattern, &pattern_with_params.params) - } else { - read_unprepared(&pattern_with_params.pattern, &pattern_with_params.params) - }; - - result + let result = execute_dml(self, raw_optional)?; + let tuple = Tuple::new(&(result,)) + .map_err(|e| SbroadError::Invalid(Entity::Tuple, Some(format!("{e:?}"))))?; + Ok(Box::new(tuple) as Box<dyn Any>) } #[allow(unused_variables)] - fn execute_cacheable_plan( + fn execute_cacheable_dql( &self, required: &mut RequiredData, raw_optional: &mut Vec<u8>, ) -> Result<Box<dyn Any>, SbroadError> { let plan_id = required.plan_id.clone(); - if !required.can_be_cached { + if !required.can_be_cached || required.query_type != QueryType::DQL { return Err(SbroadError::Invalid( Entity::Plan, - Some("Expected a plan that can be cached.".to_string()), + Some("Expected a DQL plan that can be cached.".to_string()), )); } @@ -319,6 +358,32 @@ impl StorageRuntime { } } +fn execute_non_cacheable_dql( + required: &mut RequiredData, + raw_optional: &mut Vec<u8>, +) -> Result<Box<dyn Any>, SbroadError> { + if required.can_be_cached || required.query_type != QueryType::DQL { + return Err(SbroadError::Invalid( + Entity::Plan, + Some("Expected a DQL plan that can not be cached.".to_string()), + )); + } + + let (pattern_with_params, _tmp_spaces) = compile_encoded_optional(raw_optional)?; + debug!( + Option::from("execute"), + &format!( + "Failed to execute the statement: {}", + pattern_with_params.pattern + ), + ); + warn!( + Option::from("execute"), + &format!("SQL pattern: {}", pattern_with_params.pattern), + ); + read_unprepared(&pattern_with_params.pattern, &pattern_with_params.params) +} + fn update_box_param<T>(param: &str, val: T) where T: Display, diff --git a/sbroad-cartridge/test_app/test/integration/api_test.lua b/sbroad-cartridge/test_app/test/integration/api_test.lua index bd5cb48daf..25eff7b808 100644 --- a/sbroad-cartridge/test_app/test/integration/api_test.lua +++ b/sbroad-cartridge/test_app/test/integration/api_test.lua @@ -436,27 +436,6 @@ g.test_bucket_id_in_join = function() }) end -g.test_bucket_id_function = function() - local api = cluster:server("api-1").net_box - - local r, err = api:call( - "sbroad.execute", - { - [[SELECT bucket_id('hello') FROM "space_simple_shard_key" WHERE "id" = 10]], - {} - } - ) - t.assert_equals(err, nil) - t.assert_equals(r, { - metadata = { - {name = "COL_1", type = "unsigned"}, - }, - rows = { - { 13352 }, - }, - }) -end - g.test_uppercase1 = function() local api = cluster:server("api-1").net_box diff --git a/sbroad-cartridge/test_app/test/integration/left_outer_join_test.lua b/sbroad-cartridge/test_app/test/integration/left_outer_join_test.lua index fb76e15ac2..220884bf74 100644 --- a/sbroad-cartridge/test_app/test/integration/left_outer_join_test.lua +++ b/sbroad-cartridge/test_app/test/integration/left_outer_join_test.lua @@ -1,4 +1,5 @@ local t = require('luatest') +local d = require('decimal') local left_join = t.group('left_join') local helper = require('test.helper.cluster_no_replication') local cluster = nil @@ -13,8 +14,10 @@ left_join.before_all( local r, err = api:call("sbroad.execute", { [[ - INSERT INTO "arithmetic_space" - ("id", "a", "b", "c", "d", "e", "f", "boolean_col", "string_col", "number_col") + INSERT INTO "arithmetic_space"( + "id", "a", "b", "c", "d", "e", "f", + "boolean_col", "string_col", "number_col" + ) VALUES (?,?,?,?,?,?,?,?,?,?), (?,?,?,?,?,?,?,?,?,?), (?,?,?,?,?,?,?,?,?,?), @@ -32,8 +35,10 @@ left_join.before_all( t.assert_equals(r, {row_count = 4}) r, err = api:call("sbroad.execute", { [[ - INSERT INTO "arithmetic_space2" - ("id", "a", "b", "c", "d", "e", "f", "boolean_col", "string_col", "number_col") + INSERT INTO "arithmetic_space2"( + "id", "a", "b", "c", "d", "e", "f", + "boolean_col", "string_col", "number_col" + ) VALUES (?,?,?,?,?,?,?,?,?,?), (?,?,?,?,?,?,?,?,?,?), (?,?,?,?,?,?,?,?,?,?), @@ -51,13 +56,24 @@ left_join.before_all( t.assert_equals(r, {row_count = 4}) for i =1,6 do - --yearquarter a_to b_to a_from b_from c_by_ab d_by_ab d_c_diff field1 field2 - --integer string string string string decimal decimal decimal string string - local values = {i, "a", "a", "a", "a", i, i, i, "a", "a"} + local values = { + i, -- yearquarter (integer) + "a", -- a_to (string) + "a", -- b_to (string) + "a", -- a_from (string) + "a", -- b_from (string) + d.new(i), -- c_by_ab (decimal) + d.new(i), -- d_by_ab (decimal) + d.new(i), -- d_c_diff (decimal) + "a", -- field1 (string) + "a" -- field2 (string) + } r, err = api:call("sbroad.execute", { [[ - INSERT INTO "SPACE1" - ("yearquarter","a_to","b_to","a_from","b_from","c_by_ab","d_by_ab","d_c_diff","field1","field2") + INSERT INTO "SPACE1"( + "yearquarter","a_to","b_to","a_from","b_from", + "c_by_ab","d_by_ab","d_c_diff","field1","field2" + ) VALUES (?,?,?,?,?,?,?,?,?,?) ]], values }) @@ -65,13 +81,31 @@ left_join.before_all( t.assert_equals(r, {row_count = 1}) end for i =4,10 do - --id yearquarter a b name field1 field2 field3 field4 field5 field6 field7 field8 field9 count_from count_to - --integer integer string string string integer decimal string integer string decimal decimal decimal integer integer integer - local values = {i, i, "a", "a", "a", i, i, "a", i, "a", i, i, i, i, i, i} + local values = { + i, -- id (integer) + i, -- yearquarter (integer) + "a", -- a (string) + "a", -- b (string) + "a", -- name (string) + i, -- field1 (integer) + d.new(i), -- field2 (decimal) + "a", -- field3 (string) + i, -- field4 (integer) + "a", -- field5 (string) + d.new(i), -- field6 (decimal) + d.new(i), -- field7 (decimal) + d.new(i), -- field8 (decimal) + i, -- field9 (integer) + i, -- count_from (integer) + i -- count_to (integer) + } r, err = api:call("sbroad.execute", { [[ - INSERT INTO "SPACE2" - ("id","yearquarter","a","b","name","field1","field2","field3","field4","field5","field6","field7","field8","field9","count_from","count_to") + INSERT INTO "SPACE2"( + "id","yearquarter","a","b","name","field1","field2", + "field3","field4","field5","field6","field7","field8", + "field9","count_from","count_to" + ) VALUES (?,?,?,?,?,?,?,?,?,?,?,?,?,?,?,?) ]], values }) diff --git a/sbroad-core/Cargo.toml b/sbroad-core/Cargo.toml index de93648ef6..d38ff93336 100644 --- a/sbroad-core/Cargo.toml +++ b/sbroad-core/Cargo.toml @@ -21,7 +21,7 @@ opentelemetry-jaeger = "0.16" pest = "2.0" pest_derive = "2.0" rand = "0.8" -rmp-serde = "0.14" +rmp-serde = "1.0" sbroad-proc = { path = "../sbroad-proc", version = "0.1" } serde = { version = "1.0", features = ["derive", "rc"] } serde_yaml = "0.8" diff --git a/sbroad-core/src/backend/sql/tree.rs b/sbroad-core/src/backend/sql/tree.rs index 2bd6dd79c6..2293f974aa 100644 --- a/sbroad-core/src/backend/sql/tree.rs +++ b/sbroad-core/src/backend/sql/tree.rs @@ -8,6 +8,7 @@ use crate::errors::{Action, Entity, SbroadError}; use crate::executor::ir::ExecutionPlan; use crate::ir::expression::Expression; use crate::ir::operator::{Bool, Relational, Unary}; +use crate::ir::transformation::redistribution::MotionPolicy; use crate::ir::tree::traversal::PostOrder; use crate::ir::tree::Snapshot; use crate::ir::Node; @@ -792,7 +793,35 @@ impl<'p> SyntaxPlan<'p> { let sn = SyntaxNode::new_pointer(id, None, children); Ok(self.nodes.push_syntax_node(sn)) } - Relational::Motion { .. } => { + Relational::Motion { + policy, children, .. + } => { + if let MotionPolicy::LocalSegment { .. } = policy { + #[cfg(feature = "mock")] + { + // We should materialize the subquery on the storage. + // Honestly, this SQL is not valid and should never be + // generated in runtime, but let's leave it for testing. + if let Some(child_id) = children.first() { + let sn = SyntaxNode::new_pointer( + id, + Some(self.nodes.get_syntax_node_id(*child_id)?), + vec![], + ); + return Ok(self.nodes.push_syntax_node(sn)); + } + } + #[cfg(not(feature = "mock"))] + { + return Err(SbroadError::Invalid( + Entity::Node, + Some( + "LocalSegment motion policy is not supported in the syntax plan." + .into(), + ), + )); + } + } let vtable = self.plan.get_motion_vtable(id)?; let vtable_alias = vtable.get_alias().map(String::from); let mut children: Vec<usize> = Vec::new(); @@ -1149,7 +1178,7 @@ impl<'p> SyntaxPlan<'p> { Ok(()) } - fn empty(plan: &'p ExecutionPlan) -> Self { + pub(crate) fn empty(plan: &'p ExecutionPlan) -> Self { SyntaxPlan { nodes: SyntaxNodes::with_capacity(plan.get_ir_plan().next_id()), top: None, @@ -1288,6 +1317,14 @@ impl OrderedSyntaxNodes { } Ok(result) } + + #[must_use] + pub fn empty() -> Self { + OrderedSyntaxNodes { + arena: Vec::new(), + positions: Vec::new(), + } + } } impl TryFrom<SyntaxPlan<'_>> for OrderedSyntaxNodes { diff --git a/sbroad-core/src/core.lua b/sbroad-core/src/core.lua index 265bfed17d..3c74f949be 100644 --- a/sbroad-core/src/core.lua +++ b/sbroad-core/src/core.lua @@ -1,33 +1,23 @@ -local function init_bucket_id() +local helper = require('sbroad.helper') + +local function init_bucket() + local exec_fn = helper.module_name() .. ".calculate_bucket_id" box.schema.func.create( - 'libsbroad.calculate_bucket_id', + exec_fn, { if_not_exists = true, language = 'C' } ) - - box.schema.func.create('BUCKET_ID', { - language = 'Lua', - body = [[ - function(x) - return box.func['libsbroad.calculate_bucket_id']:call({ x }) - end - ]], - if_not_exists = true, - param_list = {'string'}, - returns = 'unsigned', - aggregate = 'none', - exports = {'SQL'}, - }) end local function init_statistics() + local exec_fn = helper.module_name() .. ".init_statistics" box.schema.func.create( - 'libsbroad.init_statistics', + exec_fn, { if_not_exists = true, language = 'C' } ) end local function init() - init_bucket_id() + init_bucket() init_statistics() end diff --git a/sbroad-core/src/errors.rs b/sbroad-core/src/errors.rs index c9457ec966..f39296333a 100644 --- a/sbroad-core/src/errors.rs +++ b/sbroad-core/src/errors.rs @@ -1,5 +1,6 @@ use serde::Serialize; use std::fmt; +use tarantool::error::TransactionError; const DO_SKIP: &str = "do skip"; @@ -98,6 +99,8 @@ pub enum Entity { Table, /// corresponds to struct Target Target, + /// tarantool transaction + Transaction, /// general variant for tuple Tuple, /// general variant for type of some object @@ -157,6 +160,7 @@ impl fmt::Display for Entity { Entity::SyntaxPlan => "syntax plan".to_string(), Entity::Table => "table".to_string(), Entity::Target => "target".to_string(), + Entity::Transaction => "transaction".to_string(), Entity::Tuple => "tuple".to_string(), Entity::Type => "type".to_string(), Entity::Value => "value".to_string(), @@ -273,3 +277,9 @@ impl fmt::Display for SbroadError { } impl std::error::Error for SbroadError {} + +impl From<TransactionError> for SbroadError { + fn from(error: TransactionError) -> Self { + SbroadError::FailedTo(Action::Create, Some(Entity::Transaction), error.to_string()) + } +} diff --git a/sbroad-core/src/executor.rs b/sbroad-core/src/executor.rs index 1cd534db7d..30784d5eec 100644 --- a/sbroad-core/src/executor.rs +++ b/sbroad-core/src/executor.rs @@ -24,18 +24,17 @@ //! 6. Executes the final IR top subtree and returns the final result to the user. use std::any::Any; -use std::collections::{hash_map::Entry, HashMap}; +use std::collections::HashMap; use std::rc::Rc; use crate::errors::{Action, Entity, SbroadError}; use crate::executor::bucket::Buckets; -use crate::executor::engine::Router; +use crate::executor::engine::{helpers::materialize_values, Router, Vshard}; use crate::executor::ir::ExecutionPlan; use crate::executor::lru::Cache; -use crate::executor::vtable::VirtualTable; use crate::frontend::Ast; use crate::ir::operator::Relational; -use crate::ir::transformation::redistribution::{MotionKey, MotionPolicy, Target}; +use crate::ir::transformation::redistribution::MotionPolicy; use crate::ir::value::Value; use crate::ir::Plan; use crate::otm::{child_span, query_id}; @@ -68,7 +67,8 @@ impl Plan { #[derive(Debug)] pub struct Query<'a, C> where - C: Router, + C: Router + Vshard, + &'a C: Vshard, { /// Explain flag is_explain: bool, @@ -82,7 +82,8 @@ where impl<'a, C> Query<'a, C> where - C: Router, + C: Router + Vshard, + &'a C: Vshard, { /// Create a new query. /// @@ -179,13 +180,39 @@ where continue; } + // Local segment motions should be treated as a special case. + // 1. If we can materialize it on the router, then we should do it + // (the child node is `VALUES` of constants). + // 2. Otherwise we should skip it and dispatch the query to the segments + // (materialization would be done on the segments). + let motion = self.exec_plan.get_ir_plan().get_relation_node(*motion_id)?; + if let Relational::Motion { + policy: MotionPolicy::LocalSegment(_), + .. + } = motion + { + if let Some(virtual_table) = + materialize_values(&mut self.exec_plan, *motion_id)? + { + self.exec_plan.set_motion_vtable( + *motion_id, + virtual_table, + &self.coordinator, + )?; + self.get_mut_exec_plan().unlink_motion_subtree(*motion_id)?; + already_materialized.insert(top_id, *motion_id); + } + continue; + } + let buckets = self.bucket_discovery(top_id)?; let virtual_table = self.coordinator.materialize_motion( &mut self.exec_plan, *motion_id, &buckets, )?; - self.add_motion_result(*motion_id, virtual_table)?; + self.exec_plan + .set_motion_vtable(*motion_id, virtual_table, &self.coordinator)?; already_materialized.insert(top_id, *motion_id); } } @@ -196,95 +223,6 @@ where .dispatch(&mut self.exec_plan, top_id, &buckets) } - /// Add materialize motion result to translation map of virtual tables - /// - /// # Errors - /// - invalid motion node - #[otm_child_span("query.motion.add")] - pub fn add_motion_result( - &mut self, - motion_id: usize, - mut vtable: VirtualTable, - ) -> Result<(), SbroadError> { - let policy = if let Relational::Motion { policy, .. } = self - .get_exec_plan() - .get_ir_plan() - .get_relation_node(motion_id)? - { - policy.clone() - } else { - return Err(SbroadError::Invalid( - Entity::Node, - Some("invalid motion node".to_string()), - )); - }; - if let MotionPolicy::Segment(shard_key) = policy { - // At the moment we generate a new sharding column only for the motions - // prior the insertion node. As we support only relations with segmented - // data (Tarantool doesn't have relations with replicated data), we handle - // a case with sharding column only for a segment motion policy. - self.reshard_vtable(&mut vtable, &shard_key)?; - } - - let need_init = self.exec_plan.get_vtables().is_none(); - if need_init { - self.exec_plan.set_vtables(HashMap::new()); - } - - if let Some(vtables) = self.exec_plan.get_mut_vtables() { - vtables.insert(motion_id, Rc::new(vtable)); - } - - Ok(()) - } - - /// Reshard virtual table. - /// - /// # Errors - /// - Invalid distribution key. - pub fn reshard_vtable( - &self, - vtable: &mut VirtualTable, - sharding_key: &MotionKey, - ) -> Result<(), SbroadError> { - vtable.set_motion_key(sharding_key); - - let mut index: HashMap<u64, Vec<usize>> = HashMap::new(); - for (pos, tuple) in vtable.get_tuples().iter().enumerate() { - let mut shard_key_tuple: Vec<&Value> = Vec::new(); - for target in &sharding_key.targets { - match target { - Target::Reference(col_idx) => { - let part = tuple.get(*col_idx).ok_or_else(|| { - SbroadError::NotFound( - Entity::DistributionKey, - format!( - "failed to find a distribution key column {pos} in the tuple {tuple:?}." - ), - ) - })?; - shard_key_tuple.push(part); - } - Target::Value(ref value) => { - shard_key_tuple.push(value); - } - } - } - let bucket_id = self.coordinator.determine_bucket_id(&shard_key_tuple); - match index.entry(bucket_id) { - Entry::Vacant(entry) => { - entry.insert(vec![pos]); - } - Entry::Occupied(entry) => { - entry.into_mut().push(pos); - } - } - } - - vtable.set_index(index); - Ok(()) - } - /// Query explain /// /// # Errors diff --git a/sbroad-core/src/executor/bucket.rs b/sbroad-core/src/executor/bucket.rs index 5f0967a9dc..6d3dffc17b 100644 --- a/sbroad-core/src/executor/bucket.rs +++ b/sbroad-core/src/executor/bucket.rs @@ -1,7 +1,7 @@ use std::collections::HashSet; use crate::errors::{Action, Entity, SbroadError}; -use crate::executor::engine::Router; +use crate::executor::engine::{Router, Vshard}; use crate::executor::Query; use crate::ir::distribution::Distribution; use crate::ir::expression::Expression; @@ -95,7 +95,8 @@ impl Buckets { impl<'a, T> Query<'a, T> where - T: Router, + T: Router + Vshard, + &'a T: Vshard, { fn get_buckets_from_expr(&self, expr_id: usize) -> Result<Buckets, SbroadError> { let mut buckets: Vec<Buckets> = Vec::new(); @@ -192,9 +193,11 @@ where if buckets.is_empty() { Ok(Buckets::new_all()) } else { - Ok(buckets - .into_iter() - .fold(Ok(Buckets::new_all()), |a, b| a?.conjuct(&b)))? + Ok::<Result<Buckets, SbroadError>, SbroadError>( + buckets + .into_iter() + .fold(Ok(Buckets::new_all()), |a, b| a?.conjuct(&b)), + )? } } @@ -253,7 +256,8 @@ where // We use a `subtree_iter()` because we need DNF version of the filter/condition // expressions to determine buckets. let capacity = ir_plan.next_id(); - let mut tree = PostOrder::with_capacity(|node| ir_plan.subtree_iter(node), capacity); + let mut tree = + PostOrder::with_capacity(|node| ir_plan.exec_plan_subtree_iter(node), capacity); let nodes: Vec<usize> = tree .iter(top_id) .filter_map(|(_, id)| { @@ -275,7 +279,12 @@ where Relational::ScanRelation { output, .. } => { self.bucket_map.insert(*output, Buckets::new_all()); } - Relational::Motion { policy, output, .. } => match policy { + Relational::Motion { + children, + policy, + output, + .. + } => match policy { MotionPolicy::Full => { self.bucket_map.insert(*output, Buckets::new_all()); } @@ -289,6 +298,43 @@ where self.bucket_map .insert(*output, Buckets::new_filtered(buckets)); } + MotionPolicy::LocalSegment(_) => { + if let Ok(virtual_table) = self.exec_plan.get_motion_vtable(node_id) { + // In a case of `insert .. values ..` it is possible built a local + // segmented virtual table right on the router. So, we can use its + // buckets from the index to determine the buckets for the insert. + let buckets = virtual_table + .get_index() + .keys() + .copied() + .collect::<HashSet<u64, RepeatableState>>(); + self.bucket_map + .insert(*output, Buckets::new_filtered(buckets)); + } else { + // We'll create and populate a local segmented virtual table on the + // storage later. At the moment the best thing we can do is to copy + // child's buckets. + let child_id = children.first().ok_or_else(|| { + SbroadError::UnexpectedNumberOfValues( + "Motion node should have exactly one child".to_string(), + ) + })?; + let child_rel = + self.exec_plan.get_ir_plan().get_relation_node(*child_id)?; + let child_buckets = self + .bucket_map + .get(&child_rel.output()) + .ok_or_else(|| { + SbroadError::FailedTo( + Action::Retrieve, + Some(Entity::Buckets), + "of the child from the bucket map.".to_string(), + ) + })? + .clone(); + self.bucket_map.insert(*output, child_buckets); + } + } MotionPolicy::Local => { return Err(SbroadError::Invalid( Entity::Motion, diff --git a/sbroad-core/src/executor/bucket/tests.rs b/sbroad-core/src/executor/bucket/tests.rs index 0e7cfa7f1c..37faf0c890 100644 --- a/sbroad-core/src/executor/bucket/tests.rs +++ b/sbroad-core/src/executor/bucket/tests.rs @@ -3,7 +3,7 @@ use std::collections::HashSet; use crate::executor::bucket::Buckets; use crate::executor::engine::mock::RouterRuntimeMock; -use crate::executor::engine::Router; +use crate::executor::engine::Vshard; use crate::executor::Query; use crate::ir::helpers::RepeatableState; diff --git a/sbroad-core/src/executor/engine.rs b/sbroad-core/src/executor/engine.rs index 85f339536c..ed1de01764 100644 --- a/sbroad-core/src/executor/engine.rs +++ b/sbroad-core/src/executor/engine.rs @@ -124,9 +124,6 @@ pub trait Router: QueryCache { args: &'rec [Value], ) -> Result<Vec<&'rec Value>, SbroadError>; - /// Determine shard for query execution by sharding key value - fn determine_bucket_id(&self, s: &[&Value]) -> u64; - /// Dispatch a sql query to the shards in cluster and get the results. /// /// # Errors @@ -260,4 +257,7 @@ pub trait Vshard { /// Get a random bucket from the cluster. fn get_random_bucket(&self) -> Buckets; + + /// Determine shard for query execution by sharding key value + fn determine_bucket_id(&self, s: &[&Value]) -> u64; } diff --git a/sbroad-core/src/executor/engine/helpers.rs b/sbroad-core/src/executor/engine/helpers.rs index 1aa6b73d25..2eed3b9457 100644 --- a/sbroad-core/src/executor/engine/helpers.rs +++ b/sbroad-core/src/executor/engine/helpers.rs @@ -1,3 +1,5 @@ +use ahash::AHashMap; + use std::{ any::Any, cmp::Ordering, @@ -5,9 +7,13 @@ use std::{ rc::Rc, }; -use tarantool::tuple::{ - rmp::{self, decode::RmpRead}, - Tuple, +use tarantool::{ + space::Space, + transaction::start_transaction, + tuple::{ + rmp::{self, decode::RmpRead}, + Tuple, + }, }; use crate::{ @@ -19,12 +25,22 @@ use crate::{ errors::{Action, Entity, SbroadError}, executor::{ bucket::Buckets, - ir::ExecutionPlan, + engine::helpers::storage::runtime::read_unprepared, + ir::{ExecutionPlan, QueryType}, protocol::{Binary, EncodedOptionalData, EncodedRequiredData, OptionalData, RequiredData}, - result::ProducerResult, - vtable::VirtualTable, + result::{ConsumerResult, ProducerResult}, + vtable::{VTableTuple, VirtualTable}, + }, + ir::{ + expression::Expression, + helpers::RepeatableState, + operator::Relational, + relation::{Column, ColumnRole, Type}, + transformation::redistribution::{MotionKey, MotionPolicy}, + tree::Snapshot, + value::{EncodedValue, Value}, + Node, Plan, }, - ir::{helpers::RepeatableState, tree::Snapshot, value::Value}, }; use super::{Metadata, Router, Vshard}; @@ -45,6 +61,14 @@ pub fn normalize_name_from_sql(s: &str) -> String { format!("\"{}\"", s.to_uppercase()) } +#[must_use] +pub fn normalize_name_for_space_api(s: &str) -> String { + if let (Some('"'), Some('"')) = (s.chars().next(), s.chars().last()) { + return s.chars().skip(1).take(s.len() - 2).collect(); + } + s.to_uppercase() +} + /// A helper function to encode the execution plan into a pair of binary data: /// * required data (plan id, parameters, etc.) /// * optional data (execution plan, etc.) @@ -52,18 +76,64 @@ pub fn normalize_name_from_sql(s: &str) -> String { /// # Errors /// - Failed to encode the execution plan. pub fn encode_plan(exec_plan: ExecutionPlan) -> Result<(Binary, Binary), SbroadError> { + let query_type = exec_plan.query_type()?; + let (ordered, sub_plan_id) = match query_type { + QueryType::DQL => { + let sub_plan_id = exec_plan.get_ir_plan().pattern_id()?; + let sp_top_id = exec_plan.get_ir_plan().get_top()?; + let sp = SyntaxPlan::new(&exec_plan, sp_top_id, Snapshot::Oldest)?; + let ordered = OrderedSyntaxNodes::try_from(sp)?; + (ordered, sub_plan_id) + } + QueryType::DML => { + let plan = exec_plan.get_ir_plan(); + let sp_top_id = plan.get_top()?; + // At the moment we support only `INSERT` statement for DML. + // TODO: refactor this code when we'll support other DML statements. + let motion_id = plan.insert_child_id(sp_top_id)?; + let motion = plan.get_relation_node(motion_id)?; + let policy = if let Relational::Motion { policy, .. } = motion { + policy + } else { + return Err(SbroadError::Invalid( + Entity::Plan, + Some(format!( + "expected motion node under insert node, got: {:?}", + motion + )), + )); + }; + // No need to calculate unique key for the plan, as it's not cached. + let sub_plan_id = String::new(); + // SQL is needed only for the motion node subtree. + // HACK: we don't actually need SQL when the subtree is already + // materialized into a virtual table on the router. + let already_materialized = exec_plan.get_motion_vtable(motion_id).is_ok(); + let ordered = if already_materialized { + OrderedSyntaxNodes::empty() + } else if let MotionPolicy::LocalSegment { .. } = policy { + let motion_child_id = exec_plan.get_motion_child(motion_id)?; + let sp = SyntaxPlan::new(&exec_plan, motion_child_id, Snapshot::Oldest)?; + OrderedSyntaxNodes::try_from(sp)? + } else { + return Err(SbroadError::Invalid( + Entity::Plan, + Some(format!( + "unsupported motion policy under insert node: {:?}", + policy + )), + )); + }; + (ordered, sub_plan_id) + } + }; // We should not use the cache on the storage if the plan contains virtual tables, // as they can contain different amount of tuples that are not taken into account // when calculating the cache key. let can_be_cached = exec_plan.vtables_empty(); - let sub_plan_id = exec_plan.get_ir_plan().pattern_id()?; - let sp_top_id = exec_plan.get_ir_plan().get_top()?; - let sp = SyntaxPlan::new(&exec_plan, sp_top_id, Snapshot::Oldest)?; - let ordered = OrderedSyntaxNodes::try_from(sp)?; let nodes = ordered.to_syntax_data()?; // Virtual tables in the plan must be already filtered, so we can use all buckets here. let params = exec_plan.to_params(&nodes, &Buckets::All)?; - let query_type = exec_plan.query_type()?; let required_data = RequiredData::new(sub_plan_id, params, query_type, can_be_cached); let encoded_required_data = EncodedRequiredData::try_from(required_data)?; let raw_required_data: Vec<u8> = encoded_required_data.into(); @@ -154,6 +224,196 @@ pub fn compile_encoded_optional( ) } +#[derive(Debug)] +enum TupleBuilderCommands { + /// Take a value from the original tuple + /// at the specified position. + TakePosition(usize), + /// Set a specified value. + SetValue(Value), + /// Calculate a bucket id for the new tuple + /// using the specified motion key. + CalculateBucketId(MotionKey), +} + +type TupleBuilderPattern = Vec<TupleBuilderCommands>; + +fn insert_tuple_builder(plan: &Plan, insert_id: usize) -> Result<TupleBuilderPattern, SbroadError> { + let columns = plan.insert_columns(insert_id)?; + let columns_map: AHashMap<usize, usize> = columns + .iter() + .enumerate() + .map(|(pos, id)| (*id, pos)) + .collect::<AHashMap<_, _>>(); + let table = plan.insert_table(insert_id)?; + let mut commands = Vec::with_capacity(table.columns.len()); + for (pos, table_col) in table.columns.iter().enumerate() { + if table_col.role == ColumnRole::Sharding { + let motion_key = plan.insert_child_motion_key(insert_id)?; + commands.push(TupleBuilderCommands::CalculateBucketId(motion_key)); + } else if columns_map.contains_key(&pos) { + // It is safe to unwrap here because we have checked that + // the column is present in the tuple. + let tuple_pos = columns_map[&pos]; + commands.push(TupleBuilderCommands::TakePosition(tuple_pos)); + } else { + // FIXME: support default values other then NULL (issue #442). + commands.push(TupleBuilderCommands::SetValue(Column::default_value())); + } + } + Ok(commands) +} + +/// Execute DML on the storage. +/// +/// # Errors +/// - Failed to execute DML locally. +/// +/// # Panics +/// - Current function never panics (though it contains `unwrap()` calls). +#[allow(clippy::too_many_lines)] +pub fn execute_dml( + runtime: &impl Vshard, + raw_optional: &mut Vec<u8>, +) -> Result<ConsumerResult, SbroadError> { + let data = std::mem::take(raw_optional); + let mut optional = OptionalData::try_from(EncodedOptionalData::from(data))?; + optional.exec_plan.get_mut_ir_plan().restore_constants()?; + let plan = optional.exec_plan.get_ir_plan(); + + // At the moment the only supported DML query is `INSERT`. + // We always generate a virtual table under the `INSERT` node + // of the execution plan and prefer to execute it via space API + // instead of SQL (for performance reasons). + let insert_id = plan.get_top()?; + let insert_child_id = plan.insert_child_id(insert_id)?; + let builder = insert_tuple_builder(plan, insert_id)?; + let space_name = normalize_name_for_space_api(plan.insert_table(insert_id)?.name()); + let mut result = ConsumerResult::default(); + + // There are two ways to execute an `INSERT` query: + // 1. Execute SQL subtree under the `INSERT` node (`INSERT .. SELECT ..`) + // and then repack and insert results into the space. + // 2. A virtual table was dispatched under the `INSERT` node. + // Simply insert its tuples into the space. + + // Check is we need to execute an SQL subtree (case 1). + let build_vtable_locally = optional + .exec_plan + .get_motion_vtable(insert_child_id) + .is_err(); + if build_vtable_locally { + let subplan_top_id = optional + .exec_plan + .get_motion_subtree_root(insert_child_id)?; + let column_names = plan.get_relational_aliases(subplan_top_id)?; + optional.exec_plan.get_mut_ir_plan().restore_constants()?; + let nodes = optional.ordered.to_syntax_data()?; + let buckets = Buckets::All; + let (pattern, _) = optional.exec_plan.to_sql( + &nodes, + &buckets, + &uuid::Uuid::new_v4().as_simple().to_string(), + )?; + // TODO: first try to use storage cache with read_prepared(). + let result = read_unprepared(&pattern.pattern, &pattern.params)?; + let tuple = result.downcast::<Tuple>().map_err(|e| { + SbroadError::FailedTo( + Action::Deserialize, + Some(Entity::Tuple), + format!("motion node {insert_child_id}. {e:?}"), + ) + })?; + let mut data = tuple.decode::<Vec<ProducerResult>>().map_err(|e| { + SbroadError::FailedTo( + Action::Decode, + Some(Entity::Tuple), + format!("motion node {insert_child_id}. {e}"), + ) + })?; + let vtable = data + .get_mut(0) + .ok_or_else(|| SbroadError::NotFound(Entity::ProducerResult, "from the tuple".into()))? + // It is a DML query, so we don't need to care about the column types + // in response. So, simply use scalar type for all the columns. + .as_virtual_table(column_names, true)?; + optional + .exec_plan + .set_motion_vtable(insert_child_id, vtable, runtime)?; + } + + // Check if the virtual table have been dispatched (case 2) or built locally (case 1). + if let Some(vtables) = &mut optional.exec_plan.vtables { + if let Some(mut vtable) = vtables.mut_map().remove(&insert_child_id) { + let space = Space::find(&space_name).ok_or_else(|| { + SbroadError::Invalid(Entity::Space, Some(format!("space {space_name} not found"))) + })?; + // There are no long-living references to the virtual table on the storage + // (we can have the ones only on the router while dispatching subplans). + // So we should never produce any memory copy here with `Rc::make_mut()`. + let vtable = Rc::make_mut(&mut vtable); + let mut tuples = std::mem::take(vtable.get_mut_tuples()); + start_transaction(|| -> Result<(), SbroadError> { + for (bucket_id, positions) in vtable.get_mut_index().drain() { + for pos in positions { + let mut vt_tuple = { + let tuple = tuples.get_mut(pos).ok_or_else(|| { + SbroadError::Invalid( + Entity::VirtualTable, + Some(format!( + "tuple at position {pos} not found in virtual table" + )), + ) + })?; + std::mem::take(tuple) + }; + let mut insert_tuple = Vec::with_capacity(builder.len()); + for command in &builder { + match command { + TupleBuilderCommands::TakePosition(pos) => { + let value = { + let value = vt_tuple.get_mut(*pos).ok_or_else(|| + SbroadError::Invalid( + Entity::Tuple, + Some(format!( + "column at position {pos} not found in virtual table" + )) + ))?; + std::mem::take(value) + }; + insert_tuple.push(EncodedValue::from(value)); + } + TupleBuilderCommands::SetValue(value) => { + insert_tuple.push(EncodedValue::from(value.clone())); + } + TupleBuilderCommands::CalculateBucketId(_) => { + insert_tuple.push(EncodedValue::Unsigned(bucket_id)); + } + } + } + space.insert(&insert_tuple).map_err(|e| { + SbroadError::FailedTo( + Action::Insert, + Some(Entity::Space), + format!("{e}"), + ) + })?; + result.row_count += 1; + } + } + Ok(()) + })?; + } + } else { + return Err(SbroadError::NotFound( + Entity::VirtualTable, + "in the execution plan while executiong DML on the storage".into(), + )); + } + + Ok(result) +} + /// Format explain output into a tuple. /// /// # Errors @@ -220,6 +480,123 @@ pub fn dispatch( } } +pub(crate) fn materialize_values( + plan: &mut ExecutionPlan, + motion_node_id: usize, +) -> Result<Option<VirtualTable>, SbroadError> { + // Check that the motion node has a local segment policy. + let motion_node = plan.get_ir_plan().get_relation_node(motion_node_id)?; + if let Relational::Motion { + policy: MotionPolicy::LocalSegment(_), + .. + } = motion_node + { + } else { + return Ok(None); + }; + + // Check that the motion child is a values node with constants in the rows. + // + // When the VALUES node supports subqueries, arithmetics, etc. in addition + // to constants, we have to rewrite this code (need to check that there are + // no subqueries before node replacement). + let child_id = plan.get_motion_child(motion_node_id)?; + if let Relational::Values { .. } = plan.get_ir_plan().get_relation_node(child_id)? { + } else { + return Ok(None); + }; + let child_node_ref = plan.get_mut_ir_plan().get_mut_node(child_id)?; + let child_node = std::mem::replace(child_node_ref, Node::Parameter); + if let Node::Relational(Relational::Values { + children, output, .. + }) = child_node + { + // Build a new virtual table (check that all the rows are made of constants only). + let mut vtable = VirtualTable::new(); + vtable.get_mut_tuples().reserve(children.len()); + for row_id in children { + let row_node = plan.get_ir_plan().get_relation_node(row_id)?; + if let Relational::ValuesRow { data, children, .. } = row_node { + // Check that there are no subqueries in the values node. + // (If any we'll need to materialize them first with dispatch + // to the storages.) + if !children.is_empty() { + return Err(SbroadError::Invalid( + Entity::Node, + Some("VALUES rows supports only constants in its columns.".to_string()), + )); + } + let data = *data; + // Check that all the values are constants. + let columns_len = plan + .get_ir_plan() + .get_expression_node(data)? + .get_row_list()? + .len(); + let mut row: VTableTuple = Vec::with_capacity(columns_len); + for idx in 0..columns_len { + let column_id = + *plan + .get_ir_plan() + .get_row_list(data)? + .get(idx) + .ok_or_else(|| { + SbroadError::NotFound( + Entity::Column, + format!("at position {idx} in the row"), + ) + })?; + let column_node_ref = plan.get_mut_ir_plan().get_mut_node(column_id)?; + let column_node = std::mem::replace(column_node_ref, Node::Parameter); + if let Node::Expression(Expression::Constant { value, .. }) = column_node { + row.push(value); + } else { + return Err(SbroadError::Invalid( + Entity::Node, + Some(format!( + "VALUES rows supports only constants in its columns (got: {column_node:?})." + )), + )); + } + } + vtable.add_tuple(row); + } else { + return Err(SbroadError::Invalid( + Entity::Node, + Some(format!( + "value node child ({child_id}) is not a values row node!" + )), + )); + } + } + // Build virtual table's columns. + let output_cols = plan + .get_ir_plan() + .get_expression_node(output)? + .get_row_list()?; + let columns = vtable.get_mut_columns(); + columns.reserve(output_cols.len()); + for column_id in output_cols { + let alias = plan.get_ir_plan().get_expression_node(*column_id)?; + if let Expression::Alias { name, .. } = alias { + let column = Column { + name: name.clone(), + r#type: Type::Scalar, + role: ColumnRole::User, + }; + columns.push(column); + } else { + return Err(SbroadError::Invalid( + Entity::Node, + Some(format!("output column ({column_id}) is not an alias node!")), + )); + } + } + return Ok(Some(vtable)); + } + Ok(None) +} + /// Materialize a motion subtree into a virtual table. /// /// # Errors @@ -234,22 +611,22 @@ pub fn materialize_motion( let column_names = plan.get_ir_plan().get_relational_aliases(top_id)?; // We should get a motion alias name before we take the subtree in dispatch. let alias = plan.get_motion_alias(motion_node_id)?.map(String::from); - // We also need to find out, if the motion subtree contains values node (as a result we can retrieve - // incorrect types from the result metadata). + // We also need to find out, if the motion subtree contains values node + // (as a result we can retrieve incorrect types from the result metadata). let possibly_incorrect_types = plan.get_ir_plan().subtree_contains_values(motion_node_id)?; // Dispatch the motion subtree (it will be replaced with invalid values). let result = runtime.dispatch(plan, top_id, buckets)?; // Unlink motion node's child sub tree (it is already replaced with invalid values). plan.unlink_motion_subtree(motion_node_id)?; let mut vtable = if let Ok(tuple) = result.downcast::<Tuple>() { - let data = tuple.decode::<Vec<ProducerResult>>().map_err(|e| { + let mut data = tuple.decode::<Vec<ProducerResult>>().map_err(|e| { SbroadError::FailedTo( Action::Decode, Some(Entity::Tuple), format!("motion node {motion_node_id}. {e}"), ) })?; - data.get(0) + data.get_mut(0) .ok_or_else(|| SbroadError::NotFound(Entity::ProducerResult, "from the tuple".into()))? .as_virtual_table(column_names, possibly_incorrect_types)? } else { @@ -258,6 +635,7 @@ pub fn materialize_motion( Some("the result of the motion is not a tuple".to_string()), )); }; + if let Some(name) = alias { vtable.set_alias(&name)?; } diff --git a/sbroad-core/src/executor/engine/helpers/vshard.rs b/sbroad-core/src/executor/engine/helpers/vshard.rs index a13e65d202..66dabd4325 100644 --- a/sbroad-core/src/executor/engine/helpers/vshard.rs +++ b/sbroad-core/src/executor/engine/helpers/vshard.rs @@ -136,7 +136,7 @@ pub fn exec_ir_on_all( #[otm_child_span("query.dispatch.cartridge.some")] pub fn exec_with_filtered_buckets( - runtime: &(impl Vshard + Router), + runtime: &(impl Router + Vshard), mut sub_plan: ExecutionPlan, buckets: &Buckets, ) -> Result<Box<dyn Any>, SbroadError> { diff --git a/sbroad-core/src/executor/engine/mock.rs b/sbroad-core/src/executor/engine/mock.rs index a6bba44957..0b5902e1ea 100644 --- a/sbroad-core/src/executor/engine/mock.rs +++ b/sbroad-core/src/executor/engine/mock.rs @@ -10,12 +10,13 @@ use crate::collection; use crate::errors::{Action, Entity, SbroadError}; use crate::executor::bucket::Buckets; use crate::executor::engine::{ - helpers::{sharding_keys_from_map, sharding_keys_from_tuple}, - InitialBucket, InitialColumnStats, InitialHistogram, Router, Statistics, + helpers::{sharding_keys_from_map, sharding_keys_from_tuple, vshard::get_random_bucket}, + InitialBucket, InitialColumnStats, InitialHistogram, Router, Statistics, Vshard, }; use crate::executor::hash::bucket_id_by_tuple; -use crate::executor::ir::ExecutionPlan; +use crate::executor::ir::{ConnectionType, ExecutionPlan, QueryType}; use crate::executor::lru::{LRUCache, DEFAULT_CAPACITY}; +use crate::executor::protocol::Binary; use crate::executor::result::ProducerResult; use crate::executor::vtable::VirtualTable; use crate::executor::Cache; @@ -35,7 +36,7 @@ use super::{Metadata, QueryCache}; pub struct RouterConfigurationMock { functions: HashMap<String, Function>, tables: HashMap<String, Table>, - bucket_count: usize, + bucket_count: u64, sharding_column: String, } @@ -89,10 +90,10 @@ impl RouterConfigurationMock { #[allow(clippy::too_many_lines)] #[must_use] pub fn new() -> Self { - let name_bucket_id = normalize_name_from_sql("bucket_id"); - let fn_bucket_id = Function::new_stable(name_bucket_id.clone()); + let name_func = normalize_name_from_sql("func"); + let fn_func = Function::new_stable(name_func.clone()); let mut functions = HashMap::new(); - functions.insert(name_bucket_id, fn_bucket_id); + functions.insert(name_func, fn_func); let mut tables = HashMap::new(); @@ -313,6 +314,82 @@ impl QueryCache for RouterRuntimeMock { } } +impl Vshard for RouterRuntimeMock { + fn exec_ir_on_all( + &self, + _required: Binary, + _optional: Binary, + _query_type: QueryType, + _conn_type: ConnectionType, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Runtime, + Some("exec_ir_on_all is not supported for the mock runtime".to_string()), + )) + } + + fn bucket_count(&self) -> u64 { + self.metadata.borrow().bucket_count + } + + fn get_random_bucket(&self) -> Buckets { + get_random_bucket(self) + } + + fn determine_bucket_id(&self, s: &[&Value]) -> u64 { + bucket_id_by_tuple(s, self.bucket_count()) + } + + fn exec_ir_on_some( + &self, + _sub_plan: ExecutionPlan, + _buckets: &Buckets, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Runtime, + Some("exec_ir_on_some is not supported for the mock runtime".to_string()), + )) + } +} + +impl Vshard for &RouterRuntimeMock { + fn exec_ir_on_all( + &self, + _required: Binary, + _optional: Binary, + _query_type: QueryType, + _conn_type: ConnectionType, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Runtime, + Some("exec_ir_on_all is not supported for the mock runtime".to_string()), + )) + } + + fn bucket_count(&self) -> u64 { + self.metadata.borrow().bucket_count + } + + fn get_random_bucket(&self) -> Buckets { + get_random_bucket(self) + } + + fn determine_bucket_id(&self, s: &[&Value]) -> u64 { + bucket_id_by_tuple(s, self.bucket_count()) + } + + fn exec_ir_on_some( + &self, + _sub_plan: ExecutionPlan, + _buckets: &Buckets, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Runtime, + Some("exec_ir_on_some is not supported for the mock runtime".to_string()), + )) + } +} + impl Default for RouterRuntimeMock { fn default() -> Self { Self::new() @@ -666,10 +743,6 @@ impl Router for RouterRuntimeMock { ) -> Result<Vec<&'rec Value>, SbroadError> { sharding_keys_from_tuple(&*self.metadata.borrow(), &space, rec) } - - fn determine_bucket_id(&self, s: &[&Value]) -> u64 { - bucket_id_by_tuple(s, self.metadata.borrow().bucket_count) - } } impl Statistics for RouterRuntimeMock { diff --git a/sbroad-core/src/executor/hash.rs b/sbroad-core/src/executor/hash.rs index d6652d3eb5..3625b8d820 100644 --- a/sbroad-core/src/executor/hash.rs +++ b/sbroad-core/src/executor/hash.rs @@ -7,14 +7,14 @@ pub trait ToHashString { #[must_use] /// A simple function to calculate the bucket id from a string slice. /// `(MurMur3 hash at str) % bucket_count + 1` -pub fn str_to_bucket_id(s: &str, bucket_count: usize) -> u64 { +pub fn str_to_bucket_id(s: &str, bucket_count: u64) -> u64 { let mut hasher = Murmur3Hasher::default(); hasher.write(s.as_bytes()); - u64::from(hasher.finish()) % bucket_count as u64 + 1 + u64::from(hasher.finish()) % bucket_count + 1 } #[must_use] -pub fn bucket_id_by_tuple<T>(tuple: &[&T], bucket_count: usize) -> u64 +pub fn bucket_id_by_tuple<T>(tuple: &[&T], bucket_count: u64) -> u64 where T: ToHashString, { diff --git a/sbroad-core/src/executor/ir.rs b/sbroad-core/src/executor/ir.rs index f2aa1f104f..9f181081fc 100644 --- a/sbroad-core/src/executor/ir.rs +++ b/sbroad-core/src/executor/ir.rs @@ -5,6 +5,7 @@ use ahash::AHashMap; use serde::{Deserialize, Serialize}; use crate::errors::{Action, Entity, SbroadError}; +use crate::executor::engine::Vshard; use crate::executor::vtable::{VirtualTable, VirtualTableMap}; use crate::ir::expression::Expression; use crate::ir::operator::Relational; @@ -12,6 +13,8 @@ use crate::ir::relation::SpaceEngine; use crate::ir::transformation::redistribution::MotionPolicy; use crate::ir::tree::traversal::PostOrder; use crate::ir::{Node, Plan}; +use crate::otm::child_span; +use sbroad_proc::otm_child_span; /// Query type (used to parse the returned results). #[derive(Debug, PartialEq, Eq, Serialize, Deserialize, Clone)] @@ -98,6 +101,47 @@ impl ExecutionPlan { )) } + /// Add materialize motion result to translation map of virtual tables + /// + /// # Errors + /// - invalid motion node + #[otm_child_span("query.motion.add")] + pub fn set_motion_vtable( + &mut self, + motion_id: usize, + vtable: VirtualTable, + runtime: &impl Vshard, + ) -> Result<(), SbroadError> { + let mut vtable = vtable; + let policy = if let Relational::Motion { policy, .. } = + self.get_ir_plan().get_relation_node(motion_id)? + { + policy.clone() + } else { + return Err(SbroadError::Invalid( + Entity::Node, + Some("invalid motion node".to_string()), + )); + }; + match policy { + MotionPolicy::Segment(shard_key) | MotionPolicy::LocalSegment(shard_key) => { + vtable.reshard(&shard_key, runtime)?; + } + MotionPolicy::Full | MotionPolicy::Local => {} + } + + let need_init = self.get_vtables().is_none(); + if need_init { + self.set_vtables(HashMap::new()); + } + + if let Some(vtables) = self.get_mut_vtables() { + vtables.insert(motion_id, Rc::new(vtable)); + } + + Ok(()) + } + #[must_use] pub fn has_segmented_tables(&self) -> bool { self.vtables.as_ref().map_or(false, |vtable_map| { @@ -181,7 +225,7 @@ impl ExecutionPlan { if children.len() != 1 { return Err(SbroadError::UnexpectedNumberOfValues(format!( - "Motion node ({}) must have once child only (actual {})", + "Motion node ({}) must have a single child only (actual {})", node_id, children.len() ))); @@ -217,7 +261,7 @@ impl ExecutionPlan { if children.len() != 1 { return Err(SbroadError::UnexpectedNumberOfValues(format!( - "Sub query node ({}) must have once child only (actual {})", + "Sub query node ({}) must have a single child only (actual {})", node_id, children.len() ))); @@ -295,13 +339,20 @@ impl ExecutionPlan { })?; } - if let Relational::Motion { children, .. } = rel { + if let Relational::Motion { + children, policy, .. + } = rel + { if let Some(vtable) = self.get_vtables().map_or_else(|| None, |v| v.get(&node_id)) { new_vtables.insert(next_id, Rc::clone(vtable)); } - *children = Vec::new(); + // We should not remove the child of a local motion node. + // The subtree is needed to complie the SQL on the storage. + if !policy.is_local() { + *children = Vec::new(); + } } if let Some(children) = rel.mut_children() { diff --git a/sbroad-core/src/executor/result.rs b/sbroad-core/src/executor/result.rs index 6898d417d4..7cab30d638 100644 --- a/sbroad-core/src/executor/result.rs +++ b/sbroad-core/src/executor/result.rs @@ -2,6 +2,7 @@ use core::fmt::Debug; use serde::ser::{Serialize, SerializeMap, Serializer}; use serde::Deserialize; use tarantool::tlua::{self, LuaRead}; +use tarantool::tuple::Encode; use crate::debug; use crate::errors::{Entity, SbroadError}; @@ -92,17 +93,14 @@ impl ProducerResult { /// # Errors /// - convert to virtual table error pub fn as_virtual_table( - &self, + &mut self, column_names: Vec<String>, possibly_incorrect_types: bool, ) -> Result<VirtualTable, SbroadError> { let mut vtable = VirtualTable::new(); - for encoded_tuple in &self.rows { - let tuple: Vec<Value> = encoded_tuple - .iter() - .map(|v| Value::from(v.clone())) - .collect(); + for mut encoded_tuple in self.rows.drain(..) { + let tuple: Vec<Value> = encoded_tuple.drain(..).map(Value::from).collect(); vtable.add_tuple(tuple); } @@ -156,7 +154,7 @@ impl Serialize for ProducerResult { /// Results of query execution for `INSERT`. #[allow(clippy::module_name_repetitions)] -#[derive(LuaRead, Debug, PartialEq, Eq, Clone)] +#[derive(Debug, PartialEq, Eq, Clone)] pub struct ConsumerResult { pub row_count: u64, } @@ -187,6 +185,8 @@ impl Serialize for ConsumerResult { } } +impl Encode for ConsumerResult {} + impl Plan { /// Checks if the plan contains a `Values` node. /// diff --git a/sbroad-core/src/executor/result/tests.rs b/sbroad-core/src/executor/result/tests.rs index 4622f8911f..2bb10a9ea9 100644 --- a/sbroad-core/src/executor/result/tests.rs +++ b/sbroad-core/src/executor/result/tests.rs @@ -49,7 +49,7 @@ fn box_execute_result_serialize() { #[test] fn convert_to_vtable() { let col_names = ["id", "name", "count", "price"]; - let r = ProducerResult { + let mut r = ProducerResult { metadata: vec![ MetadataColumn::new(col_names[0].into(), "integer".into()), MetadataColumn::new(col_names[1].into(), "string".into()), diff --git a/sbroad-core/src/executor/tests.rs b/sbroad-core/src/executor/tests.rs index 97915daeab..d991369541 100644 --- a/sbroad-core/src/executor/tests.rs +++ b/sbroad-core/src/executor/tests.rs @@ -144,7 +144,7 @@ fn linker_test() { let mut virtual_table = virtual_table_23(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } query .coordinator @@ -228,7 +228,7 @@ fn union_linker_test() { let mut virtual_table = virtual_table_23(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } query .coordinator @@ -332,7 +332,7 @@ WHERE "t3"."id" = 2 AND "t8"."identification_number" = 2"#; virtual_table.set_alias("\"t8\"").unwrap(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } query .coordinator @@ -419,7 +419,7 @@ fn join_linker2_test() { virtual_table.set_alias("\"t2\"").unwrap(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } query @@ -491,7 +491,7 @@ fn join_linker3_test() { virtual_table.set_alias("\"t2\"").unwrap(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } query @@ -558,7 +558,7 @@ fn join_linker4_test() { if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_t2_id) { - query.reshard_vtable(&mut virtual_t2, key).unwrap(); + virtual_t2.reshard(key, &query.coordinator).unwrap(); } query .coordinator @@ -583,7 +583,7 @@ fn join_linker4_test() { if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_sq_id) { - query.reshard_vtable(&mut virtual_sq, key).unwrap(); + virtual_sq.reshard(key, &query.coordinator).unwrap(); } query .coordinator @@ -669,7 +669,7 @@ on q."f" = "t1"."a""#; if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_t2_id) { - query.reshard_vtable(&mut virtual_t2, key).unwrap(); + virtual_t2.reshard(key, &query.coordinator).unwrap(); } query .coordinator @@ -698,7 +698,7 @@ on q."f" = "t1"."a""#; if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_sq_id) { - query.reshard_vtable(&mut virtual_sq, key).unwrap(); + virtual_sq.reshard(key, &query.coordinator).unwrap(); } query .coordinator @@ -749,7 +749,7 @@ fn anonymous_col_index_test() { let mut virtual_t1 = virtual_table_23(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion1_id) { - query.reshard_vtable(&mut virtual_t1, key).unwrap(); + virtual_t1.reshard(key, &query.coordinator).unwrap(); } query .coordinator @@ -765,7 +765,7 @@ fn anonymous_col_index_test() { let mut virtual_t2 = virtual_table_23(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion2_id) { - query.reshard_vtable(&mut virtual_t2, key).unwrap(); + virtual_t2.reshard(key, &query.coordinator).unwrap(); } query .coordinator @@ -941,24 +941,26 @@ fn insert1_test() { EncodedValue::String(format!("Execute query on a bucket [{bucket1}]")), EncodedValue::String(String::from(PatternWithParams::new( format!( - "{} {} {}", - r#"INSERT INTO "t" ("b", "bucket_id")"#, - r#"SELECT COL_0, bucket_id (coalesce (CAST (? as string), ?) || coalesce (CAST (COL_0 as string), ?)) FROM"#, - r#"(SELECT CAST ("t"."a" as unsigned) as COL_0 FROM ((SELECT "a" FROM "TMP_test_142") as "t"))"#, + "{} {}", + r#"INSERT INTO "t" ("b")"#, + // We don't generate SQL for insertion, so in this test we just check that + // correct virtual table is used. + r#"(SELECT "a" FROM "TMP_test_113") as "t""#, ), - vec![Value::Null, Value::String("NULL".into()), Value::String("NULL".into())], + vec![], ))), ], vec![ EncodedValue::String(format!("Execute query on a bucket [{bucket2}]")), EncodedValue::String(String::from(PatternWithParams::new( format!( - "{} {} {}", - r#"INSERT INTO "t" ("b", "bucket_id")"#, - r#"SELECT COL_0, bucket_id (coalesce (CAST (? as string), ?) || coalesce (CAST (COL_0 as string), ?)) FROM"#, - r#"(SELECT CAST ("t"."a" as unsigned) as COL_0 FROM ((SELECT "a" FROM "TMP_test_142") as "t"))"#, + "{} {}", + r#"INSERT INTO "t" ("b")"#, + // We don't generate SQL for insertion, so in this test we just check that + // correct virtual table is used. + r#"(SELECT "a" FROM "TMP_test_113") as "t""#, ), - vec![Value::Null, Value::String("NULL".into()), Value::String("NULL".into())], + vec![], ))), ], ]); @@ -989,13 +991,13 @@ fn insert2_test() { EncodedValue::String(format!("Execute query on a bucket [{bucket}]")), EncodedValue::String(String::from(PatternWithParams::new( format!( - "{} {} {} {}", - r#"INSERT INTO "t" ("a", "b", "bucket_id")"#, - r#"SELECT COL_0, COL_1, bucket_id (coalesce (CAST (COL_0 as string), ?) || coalesce (CAST (COL_1 as string), ?)) FROM"#, - r#"(SELECT CAST ("t"."a" as unsigned) as COL_0, CAST ("t"."b" as unsigned) as COL_1 FROM"#, - r#"(SELECT "t"."a", "t"."b" FROM "t" WHERE ("t"."a") = (?) and ("t"."b") = (?)))"#, + "{} {}", + // We don't generate SQL for insertion, so we just check a storage plan + // and buckets this way. + r#"INSERT INTO "t" ("a", "b")"#, + r#"SELECT "t"."a", "t"."b" FROM "t" WHERE ("t"."a") = (?) and ("t"."b") = (?) "#, ), - vec![Value::from("NULL"), Value::from("NULL"), Value::from(1_u64), Value::from(2_u64)], + vec![Value::from(1_u64), Value::from(2_u64)], ))), ]]); assert_eq!(expected, result); @@ -1058,26 +1060,26 @@ fn insert3_test() { EncodedValue::String(format!("Execute query on a bucket [{bucket1}]")), EncodedValue::String(String::from(PatternWithParams::new( format!( - "{} {} {} {}", - r#"INSERT INTO "t" ("b", "a", "bucket_id")"#, - r#"SELECT COL_0, COL_1, bucket_id (coalesce (CAST (COL_1 as string), ?) || coalesce (CAST (COL_0 as string), ?)) FROM"#, - r#"(SELECT CAST ("t"."a" as unsigned) as COL_0, CAST ("t"."b" as unsigned) as COL_1 FROM"#, - r#"((SELECT "a","b" FROM "TMP_test_155") as "t"))"#, + "{} {}", + r#"INSERT INTO "t" ("b", "a")"#, + // We don't generate SQL for insertion, so in this test we just check that + // correct virtual table is used. + r#"(SELECT "a","b" FROM "TMP_test_117") as "t""#, ), - vec![Value::from("NULL"), Value::from("NULL")], + vec![], ))), ], vec![ EncodedValue::String(format!("Execute query on a bucket [{bucket2}]")), EncodedValue::String(String::from(PatternWithParams::new( format!( - "{} {} {} {}", - r#"INSERT INTO "t" ("b", "a", "bucket_id")"#, - r#"SELECT COL_0, COL_1, bucket_id (coalesce (CAST (COL_1 as string), ?) || coalesce (CAST (COL_0 as string), ?)) FROM"#, - r#"(SELECT CAST ("t"."a" as unsigned) as COL_0, CAST ("t"."b" as unsigned) as COL_1 FROM"#, - r#"((SELECT "a","b" FROM "TMP_test_155") as "t"))"#, + "{} {}", + r#"INSERT INTO "t" ("b", "a")"#, + // We don't generate SQL for insertion, so in this test we just check that + // correct virtual table is used. + r#"(SELECT "a","b" FROM "TMP_test_117") as "t""#, ), - vec![Value::from("NULL"), Value::from("NULL")], + vec![], ))), ], ]); @@ -1109,13 +1111,13 @@ fn insert4_test() { EncodedValue::String(format!("Execute query on a bucket [{bucket}]")), EncodedValue::String(String::from(PatternWithParams::new( format!( - "{} {} {} {}", - r#"INSERT INTO "t" ("b", "a", "bucket_id")"#, - r#"SELECT COL_0, COL_1, bucket_id (coalesce (CAST (COL_1 as string), ?) || coalesce (CAST (COL_0 as string), ?)) FROM"#, - r#"(SELECT CAST ("t"."b" as unsigned) as COL_0, CAST ("t"."a" as unsigned) as COL_1 FROM"#, - r#"(SELECT "t"."b", "t"."a" FROM "t" WHERE ("t"."a") = (?) and ("t"."b") = (?)))"#, + "{} {}", + // We don't generate SQL for insertion, so we just check a storage plan + // and buckets this way. + r#"INSERT INTO "t" ("b", "a")"#, + r#"SELECT "t"."b", "t"."a" FROM "t" WHERE ("t"."a") = (?) and ("t"."b") = (?) "#, ), - vec![Value::from("NULL"), Value::from("NULL"), Value::from(1_u64), Value::from(2_u64)], + vec![Value::from(1_u64), Value::from(2_u64)], ))), ]]); assert_eq!(expected, result); @@ -1173,13 +1175,13 @@ fn insert5_test() { EncodedValue::String(format!("Execute query on a bucket [{bucket}]")), EncodedValue::String(String::from(PatternWithParams::new( format!( - "{} {} {} {}", - r#"INSERT INTO "t" ("b", "a", "bucket_id")"#, - r#"SELECT COL_0, COL_1, bucket_id (coalesce (CAST (COL_1 as string), ?) || coalesce (CAST (COL_0 as string), ?)) FROM"#, - r#"(SELECT CAST ("COL_1" as unsigned) as COL_0, CAST ("COL_2" as unsigned) as COL_1 FROM"#, - r#"((SELECT "COL_1","COL_2" FROM "TMP_test_125") as "t"))"#, + "{} {}", + r#"INSERT INTO "t" ("b", "a")"#, + // We don't generate SQL for insertion, so in this test we just check that + // correct virtual table is used. + r#"(SELECT "COL_1","COL_2" FROM "TMP_test_87") as "t""#, ), - vec![Value::from("NULL"), Value::from("NULL")], + vec![], ))), ]]); assert_eq!(expected, result); @@ -1192,79 +1194,21 @@ fn insert6_test() { let coordinator = RouterRuntimeMock::new(); let mut query = Query::new(&coordinator, sql, vec![]).unwrap(); - let motion_id = *query - .exec_plan - .get_ir_plan() - .clone_slices() - .slice(0) - .unwrap() - .position(0) - .unwrap(); - - let mut virtual_table = VirtualTable::new(); - virtual_table.add_column(Column { - name: "COLUMN_5".into(), - r#type: Type::Integer, - role: ColumnRole::User, - }); - virtual_table.add_column(Column { - name: "COLUMN_6".into(), - r#type: Type::Integer, - role: ColumnRole::User, - }); - virtual_table.add_tuple(vec![Value::from(1_u64), Value::from(2_u64)]); - virtual_table.add_tuple(vec![Value::from(1_u64), Value::from(2_u64)]); - virtual_table.add_tuple(vec![Value::from(3_u64), Value::from(4_u64)]); - virtual_table.set_alias("\"t\"").unwrap(); - - query - .coordinator - .add_virtual_table(motion_id, virtual_table); - let result = *query + let _ = *query .dispatch() .unwrap() .downcast::<ProducerResult>() .unwrap(); - let mut expected = ProducerResult::new(); - let param1 = Value::from(1_u64); let param2 = Value::from(2_u64); let bucket1 = query.coordinator.determine_bucket_id(&[¶m1, ¶m2]); + assert_eq!(550, bucket1); let param3 = Value::from(3_u64); let param4 = Value::from(4_u64); let bucket2 = query.coordinator.determine_bucket_id(&[¶m3, ¶m4]); - - expected.rows.extend(vec![ - vec![ - EncodedValue::String(format!("Execute query on a bucket [{bucket1}]")), - EncodedValue::String(String::from(PatternWithParams::new( - format!( - "{} {} {} {}", - r#"INSERT INTO "t" ("a", "b", "bucket_id")"#, - r#"SELECT COL_0, COL_1, bucket_id (coalesce (CAST (COL_0 as string), ?) || coalesce (CAST (COL_1 as string), ?)) FROM"#, - r#"(SELECT CAST ("COLUMN_5" as unsigned) as COL_0, CAST ("COLUMN_6" as unsigned) as COL_1 FROM"#, - r#"((SELECT "COLUMN_5","COLUMN_6" FROM "TMP_test_94") as "t"))"#, - ), - vec![Value::from("NULL"), Value::from("NULL")], - ))), - ], - vec![ - EncodedValue::String(format!("Execute query on a bucket [{bucket2}]")), - EncodedValue::String(String::from(PatternWithParams::new( - format!( - "{} {} {} {}", - r#"INSERT INTO "t" ("a", "b", "bucket_id")"#, - r#"SELECT COL_0, COL_1, bucket_id (coalesce (CAST (COL_0 as string), ?) || coalesce (CAST (COL_1 as string), ?)) FROM"#, - r#"(SELECT CAST ("COLUMN_5" as unsigned) as COL_0, CAST ("COLUMN_6" as unsigned) as COL_1 FROM"#, - r#"((SELECT "COLUMN_5","COLUMN_6" FROM "TMP_test_94") as "t"))"#, - ), - vec![Value::from("NULL"), Value::from("NULL")], - ))), - ], - ]); - assert_eq!(expected, result); + assert_eq!(8906, bucket2); } #[test] @@ -1341,13 +1285,12 @@ fn insert8_test() { String::from( PatternWithParams::new( format!( - "{} {} {} {} {}", - r#"INSERT INTO "hash_testing" ("identification_number", "product_code", "product_units", "sys_op", "bucket_id")"#, - r#"SELECT COL_0, COL_1, COL_2, COL_3, bucket_id (coalesce (CAST (COL_0 as string), ?) || coalesce (CAST (COL_1 as string), ?)) FROM"#, - r#"(SELECT CAST ("hash_single_testing"."identification_number" as int) as COL_0, CAST ("hash_single_testing"."product_code" as string) as COL_1,"#, - r#"CAST ("hash_single_testing"."product_units" as bool) as COL_2, CAST ("hash_single_testing"."sys_op" as unsigned) as COL_3 FROM"#, - r#"((SELECT "identification_number","product_code","product_units","sys_op" FROM "TMP_test_114") as "hash_single_testing"))"#, - ), vec![Value::from("NULL"), Value::from("NULL")], + "{} {}", + r#"INSERT INTO "hash_testing" ("identification_number", "product_code", "product_units", "sys_op")"#, + // We don't generate SQL for insertion, so in this test we just check that + // correct virtual table is used. + r#"(SELECT "identification_number","product_code","product_units","sys_op" FROM "TMP_test_56") as "hash_single_testing""#, + ), vec![], ) ) ), @@ -1367,57 +1310,18 @@ fn insert9_test() { vec![Value::from(1_u64), Value::from(2_u64)], ) .unwrap(); - let motion_id = *query - .exec_plan - .get_ir_plan() - .clone_slices() - .slice(0) - .unwrap() - .position(0) - .unwrap(); - let mut virtual_table = VirtualTable::new(); - virtual_table.add_column(Column { - name: "COLUMN_1".into(), - r#type: Type::Integer, - role: ColumnRole::User, - }); - virtual_table.add_column(Column { - name: "COLUMN_2".into(), - r#type: Type::Integer, - role: ColumnRole::User, - }); - virtual_table.add_tuple(vec![Value::from(1_u64), Value::from(2_u64)]); - virtual_table.set_alias("\"t\"").unwrap(); - - query - .coordinator - .add_virtual_table(motion_id, virtual_table); - let result = *query + let _ = *query .dispatch() .unwrap() .downcast::<ProducerResult>() .unwrap(); - let mut expected = ProducerResult::new(); - let param1 = Value::from(1_u64); let param2 = Value::from(2_u64); let bucket1 = query.coordinator.determine_bucket_id(&[¶m1, ¶m2]); - expected.rows.extend(vec![vec![ - EncodedValue::String(format!("Execute query on a bucket [{bucket1}]")), - EncodedValue::String(String::from(PatternWithParams::new( - format!( - "{} {} {}", - r#"INSERT INTO "t" ("a", "b", "bucket_id")"#, - r#"SELECT COL_0, COL_1, bucket_id (coalesce (CAST (COL_0 as string), ?) || coalesce (CAST (COL_1 as string), ?)) FROM"#, - r#"(SELECT CAST ("COLUMN_1" as unsigned) as COL_0, CAST ("COLUMN_2" as unsigned) as COL_1 FROM ((SELECT "COLUMN_1","COLUMN_2" FROM "TMP_test_82") as "t"))"#, - ), - vec![Value::from("NULL"), Value::from("NULL")], - ))), - ]]); - assert_eq!(expected, result); + assert_eq!(550, bucket1); } /// Helper function to create a "test" virtual table. diff --git a/sbroad-core/src/executor/tests/bucket_id.rs b/sbroad-core/src/executor/tests/bucket_id.rs index 82f2c001be..ef19271f03 100644 --- a/sbroad-core/src/executor/tests/bucket_id.rs +++ b/sbroad-core/src/executor/tests/bucket_id.rs @@ -65,7 +65,7 @@ fn bucket2_test() { #[test] fn bucket3_test() { - let sql = r#"SELECT *, bucket_id('111') FROM "t1""#; + let sql = r#"SELECT *, func('111') FROM "t1""#; let coordinator = RouterRuntimeMock::new(); let mut query = Query::new(&coordinator, sql, vec![]).unwrap(); @@ -80,7 +80,7 @@ fn bucket3_test() { expected.rows.push(vec![ EncodedValue::String("Execute query on all buckets".to_string()), EncodedValue::String(String::from(PatternWithParams::new( - r#"SELECT "t1"."a", "t1"."b", "BUCKET_ID" (?) as "COL_1" FROM "t1""#.to_string(), + r#"SELECT "t1"."a", "t1"."b", "FUNC" (?) as "COL_1" FROM "t1""#.to_string(), vec![Value::from("111".to_string())], ))), ]); diff --git a/sbroad-core/src/executor/tests/cast.rs b/sbroad-core/src/executor/tests/cast.rs index 842da66277..6fb87f2792 100644 --- a/sbroad-core/src/executor/tests/cast.rs +++ b/sbroad-core/src/executor/tests/cast.rs @@ -121,8 +121,8 @@ fn cast13_test() { #[test] fn cast14_test() { broadcast_check( - r#"SELECT CAST(bucket_id("a") as varchar(100)) FROM "t1""#, - r#"SELECT CAST ("BUCKET_ID" ("t1"."a") as varchar(100)) as "COL_1" FROM "t1""#, + r#"SELECT CAST(func("a") as varchar(100)) FROM "t1""#, + r#"SELECT CAST ("FUNC" ("t1"."a") as varchar(100)) as "COL_1" FROM "t1""#, vec![], ); } diff --git a/sbroad-core/src/executor/tests/concat.rs b/sbroad-core/src/executor/tests/concat.rs index 8f48d040a3..d5be6daf14 100644 --- a/sbroad-core/src/executor/tests/concat.rs +++ b/sbroad-core/src/executor/tests/concat.rs @@ -13,8 +13,8 @@ fn concat1_test() { #[test] fn concat2_test() { broadcast_check( - r#"SELECT bucket_id('hello') || CAST(42 as string) FROM "t1""#, - r#"SELECT ("BUCKET_ID" (?)) || (CAST (? as string)) as "COL_1" FROM "t1""#, + r#"SELECT func('hello') || CAST(42 as string) FROM "t1""#, + r#"SELECT ("FUNC" (?)) || (CAST (? as string)) as "COL_1" FROM "t1""#, vec![Value::from("hello"), Value::from(42_u64)], ); } @@ -31,8 +31,8 @@ fn concat3_test() { #[test] fn concat4_test() { broadcast_check( - r#"SELECT "a" FROM "t1" WHERE "a" || 'a' = CAST(42 as string) || BUCKET_ID('b') || 'a'"#, - r#"SELECT "t1"."a" FROM "t1" WHERE (("t1"."a") || (?)) = ((CAST (? as string)) || (("BUCKET_ID" (?)) || (?)))"#, + r#"SELECT "a" FROM "t1" WHERE "a" || 'a' = CAST(42 as string) || FUNC('b') || 'a'"#, + r#"SELECT "t1"."a" FROM "t1" WHERE (("t1"."a") || (?)) = ((CAST (? as string)) || (("FUNC" (?)) || (?)))"#, vec![ Value::from("a"), Value::from(42_u64), diff --git a/sbroad-core/src/executor/tests/empty_motion.rs b/sbroad-core/src/executor/tests/empty_motion.rs index 02a48da109..69fb287258 100644 --- a/sbroad-core/src/executor/tests/empty_motion.rs +++ b/sbroad-core/src/executor/tests/empty_motion.rs @@ -34,7 +34,7 @@ fn empty_motion1_test() { let mut virtual_t1 = t2_empty(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion1_id) { - query.reshard_vtable(&mut virtual_t1, key).unwrap(); + virtual_t1.reshard(key, &query.coordinator).unwrap(); } query.coordinator.add_virtual_table(motion1_id, virtual_t1); let motion2_id = *query @@ -48,7 +48,7 @@ fn empty_motion1_test() { let mut virtual_t2 = t2_empty(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion2_id) { - query.reshard_vtable(&mut virtual_t2, key).unwrap(); + virtual_t2.reshard(key, &query.coordinator).unwrap(); } query.coordinator.add_virtual_table(motion2_id, virtual_t2); diff --git a/sbroad-core/src/executor/tests/subtree.rs b/sbroad-core/src/executor/tests/subtree.rs index 88ae99639a..a7d27604ba 100644 --- a/sbroad-core/src/executor/tests/subtree.rs +++ b/sbroad-core/src/executor/tests/subtree.rs @@ -25,7 +25,7 @@ fn exec_plan_subtree_test() { let mut virtual_table = virtual_table_23(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } let mut vtables: HashMap<usize, Rc<VirtualTable>> = HashMap::new(); vtables.insert(motion_id, Rc::new(virtual_table)); @@ -89,7 +89,7 @@ fn exec_plan_subtree_two_stage_groupby_test() { if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } let mut vtables: HashMap<usize, Rc<VirtualTable>> = HashMap::new(); @@ -168,7 +168,7 @@ fn exec_plan_subtree_two_stage_groupby_test_2() { virtual_table.set_alias("").unwrap(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } let mut vtables: HashMap<usize, Rc<VirtualTable>> = HashMap::new(); @@ -273,7 +273,7 @@ fn exec_plan_subtree_aggregates() { virtual_table.set_alias("").unwrap(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } let mut vtables: HashMap<usize, Rc<VirtualTable>> = HashMap::new(); @@ -360,7 +360,7 @@ fn exec_plan_subtree_aggregates_no_groupby() { virtual_table.set_alias("").unwrap(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } let mut vtables: HashMap<usize, Rc<VirtualTable>> = HashMap::new(); @@ -427,7 +427,7 @@ fn exec_plan_subtree_count_asterisk() { virtual_table.set_alias("").unwrap(); if let MotionPolicy::Segment(key) = get_motion_policy(query.exec_plan.get_ir_plan(), motion_id) { - query.reshard_vtable(&mut virtual_table, key).unwrap(); + virtual_table.reshard(key, &query.coordinator).unwrap(); } let mut vtables: HashMap<usize, Rc<VirtualTable>> = HashMap::new(); diff --git a/sbroad-core/src/executor/vtable.rs b/sbroad-core/src/executor/vtable.rs index ba36196678..06ffd59ac3 100644 --- a/sbroad-core/src/executor/vtable.rs +++ b/sbroad-core/src/executor/vtable.rs @@ -1,4 +1,4 @@ -use std::collections::{HashMap, HashSet}; +use std::collections::{hash_map::Entry, HashMap, HashSet}; use std::fmt::{Display, Formatter}; use std::rc::Rc; use std::vec; @@ -6,7 +6,7 @@ use std::vec; use serde::{Deserialize, Serialize}; use crate::errors::{Entity, SbroadError}; -use crate::executor::bucket::Buckets; +use crate::executor::{bucket::Buckets, Vshard}; use crate::ir::relation::Column; use crate::ir::transformation::redistribution::{MotionKey, Target}; use crate::ir::value::Value; @@ -106,7 +106,7 @@ impl VirtualTable { /// Gets a mutable virtual table tuples list #[must_use] - pub fn get_mut_tuples(&mut self) -> &mut [VTableTuple] { + pub fn get_mut_tuples(&mut self) -> &mut Vec<VTableTuple> { &mut self.tuples } @@ -139,6 +139,12 @@ impl VirtualTable { &self.index.value } + /// Gets virtual table mutable index + #[must_use] + pub fn get_mut_index(&mut self) -> &mut HashMap<u64, Vec<usize>> { + &mut self.index.value + } + /// Set vtable index pub fn set_index(&mut self, index: HashMap<u64, Vec<usize>>) { self.index = index.into(); @@ -230,15 +236,64 @@ impl VirtualTable { for bucket in buckets { if let Some(positions) = self.index.value.get(bucket) { + let mut new_positions: Vec<usize> = Vec::with_capacity(positions.len()); for pos in positions { result.tuples.push(self.tuples[*pos].clone()); + new_positions.push(result.tuples.len() - 1); } - result.index.value.insert(*bucket, positions.clone()); + result.index.value.insert(*bucket, new_positions); } } result } + + /// Reshard a virtual table (i.e. build a bucket index). + /// + /// # Errors + /// - Motion key is invalid. + pub fn reshard( + &mut self, + sharding_key: &MotionKey, + runtime: &impl Vshard, + ) -> Result<(), SbroadError> { + self.set_motion_key(sharding_key); + + let mut index: HashMap<u64, Vec<usize>> = HashMap::new(); + for (pos, tuple) in self.get_tuples().iter().enumerate() { + let mut shard_key_tuple: Vec<&Value> = Vec::new(); + for target in &sharding_key.targets { + match target { + Target::Reference(col_idx) => { + let part = tuple.get(*col_idx).ok_or_else(|| { + SbroadError::NotFound( + Entity::DistributionKey, + format!( + "failed to find a distribution key column {pos} in the tuple {tuple:?}." + ), + ) + })?; + shard_key_tuple.push(part); + } + Target::Value(ref value) => { + shard_key_tuple.push(value); + } + } + } + let bucket_id = runtime.determine_bucket_id(&shard_key_tuple); + match index.entry(bucket_id) { + Entry::Vacant(entry) => { + entry.insert(vec![pos]); + } + Entry::Occupied(entry) => { + entry.into_mut().push(pos); + } + } + } + + self.set_index(index); + Ok(()) + } } #[derive(Clone, Debug, Default, PartialEq, Eq, Deserialize, Serialize)] diff --git a/sbroad-core/src/frontend/sql.rs b/sbroad-core/src/frontend/sql.rs index 9ea584f2e7..b7e094ec30 100644 --- a/sbroad-core/src/frontend/sql.rs +++ b/sbroad-core/src/frontend/sql.rs @@ -213,9 +213,8 @@ impl Ast for AbstractSyntaxTree { let op_node = self.nodes.get_node(*ast_op_id)?; let op = Arithmetic::from_node_type(&op_node.rule)?; - let op_id = plan.add_arithmetic_to_plan(plan_left_id, op, plan_right_id, false)?; - Ok(op_id) + Ok::<usize, SbroadError>(op_id) }; for (_, id) in dft_post.iter(top) { diff --git a/sbroad-core/src/frontend/sql/ir/tests.rs b/sbroad-core/src/frontend/sql/ir/tests.rs index 0313672519..f07456731f 100644 --- a/sbroad-core/src/frontend/sql/ir/tests.rs +++ b/sbroad-core/src/frontend/sql/ir/tests.rs @@ -232,13 +232,9 @@ fn front_sql10() { let expected_explain = String::from( r#"insert "t" - projection (COL_0 -> COL_0, COL_1 -> COL_1, COL_2 -> COL_2, COL_3 -> COL_3, bucket_id((coalesce(('NULL', COL_0::string)) || coalesce(('NULL', COL_1::string))))) - scan - projection ("COLUMN_1"::unsigned -> COL_0, "COLUMN_2"::unsigned -> COL_1, "COLUMN_3"::unsigned -> COL_2, "COLUMN_4"::unsigned -> COL_3) - scan - motion [policy: segment([ref("COLUMN_1"), ref("COLUMN_2")])] - values - value row (data=ROW(1, 2, 3, 4)) + motion [policy: local segment([ref("COLUMN_1"), ref("COLUMN_2")])] + values + value row (data=ROW(1, 2, 3, 4)) "#, ); @@ -253,13 +249,9 @@ fn front_sql11() { let expected_explain = String::from( r#"insert "t" - projection (COL_0 -> COL_0, COL_1 -> COL_1, bucket_id((coalesce(('NULL', COL_0::string)) || coalesce(('NULL', NULL::string))))) - scan - projection ("COLUMN_1"::unsigned -> COL_0, "COLUMN_2"::unsigned -> COL_1) - scan - motion [policy: segment([ref("COLUMN_1"), value(NULL)])] - values - value row (data=ROW(1, 2)) + motion [policy: local segment([ref("COLUMN_1"), value(NULL)])] + values + value row (data=ROW(1, 2)) "#, ); @@ -274,13 +266,9 @@ fn front_sql14() { let expected_explain = String::from( r#"insert "t" - projection (COL_0 -> COL_0, COL_1 -> COL_1, bucket_id((coalesce(('NULL', COL_0::string)) || coalesce(('NULL', NULL::string))))) - scan - projection ("t"."b"::unsigned -> COL_0, "t"."d"::unsigned -> COL_1) - scan - motion [policy: segment([ref("b"), value(NULL)])] - projection ("t"."b" -> "b", "t"."d" -> "d") - scan "t" + motion [policy: segment([ref("b"), value(NULL)])] + projection ("t"."b" -> "b", "t"."d" -> "d") + scan "t" "#, ); @@ -378,7 +366,7 @@ fn front_sql20() { fn front_sql_exists_subquery_select_from_table() { let input = r#"SELECT "id" FROM "test_space" WHERE EXISTS (SELECT 0 FROM "hash_testing")"#; - let mut plan = sql_to_optimized_ir(input, vec![]); + let plan = sql_to_optimized_ir(input, vec![]); let expected_explain = String::from( r#"projection ("test_space"."id" -> "id") @@ -646,18 +634,14 @@ fn front_sql_groupby_insert() { let plan = sql_to_optimized_ir(input, vec![]); let expected_explain = String::from( r#"insert "t" - projection (COL_0 -> COL_0, COL_1 -> COL_1, bucket_id((coalesce(('NULL', COL_0::string)) || coalesce(('NULL', NULL::string))))) - scan - projection ("b"::unsigned -> COL_0, "d"::unsigned -> COL_1) - scan - motion [policy: segment([ref("b"), value(NULL)])] - projection ("column_24" -> "b", "column_25" -> "d") - group by ("column_24", "column_25") output: ("column_25" -> "column_25", "column_24" -> "column_24") - motion [policy: segment([ref("column_24"), ref("column_25")])] - scan - projection ("t"."d" -> "column_25", "t"."b" -> "column_24") - group by ("t"."b", "t"."d") output: ("t"."a" -> "a", "t"."b" -> "b", "t"."c" -> "c", "t"."d" -> "d", "t"."bucket_id" -> "bucket_id") - scan "t" + motion [policy: segment([ref("b"), value(NULL)])] + projection ("column_24" -> "b", "column_25" -> "d") + group by ("column_24", "column_25") output: ("column_25" -> "column_25", "column_24" -> "column_24") + motion [policy: segment([ref("column_24"), ref("column_25")])] + scan + projection ("t"."d" -> "column_25", "t"."b" -> "column_24") + group by ("t"."b", "t"."d") output: ("t"."a" -> "a", "t"."b" -> "b", "t"."c" -> "c", "t"."d" -> "d", "t"."bucket_id" -> "bucket_id") + scan "t" "#, ); @@ -769,7 +753,7 @@ fn front_sql_invalid_count_asterisk1() { #[test] fn front_sql_aggregates_with_subexpressions() { - let input = r#"SELECT "b", count("a" * "b" + 1), count(bucket_id("a")) FROM "t" + let input = r#"SELECT "b", count("a" * "b" + 1), count(func("a")) FROM "t" group by "b""#; let plan = sql_to_optimized_ir(input, vec![]); @@ -779,7 +763,7 @@ fn front_sql_aggregates_with_subexpressions() { group by ("column_12") output: ("column_12" -> "column_12", "count_39" -> "count_39", "count_35" -> "count_35") motion [policy: segment([ref("column_12")])] scan - projection ("t"."b" -> "column_12", count(("BUCKET_ID"(("t"."a")))) -> "count_39", count((("t"."a") * ("t"."b") + (1))) -> "count_35") + projection ("t"."b" -> "column_12", count(("FUNC"(("t"."a")))) -> "count_39", count((("t"."a") * ("t"."b") + (1))) -> "count_35") group by ("t"."b") output: ("t"."a" -> "a", "t"."b" -> "b", "t"."c" -> "c", "t"."d" -> "d", "t"."bucket_id" -> "bucket_id") scan "t" "#, @@ -1027,16 +1011,12 @@ fn front_sql_insert_single() { let expected_explain = String::from( r#"insert "t" - projection (COL_0 -> COL_0, COL_1 -> COL_1, bucket_id((coalesce(('NULL', COL_0::string)) || coalesce(('NULL', NULL::string))))) - scan - projection ("COL_1"::unsigned -> COL_0, "COL_2"::unsigned -> COL_1) - scan - motion [policy: segment([ref("COL_1"), value(NULL)])] - projection (sum(("sum_25")) -> "COL_1", sum(("count_28")) -> "COL_2") - motion [policy: full] - scan - projection (sum(("t"."b")) -> "sum_25", count(("t"."d")) -> "count_28") - scan "t" + motion [policy: segment([ref("COL_1"), value(NULL)])] + projection (sum(("sum_25")) -> "COL_1", sum(("count_28")) -> "COL_2") + motion [policy: full] + scan + projection (sum(("t"."b")) -> "sum_25", count(("t"."d")) -> "count_28") + scan "t" "#, ); @@ -1230,7 +1210,7 @@ fn front_sql_groupby_with_aggregates() { join (select "g", "e", sum("f") as "f" from "t2" group by "g", "e") as t2 on (t1."a", t2."g") = (t2."e", t1."b")"#; - let mut plan = sql_to_optimized_ir(input, vec![]); + let plan = sql_to_optimized_ir(input, vec![]); let expected_explain = String::from( r#"projection ("T1"."a" -> "a", "T1"."b" -> "b", "T1"."c" -> "c", "T2"."g" -> "g", "T2"."e" -> "e", "T2"."f" -> "f") join on ROW("T1"."a", "T2"."g") = ROW("T2"."e", "T1"."b") diff --git a/sbroad-core/src/frontend/sql/ir/tests/single.rs b/sbroad-core/src/frontend/sql/ir/tests/single.rs index f54693285c..3a18cc1667 100644 --- a/sbroad-core/src/frontend/sql/ir/tests/single.rs +++ b/sbroad-core/src/frontend/sql/ir/tests/single.rs @@ -33,7 +33,8 @@ impl Plan { Relational::Motion { policy, .. } => match policy { MotionPolicy::Full => Policy::Full, MotionPolicy::Local => Policy::Local, - MotionPolicy::Segment(MotionKey { targets }) => { + MotionPolicy::Segment(MotionKey { targets }) + | MotionPolicy::LocalSegment(MotionKey { targets }) => { let mut new_targets: Vec<String> = Vec::with_capacity(targets.len()); let aliases = self.get_relational_aliases(node_id).unwrap(); for t in targets { diff --git a/sbroad-core/src/ir/explain.rs b/sbroad-core/src/ir/explain.rs index 549ed9f1db..27f81b9937 100644 --- a/sbroad-core/src/ir/explain.rs +++ b/sbroad-core/src/ir/explain.rs @@ -9,7 +9,7 @@ use crate::ir::expression::cast::Type as CastType; use crate::ir::expression::Expression; use crate::ir::operator::{JoinKind, Relational}; use crate::ir::transformation::redistribution::{ - MotionPolicy as IrMotionPolicy, Target as IrTarget, + MotionKey as IrMotionKey, MotionPolicy as IrMotionPolicy, Target as IrTarget, }; use crate::ir::Plan; @@ -644,6 +644,7 @@ enum MotionPolicy { Full, Segment(MotionKey), Local, + LocalSegment(MotionKey), } impl Display for MotionPolicy { @@ -652,6 +653,7 @@ impl Display for MotionPolicy { MotionPolicy::Full => write!(f, "full"), MotionPolicy::Segment(mk) => write!(f, "segment({mk})"), MotionPolicy::Local => write!(f, "local"), + MotionPolicy::LocalSegment(mk) => write!(f, "local segment({mk})"), } } } @@ -705,7 +707,7 @@ impl Display for InnerJoin { } JoinKind::Inner => String::new(), }; - write!(f, "{}join on {}", kind, self.condition) + write!(f, "{kind}join on {0}", self.condition) } } @@ -936,43 +938,50 @@ impl FullExplain { })?; current_node.children.push(child); - let p = match policy { - IrMotionPolicy::Segment(s) => { - let child_id = children.first().ok_or_else(|| { - SbroadError::UnexpectedNumberOfValues( - "current node should have exactly one child".to_string(), - ) - })?; + let collect_targets = |s: &IrMotionKey| -> Result<Vec<Target>, SbroadError> { + let child_id = children.first().ok_or_else(|| { + SbroadError::UnexpectedNumberOfValues( + "current node should have exactly one child".to_string(), + ) + })?; - let child_output_id = ir.get_relation_node(*child_id)?.output(); - let col_list = - ir.get_expression_node(child_output_id)?.get_row_list()?; - - let targets = (s.targets) - .iter() - .map(|r| match r { - IrTarget::Reference(pos) => { - let col_id = *col_list.get(*pos).ok_or_else(|| { - SbroadError::NotFound( - Entity::Target, - format!("reference with position {pos}"), - ) - })?; - let col_name = ir - .get_expression_node(col_id)? - .get_alias_name()? - .to_string(); - - Ok(Target::Reference(col_name)) - } - IrTarget::Value(v) => Ok(Target::Value(v.clone())), - }) - .collect::<Result<Vec<Target>, _>>()?; + let child_output_id = ir.get_relation_node(*child_id)?.output(); + let col_list = ir.get_expression_node(child_output_id)?.get_row_list()?; + + let targets = (s.targets) + .iter() + .map(|r| match r { + IrTarget::Reference(pos) => { + let col_id = *col_list.get(*pos).ok_or_else(|| { + SbroadError::NotFound( + Entity::Target, + format!("reference with position {pos}"), + ) + })?; + let col_name = ir + .get_expression_node(col_id)? + .get_alias_name()? + .to_string(); + + Ok::<Target, SbroadError>(Target::Reference(col_name)) + } + IrTarget::Value(v) => Ok(Target::Value(v.clone())), + }) + .collect::<Result<Vec<Target>, _>>()?; + Ok(targets) + }; + let p = match policy { + IrMotionPolicy::Segment(s) => { + let targets = collect_targets(s)?; MotionPolicy::Segment(MotionKey { targets }) } IrMotionPolicy::Full => MotionPolicy::Full, IrMotionPolicy::Local => MotionPolicy::Local, + IrMotionPolicy::LocalSegment(s) => { + let targets = collect_targets(s)?; + MotionPolicy::LocalSegment(MotionKey { targets }) + } }; let m = Motion::new(p); diff --git a/sbroad-core/src/ir/explain/tests.rs b/sbroad-core/src/ir/explain/tests.rs index eab32652dc..cb9fcdf50a 100644 --- a/sbroad-core/src/ir/explain/tests.rs +++ b/sbroad-core/src/ir/explain/tests.rs @@ -315,13 +315,9 @@ fn insert_plan() { let mut actual_explain = String::new(); actual_explain.push_str( r#"insert "test_space" - projection (COL_0 -> COL_0, COL_1 -> COL_1, bucket_id((coalesce(('NULL', COL_0::string))))) - scan - projection ("COLUMN_1"::unsigned -> COL_0, "COLUMN_2"::string -> COL_1) - scan - motion [policy: segment([ref("COLUMN_1")])] - values - value row (data=ROW(1, '123')) + motion [policy: local segment([ref("COLUMN_1")])] + values + value row (data=ROW(1, '123')) "#, ); @@ -340,15 +336,11 @@ fn multiply_insert_plan() { let mut actual_explain = String::new(); actual_explain.push_str( r#"insert "test_space" - projection (COL_0 -> COL_0, COL_1 -> COL_1, bucket_id((coalesce(('NULL', COL_0::string))))) - scan - projection ("COLUMN_5"::unsigned -> COL_0, "COLUMN_6"::string -> COL_1) - scan - motion [policy: segment([ref("COLUMN_5")])] - values - value row (data=ROW(1, '123')) - value row (data=ROW(2, '456')) - value row (data=ROW(3, '789')) + motion [policy: local segment([ref("COLUMN_5")])] + values + value row (data=ROW(1, '123')) + value row (data=ROW(2, '456')) + value row (data=ROW(3, '789')) "#, ); @@ -368,13 +360,9 @@ SELECT "identification_number", "product_code" FROM "hash_testing""#; let mut actual_explain = String::new(); actual_explain.push_str( r#"insert "test_space" - projection (COL_0 -> COL_0, COL_1 -> COL_1, bucket_id((coalesce(('NULL', COL_0::string))))) - scan - projection ("hash_testing"."identification_number"::unsigned -> COL_0, "hash_testing"."product_code"::string -> COL_1) - scan - motion [policy: segment([ref("identification_number")])] - projection ("hash_testing"."identification_number" -> "identification_number", "hash_testing"."product_code" -> "product_code") - scan "hash_testing" + motion [policy: segment([ref("identification_number")])] + projection ("hash_testing"."identification_number" -> "identification_number", "hash_testing"."product_code" -> "product_code") + scan "hash_testing" "#, ); @@ -443,7 +431,7 @@ fn select_cast_plan2() { #[test] fn select_cast_plan_nested() { - let query = r#"SELECT cast(bucket_id("id") as string) FROM "test_space""#; + let query = r#"SELECT cast(func("id") as string) FROM "test_space""#; let plan = sql_to_optimized_ir(query, vec![]); @@ -452,7 +440,7 @@ fn select_cast_plan_nested() { let mut actual_explain = String::new(); actual_explain.push_str( - r#"projection ("BUCKET_ID"(("test_space"."id"))::string -> "COL_1") + r#"projection ("FUNC"(("test_space"."id"))::string -> "COL_1") scan "test_space" "#, ); @@ -462,7 +450,7 @@ fn select_cast_plan_nested() { #[test] fn select_cast_plan_nested_where() { - let query = r#"SELECT "id" FROM "test_space" WHERE cast(bucket_id("id") as string) = 1"#; + let query = r#"SELECT "id" FROM "test_space" WHERE cast(func("id") as string) = 1"#; let plan = sql_to_optimized_ir(query, vec![]); @@ -472,7 +460,7 @@ fn select_cast_plan_nested_where() { let mut actual_explain = String::new(); actual_explain.push_str( r#"projection ("test_space"."id" -> "id") - selection ROW("BUCKET_ID"(("test_space"."id"))::string) = ROW(1) + selection ROW("FUNC"(("test_space"."id"))::string) = ROW(1) scan "test_space" "#, ); @@ -482,7 +470,7 @@ fn select_cast_plan_nested_where() { #[test] fn select_cast_plan_nested_where2() { - let query = r#"SELECT "id" FROM "test_space" WHERE bucket_id(cast(42 as string)) = 1"#; + let query = r#"SELECT "id" FROM "test_space" WHERE func(cast(42 as string)) = 1"#; let plan = sql_to_optimized_ir(query, vec![]); @@ -492,7 +480,7 @@ fn select_cast_plan_nested_where2() { let mut actual_explain = String::new(); actual_explain.push_str( r#"projection ("test_space"."id" -> "id") - selection ROW("BUCKET_ID"((42::string))) = ROW(1) + selection ROW("FUNC"((42::string))) = ROW(1) scan "test_space" "#, ); diff --git a/sbroad-core/src/ir/explain/tests/concat.rs b/sbroad-core/src/ir/explain/tests/concat.rs index baffc96799..b3c75408fc 100644 --- a/sbroad-core/src/ir/explain/tests/concat.rs +++ b/sbroad-core/src/ir/explain/tests/concat.rs @@ -14,11 +14,11 @@ fn concat1_test() { #[test] fn concat2_test() { explain_check( - r#"SELECT "a" FROM "t1" WHERE CAST('1' as string) || BUCKET_ID('hello') || '2' = 42"#, + r#"SELECT "a" FROM "t1" WHERE CAST('1' as string) || FUNC('hello') || '2' = 42"#, &format!( "{}\n{}\n{}\n", r#"projection ("t1"."a" -> "a")"#, - r#" selection ROW(('1'::string) || (("BUCKET_ID"(('hello'))) || ('2'))) = ROW(42)"#, + r#" selection ROW(('1'::string) || (("FUNC"(('hello'))) || ('2'))) = ROW(42)"#, r#" scan "t1""#, ), ); diff --git a/sbroad-core/src/ir/operator.rs b/sbroad-core/src/ir/operator.rs index 2c70cbe824..173eec9fc0 100644 --- a/sbroad-core/src/ir/operator.rs +++ b/sbroad-core/src/ir/operator.rs @@ -897,7 +897,7 @@ impl Plan { )), )) } - MotionPolicy::Segment(key) => { + MotionPolicy::Segment(key) | MotionPolicy::LocalSegment(key) => { if let Ok(keyset) = KeySet::try_from(key) { self.set_dist(output, Distribution::Segment { keys: keyset })?; } else { diff --git a/sbroad-core/src/ir/relation.rs b/sbroad-core/src/ir/relation.rs index d5d8ce69e6..ecc5f561a1 100644 --- a/sbroad-core/src/ir/relation.rs +++ b/sbroad-core/src/ir/relation.rs @@ -3,7 +3,10 @@ use std::cmp::Ordering; use std::collections::{HashMap, HashSet}; use std::fmt::Formatter; -use tarantool::space::{Field, SpaceEngineType}; +use tarantool::{ + space::{Field, SpaceEngineType}, + tuple::{FieldType, KeyDef, KeyDefPart}, +}; use serde::de::{Error, MapAccess, Visitor}; use serde::ser::{Serialize as SerSerialize, SerializeMap, Serializer}; @@ -103,6 +106,22 @@ impl From<Column> for Field { } } +impl From<&Column> for FieldType { + fn from(column: &Column) -> Self { + match column.r#type { + Type::Boolean => FieldType::Boolean, + Type::Decimal => FieldType::Decimal, + Type::Double => FieldType::Double, + Type::Integer => FieldType::Integer, + Type::Number => FieldType::Number, + Type::Scalar => FieldType::Scalar, + Type::String => FieldType::String, + Type::Unsigned => FieldType::Unsigned, + Type::Array => FieldType::Array, + } + } +} + impl Column { #[must_use] pub fn default_value() -> Value { @@ -116,7 +135,7 @@ impl SerSerialize for Column { where S: Serializer, { - let mut map = serializer.serialize_map(Some(2))?; + let mut map = serializer.serialize_map(Some(3))?; map.serialize_entry("name", &self.name)?; match &self.r#type { Type::Boolean => map.serialize_entry("type", "boolean")?, @@ -129,6 +148,13 @@ impl SerSerialize for Column { Type::Unsigned => map.serialize_entry("type", "unsigned")?, Type::Array => map.serialize_entry("type", "array")?, } + map.serialize_entry( + "role", + match self.role { + ColumnRole::User => "user", + ColumnRole::Sharding => "sharding", + }, + )?; map.end() } @@ -419,6 +445,40 @@ impl Table { pub fn get_sharding_positions(&self) -> &[usize] { &self.key.positions } + + /// Get a sharding key definition for the table. + /// + /// # Errors + /// - Table internal inconsistency. + /// - Invalid sharding key position. + pub fn get_key_def(&self) -> Result<KeyDef, SbroadError> { + let mut parts = Vec::with_capacity(self.get_sharding_positions().len()); + for pos in self.get_sharding_positions() { + let column = self.columns.get(*pos).ok_or_else(|| { + SbroadError::NotFound( + Entity::Column, + format!( + "(distribution column) at position {} for Table {}", + *pos, self.name + ), + ) + })?; + let field_no = u32::try_from(*pos).map_err(|e| { + SbroadError::Invalid( + Entity::Table, + Some(format!("sharding key (position {pos}) error: {e}")), + ) + })?; + let part = KeyDefPart { + field_no, + field_type: FieldType::from(column), + is_nullable: true, + ..Default::default() + }; + parts.push(part); + } + KeyDef::new(&parts).map_err(|e| SbroadError::Invalid(Entity::Table, Some(e.to_string()))) + } } #[derive(Clone, Debug, PartialEq, Eq, Deserialize, Serialize)] diff --git a/sbroad-core/src/ir/relation/tests.rs b/sbroad-core/src/ir/relation/tests.rs index b470670974..7628305bee 100644 --- a/sbroad-core/src/ir/relation/tests.rs +++ b/sbroad-core/src/ir/relation/tests.rs @@ -234,8 +234,9 @@ fn column_msgpack_serialize() { assert_eq!( vec![ - 0x82, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, - 0x70, 0x65, 0xA7, 0x62, 0x6F, 0x6F, 0x6C, 0x65, 0x61, 0x6E, + 0x83, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, + 0x70, 0x65, 0xA7, 0x62, 0x6F, 0x6F, 0x6C, 0x65, 0x61, 0x6E, 0xA4, 0x72, 0x6F, 0x6C, + 0x65, 0xA4, 0x75, 0x73, 0x65, 0x72, ], rmp_serde::to_vec(&c).unwrap() ); @@ -248,8 +249,9 @@ fn column_msgpack_serialize() { assert_eq!( vec![ - 0x82, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, - 0x70, 0x65, 0xA6, 0x73, 0x74, 0x72, 0x69, 0x6E, 0x67, + 0x83, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, + 0x70, 0x65, 0xA6, 0x73, 0x74, 0x72, 0x69, 0x6E, 0x67, 0xA4, 0x72, 0x6F, 0x6C, 0x65, + 0xA4, 0x75, 0x73, 0x65, 0x72, ], rmp_serde::to_vec(&c).unwrap() ); @@ -262,8 +264,9 @@ fn column_msgpack_serialize() { assert_eq!( vec![ - 0x82, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, - 0x70, 0x65, 0xA7, 0x69, 0x6E, 0x74, 0x65, 0x67, 0x65, 0x72, + 0x83, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, + 0x70, 0x65, 0xA7, 0x69, 0x6E, 0x74, 0x65, 0x67, 0x65, 0x72, 0xA4, 0x72, 0x6F, 0x6C, + 0x65, 0xA4, 0x75, 0x73, 0x65, 0x72, ], rmp_serde::to_vec(&c).unwrap() ); @@ -276,8 +279,9 @@ fn column_msgpack_serialize() { assert_eq!( vec![ - 0x82, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, - 0x70, 0x65, 0xA8, 0x75, 0x6E, 0x73, 0x69, 0x67, 0x6E, 0x65, 0x64, + 0x83, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, + 0x70, 0x65, 0xA8, 0x75, 0x6E, 0x73, 0x69, 0x67, 0x6E, 0x65, 0x64, 0xA4, 0x72, 0x6F, + 0x6C, 0x65, 0xA4, 0x75, 0x73, 0x65, 0x72, ], rmp_serde::to_vec(&c).unwrap() ); @@ -290,8 +294,9 @@ fn column_msgpack_serialize() { assert_eq!( vec![ - 0x82, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, - 0x70, 0x65, 0xA6, 0x6E, 0x75, 0x6D, 0x62, 0x65, 0x72, + 0x83, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, + 0x70, 0x65, 0xA6, 0x6E, 0x75, 0x6D, 0x62, 0x65, 0x72, 0xA4, 0x72, 0x6F, 0x6C, 0x65, + 0xA4, 0x75, 0x73, 0x65, 0x72, ], rmp_serde::to_vec(&c).unwrap() ); @@ -306,8 +311,9 @@ fn column_msgpack_deserialize() { }; let expected_msgpack = vec![ - 0x82, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, 0x70, - 0x65, 0xA7, 0x62, 0x6F, 0x6F, 0x6C, 0x65, 0x61, 0x6E, + 0x83, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x6E, 0x61, 0x6D, 0x65, 0xA4, 0x74, 0x79, 0x70, + 0x65, 0xA7, 0x62, 0x6F, 0x6F, 0x6C, 0x65, 0x61, 0x6E, 0xA4, 0x72, 0x6F, 0x6C, 0x65, 0xA4, + 0x75, 0x73, 0x65, 0x72, ]; assert_eq!(expected_msgpack, rmp_serde::to_vec(&c).unwrap()); diff --git a/sbroad-core/src/ir/transformation/redistribution.rs b/sbroad-core/src/ir/transformation/redistribution.rs index 69cb897497..ee54b23190 100644 --- a/sbroad-core/src/ir/transformation/redistribution.rs +++ b/sbroad-core/src/ir/transformation/redistribution.rs @@ -79,6 +79,16 @@ pub enum MotionPolicy { Segment(MotionKey), /// No need to move data. Local, + /// Materialize a virtual table on the data node and calculate + /// tuple buckets. + LocalSegment(MotionKey), +} + +impl MotionPolicy { + #[must_use] + pub fn is_local(&self) -> bool { + matches!(self, MotionPolicy::Local | MotionPolicy::LocalSegment(_)) + } } /// Helper struct that unwraps `Expression::Bool` fields. @@ -131,32 +141,50 @@ impl Strategy { } /// Choose a policy for the inner join child (a policy with the largest motion of data wins). -fn join_policy_for_or(left_policy: &MotionPolicy, right_policy: &MotionPolicy) -> MotionPolicy { +fn join_policy_for_or( + left_policy: &MotionPolicy, + right_policy: &MotionPolicy, +) -> Result<MotionPolicy, SbroadError> { match (left_policy, right_policy) { - (MotionPolicy::Full, _) | (_, MotionPolicy::Full) => MotionPolicy::Full, - (MotionPolicy::Local, _) => right_policy.clone(), - (_, MotionPolicy::Local) => left_policy.clone(), + (MotionPolicy::LocalSegment(_), _) | (_, MotionPolicy::LocalSegment(_)) => { + Err(SbroadError::Invalid( + Entity::Motion, + Some("LocalSegment motion is not supported for joins".to_string()), + )) + } + (MotionPolicy::Full, _) | (_, MotionPolicy::Full) => Ok(MotionPolicy::Full), + (MotionPolicy::Local, _) => Ok(right_policy.clone()), + (_, MotionPolicy::Local) => Ok(left_policy.clone()), (MotionPolicy::Segment(key_left), MotionPolicy::Segment(key_right)) => { if key_left == key_right { - left_policy.clone() + Ok(left_policy.clone()) } else { - MotionPolicy::Full + Ok(MotionPolicy::Full) } } } } /// Choose a policy for the inner join child (a policy with the smallest motion of data wins). -fn join_policy_for_and(left_policy: &MotionPolicy, right_policy: &MotionPolicy) -> MotionPolicy { +fn join_policy_for_and( + left_policy: &MotionPolicy, + right_policy: &MotionPolicy, +) -> Result<MotionPolicy, SbroadError> { match (left_policy, right_policy) { - (MotionPolicy::Full, _) => right_policy.clone(), - (_, MotionPolicy::Full) => left_policy.clone(), - (MotionPolicy::Local, _) | (_, MotionPolicy::Local) => MotionPolicy::Local, + (MotionPolicy::LocalSegment(_), _) | (_, MotionPolicy::LocalSegment(_)) => { + Err(SbroadError::Invalid( + Entity::Motion, + Some("LocalSegment motion is not supported for joins".to_string()), + )) + } + (MotionPolicy::Full, _) => Ok(right_policy.clone()), + (_, MotionPolicy::Full) => Ok(left_policy.clone()), + (MotionPolicy::Local, _) | (_, MotionPolicy::Local) => Ok(MotionPolicy::Local), (MotionPolicy::Segment(key_left), MotionPolicy::Segment(key_right)) => { if key_left == key_right { - left_policy.clone() + Ok(left_policy.clone()) } else { - MotionPolicy::Full + Ok(MotionPolicy::Full) } } } @@ -934,8 +962,8 @@ impl Plan { .cloned() .unwrap_or(MotionPolicy::Full); match bool_op.op { - Bool::And => join_policy_for_and(&left_policy, &right_policy), - Bool::Or => join_policy_for_or(&left_policy, &right_policy), + Bool::And => join_policy_for_and(&left_policy, &right_policy)?, + Bool::Or => join_policy_for_or(&left_policy, &right_policy)?, _ => { return Err(SbroadError::Unsupported( Entity::Operator, @@ -1197,76 +1225,31 @@ impl Plan { Ok(Some(strategy)) } - /// Returns the strategy for the insert node. Also it - /// wraps the child node with nested sub-queries that: - /// - /// - cast expression types - /// - calculate `bucket_id` - /// - /// Full key example: - /// ```text - /// insert into t1 (a, b) values (1, 2) - /// => - /// insert into t1 (a, b, bucket_id) - /// select COL_1, COL_2, bucket_id(coalesce(cast(COL_2 as string), '') || coalesce(cast(COL_1 as string), '')) from ( - /// select - /// cast(COLUMN_1 as integer) as COL_1, cast(COLUMN_2 as unsigned) as COL_2 - /// from (values (1, 2)) - /// ) - /// ``` - /// - /// Trimmed key example: - /// ```text - /// insert into t1 (a) select c from t2 - /// => - /// insert into t1 (a, bucket_id) - /// select COL_1, bucket_id(coalesce(cast(NULL as string), '') || coalesce(cast(COL_1 as string), '')) from ( - /// select - /// cast(c as integer) as COL_1 - /// from (select c from t2) - /// ) - /// ``` fn resolve_insert_conflicts(&mut self, rel_id: usize) -> Result<Strategy, SbroadError> { - // Get the original INSERT child node id. - let orig_child_id = self.insert_child_id(rel_id)?; - self.replace_insert_child_with_cast_types_sq(rel_id)?; - - // Get a new parent node id for the original INSERT child. - let proj_id = self.insert_child_id(rel_id)?; - let proj = self.get_relation_node(proj_id)?; - - // We have generated this projection in `replace_insert_child_with_cast_types_sq()` - // method and we are sure, that there is exactly a single child in it. - let new_parent_id = *proj - .children() - .ok_or_else(|| { - SbroadError::UnexpectedNumberOfValues( - "expected non-empty amount of children in the projection node".to_string(), - ) - })? - .first() - .ok_or_else(|| { - SbroadError::UnexpectedNumberOfValues( - "expected at least a single child in projection node".to_string(), - ) - })?; + let mut map = Strategy::new(rel_id); let motion_key = self.insert_child_motion_key(rel_id)?; - self.replace_insert_child_with_bucket_id_sq(rel_id, &motion_key)?; + let child_id = self.insert_child_id(rel_id)?; + let child_output_id = self.get_relation_node(child_id)?.output(); + let child_dist = self.get_distribution(child_output_id)?; - let mut map = Strategy::new(new_parent_id); - let orig_child_output_id = self.get_relation_node(orig_child_id)?.output(); - let orig_child_dist = self.get_distribution(orig_child_output_id)?; - if let Distribution::Segment { keys } = orig_child_dist { + // Check that we can make a local segment motion. + if let Distribution::Segment { keys } = child_dist { for key in keys.iter() { let insert_mkey = MotionKey::from(key); if insert_mkey == motion_key { - map.add_child(orig_child_id, MotionPolicy::Local); + map.add_child(child_id, MotionPolicy::LocalSegment(motion_key)); return Ok(map); } } } + if let Relational::Values { .. } = self.get_relation_node(child_id)? { + if let Distribution::Replicated = child_dist { + map.add_child(child_id, MotionPolicy::LocalSegment(motion_key)); + return Ok(map); + } + } - map.add_child(orig_child_id, MotionPolicy::Segment(motion_key)); + map.add_child(child_id, MotionPolicy::Segment(motion_key)); Ok(map) } diff --git a/sbroad-core/src/ir/transformation/redistribution/groupby.rs b/sbroad-core/src/ir/transformation/redistribution/groupby.rs index d7ebfa907e..d0e9e5d9aa 100644 --- a/sbroad-core/src/ir/transformation/redistribution/groupby.rs +++ b/sbroad-core/src/ir/transformation/redistribution/groupby.rs @@ -380,12 +380,6 @@ impl<'plan> ExpressionMapper<'plan> { } } -pub struct AggregateInfo { - pub expression_top: usize, - pub aggregate: SimpleAggregate, - pub is_distinct: bool, -} - impl Plan { #[allow(unreachable_code)] fn generate_local_alias(id: usize) -> String { diff --git a/sbroad-core/src/ir/transformation/redistribution/insert.rs b/sbroad-core/src/ir/transformation/redistribution/insert.rs index 03b25c8fe9..7bf3d95152 100644 --- a/sbroad-core/src/ir/transformation/redistribution/insert.rs +++ b/sbroad-core/src/ir/transformation/redistribution/insert.rs @@ -1,13 +1,6 @@ -use std::mem::take; - use crate::errors::{Entity, SbroadError}; -use crate::executor::hash::ToHashString; -use crate::ir::expression::cast::Type as CastType; -use crate::ir::expression::Expression; -use crate::ir::function::{Behavior, Function}; use crate::ir::operator::Relational; -use crate::ir::relation::{Column, ColumnRole, Table}; -use crate::ir::value::Value; +use crate::ir::relation::{Column, Table}; use crate::ir::Plan; use ahash::AHashMap; @@ -31,22 +24,6 @@ impl Plan { )) } - pub(crate) fn set_insert_child_id( - &mut self, - insert_id: usize, - new_child: usize, - ) -> Result<(), SbroadError> { - let insert = self.get_mut_relation_node(insert_id)?; - if let Relational::Insert { children, .. } = insert { - *children = vec![new_child]; - return Ok(()); - } - Err(SbroadError::Invalid( - Entity::Node, - Some(format!("INSERT with id {insert_id}")), - )) - } - pub(crate) fn insert_child_motion_key( &self, insert_id: usize, @@ -112,20 +89,6 @@ impl Plan { )) } - pub(crate) fn insert_columns_mut( - &mut self, - insert_id: usize, - ) -> Result<&mut Vec<usize>, SbroadError> { - let insert = self.get_mut_relation_node(insert_id)?; - if let Relational::Insert { columns, .. } = insert { - return Ok(columns); - } - Err(SbroadError::Invalid( - Entity::Node, - Some(format!("INSERT with id {insert_id}")), - )) - } - pub(crate) fn insert_table(&self, insert_id: usize) -> Result<&Table, SbroadError> { let insert = self.get_relation_node(insert_id)?; if let Relational::Insert { relation, .. } = insert { @@ -138,166 +101,4 @@ impl Plan { Some(format!("INSERT with id {insert_id}")), )) } - - /// Wraps the child node with nested sub-queries that cast expression types. - /// ```text - /// insert into t1 (a) select c from t2 - /// => - /// insert into t1 (a) - /// select - /// cast(c as integer) as COL_1 - /// from (select c from t2) - /// ``` - pub(crate) fn replace_insert_child_with_cast_types_sq( - &mut self, - insert_id: usize, - ) -> Result<(), SbroadError> { - // Add sub-query. - let child_id = self.insert_child_id(insert_id)?; - let sq_id = self.add_sub_query(child_id, None)?; - let sq_output_id = self.get_relation_node(sq_id)?.output(); - self.set_distribution(sq_output_id)?; - - // Create a new row for the type-cast projection. - let proj_output_id = self.clone_expr_subtree(sq_output_id)?; - let proj_row = self.get_expression_node(proj_output_id)?.clone_row_list()?; - let mut new_row: Vec<usize> = Vec::with_capacity(proj_row.len()); - - let columns = self.insert_columns(insert_id)?.to_vec(); - for (idx, col_idx) in columns.iter().enumerate() { - let table = self.insert_table(insert_id)?; - let col = table.columns.get(*col_idx).ok_or_else(|| { - SbroadError::NotFound( - Entity::Column, - format!("in the table {table:?} at position {col_idx}"), - ) - })?; - let new_col_id: usize = match col.role { - ColumnRole::User => { - let proj_alias_id = *proj_row.get(idx).ok_or_else(|| { - SbroadError::NotFound( - Entity::Column, - format!("in the INSERT child's output at position {idx}"), - ) - })?; - let proj_alias = self.get_expression_node(proj_alias_id)?; - let proj_col_id = if let Expression::Alias { child, .. } = proj_alias { - *child - } else { - proj_alias_id - }; - // Explicitly cast the value to the type of the table column. - let col_type = CastType::try_from(&col.r#type)?; - self.add_cast(proj_col_id, col_type)? - } - ColumnRole::Sharding => { - return Err(SbroadError::Unsupported( - Entity::Operator, - Some("INSERT bucket_id column".to_string()), - )); - } - }; - let alias_id = self.nodes.add_alias(&format!("COL_{idx}"), new_col_id)?; - new_row.push(alias_id); - } - // Add projection with type cast on top of the sub-query. - let proj_id = self.add_proj_internal(sq_id, &new_row)?; - let proj_output_id = self.get_relation_node(proj_id)?.output(); - self.set_distribution(proj_output_id)?; - self.set_insert_child_id(insert_id, proj_id)?; - - Ok(()) - } - - /// Wraps the child node with nested sub-queries that insert `bucket_id` column. - /// - /// ```text - /// insert into t1 (a) select c from t2 - /// => - /// insert into t1 (a, bucket_id) - /// select - /// c, - /// bucket_id(coalesce(cast(NULL as string), 'NULL') || coalesce(cast(c as string), 'NULL')) - /// from (select c from t2) - /// ``` - pub(crate) fn replace_insert_child_with_bucket_id_sq( - &mut self, - insert_id: usize, - child_motion_key: &MotionKey, - ) -> Result<(), SbroadError> { - // Add sub-query. - let child_id = self.insert_child_id(insert_id)?; - let sq_id = self.add_sub_query(child_id, None)?; - let sq_output_id = self.get_relation_node(sq_id)?.output(); - self.set_distribution(sq_output_id)?; - - // Create a new row for the bucket_id projection. - let proj_output_id = self.clone_expr_subtree(sq_output_id)?; - self.flush_parent_in_subtree(proj_output_id)?; - let mut new_row = take( - self.get_mut_expression_node(proj_output_id)? - .get_row_list_mut()?, - ); - - // Create the string concatenation node: - // coalesce(cast(NULL as string), 'NULL') || coalesce(cast(c as string), 'NULL') - let mut columns_to_concat: Vec<usize> = Vec::with_capacity(child_motion_key.targets.len()); - for target in &child_motion_key.targets { - let alias_id = match target { - Target::Value(value) => self.add_const(value.clone()), - Target::Reference(row_pos) => *new_row.get(*row_pos).ok_or_else(|| { - SbroadError::NotFound( - Entity::Column, - format!("in the INSERT child's output at position {row_pos}"), - ) - })?, - }; - let col_expr = self.get_expression_node(alias_id)?; - let col_id = if let Expression::Alias { child, .. } = col_expr { - *child - } else { - alias_id - }; - let cast_id = self.add_cast(col_id, CastType::String)?; - let null_string_id = self.add_const(Value::String(Value::Null.to_hash_string())); - let fn_coalesce = Function::new("coalesce".to_string(), Behavior::Stable); - let coalesce_id: usize = - self.add_stable_function(&fn_coalesce, vec![cast_id, null_string_id])?; - columns_to_concat.push(coalesce_id); - } - let concat_id = if let Some((first, others)) = columns_to_concat.split_first() { - let mut left_id = *first; - for right_id in others { - let concat_id = self.add_concat(left_id, *right_id)?; - left_id = concat_id; - } - left_id - } else { - return Err(SbroadError::NotFound( - Entity::Column, - "to concatenate distribution key string to calculate bucket_id".to_string(), - )); - }; - - // Create the bucket_id function node. - let fn_bucket_id = Function::new("bucket_id".to_string(), Behavior::Stable); - let bucket_id_id: usize = self.add_stable_function(&fn_bucket_id, vec![concat_id])?; - - // Push the bucket_id column to the end of the INSERT columns' row. - let table = self.insert_table(insert_id)?; - let bucket_id_pos = table.get_bucket_id_position()?; - let columns = self.insert_columns_mut(insert_id)?; - columns.push(bucket_id_pos); - - // Push the bucket_id column to the end of projection row. - new_row.push(bucket_id_id); - - // Add projection with bucket_id on top of the sub-query. - let proj_id = self.add_proj_internal(sq_id, &new_row)?; - let proj_output_id = self.get_relation_node(proj_id)?.output(); - self.set_distribution(proj_output_id)?; - self.set_insert_child_id(insert_id, proj_id)?; - - Ok(()) - } } diff --git a/sbroad-core/src/ir/transformation/redistribution/tests/segment.rs b/sbroad-core/src/ir/transformation/redistribution/tests/segment.rs index 1ad0ffc37d..78a9fab026 100644 --- a/sbroad-core/src/ir/transformation/redistribution/tests/segment.rs +++ b/sbroad-core/src/ir/transformation/redistribution/tests/segment.rs @@ -267,7 +267,21 @@ fn insert2() { let mut plan = sql_to_ir(query, vec![]); plan.add_motions().unwrap(); - assert_eq!(plan.slices.slices().is_empty(), true); + let motion_id = *plan.slices.slice(0).unwrap().position(0).unwrap(); + let motion = plan.get_relation_node(motion_id).unwrap(); + if let Relational::Motion { policy, .. } = motion { + assert_eq!( + *policy, + MotionPolicy::LocalSegment( + (Key { + positions: vec![0, 1] + }) + .into() + ) + ); + } else { + panic!("Expected a motion node"); + } } #[test] @@ -276,7 +290,21 @@ fn insert3() { let mut plan = sql_to_ir(query, vec![]); plan.add_motions().unwrap(); - assert_eq!(plan.slices.slices().is_empty(), true); + let motion_id = *plan.slices.slice(0).unwrap().position(0).unwrap(); + let motion = plan.get_relation_node(motion_id).unwrap(); + if let Relational::Motion { policy, .. } = motion { + assert_eq!( + *policy, + MotionPolicy::LocalSegment( + (Key { + positions: vec![0, 1] + }) + .into() + ) + ); + } else { + panic!("Expected a motion node"); + } } #[test] diff --git a/sbroad-core/src/ir/tree/subtree.rs b/sbroad-core/src/ir/tree/subtree.rs index 5c2ae10b94..a99ec9694f 100644 --- a/sbroad-core/src/ir/tree/subtree.rs +++ b/sbroad-core/src/ir/tree/subtree.rs @@ -353,9 +353,12 @@ fn subtree_next<'plan>( None } Relational::Motion { - children, output, .. + children, + output, + policy, + .. } => { - if iter.need_motion_subtree() { + if policy.is_local() || iter.need_motion_subtree() { let step = *iter.get_child().borrow(); if step < children.len() { *iter.get_child().borrow_mut() += 1; diff --git a/sbroad-core/src/ir/value.rs b/sbroad-core/src/ir/value.rs index 8e3654ae7c..3349fb28f6 100644 --- a/sbroad-core/src/ir/value.rs +++ b/sbroad-core/src/ir/value.rs @@ -110,7 +110,7 @@ impl From<bool> for Trivalent { /// Values are used to keep constants in the IR tree /// or results in the virtual tables. -#[derive(Hash, PartialEq, Debug, Clone, Deserialize, Serialize)] +#[derive(Hash, PartialEq, Debug, Default, Clone, Deserialize, Serialize)] pub enum Value { /// Boolean type. Boolean(bool), @@ -121,6 +121,7 @@ pub enum Value { /// Signed integer type. Integer(i64), /// SQL NULL ("unknown" in the terms of three-valued logic). + #[default] Null, /// String type. String(String), diff --git a/sbroad-picodata/src/api.rs b/sbroad-picodata/src/api.rs index aa2f4ecf36..e5d424062f 100644 --- a/sbroad-picodata/src/api.rs +++ b/sbroad-picodata/src/api.rs @@ -3,7 +3,7 @@ use crate::runtime::storage::StorageRuntime; use sbroad::backend::sql::ir::PatternWithParams; use sbroad::debug; use sbroad::errors::{Action, SbroadError}; -use sbroad::executor::engine::{helpers::decode_msgpack, Router}; +use sbroad::executor::engine::{helpers::decode_msgpack, Vshard}; use sbroad::executor::protocol::{EncodedRequiredData, RequiredData}; use sbroad::executor::Query; use sbroad::otm::query_span; diff --git a/sbroad-picodata/src/runtime.rs b/sbroad-picodata/src/runtime.rs index 7d3ce36908..24f8b9caca 100644 --- a/sbroad-picodata/src/runtime.rs +++ b/sbroad-picodata/src/runtime.rs @@ -1,3 +1,5 @@ pub mod meta; pub mod router; pub mod storage; + +pub const DEFAULT_BUCKET_COUNT: u64 = 3000; diff --git a/sbroad-picodata/src/runtime/meta/router.rs b/sbroad-picodata/src/runtime/meta/router.rs index 8bab5952b2..1dc773d786 100644 --- a/sbroad-picodata/src/runtime/meta/router.rs +++ b/sbroad-picodata/src/runtime/meta/router.rs @@ -5,7 +5,9 @@ use sbroad::errors::{Action, Entity, SbroadError}; use sbroad::executor::engine::helpers::storage::meta::{ DEFAULT_JAEGER_AGENT_HOST, DEFAULT_JAEGER_AGENT_PORT, }; -use sbroad::executor::engine::helpers::{normalize_name_from_schema, normalize_name_from_sql}; +use sbroad::executor::engine::helpers::{ + normalize_name_for_space_api, normalize_name_from_schema, normalize_name_from_sql, +}; use sbroad::executor::engine::Metadata; use sbroad::executor::lru::DEFAULT_CAPACITY; use sbroad::ir::function::Function; @@ -16,13 +18,6 @@ use tarantool::util::Value; pub const DEFAULT_BUCKET_COLUMN: &str = "bucket_id"; -fn normalize_name_for_space_api(s: &str) -> String { - if let (Some('"'), Some('"')) = (s.chars().next(), s.chars().last()) { - return s.to_string(); - } - s.to_uppercase() -} - /// Router runtime configuration. #[derive(Clone, Debug, PartialEq, Eq)] #[allow(clippy::module_name_repetitions)] @@ -81,6 +76,7 @@ impl RouterMetadata { impl Metadata for RouterMetadata { #[allow(dead_code)] + #[allow(clippy::too_many_lines)] fn table(&self, table_name: &str) -> Result<Table, SbroadError> { let name = normalize_name_for_space_api(table_name); diff --git a/sbroad-picodata/src/runtime/router.rs b/sbroad-picodata/src/runtime/router.rs index 5e2c20ae13..b1d0fca065 100644 --- a/sbroad-picodata/src/runtime/router.rs +++ b/sbroad-picodata/src/runtime/router.rs @@ -26,16 +26,14 @@ use sbroad::{ ir::Plan, }; -use super::meta::router::RouterMetadata; +use super::{meta::router::RouterMetadata, DEFAULT_BUCKET_COUNT}; thread_local! (static PLAN_CACHE: Rc<RefCell<LRUCache<String, Plan>>> = Rc::new(RefCell::new(LRUCache::new(DEFAULT_CAPACITY, None).unwrap()))); -const DEFAULT_BUCKET_COUNT: usize = 3000; - #[allow(clippy::module_name_repetitions)] pub struct RouterRuntime { metadata: RefCell<RouterMetadata>, - bucket_count: usize, + bucket_count: u64, ir_cache: Rc<RefCell<LRUCache<String, Plan>>>, } @@ -129,13 +127,47 @@ impl Router for RouterRuntime { ) -> Result<Vec<&'rec sbroad::ir::value::Value>, SbroadError> { sharding_keys_from_tuple(&*self.metadata()?, &space, args) } +} + +impl Vshard for RouterRuntime { + fn exec_ir_on_all( + &self, + required: Binary, + optional: Binary, + query_type: QueryType, + conn_type: ConnectionType, + ) -> Result<Box<dyn Any>, SbroadError> { + exec_ir_on_all( + &*self.metadata()?, + required, + optional, + query_type, + conn_type, + ) + } + + fn bucket_count(&self) -> u64 { + self.bucket_count + } + + fn get_random_bucket(&self) -> Buckets { + get_random_bucket(self) + } fn determine_bucket_id(&self, s: &[&sbroad::ir::value::Value]) -> u64 { - bucket_id_by_tuple(s, self.bucket_count) + bucket_id_by_tuple(s, self.bucket_count()) + } + + fn exec_ir_on_some( + &self, + sub_plan: ExecutionPlan, + buckets: &Buckets, + ) -> Result<Box<dyn Any>, SbroadError> { + exec_with_filtered_buckets(self, sub_plan, buckets) } } -impl Vshard for RouterRuntime { +impl Vshard for &RouterRuntime { fn exec_ir_on_all( &self, required: Binary, @@ -153,18 +185,22 @@ impl Vshard for RouterRuntime { } fn bucket_count(&self) -> u64 { - self.bucket_count as u64 + self.bucket_count } fn get_random_bucket(&self) -> Buckets { get_random_bucket(self) } + fn determine_bucket_id(&self, s: &[&sbroad::ir::value::Value]) -> u64 { + bucket_id_by_tuple(s, self.bucket_count()) + } + fn exec_ir_on_some( &self, sub_plan: ExecutionPlan, buckets: &Buckets, ) -> Result<Box<dyn Any>, SbroadError> { - exec_with_filtered_buckets(self, sub_plan, buckets) + exec_with_filtered_buckets(*self, sub_plan, buckets) } } diff --git a/sbroad-picodata/src/runtime/storage.rs b/sbroad-picodata/src/runtime/storage.rs index 652d29f6c1..a1143a8e16 100644 --- a/sbroad-picodata/src/runtime/storage.rs +++ b/sbroad-picodata/src/runtime/storage.rs @@ -4,9 +4,10 @@ use sbroad::{ debug, errors::{Action, Entity, SbroadError}, executor::{ + bucket::Buckets, engine::{ helpers::{ - compile_encoded_optional, + compile_encoded_optional, execute_dml, storage::{ meta::StorageMetadata, runtime::{ @@ -15,21 +16,29 @@ use sbroad::{ }, PreparedStmt, }, + vshard::get_random_bucket, }, - QueryCache, + QueryCache, Vshard, }, - ir::QueryType, + hash::bucket_id_by_tuple, + ir::{ConnectionType, ExecutionPlan, QueryType}, lru::{Cache, LRUCache, DEFAULT_CAPACITY}, - protocol::RequiredData, + protocol::{Binary, RequiredData}, }, + ir::value::Value, warn, }; +use tarantool::tuple::Tuple; + +use super::DEFAULT_BUCKET_COUNT; + thread_local!(static STATEMENT_CACHE: Rc<RefCell<LRUCache<String, PreparedStmt>>> = Rc::new(RefCell::new(LRUCache::new(DEFAULT_CAPACITY, Some(Box::new(unprepare))).unwrap()))); #[allow(clippy::module_name_repetitions)] pub struct StorageRuntime { pub metadata: RefCell<StorageMetadata>, + bucket_count: u64, cache: Rc<RefCell<LRUCache<String, PreparedStmt>>>, } @@ -58,6 +67,44 @@ impl QueryCache for StorageRuntime { } } +impl Vshard for StorageRuntime { + fn exec_ir_on_all( + &self, + _required: Binary, + _optional: Binary, + _query_type: QueryType, + _conn_type: ConnectionType, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Runtime, + Some("exec_ir_on_all is not supported on the storage".to_string()), + )) + } + + fn bucket_count(&self) -> u64 { + self.bucket_count + } + + fn get_random_bucket(&self) -> Buckets { + get_random_bucket(self) + } + + fn determine_bucket_id(&self, s: &[&Value]) -> u64 { + bucket_id_by_tuple(s, self.bucket_count()) + } + + fn exec_ir_on_some( + &self, + _sub_plan: ExecutionPlan, + _buckets: &Buckets, + ) -> Result<Box<dyn Any>, SbroadError> { + Err(SbroadError::Unsupported( + Entity::Runtime, + Some("exec_ir_on_some is not supported on the storage".to_string()), + )) + } +} + impl StorageRuntime { /// Build a new storage runtime. /// @@ -66,6 +113,7 @@ impl StorageRuntime { pub fn new() -> Result<Self, SbroadError> { let runtime = STATEMENT_CACHE.with(|cache| StorageRuntime { metadata: RefCell::new(StorageMetadata::new()), + bucket_count: DEFAULT_BUCKET_COUNT, cache: cache.clone(), }); Ok(runtime) @@ -81,45 +129,39 @@ impl StorageRuntime { required: &mut RequiredData, raw_optional: &mut Vec<u8>, ) -> Result<Box<dyn Any>, SbroadError> { - if required.can_be_cached { - return self.execute_cacheable_plan(required, raw_optional); + match required.query_type { + QueryType::DML => self.execute_dml(required, raw_optional), + QueryType::DQL => { + if required.can_be_cached { + self.execute_cacheable_dql(required, raw_optional) + } else { + execute_non_cacheable_dql(required, raw_optional) + } + } } - Self::execute_non_cacheable_plan(required, raw_optional) } #[allow(unused_variables)] - fn execute_non_cacheable_plan( + fn execute_dml( + &self, required: &mut RequiredData, raw_optional: &mut Vec<u8>, ) -> Result<Box<dyn Any>, SbroadError> { - let plan_id = required.plan_id.clone(); - - if required.can_be_cached { + if required.query_type != QueryType::DML { return Err(SbroadError::Invalid( Entity::Plan, - Some("Expected a plan that can not be cached.".to_string()), + Some("Expected a DML plan.".to_string()), )); } - let (pattern_with_params, _tmp_spaces) = compile_encoded_optional(raw_optional)?; - debug!( - Option::from("execute"), - &format!( - "Failed to execute the statement: {}", - pattern_with_params.pattern - ), - ); - let result = if required.query_type == QueryType::DML { - write_unprepared(&pattern_with_params.pattern, &pattern_with_params.params) - } else { - read_unprepared(&pattern_with_params.pattern, &pattern_with_params.params) - }; - - result + let result = execute_dml(self, raw_optional)?; + let tuple = Tuple::new(&(result,)) + .map_err(|e| SbroadError::Invalid(Entity::Tuple, Some(format!("{e:?}"))))?; + Ok(Box::new(tuple) as Box<dyn Any>) } #[allow(unused_variables)] - fn execute_cacheable_plan( + fn execute_cacheable_dql( &self, required: &mut RequiredData, raw_optional: &mut Vec<u8>, @@ -227,3 +269,29 @@ impl StorageRuntime { result } } + +fn execute_non_cacheable_dql( + required: &mut RequiredData, + raw_optional: &mut Vec<u8>, +) -> Result<Box<dyn Any>, SbroadError> { + if required.can_be_cached || required.query_type != QueryType::DQL { + return Err(SbroadError::Invalid( + Entity::Plan, + Some("Expected a DQL plan that can not be cached.".to_string()), + )); + } + + let (pattern_with_params, _tmp_spaces) = compile_encoded_optional(raw_optional)?; + debug!( + Option::from("execute"), + &format!( + "Failed to execute the statement: {}", + pattern_with_params.pattern + ), + ); + warn!( + Option::from("execute"), + &format!("SQL pattern: {}", pattern_with_params.pattern), + ); + read_unprepared(&pattern_with_params.pattern, &pattern_with_params.params) +} -- GitLab