Query planner to support composite indexes on the 1st column (#4746)

This commit is contained in:
Emmanuel Keller 2024-09-16 14:17:20 +01:00 committed by GitHub
parent 4d92005125
commit 9cfdd34fc0
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
8 changed files with 272 additions and 108 deletions

View file

@ -69,7 +69,7 @@ pub(super) struct InnerQueryExecutor {
mr_entries: HashMap<MatchRef, FtEntry>,
exp_entries: HashMap<Arc<Expression>, FtEntry>,
it_entries: Vec<IteratorEntry>,
index_definitions: Vec<DefineIndexStatement>,
index_definitions: Vec<Arc<DefineIndexStatement>>,
mt_entries: HashMap<Arc<Expression>, MtEntry>,
hnsw_entries: HashMap<Arc<Expression>, HnswEntry>,
knn_bruteforce_entries: HashMap<Arc<Expression>, KnnBruteForceEntry>,
@ -87,7 +87,7 @@ pub(super) enum IteratorEntry {
}
impl IteratorEntry {
pub(super) fn explain(&self, ix_def: &[DefineIndexStatement]) -> Value {
pub(super) fn explain(&self, ix_def: &[Arc<DefineIndexStatement>]) -> Value {
match self {
Self::Single(_, io) => io.explain(ix_def),
Self::Range(_, ir, from, to) => {
@ -329,7 +329,7 @@ impl QueryExecutor {
pub(crate) fn explain(&self, itr: IteratorRef) -> Value {
match self.0.it_entries.get(itr as usize) {
Some(ie) => ie.explain(self.0.index_definitions.as_slice()),
Some(ie) => ie.explain(&self.0.index_definitions),
None => Value::None,
}
}
@ -368,7 +368,7 @@ impl QueryExecutor {
) -> Result<Option<ThingIterator>, Error> {
if let Some(ix) = self.get_index_def(io.ix_ref()) {
match ix.index {
Index::Idx => Ok(self.new_index_iterator(opt, irf, ix, io.clone()).await?),
Index::Idx => Ok(self.new_index_iterator(opt, irf, ix.clone(), io.clone()).await?),
Index::Uniq => Ok(self.new_unique_index_iterator(opt, irf, ix, io.clone()).await?),
Index::Search {
..
@ -385,7 +385,7 @@ impl QueryExecutor {
&self,
opt: &Options,
irf: IteratorRef,
ix: &DefineIndexStatement,
ix: Arc<DefineIndexStatement>,
io: IndexOption,
) -> Result<Option<ThingIterator>, Error> {
Ok(match io.op() {
@ -393,16 +393,16 @@ impl QueryExecutor {
if let Value::Number(n) = value.as_ref() {
let values = Self::get_number_variants(n);
if values.len() == 1 {
Some(Self::new_index_equal_iterator(irf, opt, ix, &values[0])?)
Some(Self::new_index_equal_iterator(irf, opt, &ix, &values[0])?)
} else {
Some(Self::new_multiple_index_equal_iterators(irf, opt, ix, values)?)
Some(Self::new_multiple_index_equal_iterators(irf, opt, &ix, values)?)
}
} else {
Some(Self::new_index_equal_iterator(irf, opt, ix, value)?)
Some(Self::new_index_equal_iterator(irf, opt, &ix, value)?)
}
}
IndexOperator::Union(value) => Some(ThingIterator::IndexUnion(
IndexUnionThingIterator::new(irf, opt.ns()?, opt.db()?, &ix.what, &ix.name, value),
IndexUnionThingIterator::new(irf, opt.ns()?, opt.db()?, &ix, value),
)),
IndexOperator::Join(ios) => {
let iterators = self.build_iterators(opt, irf, ios).await?;
@ -426,8 +426,7 @@ impl QueryExecutor {
irf,
opt.ns()?,
opt.db()?,
&ix.what,
&ix.name,
ix,
value,
)))
}
@ -527,7 +526,7 @@ impl QueryExecutor {
&self,
opt: &Options,
irf: IteratorRef,
ix: &DefineIndexStatement,
ix: &Arc<DefineIndexStatement>,
io: IndexOption,
) -> Result<Option<ThingIterator>, Error> {
Ok(match io.op() {
@ -548,11 +547,12 @@ impl QueryExecutor {
)),
IndexOperator::Join(ios) => {
let iterators = self.build_iterators(opt, irf, ios).await?;
let unique_join = Box::new(UniqueJoinThingIterator::new(irf, opt, ix, iterators)?);
let unique_join =
Box::new(UniqueJoinThingIterator::new(irf, opt, ix.clone(), iterators)?);
Some(ThingIterator::UniqueJoin(unique_join))
}
IndexOperator::Order => Some(ThingIterator::UniqueRange(
UniqueRangeThingIterator::full_range(irf, opt.ns()?, opt.db()?, &ix.what, &ix.name),
UniqueRangeThingIterator::full_range(irf, opt.ns()?, opt.db()?, ix),
)),
_ => None,
})
@ -564,15 +564,27 @@ impl QueryExecutor {
ix: &DefineIndexStatement,
value: &Value,
) -> Result<ThingIterator, Error> {
if ix.cols.len() > 1 {
// If the index is unique and the index is a composite index,
// then we have the opportunity to iterate on the first column of the index
// and consider it as a standard index (rather than a unique one)
Ok(ThingIterator::IndexEqual(IndexEqualThingIterator::new(
irf,
opt.ns()?,
opt.db()?,
ix,
value,
)))
} else {
Ok(ThingIterator::UniqueEqual(UniqueEqualThingIterator::new(
irf,
opt.ns()?,
opt.db()?,
&ix.what,
&ix.name,
ix,
value,
)))
}
}
fn new_multiple_unique_equal_iterators(
irf: IteratorRef,
@ -641,7 +653,7 @@ impl QueryExecutor {
Ok(iterators)
}
fn get_index_def(&self, ir: IndexRef) -> Option<&DefineIndexStatement> {
fn get_index_def(&self, ir: IndexRef) -> Option<&Arc<DefineIndexStatement>> {
self.0.index_definitions.get(ir as usize)
}

View file

@ -152,13 +152,21 @@ impl IndexEqualThingIterator {
irf: IteratorRef,
ns: &str,
db: &str,
ix_what: &Ident,
ix_name: &Ident,
ix: &DefineIndexStatement,
v: &Value,
) -> Self {
let a = Array::from(v.clone());
let beg = Index::prefix_ids_beg(ns, db, ix_what, ix_name, &a);
let end = Index::prefix_ids_end(ns, db, ix_what, ix_name, &a);
let (beg, end) = if ix.cols.len() == 1 {
(
Index::prefix_ids_beg(ns, db, &ix.what, &ix.name, &a),
Index::prefix_ids_end(ns, db, &ix.what, &ix.name, &a),
)
} else {
(
Index::prefix_ids_composite_beg(ns, db, &ix.what, &ix.name, &a),
Index::prefix_ids_composite_end(ns, db, &ix.what, &ix.name, &a),
)
};
Self {
irf,
beg,
@ -343,8 +351,7 @@ impl IndexUnionThingIterator {
irf: IteratorRef,
ns: &str,
db: &str,
ix_what: &Ident,
ix_name: &Ident,
ix: &DefineIndexStatement,
a: &Value,
) -> Self {
// We create a VecDeque to hold the prefix keys (begin and end) for each value in the array.
@ -352,8 +359,8 @@ impl IndexUnionThingIterator {
a.0.iter()
.map(|v| {
let a = Array::from(v.clone());
let beg = Index::prefix_ids_beg(ns, db, ix_what, ix_name, &a);
let end = Index::prefix_ids_end(ns, db, ix_what, ix_name, &a);
let beg = Index::prefix_ids_beg(ns, db, &ix.what, &ix.name, &a);
let end = Index::prefix_ids_end(ns, db, &ix.what, &ix.name, &a);
(beg, end)
})
.collect()
@ -392,8 +399,7 @@ impl IndexUnionThingIterator {
struct JoinThingIterator {
ns: String,
db: String,
ix_what: Ident,
ix_name: Ident,
ix: Arc<DefineIndexStatement>,
remote_iterators: VecDeque<ThingIterator>,
current_remote: Option<ThingIterator>,
current_remote_batch: VecDeque<CollectorRecord>,
@ -404,14 +410,13 @@ struct JoinThingIterator {
impl JoinThingIterator {
pub(super) fn new(
opt: &Options,
ix: &DefineIndexStatement,
ix: Arc<DefineIndexStatement>,
remote_iterators: VecDeque<ThingIterator>,
) -> Result<Self, Error> {
Ok(Self {
ns: opt.ns()?.to_string(),
db: opt.db()?.to_string(),
ix_what: ix.what.clone(),
ix_name: ix.name.clone(),
ix,
current_remote: None,
current_remote_batch: VecDeque::with_capacity(1),
remote_iterators,
@ -451,15 +456,14 @@ impl JoinThingIterator {
new_iter: F,
) -> Result<bool, Error>
where
F: Fn(&str, &str, &Ident, &Ident, Value) -> ThingIterator,
F: Fn(&str, &str, &DefineIndexStatement, Value) -> ThingIterator,
{
while !ctx.is_done() {
while let Some((thing, _, _)) = self.current_remote_batch.pop_front() {
let k: Key = thing.as_ref().into();
let value = Value::from(thing.as_ref().clone());
if self.distinct.insert(k, true).is_none() {
self.current_local =
Some(new_iter(&self.ns, &self.db, &self.ix_what, &self.ix_name, value));
self.current_local = Some(new_iter(&self.ns, &self.db, &self.ix, value));
return Ok(true);
}
}
@ -478,7 +482,7 @@ impl JoinThingIterator {
new_iter: F,
) -> Result<B, Error>
where
F: Fn(&str, &str, &Ident, &Ident, Value) -> ThingIterator + Copy,
F: Fn(&str, &str, &DefineIndexStatement, Value) -> ThingIterator + Copy,
{
while !ctx.is_done() {
if let Some(current_local) = &mut self.current_local {
@ -501,7 +505,7 @@ impl IndexJoinThingIterator {
pub(super) fn new(
irf: IteratorRef,
opt: &Options,
ix: &DefineIndexStatement,
ix: Arc<DefineIndexStatement>,
remote_iterators: VecDeque<ThingIterator>,
) -> Result<Self, Error> {
Ok(Self(irf, JoinThingIterator::new(opt, ix, remote_iterators)?))
@ -513,8 +517,8 @@ impl IndexJoinThingIterator {
tx: &Transaction,
limit: u32,
) -> Result<B, Error> {
let new_iter = |ns: &str, db: &str, ix_what: &Ident, ix_name: &Ident, value: Value| {
let it = IndexEqualThingIterator::new(self.0, ns, db, ix_what, ix_name, &value);
let new_iter = |ns: &str, db: &str, ix: &DefineIndexStatement, value: Value| {
let it = IndexEqualThingIterator::new(self.0, ns, db, ix, &value);
ThingIterator::IndexEqual(it)
};
self.1.next_batch(ctx, tx, limit, new_iter).await
@ -531,12 +535,11 @@ impl UniqueEqualThingIterator {
irf: IteratorRef,
ns: &str,
db: &str,
ix_what: &Ident,
ix_name: &Ident,
ix: &DefineIndexStatement,
v: &Value,
) -> Self {
let a = Array::from(v.to_owned());
let key = Index::new(ns, db, ix_what, ix_name, &a, None).into();
let key = Index::new(ns, db, &ix.what, &ix.name, &a, None).into();
Self {
irf,
key: Some(key),
@ -584,14 +587,13 @@ impl UniqueRangeThingIterator {
irf: IteratorRef,
ns: &str,
db: &str,
ix_what: &Ident,
ix_name: &Ident,
ix: &DefineIndexStatement,
) -> Self {
let full_range = RangeValue {
value: Value::None,
inclusive: true,
};
Self::new(irf, ns, db, ix_what, ix_name, &full_range, &full_range)
Self::new(irf, ns, db, &ix.what, &ix.name, &full_range, &full_range)
}
fn compute_beg(
@ -720,7 +722,7 @@ impl UniqueJoinThingIterator {
pub(super) fn new(
irf: IteratorRef,
opt: &Options,
ix: &DefineIndexStatement,
ix: Arc<DefineIndexStatement>,
remote_iterators: VecDeque<ThingIterator>,
) -> Result<Self, Error> {
Ok(Self(irf, JoinThingIterator::new(opt, ix, remote_iterators)?))
@ -732,8 +734,8 @@ impl UniqueJoinThingIterator {
tx: &Transaction,
limit: u32,
) -> Result<B, Error> {
let new_iter = |ns: &str, db: &str, ix_what: &Ident, ix_name: &Ident, value: Value| {
let it = UniqueEqualThingIterator::new(self.0, ns, db, ix_what, ix_name, &value);
let new_iter = |ns: &str, db: &str, ix: &DefineIndexStatement, value: Value| {
let it = UniqueEqualThingIterator::new(self.0, ns, db, ix, &value);
ThingIterator::UniqueEqual(it)
};
self.1.next_batch(ctx, tx, limit, new_iter).await

View file

@ -1,6 +1,6 @@
use crate::err::Error;
use crate::idx::ft::MatchRef;
use crate::idx::planner::tree::{GroupRef, IdiomPosition, IndexRef, Node};
use crate::idx::planner::tree::{GroupRef, IdiomCol, IdiomPosition, IndexRef, Node};
use crate::sql::statements::DefineIndexStatement;
use crate::sql::with::With;
use crate::sql::{Array, Expression, Idiom, Number, Object};
@ -174,8 +174,13 @@ pub(super) enum Plan {
pub(super) struct IndexOption {
/// A reference to the index definition
ix_ref: IndexRef,
id: Idiom,
/// The idiom matching this index
id: Arc<Idiom>,
/// The index of the idiom in the index columns
id_col: IdiomCol,
/// The position of the idiom in the expression (Left or Right)
id_pos: IdiomPosition,
/// The index operator
op: Arc<IndexOperator>,
}
@ -195,13 +200,15 @@ pub(super) enum IndexOperator {
impl IndexOption {
pub(super) fn new(
ix_ref: IndexRef,
id: Idiom,
id: Arc<Idiom>,
id_col: IdiomCol,
id_pos: IdiomPosition,
op: IndexOperator,
) -> Self {
Self {
ix_ref,
id,
id_col,
id_pos,
op: Arc::new(op),
}
@ -236,7 +243,7 @@ impl IndexOption {
v.clone()
}
pub(crate) fn explain(&self, ix_def: &[DefineIndexStatement]) -> Value {
pub(crate) fn explain(&self, ix_def: &[Arc<DefineIndexStatement>]) -> Value {
let mut e = HashMap::new();
if let Some(ix) = ix_def.get(self.ix_ref as usize) {
e.insert("index", Value::from(ix.name.0.to_owned()));
@ -452,14 +459,16 @@ mod tests {
let mut set = HashSet::new();
let io1 = IndexOption::new(
1,
Idiom::parse("test"),
Idiom::parse("test").into(),
0,
IdiomPosition::Right,
IndexOperator::Equality(Value::Array(Array::from(vec!["test"])).into()),
);
let io2 = IndexOption::new(
1,
Idiom::parse("test"),
Idiom::parse("test").into(),
0,
IdiomPosition::Right,
IndexOperator::Equality(Value::Array(Array::from(vec!["test"])).into()),
);

View file

@ -61,14 +61,14 @@ struct TreeBuilder<'a> {
with: Option<&'a With>,
first_order: Option<&'a Order>,
schemas: HashMap<Table, SchemaCache>,
idioms_indexes: HashMap<Table, HashMap<Idiom, LocalIndexRefs>>,
idioms_indexes: HashMap<Table, HashMap<Arc<Idiom>, LocalIndexRefs>>,
resolved_expressions: HashMap<Arc<Expression>, ResolvedExpression>,
resolved_idioms: HashMap<Idiom, Node>,
resolved_idioms: HashMap<Arc<Idiom>, Node>,
index_map: IndexesMap,
with_indexes: Vec<IndexRef>,
knn_brute_force_expressions: HashMap<Arc<Expression>, KnnBruteForceExpression>,
knn_expressions: KnnExpressions,
idioms_record_options: HashMap<Idiom, RecordOptions>,
idioms_record_options: HashMap<Arc<Idiom>, RecordOptions>,
group_sequence: GroupRef,
root: Option<Node>,
knn_condition: Option<Cond>,
@ -80,8 +80,9 @@ pub(super) struct RecordOptions {
remotes: RemoteIndexRefs,
}
pub(super) type LocalIndexRefs = Vec<IndexRef>;
pub(super) type RemoteIndexRefs = Arc<Vec<(Idiom, LocalIndexRefs)>>;
pub(super) type IdiomCol = usize;
pub(super) type LocalIndexRefs = Vec<(IndexRef, IdiomCol)>;
pub(super) type RemoteIndexRefs = Arc<Vec<(Arc<Idiom>, LocalIndexRefs)>>;
impl<'a> TreeBuilder<'a> {
fn new(
@ -138,13 +139,17 @@ impl<'a> TreeBuilder<'a> {
if let Some(o) = self.first_order {
if !o.random && o.direction {
if let Node::IndexedField(id, irf) = self.resolve_idiom(&o.order).await? {
if let Some(ix_ref) = irf.first().cloned() {
for (ix_ref, id_col) in &irf {
if *id_col == 0 {
self.index_map.order_limit = Some(IndexOption::new(
ix_ref,
*ix_ref,
id,
*id_col,
IdiomPosition::None,
IndexOperator::Order,
));
break;
}
}
}
}
@ -235,14 +240,14 @@ impl<'a> TreeBuilder<'a> {
async fn resolve_idiom(&mut self, i: &Idiom) -> Result<Node, Error> {
let tx = self.ctx.tx();
self.lazy_load_schema_resolver(&tx, self.table).await?;
let i = Arc::new(i.clone());
// Try to detect if it matches an index
let n = if let Some(schema) = self.schemas.get(self.table).cloned() {
let irs = self.resolve_indexes(self.table, i, &schema);
let irs = self.resolve_indexes(self.table, &i, &schema);
if !irs.is_empty() {
Node::IndexedField(i.clone(), irs)
} else if let Some(ro) =
self.resolve_record_field(&tx, schema.fields.as_ref(), i).await?
self.resolve_record_field(&tx, schema.fields.as_ref(), &i).await?
{
// Try to detect an indexed record field
Node::RecordField(i.clone(), ro)
@ -256,7 +261,7 @@ impl<'a> TreeBuilder<'a> {
Ok(n)
}
fn resolve_indexes(&mut self, t: &Table, i: &Idiom, schema: &SchemaCache) -> Vec<IndexRef> {
fn resolve_indexes(&mut self, t: &Table, i: &Idiom, schema: &SchemaCache) -> LocalIndexRefs {
if let Some(m) = self.idioms_indexes.get(t) {
if let Some(irs) = m.get(i).cloned() {
return irs;
@ -264,21 +269,22 @@ impl<'a> TreeBuilder<'a> {
}
let mut irs = Vec::new();
for ix in schema.indexes.iter() {
if ix.cols.len() == 1 && ix.cols[0].eq(i) {
if let Some(idiom_index) = ix.cols.iter().position(|p| p.eq(i)) {
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);
}
}
self.index_map.definitions.push(ix.clone());
irs.push(ixr);
self.index_map.definitions.push(ix.clone().into());
irs.push((ixr, idiom_index));
}
}
let i = Arc::new(i.clone());
if let Some(e) = self.idioms_indexes.get_mut(t) {
e.insert(i.clone(), irs.clone());
e.insert(i, irs.clone());
} else {
self.idioms_indexes.insert(t.clone(), HashMap::from([(i.clone(), irs.clone())]));
self.idioms_indexes.insert(t.clone(), HashMap::from([(i, irs.clone())]));
}
irs
}
@ -287,7 +293,7 @@ impl<'a> TreeBuilder<'a> {
&mut self,
tx: &Transaction,
fields: &[DefineFieldStatement],
idiom: &Idiom,
idiom: &Arc<Idiom>,
) -> Result<Option<RecordOptions>, Error> {
for field in fields.iter() {
if let Some(Kind::Record(tables)) = &field.kind {
@ -305,12 +311,12 @@ impl<'a> TreeBuilder<'a> {
return Ok(None);
}
let remote_field = Idiom::from(remote_field);
let remote_field = Arc::new(Idiom::from(remote_field));
let mut remotes = vec![];
for table in tables {
self.lazy_load_schema_resolver(tx, table).await?;
if let Some(shema) = self.schemas.get(table).cloned() {
let remote_irs = self.resolve_indexes(table, &remote_field, &shema);
if let Some(schema) = self.schemas.get(table).cloned() {
let remote_irs = self.resolve_indexes(table, &remote_field, &schema);
remotes.push((remote_field.clone(), remote_irs));
} else {
return Ok(None);
@ -400,43 +406,44 @@ impl<'a> TreeBuilder<'a> {
fn lookup_index_options(
&mut self,
o: &Operator,
id: &Idiom,
id: &Arc<Idiom>,
node: &Node,
exp: &Arc<Expression>,
p: IdiomPosition,
local_irs: LocalIndexRefs,
remote_irs: Option<RemoteIndexRefs>,
local_irs: &LocalIndexRefs,
remote_irs: Option<&RemoteIndexRefs>,
) -> Result<Option<IndexOption>, Error> {
if let Some(remote_irs) = remote_irs {
let mut remote_ios = Vec::with_capacity(remote_irs.len());
for (id, irs) in remote_irs.iter() {
if let Some(io) = self.lookup_index_option(irs.as_slice(), o, id, node, exp, p)? {
if let Some(io) = self.lookup_index_option(irs, o, id, node, exp, p)? {
remote_ios.push(io);
} else {
return Ok(None);
}
}
if let Some(ir) = self.lookup_join_index_ref(local_irs.as_slice()) {
let io = IndexOption::new(ir, id.clone(), p, IndexOperator::Join(remote_ios));
if let Some((irf, id_col)) = self.lookup_join_index_ref(local_irs) {
let io =
IndexOption::new(irf, id.clone(), id_col, p, IndexOperator::Join(remote_ios));
return Ok(Some(io));
}
return Ok(None);
}
let io = self.lookup_index_option(local_irs.as_slice(), o, id, node, exp, p)?;
let io = self.lookup_index_option(local_irs, o, id, node, exp, p)?;
Ok(io)
}
fn lookup_index_option(
&mut self,
irs: &[IndexRef],
irs: &LocalIndexRefs,
op: &Operator,
id: &Idiom,
id: &Arc<Idiom>,
n: &Node,
e: &Arc<Expression>,
p: IdiomPosition,
) -> Result<Option<IndexOption>, Error> {
for ir in irs {
if let Some(ix) = self.index_map.definitions.get(*ir as usize) {
for (irf, id_col) in irs.iter().filter(|(_, id_col)| 0.eq(id_col)) {
if let Some(ix) = self.index_map.definitions.get(*irf as usize) {
let op = match &ix.index {
Index::Idx => self.eval_index_operator(op, n, p),
Index::Uniq => self.eval_index_operator(op, n, p),
@ -447,7 +454,7 @@ impl<'a> TreeBuilder<'a> {
Index::Hnsw(_) => self.eval_hnsw_knn(e, op, n)?,
};
if let Some(op) = op {
let io = IndexOption::new(*ir, id.clone(), p, op);
let io = IndexOption::new(*irf, id.clone(), *id_col, p, op);
self.index_map.options.push((e.clone(), io.clone()));
return Ok(Some(io));
}
@ -456,11 +463,11 @@ impl<'a> TreeBuilder<'a> {
Ok(None)
}
fn lookup_join_index_ref(&self, irs: &[IndexRef]) -> Option<IndexRef> {
for ir in irs {
if let Some(ix) = self.index_map.definitions.get(*ir as usize) {
fn lookup_join_index_ref(&self, irs: &LocalIndexRefs) -> Option<(IndexRef, IdiomCol)> {
for (irf, id_col) in irs.iter().filter(|(_, id_col)| 0.eq(id_col)) {
if let Some(ix) = self.index_map.definitions.get(*irf as usize) {
match &ix.index {
Index::Idx | Index::Uniq => return Some(*ir),
Index::Idx | Index::Uniq => return Some((*irf, *id_col)),
_ => {}
};
}
@ -581,7 +588,7 @@ pub(super) type IndexRef = u16;
#[derive(Default)]
pub(super) struct IndexesMap {
pub(super) options: Vec<(Arc<Expression>, IndexOption)>,
pub(super) definitions: Vec<DefineIndexStatement>,
pub(super) definitions: Vec<Arc<DefineIndexStatement>>,
pub(super) order_limit: Option<IndexOption>,
}
@ -613,9 +620,9 @@ pub(super) enum Node {
right: Arc<Node>,
exp: Arc<Expression>,
},
IndexedField(Idiom, Vec<IndexRef>),
RecordField(Idiom, RecordOptions),
NonIndexedField(Idiom),
IndexedField(Arc<Idiom>, LocalIndexRefs),
RecordField(Arc<Idiom>, RecordOptions),
NonIndexedField(Arc<Idiom>),
Computable,
Computed(Arc<Value>),
Unsupported(String),
@ -632,10 +639,10 @@ impl Node {
pub(super) fn is_indexed_field(
&self,
) -> Option<(&Idiom, LocalIndexRefs, Option<RemoteIndexRefs>)> {
) -> Option<(&Arc<Idiom>, &LocalIndexRefs, Option<&RemoteIndexRefs>)> {
match self {
Self::IndexedField(id, irs) => Some((id, irs.clone(), None)),
Self::RecordField(id, ro) => Some((id, ro.locals.clone(), Some(ro.remotes.clone()))),
Self::IndexedField(id, irs) => Some((id, irs, None)),
Self::RecordField(id, ro) => Some((id, &ro.locals, Some(&ro.remotes))),
_ => None,
}
}

View file

@ -172,13 +172,26 @@ impl<'a> Index<'a> {
beg.extend_from_slice(&[0xff]);
beg
}
pub fn prefix_ids_composite_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.last_mut().unwrap() = 0x00;
beg
}
pub fn prefix_ids_composite_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.last_mut().unwrap() = 0xff;
beg
}
}
#[cfg(test)]
mod tests {
use super::*;
#[test]
fn key() {
use super::*;
#[rustfmt::skip]
let fd = vec!["testfd1", "testfd2"].into();
let id = "testid".into();
@ -192,4 +205,26 @@ mod tests {
let dec = Index::decode(&enc).unwrap();
assert_eq!(val, dec);
}
#[test]
fn key_none() {
let fd = vec!["testfd1", "testfd2"].into();
let val = Index::new("testns", "testdb", "testtb", "testix", &fd, None);
let enc = Index::encode(&val).unwrap();
assert_eq!(
enc,
b"/*testns\0*testdb\0*testtb\0+testix\0*\0\0\0\x04testfd1\0\0\0\0\x04testfd2\0\x01\0"
);
}
#[test]
fn check_composite() {
let fd = vec!["testfd1"].into();
let enc = Index::prefix_ids_composite_beg("testns", "testdb", "testtb", "testix", &fd);
assert_eq!(enc, b"/*testns\0*testdb\0*testtb\0+testix\0*\0\0\0\x04testfd1\0\x00");
let enc = Index::prefix_ids_composite_end("testns", "testdb", "testtb", "testix", &fd);
assert_eq!(enc, b"/*testns\0*testdb\0*testtb\0+testix\0*\0\0\0\x04testfd1\0\xff");
}
}

View file

@ -27,6 +27,7 @@ impl SavedValue {
}
}
#[cfg(any(feature = "kv-surrealkv", feature = "kv-fdb", feature = "kv-tikv"))]
pub(super) fn get_val(&self) -> Option<&Val> {
self.saved_val.as_ref()
}

View file

@ -344,7 +344,7 @@ 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())
self.expect_value(value(val).unwrap_or_else(|_| panic!("INVALID VALUE:\n{val}")))
}
#[allow(dead_code)]

View file

@ -2965,3 +2965,101 @@ async fn select_from_unique_index_descending() -> Result<(), Error> {
//
Ok(())
}
async fn select_composite_index(unique: bool) -> Result<(), Error> {
//
let sql = format!(
"
DEFINE INDEX t_idx ON TABLE t COLUMNS on, value {};
CREATE t:1 SET on = true, value = 1;
CREATE t:2 SET on = false, value = 1;
CREATE t:3 SET on = true, value = 2;
CREATE t:4 SET on = false, value = 2;
SELECT * FROM t WHERE on = true EXPLAIN;
SELECT * FROM t WHERE on = true;
SELECT * FROM t WHERE on = false AND value = 2 EXPLAIN;
SELECT * FROM t WHERE on = false AND value = 2;
",
if unique {
"UNIQUE"
} else {
""
}
);
let mut t = Test::new(&sql).await?;
//
t.expect_size(9)?;
t.skip_ok(5)?;
//
t.expect_vals(&[
"[
{
detail: {
plan: {
index: 't_idx',
operator: '=',
value: true
},
table: 't'
},
operation: 'Iterate Index'
},
{
detail: {
type: 'Memory'
},
operation: 'Collector'
}
]",
"[
{
id: t:1,
on: true,
value: 1
},
{
id: t:3,
on: true,
value: 2
}
]",
"[
{
detail: {
plan: {
index: 't_idx',
operator: '=',
value: false
},
table: 't'
},
operation: 'Iterate Index'
},
{
detail: {
type: 'Memory'
},
operation: 'Collector'
}
]",
"[
{
id: t:4,
on: false,
value: 2
}
]",
])?;
//
Ok(())
}
#[tokio::test]
async fn select_composite_standard_index() -> Result<(), Error> {
select_composite_index(false).await
}
#[tokio::test]
async fn select_composite_unique_index() -> Result<(), Error> {
select_composite_index(true).await
}