Feature: Support of range queries in the query planner (#2656)

This commit is contained in:
Emmanuel Keller 2023-09-12 16:43:32 +01:00 committed by GitHub
parent 3ab03b00a8
commit e4c6d817c4
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
12 changed files with 1201 additions and 256 deletions

View file

@ -27,7 +27,7 @@ impl SyncDistinct {
}
fn is_distinct(ctx: &Context<'_>, i: &Iterable) -> bool {
if let Iterable::Index(t, ir, _) = i {
if let Iterable::Index(t, ir) = i {
if let Some(pla) = ctx.get_query_planner() {
if let Some(exe) = pla.get_query_executor(&t.0) {
return exe.is_distinct(*ir);

View file

@ -17,7 +17,7 @@ impl Explanation {
Some(e) => {
let mut exp = Self::default();
for i in iterables {
exp.add_iter(i);
exp.add_iter(ctx, i);
}
if let Some(qp) = ctx.get_query_planner() {
for reason in qp.fallbacks() {
@ -29,8 +29,8 @@ impl Explanation {
}
}
fn add_iter(&mut self, iter: &Iterable) {
self.0.push(ExplainItem::new_iter(iter));
fn add_iter(&mut self, ctx: &Context<'_>, iter: &Iterable) {
self.0.push(ExplainItem::new_iter(ctx, iter));
}
pub(super) fn add_fetch(&mut self, count: usize) {
@ -68,7 +68,7 @@ impl ExplainItem {
}
}
fn new_iter(iter: &Iterable) -> Self {
fn new_iter(ctx: &Context<'_>, iter: &Iterable) -> Self {
match iter {
Iterable::Value(v) => Self {
name: "Iterate Value".into(),
@ -102,10 +102,18 @@ impl ExplainItem {
("thing-3", Value::Thing(t3.to_owned())),
],
},
Iterable::Index(t, _, io) => Self {
name: "Iterate Index".into(),
details: vec![("table", Value::from(t.0.to_owned())), ("plan", io.explain())],
},
Iterable::Index(t, ir) => {
let mut details = vec![("table", Value::from(t.0.to_owned()))];
if let Some(qp) = ctx.get_query_planner() {
if let Some(exe) = qp.get_query_executor(&t.0) {
details.push(("plan", exe.explain(*ir)));
}
}
Self {
name: "Iterate Index".into(),
details,
}
}
}
}
}

View file

@ -10,7 +10,6 @@ use crate::doc::Document;
use crate::err::Error;
use crate::idx::ft::docids::DocId;
use crate::idx::planner::executor::IteratorRef;
use crate::idx::planner::plan::IndexOption;
use crate::sql::array::Array;
use crate::sql::edges::Edges;
use crate::sql::field::Field;
@ -32,7 +31,7 @@ pub(crate) enum Iterable {
Edges(Edges),
Mergeable(Thing, Value),
Relatable(Thing, Thing, Thing),
Index(Table, IteratorRef, IndexOption),
Index(Table, IteratorRef),
}
pub(crate) struct Processed {

View file

@ -6,7 +6,6 @@ use crate::dbs::distinct::SyncDistinct;
use crate::dbs::{Iterable, Iterator, Operable, Options, Processed, Statement, Transaction};
use crate::err::Error;
use crate::idx::planner::executor::IteratorRef;
use crate::idx::planner::plan::IndexOption;
use crate::key::{graph, thing};
use crate::sql::dir::Dir;
use crate::sql::{Edges, Range, Table, Thing, Value};
@ -97,9 +96,7 @@ impl<'a> Processor<'a> {
Iterable::Table(v) => self.process_table(ctx, opt, txn, stm, v).await?,
Iterable::Range(v) => self.process_range(ctx, opt, txn, stm, v).await?,
Iterable::Edges(e) => self.process_edge(ctx, opt, txn, stm, e).await?,
Iterable::Index(t, ir, io) => {
self.process_index(ctx, opt, txn, stm, t, ir, io).await?
}
Iterable::Index(t, ir) => self.process_index(ctx, opt, txn, stm, t, ir).await?,
Iterable::Mergeable(v, o) => {
self.process_mergeable(ctx, opt, txn, stm, v, o).await?
}
@ -545,13 +542,12 @@ impl<'a> Processor<'a> {
stm: &Statement<'_>,
table: Table,
ir: IteratorRef,
io: IndexOption,
) -> Result<(), Error> {
// Check that the table exists
txn.lock().await.check_ns_db_tb(opt.ns(), opt.db(), &table.0, opt.strict).await?;
if let Some(pla) = ctx.get_query_planner() {
if let Some(exe) = pla.get_query_executor(&table.0) {
if let Some(mut iterator) = exe.new_iterator(opt, ir, io).await? {
if let Some(mut iterator) = exe.new_iterator(opt, ir).await? {
let mut things = iterator.next_batch(txn, PROCESSOR_BATCH_SIZE).await?;
while !things.is_empty() {
// Check if the context is finished

View file

@ -181,10 +181,8 @@ pub(crate) async fn matches(
// it means that we are using an Iterator::Index
// and we are iterating over documents that already matches the expression.
if let Some(ir) = doc.ir {
if let Some(e) = exe.get_iterator_expression(ir) {
if e.eq(exp) {
return Ok(Value::Bool(true));
}
if exe.is_iterator_expression(ir, exp) {
return Ok(Value::Bool(true));
}
}
// Evaluate the matches

View file

@ -6,36 +6,60 @@ use crate::idx::ft::termdocs::TermsDocs;
use crate::idx::ft::terms::TermId;
use crate::idx::ft::{FtIndex, MatchRef};
use crate::idx::planner::iterators::{
MatchesThingIterator, NonUniqueEqualThingIterator, ThingIterator, UniqueEqualThingIterator,
IndexEqualThingIterator, IndexRangeThingIterator, MatchesThingIterator, ThingIterator,
UniqueEqualThingIterator, UniqueRangeThingIterator,
};
use crate::idx::planner::plan::IndexOption;
use crate::idx::planner::tree::IndexMap;
use crate::idx::planner::plan::IndexOperator::Matches;
use crate::idx::planner::plan::{IndexOperator, IndexOption, RangeValue};
use crate::idx::planner::tree::{IndexMap, IndexRef};
use crate::idx::trees::store::TreeStoreType;
use crate::idx::IndexKeyBase;
use crate::kvs;
use crate::kvs::Key;
use crate::sql::index::Index;
use crate::sql::{Expression, Operator, Table, Thing, Value};
use std::collections::HashMap;
use crate::sql::statements::DefineIndexStatement;
use crate::sql::{Expression, Object, Table, Thing, Value};
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use tokio::sync::RwLock;
pub(crate) type IteratorRef = u16;
pub(crate) struct QueryExecutor {
table: String,
ft_map: HashMap<String, FtIndex>,
ft_map: HashMap<IndexRef, FtIndex>,
mr_entries: HashMap<MatchRef, FtEntry>,
exp_entries: HashMap<Expression, FtEntry>,
iterators: Vec<Expression>,
exp_entries: HashMap<Arc<Expression>, FtEntry>,
it_entries: Vec<IteratorEntry>,
index_definitions: HashMap<IndexRef, DefineIndexStatement>,
}
pub(crate) type IteratorRef = u16;
pub(super) enum IteratorEntry {
Single(Arc<Expression>, IndexOption),
Range(HashSet<Arc<Expression>>, IndexRef, RangeValue, RangeValue),
}
impl IteratorEntry {
pub(super) fn explain(&self, e: &mut HashMap<&str, Value>) -> IndexRef {
match self {
Self::Single(_, io) => {
io.explain(e);
io.ir()
}
Self::Range(_, ir, from, to) => {
e.insert("from", Value::from(from));
e.insert("to", Value::from(to));
*ir
}
}
}
}
impl QueryExecutor {
pub(super) async fn new(
opt: &Options,
txn: &Transaction,
table: &Table,
index_map: IndexMap,
im: IndexMap,
) -> Result<Self, Error> {
let mut run = txn.lock().await;
@ -45,28 +69,29 @@ impl QueryExecutor {
// Create all the instances of FtIndex
// Build the FtEntries and map them to Expressions and MatchRef
for (exp, io) in index_map.consume() {
for (exp, io) in im.options {
let mut entry = None;
if let Index::Search(p) = &io.ix().index {
let ixn = &io.ix().name.0;
if let Some(ft) = ft_map.get(ixn) {
if entry.is_none() {
entry = FtEntry::new(&mut run, ft, io).await?;
let ir = io.ir();
if let Some(idx_def) = im.definitions.get(&ir) {
if let Index::Search(p) = &idx_def.index {
if let Some(ft) = ft_map.get(&ir) {
if entry.is_none() {
entry = FtEntry::new(&mut run, ft, io).await?;
}
} else {
let ikb = IndexKeyBase::new(opt, idx_def);
let az = run.get_db_analyzer(opt.ns(), opt.db(), p.az.as_str()).await?;
let ft = FtIndex::new(&mut run, az, ikb, p, TreeStoreType::Read).await?;
if entry.is_none() {
entry = FtEntry::new(&mut run, &ft, io).await?;
}
ft_map.insert(ir, ft);
}
} else {
let ikb = IndexKeyBase::new(opt, io.ix());
let az = run.get_db_analyzer(opt.ns(), opt.db(), p.az.as_str()).await?;
let ft = FtIndex::new(&mut run, az, ikb, p, TreeStoreType::Read).await?;
let ixn = ixn.to_owned();
if entry.is_none() {
entry = FtEntry::new(&mut run, &ft, io).await?;
}
ft_map.insert(ixn, ft);
}
}
if let Some(e) = entry {
if let Some(mr) = e.0.index_option.match_ref() {
if let Matches(_, Some(mr)) = e.0.index_option.op() {
if mr_entries.insert(*mr, e.clone()).is_some() {
return Err(Error::DuplicatedMatchRef {
mr: *mr,
@ -82,22 +107,41 @@ impl QueryExecutor {
ft_map,
mr_entries,
exp_entries,
iterators: Vec::new(),
it_entries: Vec::new(),
index_definitions: im.definitions,
})
}
pub(super) fn add_iterator(&mut self, exp: Expression) -> IteratorRef {
let ir = self.iterators.len();
self.iterators.push(exp);
pub(super) fn add_iterator(&mut self, it_entry: IteratorEntry) -> IteratorRef {
let ir = self.it_entries.len();
self.it_entries.push(it_entry);
ir as IteratorRef
}
pub(crate) fn is_distinct(&self, ir: IteratorRef) -> bool {
(ir as usize) < self.iterators.len()
(ir as usize) < self.it_entries.len()
}
pub(crate) fn get_iterator_expression(&self, ir: IteratorRef) -> Option<&Expression> {
self.iterators.get(ir as usize)
pub(crate) fn is_iterator_expression(&self, ir: IteratorRef, exp: &Expression) -> bool {
match self.it_entries.get(ir as usize) {
Some(IteratorEntry::Single(e, ..)) => exp.eq(e.as_ref()),
Some(IteratorEntry::Range(es, ..)) => es.contains(exp),
_ => false,
}
}
pub(crate) fn explain(&self, itr: IteratorRef) -> Value {
match self.it_entries.get(itr as usize) {
Some(ie) => {
let mut e = HashMap::default();
let ir = ie.explain(&mut e);
if let Some(ix) = self.index_definitions.get(&ir) {
e.insert("index", Value::from(ix.name.0.to_owned()));
}
Value::from(Object::from(e))
}
None => Value::None,
}
}
fn get_match_ref(match_ref: &Value) -> Option<MatchRef> {
@ -113,43 +157,87 @@ impl QueryExecutor {
&self,
opt: &Options,
ir: IteratorRef,
io: IndexOption,
) -> Result<Option<ThingIterator>, Error> {
match &io.ix().index {
Index::Idx => Self::new_index_iterator(opt, io),
Index::Uniq => Self::new_unique_index_iterator(opt, io),
Index::Search {
..
} => self.new_search_index_iterator(ir, io).await,
_ => Err(Error::FeatureNotYetImplemented {
feature: "VectorSearch iterator".to_string(),
}),
if let Some(it_entry) = self.it_entries.get(ir as usize) {
match it_entry {
IteratorEntry::Single(_, io) => {
if let Some(ix) = self.index_definitions.get(&io.ir()) {
match ix.index {
Index::Idx => Self::new_index_iterator(opt, ix, io.clone()),
Index::Uniq => Self::new_unique_index_iterator(opt, ix, io.clone()),
Index::Search {
..
} => self.new_search_index_iterator(ir, io.clone()).await,
Index::MTree(_) => Err(Error::FeatureNotYetImplemented {
feature: "VectorSearch iterator".to_string(),
}),
}
} else {
Ok(None)
}
}
IteratorEntry::Range(_, ir, from, to) => {
Ok(self.new_range_iterator(opt, *ir, from, to)?)
}
}
} else {
Ok(None)
}
}
fn new_index_iterator(opt: &Options, io: IndexOption) -> Result<Option<ThingIterator>, Error> {
if io.op() == &Operator::Equal {
return Ok(Some(ThingIterator::NonUniqueEqual(NonUniqueEqualThingIterator::new(
opt,
io.ix(),
io.array(),
)?)));
fn new_index_iterator(
opt: &Options,
ix: &DefineIndexStatement,
io: IndexOption,
) -> Result<Option<ThingIterator>, Error> {
match io.op() {
IndexOperator::Equality(array) => {
Ok(Some(ThingIterator::IndexEqual(IndexEqualThingIterator::new(opt, ix, array)?)))
}
IndexOperator::RangePart(_, _) => Ok(None), // TODO
_ => Ok(None),
}
}
fn new_range_iterator(
&self,
opt: &Options,
ir: IndexRef,
from: &RangeValue,
to: &RangeValue,
) -> Result<Option<ThingIterator>, Error> {
if let Some(ix) = self.index_definitions.get(&ir) {
match ix.index {
Index::Idx => {
return Ok(Some(ThingIterator::IndexRange(IndexRangeThingIterator::new(
opt, ix, from, to,
))))
}
Index::Uniq => {
return Ok(Some(ThingIterator::UniqueRange(UniqueRangeThingIterator::new(
opt, ix, from, to,
))))
}
_ => {}
}
}
Ok(None)
}
fn new_unique_index_iterator(
opt: &Options,
ix: &DefineIndexStatement,
io: IndexOption,
) -> Result<Option<ThingIterator>, Error> {
if io.op() == &Operator::Equal {
return Ok(Some(ThingIterator::UniqueEqual(UniqueEqualThingIterator::new(
opt,
io.ix(),
io.array(),
)?)));
match io.op() {
IndexOperator::Equality(array) => {
Ok(Some(ThingIterator::UniqueEqual(UniqueEqualThingIterator::new(opt, ix, array)?)))
}
IndexOperator::RangePart(_, _) => {
todo!()
}
_ => Ok(None),
}
Ok(None)
}
async fn new_search_index_iterator(
@ -157,11 +245,10 @@ impl QueryExecutor {
ir: IteratorRef,
io: IndexOption,
) -> Result<Option<ThingIterator>, Error> {
if let Some(exp) = self.iterators.get(ir as usize) {
if let Operator::Matches(_) = io.op() {
let ixn = &io.ix().name.0;
if let Some(fti) = self.ft_map.get(ixn) {
if let Some(fte) = self.exp_entries.get(exp) {
if let Some(IteratorEntry::Single(exp, ..)) = self.it_entries.get(ir as usize) {
if let Matches(_, _) = io.op() {
if let Some(fti) = self.ft_map.get(&io.ir()) {
if let Some(fte) = self.exp_entries.get(exp.as_ref()) {
let it = MatchesThingIterator::new(fti, fte.0.terms_docs.clone()).await?;
return Ok(Some(ThingIterator::Matches(it)));
}
@ -223,7 +310,7 @@ impl QueryExecutor {
fn get_ft_entry_and_index(&self, match_ref: &Value) -> Option<(&FtEntry, &FtIndex)> {
if let Some(e) = self.get_ft_entry(match_ref) {
if let Some(ft) = self.ft_map.get(&e.0.index_option.ix().name.0) {
if let Some(ft) = self.ft_map.get(&e.0.index_option.ir()) {
return Some((e, ft));
}
}
@ -304,7 +391,7 @@ impl FtEntry {
ft: &FtIndex,
io: IndexOption,
) -> Result<Option<Self>, Error> {
if let Some(qs) = io.qs() {
if let Matches(qs, _) = io.op() {
let terms = ft.extract_terms(tx, qs.to_owned()).await?;
let terms_docs = Arc::new(ft.get_terms_docs(tx, &terms).await?);
Ok(Some(Self(Arc::new(Inner {

View file

@ -3,14 +3,17 @@ use crate::err::Error;
use crate::idx::ft::docids::{DocId, NO_DOC_ID};
use crate::idx::ft::termdocs::TermsDocs;
use crate::idx::ft::{FtIndex, HitsIterator};
use crate::key;
use crate::idx::planner::plan::RangeValue;
use crate::key::index::Index;
use crate::kvs::Key;
use crate::sql::statements::DefineIndexStatement;
use crate::sql::{Array, Thing};
use crate::sql::{Array, Thing, Value};
pub(crate) enum ThingIterator {
NonUniqueEqual(NonUniqueEqualThingIterator),
IndexEqual(IndexEqualThingIterator),
IndexRange(IndexRangeThingIterator),
UniqueEqual(UniqueEqualThingIterator),
UniqueRange(UniqueRangeThingIterator),
Matches(MatchesThingIterator),
}
@ -21,26 +24,24 @@ impl ThingIterator {
size: u32,
) -> Result<Vec<(Thing, DocId)>, Error> {
match self {
ThingIterator::NonUniqueEqual(i) => i.next_batch(tx, size).await,
ThingIterator::UniqueEqual(i) => i.next_batch(tx, size).await,
ThingIterator::IndexEqual(i) => i.next_batch(tx, size).await,
ThingIterator::UniqueEqual(i) => i.next_batch(tx).await,
ThingIterator::IndexRange(i) => i.next_batch(tx, size).await,
ThingIterator::UniqueRange(i) => i.next_batch(tx, size).await,
ThingIterator::Matches(i) => i.next_batch(tx, size).await,
}
}
}
pub(crate) struct NonUniqueEqualThingIterator {
pub(crate) struct IndexEqualThingIterator {
beg: Vec<u8>,
end: Vec<u8>,
}
impl NonUniqueEqualThingIterator {
pub(super) fn new(
opt: &Options,
ix: &DefineIndexStatement,
v: &Array,
) -> Result<NonUniqueEqualThingIterator, Error> {
let (beg, end) =
key::index::Index::range_all_ids(opt.ns(), opt.db(), &ix.what, &ix.name, v);
impl IndexEqualThingIterator {
pub(super) fn new(opt: &Options, ix: &DefineIndexStatement, v: &Array) -> Result<Self, Error> {
let beg = Index::prefix_ids_beg(opt.ns(), opt.db(), &ix.what, &ix.name, v);
let end = Index::prefix_ids_end(opt.ns(), opt.db(), &ix.what, &ix.name, v);
Ok(Self {
beg,
end,
@ -64,23 +65,127 @@ impl NonUniqueEqualThingIterator {
}
}
struct RangeScan {
beg: Vec<u8>,
end: Vec<u8>,
beg_excl: Option<Vec<u8>>,
end_excl: Option<Vec<u8>>,
}
impl RangeScan {
fn new(beg: Vec<u8>, beg_incl: bool, end: Vec<u8>, end_incl: bool) -> Self {
let beg_excl = if !beg_incl {
Some(beg.clone())
} else {
None
};
let end_excl = if !end_incl {
Some(end.clone())
} else {
None
};
Self {
beg,
end,
beg_excl,
end_excl,
}
}
fn matches(&mut self, k: &Key) -> bool {
if let Some(b) = &self.beg_excl {
if b.eq(k) {
self.beg_excl = None;
return false;
}
}
if let Some(e) = &self.end_excl {
if e.eq(k) {
self.end_excl = None;
return false;
}
}
true
}
}
pub(crate) struct IndexRangeThingIterator {
r: RangeScan,
}
impl IndexRangeThingIterator {
pub(super) fn new(
opt: &Options,
ix: &DefineIndexStatement,
from: &RangeValue,
to: &RangeValue,
) -> Self {
let beg = Self::compute_beg(opt, ix, from);
let end = Self::compute_end(opt, ix, to);
Self {
r: RangeScan::new(beg, from.inclusive, end, to.inclusive),
}
}
fn compute_beg(opt: &Options, ix: &DefineIndexStatement, from: &RangeValue) -> Vec<u8> {
if from.value == Value::None {
return Index::prefix_beg(opt.ns(), opt.db(), &ix.what, &ix.name);
}
let fd = Array::from(from.value.to_owned());
if from.inclusive {
Index::prefix_ids_beg(opt.ns(), opt.db(), &ix.what, &ix.name, &fd)
} else {
Index::prefix_ids_end(opt.ns(), opt.db(), &ix.what, &ix.name, &fd)
}
}
fn compute_end(opt: &Options, ix: &DefineIndexStatement, to: &RangeValue) -> Vec<u8> {
if to.value == Value::None {
return Index::prefix_end(opt.ns(), opt.db(), &ix.what, &ix.name);
}
let fd = Array::from(to.value.to_owned());
if to.inclusive {
Index::prefix_ids_end(opt.ns(), opt.db(), &ix.what, &ix.name, &fd)
} else {
Index::prefix_ids_beg(opt.ns(), opt.db(), &ix.what, &ix.name, &fd)
}
}
async fn next_batch(
&mut self,
txn: &Transaction,
limit: u32,
) -> Result<Vec<(Thing, DocId)>, Error> {
let min = self.r.beg.clone();
let max = self.r.end.clone();
let res = txn.lock().await.scan(min..max, limit).await?;
if let Some((key, _)) = res.last() {
self.r.beg = key.clone();
self.r.beg.push(0x00);
}
let mut r = Vec::with_capacity(res.len());
for (k, v) in res {
if self.r.matches(&k) {
r.push((v.into(), NO_DOC_ID));
}
}
Ok(r)
}
}
pub(crate) struct UniqueEqualThingIterator {
key: Option<Key>,
}
impl UniqueEqualThingIterator {
pub(super) fn new(opt: &Options, ix: &DefineIndexStatement, a: &Array) -> Result<Self, Error> {
let key = key::index::Index::new(opt.ns(), opt.db(), &ix.what, &ix.name, a, None).into();
let key = Index::new(opt.ns(), opt.db(), &ix.what, &ix.name, a, None).into();
Ok(Self {
key: Some(key),
})
}
async fn next_batch(
&mut self,
txn: &Transaction,
_limit: u32,
) -> Result<Vec<(Thing, DocId)>, Error> {
async fn next_batch(&mut self, txn: &Transaction) -> Result<Vec<(Thing, DocId)>, Error> {
if let Some(key) = self.key.take() {
if let Some(val) = txn.lock().await.get(key).await? {
return Ok(vec![(val.into(), NO_DOC_ID)]);
@ -90,6 +195,86 @@ impl UniqueEqualThingIterator {
}
}
pub(crate) struct UniqueRangeThingIterator {
r: RangeScan,
done: bool,
}
impl UniqueRangeThingIterator {
pub(super) fn new(
opt: &Options,
ix: &DefineIndexStatement,
from: &RangeValue,
to: &RangeValue,
) -> Self {
let beg = Self::compute_beg(opt, ix, from);
let end = Self::compute_end(opt, ix, to);
Self {
r: RangeScan::new(beg, from.inclusive, end, to.inclusive),
done: false,
}
}
fn compute_beg(opt: &Options, ix: &DefineIndexStatement, from: &RangeValue) -> Vec<u8> {
if from.value == Value::None {
return Index::prefix_beg(opt.ns(), opt.db(), &ix.what, &ix.name);
}
Index::new(
opt.ns(),
opt.db(),
&ix.what,
&ix.name,
&Array::from(from.value.to_owned()),
None,
)
.encode()
.unwrap()
}
fn compute_end(opt: &Options, ix: &DefineIndexStatement, to: &RangeValue) -> Vec<u8> {
if to.value == Value::None {
return Index::prefix_end(opt.ns(), opt.db(), &ix.what, &ix.name);
}
Index::new(opt.ns(), opt.db(), &ix.what, &ix.name, &Array::from(to.value.to_owned()), None)
.encode()
.unwrap()
}
async fn next_batch(
&mut self,
txn: &Transaction,
mut limit: u32,
) -> Result<Vec<(Thing, DocId)>, Error> {
if self.done {
return Ok(vec![]);
}
let min = self.r.beg.clone();
let max = self.r.end.clone();
limit += 1;
let mut tx = txn.lock().await;
let res = tx.scan(min..max, limit).await?;
let mut r = Vec::with_capacity(res.len());
for (k, v) in res {
limit -= 1;
if limit == 0 {
self.r.beg = k;
return Ok(r);
}
if self.r.matches(&k) {
r.push((v.into(), NO_DOC_ID));
}
}
let end = self.r.end.clone();
if self.r.matches(&end) {
if let Some(v) = tx.get(end).await? {
r.push((v.into(), NO_DOC_ID));
}
}
self.done = true;
Ok(r)
}
}
pub(crate) struct MatchesThingIterator {
hits: Option<HitsIterator>,
}

View file

@ -6,7 +6,7 @@ mod tree;
use crate::ctx::Context;
use crate::dbs::{Iterable, Iterator, Options, Transaction};
use crate::err::Error;
use crate::idx::planner::executor::QueryExecutor;
use crate::idx::planner::executor::{IteratorEntry, QueryExecutor};
use crate::idx::planner::plan::{Plan, PlanBuilder};
use crate::idx::planner::tree::Tree;
use crate::sql::with::With;
@ -42,23 +42,29 @@ impl<'a> QueryPlanner<'a> {
t: Table,
it: &mut Iterator,
) -> Result<(), Error> {
match Tree::build(ctx, self.opt, txn, &t, self.cond).await? {
Some((node, im)) => {
match Tree::build(ctx, self.opt, txn, &t, self.cond, self.with).await? {
Some((node, im, with_indexes)) => {
let mut exe = QueryExecutor::new(self.opt, txn, &t, im).await?;
match PlanBuilder::build(node, self.with)? {
match PlanBuilder::build(node, self.with, with_indexes)? {
Plan::SingleIndex(exp, io) => {
let ir = exe.add_iterator(exp);
it.ingest(Iterable::Index(t.clone(), ir, io));
let ir = exe.add_iterator(IteratorEntry::Single(exp, io));
it.ingest(Iterable::Index(t.clone(), ir));
self.executors.insert(t.0.clone(), exe);
}
Plan::MultiIndex(v) => {
for (exp, io) in v {
let ir = exe.add_iterator(exp);
it.ingest(Iterable::Index(t.clone(), ir, io));
let ir = exe.add_iterator(IteratorEntry::Single(exp, io));
it.ingest(Iterable::Index(t.clone(), ir));
self.requires_distinct = true;
}
self.executors.insert(t.0.clone(), exe);
}
Plan::SingleIndexMultiExpression(ixn, rq) => {
let ir =
exe.add_iterator(IteratorEntry::Range(rq.exps, ixn, rq.from, rq.to));
it.ingest(Iterable::Index(t.clone(), ir));
self.executors.insert(t.0.clone(), exe);
}
Plan::TableIterator(fallback) => {
if let Some(fallback) = fallback {
self.fallbacks.push(fallback);

View file

@ -1,31 +1,35 @@
use crate::err::Error;
use crate::idx::ft::MatchRef;
use crate::idx::planner::tree::Node;
use crate::sql::statements::DefineIndexStatement;
use crate::idx::planner::tree::{IndexRef, Node};
use crate::sql::with::With;
use crate::sql::{Array, Object};
use crate::sql::{Expression, Idiom, Operator, Value};
use std::collections::HashMap;
use std::collections::hash_map::Entry;
use std::collections::{HashMap, HashSet};
use std::hash::Hash;
use std::sync::Arc;
pub(super) struct PlanBuilder<'a> {
indexes: Vec<(Expression, IndexOption)>,
with: &'a Option<With>,
pub(super) struct PlanBuilder {
indexes: Vec<(Arc<Expression>, IndexOption)>,
range_queries: HashMap<IndexRef, RangeQueryBuilder>,
with_indexes: Vec<IndexRef>,
all_and: bool,
all_exp_with_index: bool,
}
impl<'a> PlanBuilder<'a> {
pub(super) fn build(root: Node, with: &'a Option<With>) -> Result<Plan, Error> {
if let Some(with) = with {
if matches!(with, With::NoIndex) {
return Ok(Plan::TableIterator(Some("WITH NOINDEX".to_string())));
}
impl PlanBuilder {
pub(super) fn build(
root: Node,
with: &Option<With>,
with_indexes: Vec<IndexRef>,
) -> Result<Plan, Error> {
if let Some(With::NoIndex) = with {
return Ok(Plan::TableIterator(Some("WITH NOINDEX".to_string())));
}
let mut b = PlanBuilder {
indexes: Vec::new(),
with,
range_queries: HashMap::new(),
with_indexes,
all_and: true,
all_exp_with_index: true,
};
@ -37,8 +41,15 @@ impl<'a> PlanBuilder<'a> {
if b.indexes.is_empty() {
return Ok(Plan::TableIterator(Some("NO INDEX FOUND".to_string())));
}
// If every boolean operator are AND then we can use the single index plan
if b.all_and {
// TODO: This is currently pretty arbitrary
// We take the "first" range query if one is available
if let Some((ir, rq)) = b.range_queries.drain().take(1).next() {
return Ok(Plan::SingleIndexMultiExpression(ir, rq));
}
// Otherwise we take the first single index option
if let Some((e, i)) = b.indexes.pop() {
return Ok(Plan::SingleIndex(e, i));
}
@ -53,10 +64,8 @@ impl<'a> PlanBuilder<'a> {
// 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 let Some(With::Index(ixs)) = self.with {
if !ixs.contains(&io.ix().name.0) {
return None;
}
if !self.with_indexes.is_empty() && !self.with_indexes.contains(&io.ir()) {
return None;
}
}
io
@ -101,15 +110,28 @@ impl<'a> PlanBuilder<'a> {
}
}
fn add_index_option(&mut self, e: Expression, i: IndexOption) {
self.indexes.push((e, i));
fn add_index_option(&mut self, exp: Arc<Expression>, io: IndexOption) {
if let IndexOperator::RangePart(o, v) = io.op() {
match self.range_queries.entry(io.ir()) {
Entry::Occupied(mut e) => {
e.get_mut().add(exp.clone(), o, v);
}
Entry::Vacant(e) => {
let mut b = RangeQueryBuilder::default();
b.add(exp.clone(), o, v);
e.insert(b);
}
}
}
self.indexes.push((exp, io));
}
}
pub(super) enum Plan {
TableIterator(Option<String>),
SingleIndex(Expression, IndexOption),
MultiIndex(Vec<(Expression, IndexOption)>),
SingleIndex(Arc<Expression>, IndexOption),
MultiIndex(Vec<(Arc<Expression>, IndexOption)>),
SingleIndexMultiExpression(IndexRef, RangeQueryBuilder),
}
#[derive(Debug, Clone, Eq, PartialEq, Hash)]
@ -117,97 +139,172 @@ pub(crate) struct IndexOption(Arc<Inner>);
#[derive(Debug, Eq, PartialEq, Hash)]
pub(super) struct Inner {
ix: DefineIndexStatement,
ir: IndexRef,
id: Idiom,
a: Array,
qs: Option<String>,
op: Operator,
mr: Option<MatchRef>,
op: IndexOperator,
}
#[derive(Debug, Eq, PartialEq, Hash)]
pub(super) enum IndexOperator {
Equality(Array),
RangePart(Operator, Value),
Matches(String, Option<MatchRef>),
}
impl IndexOption {
pub(super) fn new(
ix: DefineIndexStatement,
id: Idiom,
op: Operator,
a: Array,
qs: Option<String>,
mr: Option<MatchRef>,
) -> Self {
pub(super) fn new(ir: IndexRef, id: Idiom, op: IndexOperator) -> Self {
Self(Arc::new(Inner {
ix,
ir,
id,
op,
a,
qs,
mr,
}))
}
pub(super) fn ix(&self) -> &DefineIndexStatement {
&self.0.ix
pub(super) fn ir(&self) -> IndexRef {
self.0.ir
}
pub(super) fn op(&self) -> &Operator {
pub(super) fn op(&self) -> &IndexOperator {
&self.0.op
}
pub(super) fn array(&self) -> &Array {
&self.0.a
}
pub(super) fn qs(&self) -> Option<&String> {
self.0.qs.as_ref()
}
pub(super) fn id(&self) -> &Idiom {
&self.0.id
}
pub(super) fn match_ref(&self) -> Option<&MatchRef> {
self.0.mr.as_ref()
pub(crate) fn explain(&self, e: &mut HashMap<&str, Value>) {
match self.op() {
IndexOperator::Equality(a) => {
let v = if a.len() == 1 {
a[0].clone()
} else {
Value::Array(a.clone())
};
e.insert("operator", Value::from(Operator::Equal.to_string()));
e.insert("value", v);
}
IndexOperator::Matches(qs, a) => {
e.insert("operator", Value::from(Operator::Matches(*a).to_string()));
e.insert("value", Value::from(qs.to_owned()));
}
IndexOperator::RangePart(op, v) => {
e.insert("operator", Value::from(op.to_string()));
e.insert("value", v.to_owned());
}
};
}
}
#[derive(Debug, Default, Eq, PartialEq, Hash)]
pub(super) struct RangeValue {
pub(super) value: Value,
pub(super) inclusive: bool,
}
impl RangeValue {
fn set_to(&mut self, v: &Value) {
if self.value.is_none() {
self.value = v.clone();
return;
}
if self.value.lt(v) {
self.value = v.clone();
self.inclusive = false;
}
}
pub(crate) fn explain(&self) -> Value {
let v = if self.0.a.len() == 1 {
self.0.a[0].clone()
} else {
Value::Array(self.0.a.clone())
};
Value::Object(Object::from(HashMap::from([
("index", Value::from(self.ix().name.0.to_owned())),
("operator", Value::from(self.op().to_string())),
("value", v),
fn set_to_inclusive(&mut self, v: &Value) {
if self.value.is_none() {
self.value = v.clone();
self.inclusive = true;
return;
}
if self.inclusive {
if self.value.lt(v) {
self.value = v.clone();
}
} else if self.value.le(v) {
self.value = v.clone();
self.inclusive = true;
}
}
fn set_from(&mut self, v: &Value) {
if self.value.is_none() {
self.value = v.clone();
return;
}
if self.value.gt(v) {
self.value = v.clone();
self.inclusive = false;
}
}
fn set_from_inclusive(&mut self, v: &Value) {
if self.value.is_none() {
self.value = v.clone();
self.inclusive = true;
return;
}
if self.inclusive {
if self.value.gt(v) {
self.value = v.clone();
}
} else if self.value.ge(v) {
self.value = v.clone();
self.inclusive = true;
}
}
}
impl From<&RangeValue> for Value {
fn from(rv: &RangeValue) -> Self {
Value::from(Object::from(HashMap::from([
("value", rv.value.to_owned()),
("inclusive", Value::from(rv.inclusive)),
])))
}
}
#[derive(Default, Debug)]
pub(super) struct RangeQueryBuilder {
pub(super) exps: HashSet<Arc<Expression>>,
pub(super) from: RangeValue,
pub(super) to: RangeValue,
}
impl RangeQueryBuilder {
fn add(&mut self, exp: Arc<Expression>, op: &Operator, v: &Value) {
match op {
Operator::LessThan => self.to.set_to(v),
Operator::LessThanOrEqual => self.to.set_to_inclusive(v),
Operator::MoreThan => self.from.set_from(v),
Operator::MoreThanOrEqual => self.from.set_from_inclusive(v),
_ => return,
}
self.exps.insert(exp);
}
}
#[cfg(test)]
mod tests {
use crate::idx::planner::plan::IndexOption;
use crate::sql::statements::DefineIndexStatement;
use crate::sql::{Array, Idiom, Operator};
use crate::idx::planner::plan::{IndexOperator, IndexOption, RangeValue};
use crate::sql::{Array, Idiom, Value};
use std::collections::HashSet;
#[test]
fn test_hash_index_option() {
let mut set = HashSet::new();
let io1 = IndexOption::new(
DefineIndexStatement::default(),
1,
Idiom::from("a.b".to_string()),
Operator::Equal,
Array::from(vec!["test"]),
None,
None,
IndexOperator::Equality(Array::from(vec!["test"])),
);
let io2 = IndexOption::new(
DefineIndexStatement::default(),
1,
Idiom::from("a.b".to_string()),
Operator::Equal,
Array::from(vec!["test"]),
None,
None,
IndexOperator::Equality(Array::from(vec!["test"])),
);
set.insert(io1);
@ -216,4 +313,94 @@ mod tests {
assert_eq!(set.len(), 1);
}
#[test]
fn test_range_default_value() {
let r = RangeValue::default();
assert_eq!(r.value, Value::None);
assert_eq!(r.inclusive, false);
}
#[test]
fn test_range_value_from_inclusive() {
let mut r = RangeValue::default();
r.set_from_inclusive(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, true);
r.set_from_inclusive(&10.into());
assert_eq!(r.value, 10.into());
assert_eq!(r.inclusive, true);
r.set_from_inclusive(&20.into());
assert_eq!(r.value, 10.into());
assert_eq!(r.inclusive, true);
}
#[test]
fn test_range_value_from() {
let mut r = RangeValue::default();
r.set_from(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, false);
r.set_from(&10.into());
assert_eq!(r.value, 10.into());
assert_eq!(r.inclusive, false);
r.set_from(&20.into());
assert_eq!(r.value, 10.into());
assert_eq!(r.inclusive, false);
}
#[test]
fn test_range_value_to_inclusive() {
let mut r = RangeValue::default();
r.set_to_inclusive(&10.into());
assert_eq!(r.value, 10.into());
assert_eq!(r.inclusive, true);
r.set_to_inclusive(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, true);
r.set_to_inclusive(&10.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, true);
}
#[test]
fn test_range_value_to() {
let mut r = RangeValue::default();
r.set_to(&10.into());
assert_eq!(r.value, 10.into());
assert_eq!(r.inclusive, false);
r.set_to(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, false);
r.set_to(&10.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, false);
}
#[test]
fn test_range_value_to_switch_inclusive() {
let mut r = RangeValue::default();
r.set_to(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, false);
r.set_to_inclusive(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, true);
r.set_to(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, true);
}
#[test]
fn test_range_value_from_switch_inclusive() {
let mut r = RangeValue::default();
r.set_from(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, false);
r.set_from_inclusive(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, true);
r.set_from(&20.into());
assert_eq!(r.value, 20.into());
assert_eq!(r.inclusive, true);
}
}

View file

@ -1,10 +1,10 @@
use crate::ctx::Context;
use crate::dbs::{Options, Transaction};
use crate::err::Error;
use crate::idx::planner::plan::IndexOption;
use crate::idx::planner::plan::{IndexOperator, IndexOption};
use crate::sql::index::Index;
use crate::sql::statements::DefineIndexStatement;
use crate::sql::{Array, Cond, Expression, Idiom, Operator, Subquery, Table, Value};
use crate::sql::{Array, Cond, Expression, Idiom, Operator, Subquery, Table, Value, With};
use async_recursion::async_recursion;
use std::collections::HashMap;
use std::sync::Arc;
@ -12,7 +12,7 @@ use std::sync::Arc;
pub(super) struct Tree {}
impl Tree {
/// Traverse the all the conditions and extract every expression
/// Traverse all the conditions and extract every expression
/// that can be resolved by an index.
pub(super) async fn build<'a>(
ctx: &'a Context<'_>,
@ -20,18 +20,26 @@ impl Tree {
txn: &'a Transaction,
table: &'a Table,
cond: &'a Option<Cond>,
) -> Result<Option<(Node, IndexMap)>, Error> {
with: &'a Option<With>,
) -> Result<Option<(Node, IndexMap, Vec<IndexRef>)>, Error> {
let with_indexes = match with {
Some(With::Index(ixs)) => Vec::with_capacity(ixs.len()),
_ => vec![],
};
let mut b = TreeBuilder {
ctx,
opt,
txn,
table,
with,
indexes: None,
index_lookup: Default::default(),
index_map: IndexMap::default(),
with_indexes,
};
let mut res = None;
if let Some(cond) = cond {
res = Some((b.eval_value(&cond.0).await?, b.index_map));
res = Some((b.eval_value(&cond.0).await?, b.index_map, b.with_indexes));
}
Ok(res)
}
@ -42,12 +50,18 @@ struct TreeBuilder<'a> {
opt: &'a Options,
txn: &'a Transaction,
table: &'a Table,
with: &'a Option<With>,
indexes: Option<Arc<[DefineIndexStatement]>>,
index_lookup: HashMap<Idiom, IndexRef>,
index_map: IndexMap,
with_indexes: Vec<IndexRef>,
}
impl<'a> TreeBuilder<'a> {
async fn find_index(&mut self, i: &Idiom) -> Result<Option<DefineIndexStatement>, Error> {
async fn find_index(&mut self, i: &Idiom) -> Result<Option<IndexRef>, Error> {
if let Some(ir) = self.index_lookup.get(i) {
return Ok(Some(*ir));
}
if self.indexes.is_none() {
let indexes = self
.txn
@ -61,7 +75,15 @@ impl<'a> TreeBuilder<'a> {
if let Some(indexes) = &self.indexes {
for ix in indexes.as_ref() {
if ix.cols.len() == 1 && ix.cols[0].eq(i) {
return Ok(Some(ix.clone()));
let ir = self.index_lookup.len() as IndexRef;
if let Some(With::Index(ixs)) = self.with {
if ixs.contains(&ix.name.0) {
self.with_indexes.push(ir);
}
}
self.index_lookup.insert(i.clone(), ir);
self.index_map.definitions.insert(ir, ix.clone());
return Ok(Some(ir));
}
}
}
@ -107,12 +129,12 @@ impl<'a> TreeBuilder<'a> {
} => {
let left = self.eval_value(l).await?;
let right = self.eval_value(r).await?;
if let Some(io) = self.index_map.0.get(e) {
if let Some(io) = self.index_map.options.get(e) {
return Ok(Node::Expression {
io: Some(io.clone()),
left: Box::new(left),
right: Box::new(right),
exp: e.clone(),
exp: Arc::new(e.clone()),
});
}
let mut io = None;
@ -125,7 +147,7 @@ impl<'a> TreeBuilder<'a> {
io,
left: Box::new(left),
right: Box::new(right),
exp: e.clone(),
exp: Arc::new(e.clone()),
})
}
}
@ -133,43 +155,49 @@ impl<'a> TreeBuilder<'a> {
fn lookup_index_option(
&mut self,
ix: &DefineIndexStatement,
ir: IndexRef,
op: &Operator,
id: &Idiom,
v: &Node,
e: &Expression,
) -> Option<IndexOption> {
if let Some(v) = v.is_scalar() {
let (found, mr, qs) = match &ix.index {
Index::Idx => (Operator::Equal.eq(op), None, None),
Index::Uniq => (Operator::Equal.eq(op), None, None),
Index::Search {
..
} => {
if let Operator::Matches(mr) = op {
(true, *mr, Some(v.clone().to_raw_string()))
} else {
(false, None, None)
if let Some(ix) = self.index_map.definitions.get(&ir) {
let op = match &ix.index {
Index::Idx => Self::eval_index_operator(op, v),
Index::Uniq => Self::eval_index_operator(op, v),
Index::Search {
..
} => {
if let Operator::Matches(mr) = op {
Some(IndexOperator::Matches(v.clone().to_raw_string(), *mr))
} else {
None
}
}
Index::MTree(_) => None,
};
if let Some(op) = op {
let io = IndexOption::new(ir, id.clone(), op);
self.index_map.options.insert(Arc::new(e.clone()), io.clone());
return Some(io);
}
Index::MTree(_) => (false, None, None),
};
if found {
let io = IndexOption::new(
ix.clone(),
id.clone(),
op.to_owned(),
Array::from(v.clone()),
qs,
mr,
);
self.index_map.0.insert(e.clone(), io.clone());
return Some(io);
}
}
None
}
fn eval_index_operator(op: &Operator, v: &Value) -> Option<IndexOperator> {
match op {
Operator::Equal => Some(IndexOperator::Equality(Array::from(v.clone()))),
Operator::LessThan
| Operator::LessThanOrEqual
| Operator::MoreThan
| Operator::MoreThanOrEqual => Some(IndexOperator::RangePart(op.clone(), v.clone())),
_ => None,
}
}
async fn eval_subquery(&mut self, s: &Subquery) -> Result<Node, Error> {
match s {
Subquery::Value(v) => self.eval_value(v).await,
@ -178,14 +206,13 @@ impl<'a> TreeBuilder<'a> {
}
}
pub(super) type IndexRef = u16;
/// For each expression the a possible index option
#[derive(Default)]
pub(super) struct IndexMap(HashMap<Expression, IndexOption>);
impl IndexMap {
pub(super) fn consume(self) -> HashMap<Expression, IndexOption> {
self.0
}
pub(super) struct IndexMap {
pub(super) options: HashMap<Arc<Expression>, IndexOption>,
pub(super) definitions: HashMap<IndexRef, DefineIndexStatement>,
}
#[derive(Debug, Clone, Eq, PartialEq, Hash)]
@ -194,9 +221,9 @@ pub(super) enum Node {
io: Option<IndexOption>,
left: Box<Node>,
right: Box<Node>,
exp: Expression,
exp: Arc<Expression>,
},
IndexedField(Idiom, DefineIndexStatement),
IndexedField(Idiom, IndexRef),
NonIndexedField,
Scalar(Value),
Unsupported(String),
@ -211,9 +238,9 @@ impl Node {
}
}
pub(super) fn is_indexed_field(&self) -> Option<(&Idiom, &DefineIndexStatement)> {
pub(super) fn is_indexed_field(&self) -> Option<(&Idiom, IndexRef)> {
if let Node::IndexedField(id, ix) = self {
Some((id, ix))
Some((id, *ix))
} else {
None
}

View file

@ -124,20 +124,40 @@ impl<'a> Index<'a> {
}
}
pub fn range(ns: &str, db: &str, tb: &str, ix: &str) -> Range<Vec<u8>> {
let mut beg = Prefix::new(ns, db, tb, ix).encode().unwrap();
beg.extend_from_slice(&[0x00]);
let mut end = Prefix::new(ns, db, tb, ix).encode().unwrap();
end.extend_from_slice(&[0xff]);
beg..end
fn prefix(ns: &str, db: &str, tb: &str, ix: &str) -> Vec<u8> {
Prefix::new(ns, db, tb, ix).encode().unwrap()
}
pub fn range_all_ids(ns: &str, db: &str, tb: &str, ix: &str, fd: &Array) -> (Vec<u8>, Vec<u8>) {
let mut beg = PrefixIds::new(ns, db, tb, ix, fd).encode().unwrap();
pub fn prefix_beg(ns: &str, db: &str, tb: &str, ix: &str) -> Vec<u8> {
let mut beg = Self::prefix(ns, db, tb, ix);
beg.extend_from_slice(&[0x00]);
let mut end = PrefixIds::new(ns, db, tb, ix, fd).encode().unwrap();
end.extend_from_slice(&[0xff]);
(beg, end)
beg
}
pub fn prefix_end(ns: &str, db: &str, tb: &str, ix: &str) -> Vec<u8> {
let mut beg = Self::prefix(ns, db, tb, ix);
beg.extend_from_slice(&[0xff]);
beg
}
pub fn range(ns: &str, db: &str, tb: &str, ix: &str) -> Range<Vec<u8>> {
Self::prefix_beg(ns, db, tb, ix)..Self::prefix_end(ns, db, tb, ix)
}
fn prefix_ids(ns: &str, db: &str, tb: &str, ix: &str, fd: &Array) -> Vec<u8> {
PrefixIds::new(ns, db, tb, ix, fd).encode().unwrap()
}
pub fn prefix_ids_beg(ns: &str, db: &str, tb: &str, ix: &str, fd: &Array) -> Vec<u8> {
let mut beg = Self::prefix_ids(ns, db, tb, ix, fd);
beg.extend_from_slice(&[0x00]);
beg
}
pub fn prefix_ids_end(ns: &str, db: &str, tb: &str, ix: &str, fd: &Array) -> Vec<u8> {
let mut beg = Self::prefix_ids(ns, db, tb, ix, fd);
beg.extend_from_slice(&[0xff]);
beg
}
}

View file

@ -1,4 +1,5 @@
mod parse;
use parse::Parse;
mod helpers;
use helpers::new_ds;
@ -8,7 +9,7 @@ use surrealdb::sql::Value;
#[tokio::test]
async fn select_where_iterate_three_multi_index() -> Result<(), Error> {
let mut res = execute_test(&three_multi_index_query("", ""), 12).await?;
let mut res = execute_test(&three_multi_index_query("", ""), 12, 8).await?;
check_result(&mut res, "[{ name: 'Jaime' }, { name: 'Tobie' }, { name: 'Lizzie' }]")?;
// OR results
check_result(&mut res, THREE_MULTI_INDEX_EXPLAIN)?;
@ -20,7 +21,7 @@ async fn select_where_iterate_three_multi_index() -> Result<(), Error> {
#[tokio::test]
async fn select_where_iterate_three_multi_index_parallel() -> Result<(), Error> {
let mut res = execute_test(&three_multi_index_query("", "PARALLEL"), 12).await?;
let mut res = execute_test(&three_multi_index_query("", "PARALLEL"), 12, 8).await?;
// OR results
check_result(&mut res, "[{ name: 'Jaime' }, { name: 'Tobie' }, { name: 'Lizzie' }]")?;
check_result(&mut res, THREE_MULTI_INDEX_EXPLAIN)?;
@ -32,9 +33,12 @@ async fn select_where_iterate_three_multi_index_parallel() -> Result<(), Error>
#[tokio::test]
async fn select_where_iterate_three_multi_index_with_all_index() -> Result<(), Error> {
let mut res =
execute_test(&three_multi_index_query("WITH INDEX uniq_name,idx_genre,ft_company", ""), 12)
.await?;
let mut res = execute_test(
&three_multi_index_query("WITH INDEX uniq_name,idx_genre,ft_company", ""),
12,
8,
)
.await?;
// OR results
check_result(&mut res, "[{ name: 'Jaime' }, { name: 'Tobie' }, { name: 'Lizzie' }]")?;
check_result(&mut res, THREE_MULTI_INDEX_EXPLAIN)?;
@ -46,7 +50,8 @@ async fn select_where_iterate_three_multi_index_with_all_index() -> Result<(), E
#[tokio::test]
async fn select_where_iterate_three_multi_index_with_one_ft_index() -> Result<(), Error> {
let mut res = execute_test(&three_multi_index_query("WITH INDEX ft_company", ""), 12).await?;
let mut res =
execute_test(&three_multi_index_query("WITH INDEX ft_company", ""), 12, 8).await?;
// OR results
check_result(&mut res, "[{ name: 'Jaime' }, { name: 'Lizzie' }, { name: 'Tobie' } ]")?;
check_result(&mut res, THREE_TABLE_EXPLAIN)?;
@ -58,7 +63,7 @@ async fn select_where_iterate_three_multi_index_with_one_ft_index() -> Result<()
#[tokio::test]
async fn select_where_iterate_three_multi_index_with_one_index() -> Result<(), Error> {
let mut res = execute_test(&three_multi_index_query("WITH INDEX uniq_name", ""), 12).await?;
let mut res = execute_test(&three_multi_index_query("WITH INDEX uniq_name", ""), 12, 8).await?;
// OR results
check_result(&mut res, "[{ name: 'Jaime' }, { name: 'Lizzie' }, { name: 'Tobie' } ]")?;
check_result(&mut res, THREE_TABLE_EXPLAIN)?;
@ -70,7 +75,7 @@ async fn select_where_iterate_three_multi_index_with_one_index() -> Result<(), E
#[tokio::test]
async fn select_where_iterate_two_multi_index() -> Result<(), Error> {
let mut res = execute_test(&two_multi_index_query("", ""), 9).await?;
let mut res = execute_test(&two_multi_index_query("", ""), 9, 5).await?;
// OR results
check_result(&mut res, "[{ name: 'Jaime' }, { name: 'Tobie' }]")?;
check_result(&mut res, TWO_MULTI_INDEX_EXPLAIN)?;
@ -82,7 +87,7 @@ async fn select_where_iterate_two_multi_index() -> Result<(), Error> {
#[tokio::test]
async fn select_where_iterate_two_multi_index_with_one_index() -> Result<(), Error> {
let mut res = execute_test(&two_multi_index_query("WITH INDEX idx_genre", ""), 9).await?;
let mut res = execute_test(&two_multi_index_query("WITH INDEX idx_genre", ""), 9, 5).await?;
// OR results
check_result(&mut res, "[{ name: 'Jaime' }, { name: 'Tobie' }]")?;
check_result(&mut res, &table_explain(2))?;
@ -95,7 +100,7 @@ async fn select_where_iterate_two_multi_index_with_one_index() -> Result<(), Err
#[tokio::test]
async fn select_where_iterate_two_multi_index_with_two_index() -> Result<(), Error> {
let mut res =
execute_test(&two_multi_index_query("WITH INDEX idx_genre,uniq_name", ""), 9).await?;
execute_test(&two_multi_index_query("WITH INDEX idx_genre,uniq_name", ""), 9, 5).await?;
// OR results
check_result(&mut res, "[{ name: 'Jaime' }, { name: 'Tobie' }]")?;
check_result(&mut res, TWO_MULTI_INDEX_EXPLAIN)?;
@ -107,7 +112,7 @@ async fn select_where_iterate_two_multi_index_with_two_index() -> Result<(), Err
#[tokio::test]
async fn select_where_iterate_two_no_index() -> Result<(), Error> {
let mut res = execute_test(&two_multi_index_query("WITH NOINDEX", ""), 9).await?;
let mut res = execute_test(&two_multi_index_query("WITH NOINDEX", ""), 9, 5).await?;
// OR results
check_result(&mut res, "[{ name: 'Jaime' }, { name: 'Tobie' }]")?;
check_result(&mut res, &table_explain_no_index(2))?;
@ -117,13 +122,17 @@ async fn select_where_iterate_two_no_index() -> Result<(), Error> {
Ok(())
}
async fn execute_test(sql: &str, expected_result: usize) -> Result<Vec<Response>, Error> {
async fn execute_test(
sql: &str,
expected_result: usize,
check_results: usize,
) -> Result<Vec<Response>, Error> {
let dbs = new_ds().await?;
let ses = Session::owner().with_ns("test").with_db("test");
let mut res = dbs.execute(sql, &ses, None).await?;
assert_eq!(res.len(), expected_result);
// Check that the setup is ok
for _ in 0..(expected_result - 4) {
for _ in 0..check_results {
let _ = res.remove(0).result?;
}
Ok(res)
@ -414,3 +423,426 @@ async fn select_unsupported_unary_operator() -> Result<(), Error> {
assert_eq!(format!("{:#}", tmp), format!("{:#}", val));
Ok(())
}
fn range_test(unique: bool, from_incl: bool, to_incl: bool) -> String {
let from_op = if from_incl {
">="
} else {
">"
};
let to_op = if to_incl {
"<="
} else {
"<"
};
format!(
"DEFINE INDEX year ON TABLE test COLUMNS year {};
CREATE test:0 SET year = 2000;
CREATE test:10 SET year = 2010;
CREATE test:15 SET year = 2015;
CREATE test:16 SET year = {};
CREATE test:20 SET year = 2020;
SELECT id FROM test WHERE year {} 2000 AND year {} 2020 EXPLAIN;
SELECT id FROM test WHERE year {} 2000 AND year {} 2020;",
if unique {
"UNIQUE"
} else {
""
},
if unique {
"2016"
} else {
"2015"
},
from_op,
to_op,
from_op,
to_op,
)
}
async fn select_range(
unique: bool,
from_incl: bool,
to_incl: bool,
explain: &str,
result: &str,
) -> Result<(), Error> {
let mut res = execute_test(&range_test(unique, from_incl, to_incl), 8, 6).await?;
{
let tmp = res.remove(0).result?;
let val = Value::parse(explain);
assert_eq!(format!("{:#}", tmp), format!("{:#}", val));
}
{
let tmp = res.remove(0).result?;
let val = Value::parse(result);
assert_eq!(format!("{:#}", tmp), format!("{:#}", val));
}
Ok(())
}
const EXPLAIN_FROM_TO: &str = r"[
{
detail: {
plan: {
from: {
inclusive: false,
value: 2000
},
index: 'year',
to: {
inclusive: false,
value: 2020
}
},
table: 'test'
},
operation: 'Iterate Index'
}
]";
const RESULT_FROM_TO: &str = r"[
{
id: test:10,
},
{
id: test:15,
},
{
id: test:16,
}
]";
#[tokio::test]
async fn select_index_range_from_to() -> Result<(), Error> {
select_range(false, false, false, EXPLAIN_FROM_TO, RESULT_FROM_TO).await
}
#[tokio::test]
async fn select_unique_range_from_to() -> Result<(), Error> {
select_range(true, false, false, EXPLAIN_FROM_TO, RESULT_FROM_TO).await
}
const EXPLAIN_FROM_INCL_TO: &str = r"[
{
detail: {
plan: {
from: {
inclusive: true,
value: 2000
},
index: 'year',
to: {
inclusive: false,
value: 2020
}
},
table: 'test'
},
operation: 'Iterate Index'
}
]";
const RESULT_FROM_INCL_TO: &str = r"[
{
id: test:0,
},
{
id: test:10,
},
{
id: test:15,
},
{
id: test:16,
}
]";
#[tokio::test]
async fn select_index_range_from_incl_to() -> Result<(), Error> {
select_range(false, true, false, EXPLAIN_FROM_INCL_TO, RESULT_FROM_INCL_TO).await
}
#[tokio::test]
async fn select_unique_range_from_incl_to() -> Result<(), Error> {
select_range(true, true, false, EXPLAIN_FROM_INCL_TO, RESULT_FROM_INCL_TO).await
}
const EXPLAIN_FROM_TO_INCL: &str = r"[
{
detail: {
plan: {
from: {
inclusive: false,
value: 2000
},
index: 'year',
to: {
inclusive: true,
value: 2020
}
},
table: 'test'
},
operation: 'Iterate Index'
}
]";
const RESULT_FROM_TO_INCL: &str = r"[
{
id: test:10,
},
{
id: test:15,
},
{
id: test:16,
},
{
id: test:20,
},
]";
#[tokio::test]
async fn select_index_range_from_to_incl() -> Result<(), Error> {
select_range(false, false, true, EXPLAIN_FROM_TO_INCL, RESULT_FROM_TO_INCL).await
}
#[tokio::test]
async fn select_unique_range_from_to_incl() -> Result<(), Error> {
select_range(true, false, true, EXPLAIN_FROM_TO_INCL, RESULT_FROM_TO_INCL).await
}
const EXPLAIN_FROM_INCL_TO_INCL: &str = r"[
{
detail: {
plan: {
from: {
inclusive: true,
value: 2000
},
index: 'year',
to: {
inclusive: true,
value: 2020
}
},
table: 'test'
},
operation: 'Iterate Index'
}
]";
const RESULT_FROM_INCL_TO_INCL: &str = r"[
{
id: test:0,
},
{
id: test:10,
},
{
id: test:15,
},
{
id: test:16,
},
{
id: test:20,
},
]";
#[tokio::test]
async fn select_index_range_from_incl_to_incl() -> Result<(), Error> {
select_range(false, true, true, EXPLAIN_FROM_INCL_TO_INCL, RESULT_FROM_INCL_TO_INCL).await
}
#[tokio::test]
async fn select_unique_range_from_incl_to_incl() -> Result<(), Error> {
select_range(true, true, true, EXPLAIN_FROM_INCL_TO_INCL, RESULT_FROM_INCL_TO_INCL).await
}
fn single_range_operator_test(unique: bool, op: &str) -> String {
format!(
"DEFINE INDEX year ON TABLE test COLUMNS year {};
CREATE test:10 SET year = 2010;
CREATE test:15 SET year = 2015;
CREATE test:20 SET year = 2020;
SELECT id FROM test WHERE year {} 2015 EXPLAIN;
SELECT id FROM test WHERE year {} 2015;",
if unique {
"UNIQUE"
} else {
""
},
op,
op,
)
}
async fn select_single_range_operator(
unique: bool,
op: &str,
explain: &str,
result: &str,
) -> Result<(), Error> {
let mut res = execute_test(&single_range_operator_test(unique, op), 6, 4).await?;
{
let tmp = res.remove(0).result?;
let val = Value::parse(explain);
assert_eq!(format!("{:#}", tmp), format!("{:#}", val));
}
{
let tmp = res.remove(0).result?;
let val = Value::parse(result);
assert_eq!(format!("{:#}", tmp), format!("{:#}", val));
}
Ok(())
}
const EXPLAIN_LESS: &str = r"[
{
detail: {
plan: {
from: {
inclusive: false,
value: None
},
index: 'year',
to: {
inclusive: false,
value: 2015
}
},
table: 'test'
},
operation: 'Iterate Index'
}
]";
const RESULT_LESS: &str = r"[
{
id: test:10,
}
]";
#[tokio::test]
async fn select_index_single_range_operator_less() -> Result<(), Error> {
select_single_range_operator(false, "<", EXPLAIN_LESS, RESULT_LESS).await
}
#[tokio::test]
async fn select_unique_single_range_operator_less() -> Result<(), Error> {
select_single_range_operator(true, "<", EXPLAIN_LESS, RESULT_LESS).await
}
const EXPLAIN_LESS_OR_EQUAL: &str = r"[
{
detail: {
plan: {
from: {
inclusive: false,
value: None
},
index: 'year',
to: {
inclusive: true,
value: 2015
}
},
table: 'test'
},
operation: 'Iterate Index'
}
]";
const RESULT_LESS_OR_EQUAL: &str = r"[
{
id: test:10,
},
{
id: test:15,
}
]";
#[tokio::test]
async fn select_index_single_range_operator_less_or_equal() -> Result<(), Error> {
select_single_range_operator(false, "<=", EXPLAIN_LESS_OR_EQUAL, RESULT_LESS_OR_EQUAL).await
}
#[tokio::test]
async fn select_unique_single_range_operator_less_or_equal() -> Result<(), Error> {
select_single_range_operator(true, "<=", EXPLAIN_LESS_OR_EQUAL, RESULT_LESS_OR_EQUAL).await
}
const EXPLAIN_MORE: &str = r"[
{
detail: {
plan: {
from: {
inclusive: false,
value: 2015
},
index: 'year',
to: {
inclusive: false,
value: None
}
},
table: 'test'
},
operation: 'Iterate Index'
}
]";
const RESULT_MORE: &str = r"[
{
id: test:20,
}
]";
#[tokio::test]
async fn select_index_single_range_operator_more() -> Result<(), Error> {
select_single_range_operator(false, ">", EXPLAIN_MORE, RESULT_MORE).await
}
#[tokio::test]
async fn select_unique_single_range_operator_more() -> Result<(), Error> {
select_single_range_operator(true, ">", EXPLAIN_MORE, RESULT_MORE).await
}
const EXPLAIN_MORE_OR_EQUAL: &str = r"[
{
detail: {
plan: {
from: {
inclusive: true,
value: 2015
},
index: 'year',
to: {
inclusive: false,
value: None
}
},
table: 'test'
},
operation: 'Iterate Index'
}
]";
const RESULT_MORE_OR_EQUAL: &str = r"[
{
id: test:15,
},
{
id: test:20,
}
]";
#[tokio::test]
async fn select_index_single_range_operator_more_or_equal() -> Result<(), Error> {
select_single_range_operator(false, ">=", EXPLAIN_MORE_OR_EQUAL, RESULT_MORE_OR_EQUAL).await
}
#[tokio::test]
async fn select_unique_single_range_operator_more_or_equal() -> Result<(), Error> {
select_single_range_operator(true, ">=", EXPLAIN_MORE_OR_EQUAL, RESULT_MORE_OR_EQUAL).await
}