Compare commits

...

10 commits

Author SHA1 Message Date
Micha de Vries
3d10df0fcb
Fixes ULID/UUID gen, programmatically generating ranges, RETURN inside FOR/IF and improves arithmetic operations (#4847)
Some checks failed
Benchmark / Benchmark engines (kv-rocksdb, lib-rocksdb) (push) Has been cancelled
Benchmark / Benchmark engines (kv-rocksdb, sdk-rocksdb) (push) Has been cancelled
Benchmark / Benchmark engines (kv-surrealkv, lib-surrealkv) (push) Has been cancelled
Benchmark / Benchmark engines (kv-surrealkv, sdk-surrealkv) (push) Has been cancelled
Continuous integration / Check format (push) Has been cancelled
Continuous integration / Check workspace (push) Has been cancelled
Continuous integration / Check workspace MSRV (push) Has been cancelled
Continuous integration / Check fuzzing (push) Has been cancelled
Continuous integration / Check Wasm (push) Has been cancelled
Continuous integration / Check clippy (push) Has been cancelled
Continuous integration / CLI integration tests (push) Has been cancelled
Continuous integration / HTTP integration tests (push) Has been cancelled
Continuous integration / WebSocket integration tests (push) Has been cancelled
Continuous integration / ML integration tests (push) Has been cancelled
Continuous integration / Test workspace (push) Has been cancelled
Continuous integration / GraphQL integration (push) Has been cancelled
Continuous integration / Test SDK build (push) Has been cancelled
Continuous integration / WebSocket engine (push) Has been cancelled
Continuous integration / HTTP engine (push) Has been cancelled
Continuous integration / Any engine (push) Has been cancelled
Continuous integration / Memory engine (push) Has been cancelled
Continuous integration / File engine (push) Has been cancelled
Continuous integration / RocksDB engine (push) Has been cancelled
Continuous integration / SurrealKV engine (push) Has been cancelled
Continuous integration / TiKV engine (push) Has been cancelled
Continuous integration / FoundationDB engine 7.1 (push) Has been cancelled
Continuous integration / FoundationDB engine 7.3 (push) Has been cancelled
Continuous integration / Database Upgrade from previous versions (push) Has been cancelled
Nix / Build Docker image (push) Has been cancelled
Nix / Build native Linux binary (push) Has been cancelled
Co-authored-by: Tobie Morgan Hitchcock <tobie@surrealdb.com>
2024-09-21 14:51:39 +00:00
Rushmore Mushambi
18eb778720
Add a test for including an ID in the content body (#4849) 2024-09-20 21:51:42 +00:00
Mees Delzenne
eff4936dc3
Fix inconsitent numeric object key (#4851) 2024-09-20 15:18:46 +00:00
Tobie Morgan Hitchcock
8cca86028b
Make SurrealCS connection pool size configurable (#4826) 2024-09-20 00:54:58 +01:00
Chiru B
3b21249e99
GraphQL comments without appending and fields (#4808)
Co-authored-by: itsezc <itsezc>
2024-09-19 16:19:47 +00:00
Emmanuel Keller
ebec244b01
Fix Query Planner Ignoring Isolated predicates in WHERE Clause (#4806) 2024-09-19 15:17:25 +00:00
Rushmore Mushambi
a3b6b2c5db
Fix version patching (#4840) 2024-09-19 14:29:07 +00:00
Gerard Guillemas Martos
957ff739bb
Fix failed authenticate resulting in session reset (#4839) 2024-09-19 14:17:29 +00:00
Mees Delzenne
491b549f1d
Fix decimal parsing (#4838) 2024-09-19 14:14:32 +00:00
Gerard Guillemas Martos
99db7f2a82
Explicitly check for token expiration error (#4836) 2024-09-19 13:22:11 +00:00
29 changed files with 505 additions and 130 deletions

View file

@ -684,7 +684,7 @@ jobs:
# Update the version to a nightly one
sed -i "s#^version = \".*\"#version = \"${version}\"#" sdk/Cargo.toml
sed -i "s#^version = \".*\"#version = \"${version}\"#" core/Cargo.toml
sed -i "s#surrealdb-core = { version = \"${major}\"#surrealdb-core = { version = \"=${version}\"#" sdk/Cargo.toml
sed -i "s#surrealdb-core = { version = \"=${currentVersion}\"#surrealdb-core = { version = \"=${version}\"#" sdk/Cargo.toml
- name: Patch crate name and description
if: ${{ inputs.environment != 'stable' }}

19
Cargo.lock generated
View file

@ -3503,24 +3503,17 @@ dependencies = [
[[package]]
name = "nanoservices-utils"
version = "0.1.4"
version = "0.1.5"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "c6a95cdb2ba5dc6584099b1aac8da0a6f80e12ad5f9381a19d232c1fee714793"
checksum = "63a4117f6ebfcfe84722ef283fd6e3bfcaa141718f3ff092343bb6cb98482e03"
dependencies = [
"actix-web",
"bincode",
"bitcode",
"bytes",
"chrono",
"futures",
"jsonwebtoken",
"paste",
"revision 0.7.1",
"serde",
"serde_json",
"thiserror",
"tokio",
"tokio-util",
]
[[package]]
@ -5990,9 +5983,9 @@ dependencies = [
[[package]]
name = "surrealcs"
version = "0.3.1"
version = "0.3.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "457eb2adc59f9a9ba337e1fa3a6af63e09524e4adf01921b9d7079917cd7f694"
checksum = "b28a50184a9a743ac180985d8b94ac4c93e4afaa0140f1057d3b11d4d36b9e8b"
dependencies = [
"bincode",
"bytes",
@ -6011,9 +6004,9 @@ dependencies = [
[[package]]
name = "surrealcs-kernel"
version = "0.3.1"
version = "0.3.2"
source = "registry+https://github.com/rust-lang/crates.io-index"
checksum = "f4a79da58bfc886b93a431a463f956184d4b0a9d3672833544a1b718ff86b0df"
checksum = "ab2a5ba4af373b48fa241a81676d0172fc92e563df217282c472252b90beafee"
dependencies = [
"bincode",
"chrono",

View file

@ -147,7 +147,7 @@ sha2 = "0.10.8"
snap = "1.1.0"
storekey = "0.5.0"
subtle = "2.6"
surrealcs = { version = "0.3.1", optional = true }
surrealcs = { version = "0.3.2", optional = true }
surrealkv = { version = "0.3.6", optional = true }
surrealml = { version = "0.1.1", optional = true, package = "surrealml-core" }
tempfile = { version = "3.10.1", optional = true }

View file

@ -387,8 +387,12 @@ impl<'a> Executor<'a> {
.await;
ctx = MutableContext::unfreeze(c)?;
// Check if this is a RETURN statement
let can_return =
matches!(stm, Statement::Output(_) | Statement::Value(_));
let can_return = matches!(
stm,
Statement::Output(_)
| Statement::Value(_) | Statement::Ifelse(_)
| Statement::Foreach(_)
);
// Catch global timeout
let res = match ctx.is_timedout() {
true => Err(Error::QueryTimedout),

View file

@ -1151,6 +1151,10 @@ pub enum Error {
#[error("Found a non-computed value where they are not allowed")]
NonComputed,
/// Represents a failure in timestamp arithmetic related to database internals
#[error("Failed to compute: \"{0}\", as the operation results in an overflow.")]
ArithmeticOverflow(String),
}
impl From<Error> for String {

View file

@ -153,7 +153,19 @@ pub fn time((range,): (Option<(i64, i64)>,)) -> Result<Value, Error> {
pub fn ulid((timestamp,): (Option<Datetime>,)) -> Result<Value, Error> {
let ulid = match timestamp {
Some(timestamp) => Ulid::from_datetime(timestamp.0.into()),
Some(timestamp) => {
#[cfg(target_arch = "wasm32")]
if timestamp.0 < chrono::DateTime::UNIX_EPOCH {
return Err(Error::InvalidArguments {
name: String::from("rand::ulid"),
message: format!(
"To generate a ULID from a datetime, it must be a time beyond UNIX epoch."
),
});
}
Ulid::from_datetime(timestamp.0.into())
}
None => Ulid::new(),
};
@ -162,7 +174,19 @@ pub fn ulid((timestamp,): (Option<Datetime>,)) -> Result<Value, Error> {
pub fn uuid((timestamp,): (Option<Datetime>,)) -> Result<Value, Error> {
let uuid = match timestamp {
Some(timestamp) => Uuid::new_v7_from_datetime(timestamp),
Some(timestamp) => {
#[cfg(target_arch = "wasm32")]
if timestamp.0 < chrono::DateTime::UNIX_EPOCH {
return Err(Error::InvalidArguments {
name: String::from("rand::ulid"),
message: format!(
"To generate a ULID from a datetime, it must be a time beyond UNIX epoch."
),
});
}
Uuid::new_v7_from_datetime(timestamp)
}
None => Uuid::new(),
};
Ok(uuid.into())
@ -181,7 +205,19 @@ pub mod uuid {
pub fn v7((timestamp,): (Option<Datetime>,)) -> Result<Value, Error> {
let uuid = match timestamp {
Some(timestamp) => Uuid::new_v7_from_datetime(timestamp),
Some(timestamp) => {
#[cfg(target_arch = "wasm32")]
if timestamp.0 < chrono::DateTime::UNIX_EPOCH {
return Err(Error::InvalidArguments {
name: String::from("rand::ulid"),
message: format!(
"To generate a ULID from a datetime, it must be a time beyond UNIX epoch."
),
});
}
Uuid::new_v7_from_datetime(timestamp)
}
None => Uuid::new(),
};
Ok(uuid.into())

View file

@ -1,3 +1,5 @@
use std::ops::Deref;
use crate::ctx::Context;
use crate::dbs::Options;
use crate::doc::CursorDoc;
@ -146,6 +148,7 @@ pub fn thing((arg1, arg2): (Value, Option<Value>)) -> Result<Value, Error> {
Value::Array(v) => v.into(),
Value::Object(v) => v.into(),
Value::Number(v) => v.into(),
Value::Range(v) => v.deref().to_owned().try_into()?,
v => v.as_string().into(),
},
})),

View file

@ -185,7 +185,7 @@ pub async fn generate_schema(
let desc = current.get("desc");
match (asc, desc) {
(Some(_), Some(_)) => {
return Err("Found both asc and desc in order".into());
return Err("Found both ASC and DESC in order".into());
}
(Some(GqlValue::Enum(a)), None) => {
orders.push(order!(asc, a.as_str()))
@ -281,7 +281,7 @@ pub async fn generate_schema(
})
},
)
.description(format!("Generated from table `{}`{}\nallows querying a table with filters", tb.name, if let Some(ref c) = &tb.comment {format!("\n{c}")} else {"".to_string()}))
.description(format!("{}", if let Some(ref c) = &tb.comment { format!("{c}") } else { format!("Generated from table `{}`\nallows querying a table with filters", tb.name) }))
.argument(limit_input!())
.argument(start_input!())
.argument(InputValue::new("order", TypeRef::named(&table_order_name)))
@ -329,12 +329,11 @@ pub async fn generate_schema(
},
)
.description(format!(
"Generated from table `{}`{}\nallows querying a single record in a table by id",
tb.name,
"{}",
if let Some(ref c) = &tb.comment {
format!("\n{c}")
format!("{c}")
} else {
"".to_string()
format!("Generated from table `{}`\nallows querying a single record in a table by ID", tb.name)
}
))
.argument(id_input!()),
@ -371,11 +370,20 @@ pub async fn generate_schema(
table_filter = table_filter
.field(InputValue::new(fd.name.to_string(), TypeRef::named(type_filter_name)));
table_ty_obj = table_ty_obj.field(Field::new(
fd.name.to_string(),
fd_type,
make_table_field_resolver(fd_name.as_str(), fd.kind.clone()),
));
table_ty_obj = table_ty_obj
.field(Field::new(
fd.name.to_string(),
fd_type,
make_table_field_resolver(fd_name.as_str(), fd.kind.clone()),
))
.description(format!(
"{}",
if let Some(ref c) = fd.comment {
format!("{c}")
} else {
"".to_string()
}
));
}
types.push(Type::Object(table_ty_obj));
@ -421,7 +429,7 @@ pub async fn generate_schema(
}
})
})
.description("allows fetching arbitrary records".to_string())
.description("Allows fetching arbitrary records".to_string())
.argument(id_input!()),
);
@ -476,7 +484,7 @@ pub async fn generate_schema(
schema,
"uuid",
Kind::Uuid,
"a string encoded uuid",
"String encoded UUID",
"https://datatracker.ietf.org/doc/html/rfc4122"
);

View file

@ -1539,7 +1539,11 @@ mod tests {
let mut sess = Session::default();
let res = token(&ds, &mut sess, &enc).await;
assert!(res.is_err(), "Unexpected success signing in with expired token: {:?}", res);
match res {
Err(Error::ExpiredToken) => {} // ok
Err(err) => panic!("Unexpected error signing in with expired token: {:?}", err),
res => panic!("Unexpected success signing in with expired token: {:?}", res),
}
}
#[tokio::test]

View file

@ -112,7 +112,15 @@ impl QueryPlanner {
tree.knn_condition,
)
.await?;
match PlanBuilder::build(tree.root, params, tree.with_indexes, order)? {
match PlanBuilder::build(
tree.root,
params,
tree.with_indexes,
order,
tree.all_and_groups,
tree.all_and,
tree.all_expressions_with_index,
)? {
Plan::SingleIndex(exp, io) => {
if io.require_distinct() {
self.requires_distinct = true;

View file

@ -17,33 +17,27 @@ pub(super) struct PlanBuilder {
has_indexes: bool,
/// List of expressions that are not ranges, backed by an index
non_range_indexes: Vec<(Arc<Expression>, IndexOption)>,
/// List of indexes involved in this plan
with_indexes: Vec<IndexRef>,
/// List of indexes allowed in this plan
with_indexes: Option<Vec<IndexRef>>,
/// Group each possible optimisations local to a SubQuery
groups: BTreeMap<GroupRef, Group>, // The order matters because we want the plan to be consistent across repeated queries.
/// Does a group contains only AND relations?
all_and_groups: HashMap<GroupRef, bool>,
/// Does the whole query contains only AND relations?
all_and: bool,
/// Is every expression backed by an index?
all_exp_with_index: bool,
}
impl PlanBuilder {
pub(super) fn build(
root: Option<Node>,
params: &QueryPlannerParams,
with_indexes: Vec<IndexRef>,
with_indexes: Option<Vec<IndexRef>>,
order: Option<IndexOption>,
all_and_groups: HashMap<GroupRef, bool>,
all_and: bool,
all_expressions_with_index: bool,
) -> Result<Plan, Error> {
let mut b = PlanBuilder {
has_indexes: false,
non_range_indexes: Default::default(),
groups: Default::default(),
with_indexes,
all_and_groups: Default::default(),
all_and: true,
all_exp_with_index: true,
};
// If we only count and there are no conditions and no aggregations, then we can only scan keys
@ -61,7 +55,7 @@ impl PlanBuilder {
}
// If every boolean operator are AND then we can use the single index plan
if b.all_and {
if all_and {
// TODO: This is currently pretty arbitrary
// We take the "first" range query if one is available
if let Some((_, group)) = b.groups.into_iter().next() {
@ -79,10 +73,10 @@ impl PlanBuilder {
}
}
// If every expression is backed by an index with can use the MultiIndex plan
else if b.all_exp_with_index {
else if all_expressions_with_index {
let mut ranges = Vec::with_capacity(b.groups.len());
for (depth, group) in b.groups {
if b.all_and_groups.get(&depth) == Some(&true) {
for (gr, group) in b.groups {
if all_and_groups.get(&gr) == Some(&true) {
group.take_union_ranges(&mut ranges);
} else {
group.take_intersect_ranges(&mut ranges);
@ -100,9 +94,11 @@ impl PlanBuilder {
// Check if we have an explicit list of index we can use
fn filter_index_option(&self, io: Option<&IndexOption>) -> Option<IndexOption> {
if let Some(io) = &io {
if !self.with_indexes.is_empty() && !self.with_indexes.contains(&io.ix_ref()) {
return None;
if let Some(io) = io {
if let Some(wi) = &self.with_indexes {
if !wi.contains(&io.ix_ref()) {
return None;
}
}
}
io.cloned()
@ -117,11 +113,8 @@ impl PlanBuilder {
right,
exp,
} => {
let is_bool = self.check_boolean_operator(*group, exp.operator());
if let Some(io) = self.filter_index_option(io.as_ref()) {
self.add_index_option(*group, exp.clone(), io);
} else if self.all_exp_with_index && !is_bool {
self.all_exp_with_index = false;
}
self.eval_node(left)?;
self.eval_node(right)?;
@ -132,26 +125,6 @@ impl PlanBuilder {
}
}
fn check_boolean_operator(&mut self, gr: GroupRef, op: &Operator) -> bool {
match op {
Operator::Neg | Operator::Or => {
if self.all_and {
self.all_and = false;
}
self.all_and_groups.entry(gr).and_modify(|b| *b = false).or_insert(false);
true
}
Operator::And => {
self.all_and_groups.entry(gr).or_insert(true);
true
}
_ => {
self.all_and_groups.entry(gr).or_insert(true);
false
}
}
}
fn add_index_option(&mut self, group_ref: GroupRef, exp: Arc<Expression>, io: IndexOption) {
if let IndexOperator::RangePart(_, _) = io.op() {
let level = self.groups.entry(group_ref).or_default();

View file

@ -20,10 +20,16 @@ use std::sync::Arc;
pub(super) struct Tree {
pub(super) root: Option<Node>,
pub(super) index_map: IndexesMap,
pub(super) with_indexes: Vec<IndexRef>,
pub(super) with_indexes: Option<Vec<IndexRef>>,
pub(super) knn_expressions: KnnExpressions,
pub(super) knn_brute_force_expressions: KnnBruteForceExpressions,
pub(super) knn_condition: Option<Cond>,
/// Is every expression backed by an index?
pub(super) all_expressions_with_index: bool,
/// Does the whole query contains only AND relations?
pub(super) all_and: bool,
/// Does a group contains only AND relations?
pub(super) all_and_groups: HashMap<GroupRef, bool>,
}
impl Tree {
@ -50,6 +56,10 @@ impl Tree {
knn_expressions: b.knn_expressions,
knn_brute_force_expressions: b.knn_brute_force_expressions,
knn_condition: b.knn_condition,
all_expressions_with_index: b.leaf_nodes_count > 0
&& b.leaf_nodes_with_index_count == b.leaf_nodes_count,
all_and: b.all_and.unwrap_or(true),
all_and_groups: b.all_and_groups,
})
}
}
@ -65,13 +75,17 @@ struct TreeBuilder<'a> {
resolved_expressions: HashMap<Arc<Expression>, ResolvedExpression>,
resolved_idioms: HashMap<Arc<Idiom>, Node>,
index_map: IndexesMap,
with_indexes: Vec<IndexRef>,
with_indexes: Option<Vec<IndexRef>>,
knn_brute_force_expressions: HashMap<Arc<Expression>, KnnBruteForceExpression>,
knn_expressions: KnnExpressions,
idioms_record_options: HashMap<Arc<Idiom>, RecordOptions>,
group_sequence: GroupRef,
root: Option<Node>,
knn_condition: Option<Cond>,
leaf_nodes_count: usize,
leaf_nodes_with_index_count: usize,
all_and: Option<bool>,
all_and_groups: HashMap<GroupRef, bool>,
}
#[derive(Debug, Clone, Eq, PartialEq, Hash)]
@ -93,8 +107,8 @@ impl<'a> TreeBuilder<'a> {
orders: Option<&'a Orders>,
) -> Self {
let with_indexes = match with {
Some(With::Index(ixs)) => Vec::with_capacity(ixs.len()),
_ => vec![],
Some(With::Index(ixs)) => Some(Vec::with_capacity(ixs.len())),
_ => None,
};
let first_order = if let Some(o) = orders {
o.0.first()
@ -119,6 +133,10 @@ impl<'a> TreeBuilder<'a> {
group_sequence: 0,
root: None,
knn_condition: None,
all_and: None,
all_and_groups: Default::default(),
leaf_nodes_count: 0,
leaf_nodes_with_index_count: 0,
}
}
@ -188,7 +206,10 @@ impl<'a> TreeBuilder<'a> {
| Value::Param(_)
| Value::Null
| Value::None
| Value::Function(_) => Ok(Node::Computable),
| Value::Function(_) => {
self.leaf_nodes_count += 1;
Ok(Node::Computable)
}
Value::Array(a) => self.eval_array(stk, a).await,
Value::Subquery(s) => self.eval_subquery(stk, s).await,
_ => Ok(Node::Unsupported(format!("Unsupported value: {}", v))),
@ -207,6 +228,7 @@ impl<'a> TreeBuilder<'a> {
}
async fn eval_array(&mut self, stk: &mut Stk, a: &Array) -> Result<Node, Error> {
self.leaf_nodes_count += 1;
let mut values = Vec::with_capacity(a.len());
for v in &a.0 {
values.push(stk.run(|stk| v.compute(stk, self.ctx, self.opt, None)).await?);
@ -220,6 +242,7 @@ impl<'a> TreeBuilder<'a> {
group: GroupRef,
i: &Idiom,
) -> Result<Node, Error> {
self.leaf_nodes_count += 1;
// Check if the idiom has already been resolved
if let Some(node) = self.resolved_idioms.get(i).cloned() {
return Ok(node);
@ -273,7 +296,11 @@ impl<'a> TreeBuilder<'a> {
let ixr = self.index_map.definitions.len() as IndexRef;
if let Some(With::Index(ixs)) = &self.with {
if ixs.contains(&ix.name.0) {
self.with_indexes.push(ixr);
if let Some(wi) = &mut self.with_indexes {
wi.push(ixr);
} else {
self.with_indexes = Some(vec![ixr]);
}
}
}
self.index_map.definitions.push(ix.clone().into());
@ -343,7 +370,10 @@ impl<'a> TreeBuilder<'a> {
match e {
Expression::Unary {
..
} => Ok(Node::Unsupported("unary expressions not supported".to_string())),
} => {
self.leaf_nodes_count += 1;
Ok(Node::Unsupported("unary expressions not supported".to_string()))
}
Expression::Binary {
l,
o,
@ -353,6 +383,7 @@ impl<'a> TreeBuilder<'a> {
if let Some(re) = self.resolved_expressions.get(e).cloned() {
return Ok(re.into());
}
self.check_boolean_operator(group, o);
let left = stk.run(|stk| self.eval_value(stk, group, l)).await?;
let right = stk.run(|stk| self.eval_value(stk, group, r)).await?;
// If both values are computable, then we can delegate the computation to the parent
@ -362,9 +393,8 @@ impl<'a> TreeBuilder<'a> {
let exp = Arc::new(e.clone());
let left = Arc::new(self.compute(stk, l, left).await?);
let right = Arc::new(self.compute(stk, r, right).await?);
let mut io = None;
if let Some((id, local_irs, remote_irs)) = left.is_indexed_field() {
io = self.lookup_index_options(
let io = if let Some((id, local_irs, remote_irs)) = left.is_indexed_field() {
self.lookup_index_options(
o,
id,
&right,
@ -372,9 +402,9 @@ impl<'a> TreeBuilder<'a> {
IdiomPosition::Left,
local_irs,
remote_irs,
)?;
)?
} else if let Some((id, local_irs, remote_irs)) = right.is_indexed_field() {
io = self.lookup_index_options(
self.lookup_index_options(
o,
id,
&left,
@ -382,13 +412,16 @@ impl<'a> TreeBuilder<'a> {
IdiomPosition::Right,
local_irs,
remote_irs,
)?;
}
)?
} else {
None
};
if let Some(id) = left.is_field() {
self.eval_bruteforce_knn(id, &right, &exp)?;
} else if let Some(id) = right.is_field() {
self.eval_bruteforce_knn(id, &left, &exp)?;
}
self.check_leaf_node_with_index(io.as_ref());
let re = ResolvedExpression {
group,
exp: exp.clone(),
@ -402,6 +435,37 @@ impl<'a> TreeBuilder<'a> {
}
}
fn check_boolean_operator(&mut self, gr: GroupRef, op: &Operator) {
match op {
Operator::Neg | Operator::Or => {
if self.all_and != Some(false) {
self.all_and = Some(false);
}
self.all_and_groups.entry(gr).and_modify(|b| *b = false).or_insert(false);
}
Operator::And => {
if self.all_and.is_none() {
self.all_and = Some(true);
}
self.all_and_groups.entry(gr).or_insert(true);
}
_ => {
self.all_and_groups.entry(gr).or_insert(true);
}
}
}
fn check_leaf_node_with_index(&mut self, io: Option<&IndexOption>) {
if let Some(io) = io {
if let Some(wi) = &self.with_indexes {
if !wi.contains(&io.ix_ref()) {
return;
}
}
self.leaf_nodes_with_index_count += 2;
}
}
#[allow(clippy::too_many_arguments)]
fn lookup_index_options(
&mut self,

View file

@ -0,0 +1,4 @@
use std::sync::LazyLock;
pub static SURREALCS_CONNECTION_POOL_SIZE: LazyLock<usize> =
lazy_env_parse_or_else!("SURREAL_SURREALCS_CONNECTION_POOL_SIZE", usize, |_| num_cpus::get());

View file

@ -1,5 +1,7 @@
#![cfg(feature = "kv-surrealcs")]
mod cnf;
use crate::err::Error;
use crate::key::debug::Sprintable;
use crate::kvs::savepoint::{SaveOperation, SavePointImpl, SavePoints};
@ -72,7 +74,7 @@ impl Drop for Transaction {
impl Datastore {
/// Open a new database
pub(crate) async fn new(path: &str) -> Result<Datastore, Error> {
match create_connection_pool(path, None).await {
match create_connection_pool(path, Some(*cnf::SURREALCS_CONNECTION_POOL_SIZE)).await {
Ok(_) => Ok(Datastore {}),
Err(_) => {
Err(Error::Ds("Cannot connect to the `surrealcs` storage engine".to_string()))

View file

@ -172,9 +172,12 @@ pub trait RpcContext {
return Err(RpcError::InvalidParams);
};
let mut tmp_session = mem::take(self.session_mut());
crate::iam::verify::token(self.kvs(), &mut tmp_session, &token.0).await?;
let out: Result<(), RpcError> =
crate::iam::verify::token(self.kvs(), &mut tmp_session, &token.0)
.await
.map_err(Into::into);
*self.session_mut() = tmp_session;
Ok(Value::None.into())
out.map(|_| Value::None.into())
}
// ------------------------------

View file

@ -1,3 +1,4 @@
use crate::err::Error;
use crate::sql::duration::Duration;
use crate::sql::strand::Strand;
use crate::syn;
@ -11,6 +12,7 @@ use std::str;
use std::str::FromStr;
use super::escape::quote_str;
use super::value::TrySub;
pub(crate) const TOKEN: &str = "$surrealdb::private::sql::Datetime";
@ -108,3 +110,13 @@ impl ops::Sub<Self> for Datetime {
}
}
}
impl TrySub for Datetime {
type Output = Duration;
fn try_sub(self, other: Self) -> Result<Duration, Error> {
(self.0 - other.0)
.to_std()
.map_err(|_| Error::ArithmeticOverflow(format!("{self} - {other}")))
.map(Duration::from)
}
}

View file

@ -1,3 +1,4 @@
use crate::err::Error;
use crate::sql::datetime::Datetime;
use crate::sql::statements::info::InfoStructure;
use crate::sql::strand::Strand;
@ -12,6 +13,8 @@ use std::ops::Deref;
use std::str::FromStr;
use std::time;
use super::value::{TryAdd, TrySub};
pub(crate) static SECONDS_PER_YEAR: u64 = 365 * SECONDS_PER_DAY;
pub(crate) static SECONDS_PER_WEEK: u64 = 7 * SECONDS_PER_DAY;
pub(crate) static SECONDS_PER_DAY: u64 = 24 * SECONDS_PER_HOUR;
@ -235,6 +238,16 @@ impl ops::Add for Duration {
}
}
impl TryAdd for Duration {
type Output = Self;
fn try_add(self, other: Self) -> Result<Self, Error> {
self.0
.checked_add(other.0)
.ok_or_else(|| Error::ArithmeticOverflow(format!("{self} + {other}")))
.map(Duration::from)
}
}
impl<'a, 'b> ops::Add<&'b Duration> for &'a Duration {
type Output = Duration;
fn add(self, other: &'b Duration) -> Duration {
@ -245,6 +258,16 @@ impl<'a, 'b> ops::Add<&'b Duration> for &'a Duration {
}
}
impl<'a, 'b> TryAdd<&'b Duration> for &'a Duration {
type Output = Duration;
fn try_add(self, other: &'b Duration) -> Result<Duration, Error> {
self.0
.checked_add(other.0)
.ok_or_else(|| Error::ArithmeticOverflow(format!("{self} + {other}")))
.map(Duration::from)
}
}
impl ops::Sub for Duration {
type Output = Self;
fn sub(self, other: Self) -> Self {
@ -255,6 +278,16 @@ impl ops::Sub for Duration {
}
}
impl TrySub for Duration {
type Output = Self;
fn try_sub(self, other: Self) -> Result<Self, Error> {
self.0
.checked_sub(other.0)
.ok_or_else(|| Error::ArithmeticOverflow(format!("{self} - {other}")))
.map(Duration::from)
}
}
impl<'a, 'b> ops::Sub<&'b Duration> for &'a Duration {
type Output = Duration;
fn sub(self, other: &'b Duration) -> Duration {
@ -265,26 +298,68 @@ impl<'a, 'b> ops::Sub<&'b Duration> for &'a Duration {
}
}
impl<'a, 'b> TrySub<&'b Duration> for &'a Duration {
type Output = Duration;
fn try_sub(self, other: &'b Duration) -> Result<Duration, Error> {
self.0
.checked_sub(other.0)
.ok_or_else(|| Error::ArithmeticOverflow(format!("{self} - {other}")))
.map(Duration::from)
}
}
impl ops::Add<Datetime> for Duration {
type Output = Datetime;
fn add(self, other: Datetime) -> Datetime {
match chrono::Duration::from_std(self.0) {
Ok(d) => Datetime::from(other.0 + d),
Ok(d) => match other.0.checked_add_signed(d) {
Some(v) => Datetime::from(v),
None => Datetime::default(),
},
Err(_) => Datetime::default(),
}
}
}
impl TryAdd<Datetime> for Duration {
type Output = Datetime;
fn try_add(self, other: Datetime) -> Result<Datetime, Error> {
match chrono::Duration::from_std(self.0) {
Ok(d) => match other.0.checked_add_signed(d) {
Some(v) => Ok(Datetime::from(v)),
None => Err(Error::ArithmeticOverflow(format!("{self} + {other}"))),
},
Err(_) => Err(Error::ArithmeticOverflow(format!("{self} + {other}"))),
}
}
}
impl ops::Sub<Datetime> for Duration {
type Output = Datetime;
fn sub(self, other: Datetime) -> Datetime {
match chrono::Duration::from_std(self.0) {
Ok(d) => Datetime::from(other.0 - d),
Ok(d) => match other.0.checked_sub_signed(d) {
Some(v) => Datetime::from(v),
None => Datetime::default(),
},
Err(_) => Datetime::default(),
}
}
}
impl TrySub<Datetime> for Duration {
type Output = Datetime;
fn try_sub(self, other: Datetime) -> Result<Datetime, Error> {
match chrono::Duration::from_std(self.0) {
Ok(d) => match other.0.checked_sub_signed(d) {
Some(v) => Ok(Datetime::from(v)),
None => Err(Error::ArithmeticOverflow(format!("{self} - {other}"))),
},
Err(_) => Err(Error::ArithmeticOverflow(format!("{self} - {other}"))),
}
}
}
impl Sum<Self> for Duration {
fn sum<I>(iter: I) -> Duration
where

View file

@ -79,15 +79,12 @@ pub fn escape_ident(s: &str) -> Cow<'_, str> {
#[inline]
pub fn escape_normal<'a>(s: &'a str, l: char, r: char, e: &str) -> Cow<'a, str> {
// Loop over each character
for x in s.bytes() {
// Check if character is allowed
if !(x.is_ascii_alphanumeric() || x == b'_') {
return Cow::Owned(format!("{l}{}{r}", s.replace(r, e)));
}
// Is there no need to escape the value?
if s.bytes().all(|x| x.is_ascii_alphanumeric() || x == b'_') {
return Cow::Borrowed(s);
}
// Output the value
Cow::Borrowed(s)
Cow::Owned(format!("{l}{}{r}", s.replace(r, e)))
}
pub fn escape_reserved_keyword(s: &str) -> Option<String> {

View file

@ -1,3 +1,4 @@
use crate::err::Error;
use crate::sql::escape::quote_plain_str;
use revision::revisioned;
use serde::{Deserialize, Serialize};
@ -6,6 +7,8 @@ use std::ops::Deref;
use std::ops::{self};
use std::str;
use super::value::TryAdd;
pub(crate) const TOKEN: &str = "$surrealdb::private::sql::Strand";
/// A string that doesn't contain NUL bytes.
@ -72,6 +75,18 @@ impl ops::Add for Strand {
}
}
impl TryAdd for Strand {
type Output = Self;
fn try_add(mut self, other: Self) -> Result<Self, Error> {
if self.0.try_reserve(other.len()).is_ok() {
self.0.push_str(other.as_str());
Ok(self)
} else {
Err(Error::ArithmeticOverflow(format!("{self} + {other}")))
}
}
}
// serde(with = no_nul_bytes) will (de)serialize with no NUL bytes.
pub(crate) mod no_nul_bytes {
use serde::{

View file

@ -2981,10 +2981,10 @@ impl TryAdd for Value {
fn try_add(self, other: Self) -> Result<Self, Error> {
Ok(match (self, other) {
(Self::Number(v), Self::Number(w)) => Self::Number(v.try_add(w)?),
(Self::Strand(v), Self::Strand(w)) => Self::Strand(v + w),
(Self::Datetime(v), Self::Duration(w)) => Self::Datetime(w + v),
(Self::Duration(v), Self::Datetime(w)) => Self::Datetime(v + w),
(Self::Duration(v), Self::Duration(w)) => Self::Duration(v + w),
(Self::Strand(v), Self::Strand(w)) => Self::Strand(v.try_add(w)?),
(Self::Datetime(v), Self::Duration(w)) => Self::Datetime(w.try_add(v)?),
(Self::Duration(v), Self::Datetime(w)) => Self::Datetime(v.try_add(w)?),
(Self::Duration(v), Self::Duration(w)) => Self::Duration(v.try_add(w)?),
(v, w) => return Err(Error::TryAdd(v.to_raw_string(), w.to_raw_string())),
})
}
@ -2994,7 +2994,7 @@ impl TryAdd for Value {
pub(crate) trait TrySub<Rhs = Self> {
type Output;
fn try_sub(self, v: Self) -> Result<Self::Output, Error>;
fn try_sub(self, v: Rhs) -> Result<Self::Output, Error>;
}
impl TrySub for Value {
@ -3002,10 +3002,10 @@ impl TrySub for Value {
fn try_sub(self, other: Self) -> Result<Self, Error> {
Ok(match (self, other) {
(Self::Number(v), Self::Number(w)) => Self::Number(v.try_sub(w)?),
(Self::Datetime(v), Self::Datetime(w)) => Self::Duration(v - w),
(Self::Datetime(v), Self::Duration(w)) => Self::Datetime(w - v),
(Self::Duration(v), Self::Datetime(w)) => Self::Datetime(v - w),
(Self::Duration(v), Self::Duration(w)) => Self::Duration(v - w),
(Self::Datetime(v), Self::Datetime(w)) => Self::Duration(v.try_sub(w)?),
(Self::Datetime(v), Self::Duration(w)) => Self::Datetime(w.try_sub(v)?),
(Self::Duration(v), Self::Datetime(w)) => Self::Datetime(v.try_sub(w)?),
(Self::Duration(v), Self::Duration(w)) => Self::Duration(v.try_sub(w)?),
(v, w) => return Err(Error::TrySub(v.to_raw_string(), w.to_raw_string())),
})
}

View file

@ -82,9 +82,16 @@ pub fn numeric(lexer: &mut Lexer, start: Token) -> Result<Numeric, SyntaxError>
match start.kind {
t!("-") | t!("+") => number(lexer, start).map(Numeric::Number),
TokenKind::Digits => match lexer.reader.peek() {
Some(b'n' | b'm' | b's' | b'h' | b'y' | b'd' | b'w') => {
Some(b'n' | b'm' | b's' | b'h' | b'y' | b'w') => {
duration(lexer, start).map(Numeric::Duration)
}
Some(b'd') => {
if lexer.reader.peek1() == Some(b'e') {
number(lexer, start).map(Numeric::Number)
} else {
duration(lexer, start).map(Numeric::Duration)
}
}
Some(x) if !x.is_ascii() => duration(lexer, start).map(Numeric::Duration),
_ => number(lexer, start).map(Numeric::Number),
},

View file

@ -3,9 +3,10 @@ use std::collections::BTreeMap;
use reblessive::Stk;
use crate::{
sql::{Block, Geometry, Number, Object, Strand, Value},
sql::{Block, Geometry, Object, Strand, Value},
syn::{
error::bail,
lexer::compound,
parser::{enter_object_recursion, mac::expected, ParseResult, Parser},
token::{t, Glued, Span, TokenKind},
},
@ -602,10 +603,8 @@ impl Parser<'_> {
match token.kind {
x if Self::kind_is_keyword_like(x) => {
self.pop_peek();
let str = self.lexer.reader.span(token.span);
// Lexer should ensure that the token is valid utf-8
let str = std::str::from_utf8(str).unwrap().to_owned();
Ok(str)
let str = self.lexer.span_str(token.span);
Ok(str.to_string())
}
TokenKind::Identifier => {
self.pop_peek();
@ -616,9 +615,16 @@ impl Parser<'_> {
let str = self.next_token_value::<Strand>()?.0;
Ok(str)
}
TokenKind::Digits | TokenKind::Glued(Glued::Number) => {
let number = self.next_token_value::<Number>()?.to_string();
Ok(number)
TokenKind::Digits => {
self.pop_peek();
let span = self.lexer.lex_compound(token, compound::number)?.span;
let str = self.lexer.span_str(span);
Ok(str.to_string())
}
TokenKind::Glued(Glued::Number) => {
self.pop_peek();
let str = self.lexer.span_str(token.span);
Ok(str.to_string())
}
_ => unexpected!(self, token, "an object key"),
}

View file

@ -9,6 +9,11 @@ mod stmt;
mod streaming;
mod value;
#[test]
fn parse_large_test_file() {
test_parse!(parse_query, include_str!("../../../../test.surql")).unwrap();
}
#[test]
fn multiple_semicolons() {
let res = test_parse!(parse_query, r#";;"#).unwrap();
@ -109,6 +114,5 @@ fn escaped_params_backtick() {
#[test]
fn parse_immediate_insert_subquery() {
let res =
test_parse!(parse_query, r#"LET $insert = INSERT INTO t (SELECT true FROM 1);"#).unwrap();
test_parse!(parse_query, r#"LET $insert = INSERT INTO t (SELECT true FROM 1);"#).unwrap();
}

View file

@ -1,6 +1,7 @@
use std::collections::BTreeMap;
use reblessive::Stack;
use rust_decimal::Decimal;
use crate::{
sql::{
@ -37,6 +38,16 @@ fn parse_coordinate() {
assert_eq!(x.y(), -18.0);
}
#[test]
fn parse_numeric_object_key() {
let v = test_parse!(parse_value_table, "{ 00: 0 }").unwrap();
let Value::Object(object) = v else {
panic!("not an object");
};
assert!(object.len() == 1);
assert_eq!(object.get("00").cloned(), Some(Value::Number(Number::Int(0))));
}
#[test]
fn parse_like_operator() {
test_parse!(parse_value_field, "a ~ b").unwrap();
@ -156,6 +167,12 @@ fn parse_i64() {
assert_eq!(res, Value::Number(Number::Int(i64::MAX)));
}
#[test]
fn parse_decimal() {
let res = test_parse!(parse_value_field, r#" 0dec "#).unwrap();
assert_eq!(res, Value::Number(Number::Decimal(Decimal::ZERO)));
}
#[test]
fn constant_lowercase() {
let out = test_parse!(parse_value_field, r#" math::pi "#).unwrap();

View file

@ -54,7 +54,7 @@ mod api_integration {
name: String,
}
#[derive(Debug, Clone, Deserialize, PartialEq, PartialOrd)]
#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, PartialOrd)]
struct RecordBuf {
id: RecordId,
name: String,

View file

@ -502,6 +502,74 @@ async fn create_record_with_id_with_content() {
);
}
#[test_log::test(tokio::test)]
async fn create_record_with_id_in_content() {
#[derive(Debug, Serialize, Deserialize)]
pub struct Person {
pub id: u32,
pub name: &'static str,
pub job: &'static str,
}
#[derive(Debug, Serialize, Deserialize)]
pub struct Record {
#[allow(dead_code)]
pub id: RecordId,
}
let (permit, db) = new_db().await;
db.use_ns(NS).use_db(Ulid::new().to_string()).await.unwrap();
drop(permit);
let record: Option<RecordBuf> = db
.create(("user", "john"))
.content(RecordBuf {
id: RecordId::from_table_key("user", "john"),
name: "John Doe".to_owned(),
})
.await
.unwrap();
assert_eq!(record.unwrap().id, "user:john".parse::<RecordId>().unwrap());
let error = db
.create::<Option<RecordBuf>>(("user", "john"))
.content(RecordBuf {
id: RecordId::from_table_key("user", "jane"),
name: "John Doe".to_owned(),
})
.await
.unwrap_err();
match error {
surrealdb::Error::Db(DbError::IdMismatch {
..
}) => {}
surrealdb::Error::Api(ApiError::Query {
..
}) => {}
error => panic!("unexpected error; {error:?}"),
}
let _: Option<Record> = db
.create("person")
.content(Person {
id: 1010,
name: "Max Mustermann",
job: "chef",
})
.await
.unwrap();
let _: Option<Record> = db
.update(("person", 1010))
.content(Person {
id: 1010,
name: "Max Mustermann",
job: "IT Tech",
})
.await
.unwrap();
}
#[test_log::test(tokio::test)]
async fn insert_table() {
let (permit, db) = new_db().await;

View file

@ -1,5 +1,5 @@
use std::collections::HashMap;
use std::fmt::{Debug, Formatter};
use std::fmt::{Debug, Display, Formatter};
use std::future::Future;
use std::sync::Arc;
use std::thread::Builder;
@ -313,7 +313,11 @@ impl Test {
/// Panics if the expected value is not equal to the actual value.
/// Compliant with NaN and Constants.
#[allow(dead_code)]
pub fn expect_value(&mut self, val: Value) -> Result<&mut Self, Error> {
pub fn expect_value_info<I: Display>(
&mut self,
val: Value,
info: I,
) -> Result<&mut Self, Error> {
let tmp = self.next_value()?;
// Then check they are indeed the same values
//
@ -324,14 +328,19 @@ impl Test {
val
};
if val.is_nan() {
assert!(tmp.is_nan(), "Expected NaN but got: {tmp}");
assert!(tmp.is_nan(), "Expected NaN but got {info}: {tmp}");
} else {
assert_eq!(tmp, val, "{tmp:#}");
assert_eq!(tmp, val, "{info} {tmp:#}");
}
//
Ok(self)
}
#[allow(dead_code)]
pub fn expect_value(&mut self, val: Value) -> Result<&mut Self, Error> {
self.expect_value_info(val, "")
}
/// Expect values in the given slice to be present in the responses, following the same order.
#[allow(dead_code)]
pub fn expect_values(&mut self, values: &[Value]) -> Result<&mut Self, Error> {
@ -344,7 +353,15 @@ impl Test {
/// Expect the given value to be equals to the next response.
#[allow(dead_code)]
pub fn expect_val(&mut self, val: &str) -> Result<&mut Self, Error> {
self.expect_value(value(val).unwrap_or_else(|_| panic!("INVALID VALUE:\n{val}")))
self.expect_val_info(val, "")
}
#[allow(dead_code)]
pub fn expect_val_info<I: Display>(&mut self, val: &str, info: I) -> Result<&mut Self, Error> {
self.expect_value_info(
value(val).unwrap_or_else(|_| panic!("INVALID VALUE {info}:\n{val}")),
info,
)
}
#[allow(dead_code)]

View file

@ -188,7 +188,7 @@ fn three_multi_index_query(with: &str, parallel: &str) -> String {
DEFINE INDEX uniq_name ON TABLE person COLUMNS name UNIQUE;
DEFINE INDEX idx_genre ON TABLE person COLUMNS genre;
SELECT name FROM person {with} WHERE name = 'Jaime' OR genre = 'm' OR company @@ 'surrealdb' {parallel};
SELECT name FROM person {with} WHERE name = 'Jaime' OR genre = 'm' OR company @@ 'surrealdb' {parallel} EXPLAIN FULL;
SELECT name FROM person {with} WHERE name = 'Jaime' OR genre = 'm' OR company @@ 'surrealdb' {parallel} EXPLAIN FULL;
SELECT name FROM person {with} WHERE name = 'Jaime' AND genre = 'm' AND company @@ 'surrealdb' {parallel};
SELECT name FROM person {with} WHERE name = 'Jaime' AND genre = 'm' AND company @@ 'surrealdb' {parallel} EXPLAIN FULL;")
}
@ -3063,3 +3063,35 @@ async fn select_composite_standard_index() -> Result<(), Error> {
async fn select_composite_unique_index() -> Result<(), Error> {
select_composite_index(true).await
}
#[tokio::test]
async fn select_where_index_boolean_behaviour() -> Result<(), Error> {
let sql = r"
DEFINE INDEX flagIndex ON TABLE test COLUMNS flag;
CREATE test:t CONTENT { flag:true };
CREATE test:f CONTENT { flag:false };
SELECT * FROM test;
SELECT * FROM test WITH NOINDEX WHERE (true OR flag=true);
SELECT * FROM test WITH NOINDEX WHERE (true OR flag==true);
SELECT * FROM test WHERE (true OR flag=true);
SELECT * FROM test WHERE (true OR flag==true);";
let mut t = Test::new(sql).await?;
t.expect_size(8)?;
t.skip_ok(3)?;
for i in 0..5 {
t.expect_val_info(
"[
{
flag: false,
id: test:f
},
{
flag: true,
id: test:t
}
]",
i,
)?;
}
Ok(())
}

View file

@ -1505,6 +1505,25 @@ async fn session_reauthentication_expired() {
server.finish().unwrap();
}
#[test(tokio::test)]
async fn session_failed_reauthentication() {
// Setup database server without authentication
let (addr, mut server) = common::start_server_without_auth().await.unwrap();
// Connect to WebSocket
let mut socket = Socket::connect(&addr, SERVER, FORMAT).await.unwrap();
// Specify a namespace and database to use
socket.send_message_use(Some(NS), Some(DB)).await.unwrap();
// Check that we have are have a database and namespace selected
socket.send_message_query("INFO FOR DB").await.unwrap();
// Authenticate using an invalid token
socket.send_request("authenticate", json!(["invalid",])).await.unwrap();
// Check to see if we still have a namespace and database selected
let res = socket.send_message_query("INFO FOR DB").await.unwrap();
assert_eq!(res[0]["status"], "OK", "result: {res:?}");
// Test passed
server.finish().unwrap();
}
#[test(tokio::test)]
async fn session_use_change_database() {
// Setup database server