Skip to content

Commit

Permalink
feat(frontend): support functional indexes creation (risingwavelabs#8976
Browse files Browse the repository at this point in the history
)
  • Loading branch information
chenzl25 authored Apr 9, 2023
1 parent d8926b8 commit ced72de
Show file tree
Hide file tree
Showing 9 changed files with 276 additions and 179 deletions.
6 changes: 3 additions & 3 deletions e2e_test/batch/catalog/pg_attribute.slt.part
Original file line number Diff line number Diff line change
Expand Up @@ -38,9 +38,9 @@ select i.relname, a.attname, ix.indkey from pg_catalog.pg_class t
join pg_catalog.pg_attribute a on t.oid = a.attrelid and a.attnum = ANY(ix.indkey)
where t.relname = 'tmp' order by a.attnum;
----
tmp_idx id1 {2,1,3}
tmp_idx id2 {2,1,3}
tmp_idx id3 {2,1,3}
tmp_idx id2 {2,3,4,5}
tmp_idx id3 {2,3,4,5}
tmp_idx id4 {2,3,4,5}

statement ok
drop table tmp;
6 changes: 3 additions & 3 deletions e2e_test/batch/catalog/pg_index.slt.part
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@ select ix.indnatts, ix.indkey from pg_catalog.pg_class t
join pg_catalog.pg_class i on i.oid = ix.indexrelid
where t.relname = 'tmp' and i.relname = 'tmp_id2_idx';
----
1 {2}
2 {2,3}

statement ok
create index tmp_id2_idx_include_id1 on tmp(id2) include(id1);
Expand All @@ -21,7 +21,7 @@ select ix.indnatts, ix.indkey from pg_catalog.pg_class t
join pg_catalog.pg_class i on i.oid = ix.indexrelid
where t.relname = 'tmp' and i.relname = 'tmp_id2_idx_include_id1';
----
2 {2,1}
3 {2,3,4}

statement ok
create index tmp_id1_id2_idx on tmp(id1, id2);
Expand All @@ -32,7 +32,7 @@ select ix.indnatts, ix.indkey from pg_catalog.pg_class t
join pg_catalog.pg_class i on i.oid = ix.indexrelid
where t.relname = 'tmp' and i.relname = 'tmp_id1_id2_idx';
----
2 {1,2}
3 {2,3,4}

statement ok
drop table tmp;
2 changes: 1 addition & 1 deletion e2e_test/ddl/alter_table_column.slt
Original file line number Diff line number Diff line change
Expand Up @@ -215,7 +215,7 @@ statement ok
create table t(id int primary key, a int, b varchar);

statement ok
create index idx on t(a);
create index idx on t(a, lower(b));

statement ok
alter table t add column c int;
Expand Down
22 changes: 21 additions & 1 deletion e2e_test/ddl/index.slt
Original file line number Diff line number Diff line change
Expand Up @@ -113,4 +113,24 @@ statement ok
drop index index_on_quoted_column2;

statement ok
drop table t2;
drop table t2;

# create functional indexes
statement ok

create table t (j jsonb);

statement ok
insert into t values ('{"k": "abc" }'::jsonb);

statement ok
create index idx on t(j->'k');

# query functional indexes
query II
select * from idx;
----
"abc" {"k": "abc"}

statement ok
drop table t;
8 changes: 8 additions & 0 deletions src/frontend/planner_test/tests/testdata/index_selection.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -671,3 +671,11 @@
└─BatchLookupJoin { type: Inner, predicate: t2.d1 = idx_t1.c2, output: all }
└─BatchExchange { order: [], dist: UpstreamHashShard(t2.d1) }
└─BatchScan { table: t2, columns: [t2.d1, t2.d2], distribution: SomeShard }
- sql: |
create table t (j jsonb);
explain create index idx on t(j->'k');
explain_output: |
StreamMaterialize { columns: [JSONB_ACCESS_INNER, j, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [JSONB_ACCESS_INNER, t._row_id], pk_conflict: "NoCheck" }
└─StreamExchange { dist: HashShard($expr1) }
└─StreamProject { exprs: [JsonbAccessInner(t.j, 'k':Varchar) as $expr1, t.j, t._row_id] }
└─StreamTableScan { table: t, columns: [j, _row_id] }
46 changes: 22 additions & 24 deletions src/frontend/src/catalog/index_catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,12 @@ use std::sync::Arc;

use itertools::Itertools;
use risingwave_common::catalog::IndexId;
use risingwave_common::types::DataType;
use risingwave_common::util::sort_util::ColumnOrder;
use risingwave_pb::catalog::PbIndex;
use risingwave_pb::expr::expr_node::RexNode;

use super::ColumnId;
use crate::catalog::{DatabaseId, RelationCatalog, SchemaId, TableCatalog};
use crate::expr::{Expr, InputRef};
use crate::expr::{Expr, ExprImpl};
use crate::user::UserId;

#[derive(Clone, Debug, PartialEq, Eq, Hash)]
Expand All @@ -33,10 +31,11 @@ pub struct IndexCatalog {

pub name: String,

/// Only `InputRef` type index is supported Now.
/// Only `InputRef` and `FuncCall` type index is supported Now.
/// The index of `InputRef` is the column index of the primary table.
/// index_item size is equal to index table columns size
pub index_item: Vec<InputRef>,
/// The index_item size is equal to the index table columns size
/// The input args of `FuncCall` is also the column index of the primary table.
pub index_item: Vec<ExprImpl>,

pub index_table: Arc<TableCatalog>,

Expand All @@ -55,30 +54,29 @@ impl IndexCatalog {
index_table: &TableCatalog,
primary_table: &TableCatalog,
) -> Self {
let index_item = index_prost
let index_item: Vec<ExprImpl> = index_prost
.index_item
.iter()
.map(|x| match x.rex_node.as_ref().unwrap() {
RexNode::InputRef(input_col_idx) => InputRef {
index: *input_col_idx as usize,
data_type: DataType::from(x.return_type.as_ref().unwrap()),
},
RexNode::FuncCall(_) => unimplemented!(),
_ => unreachable!(),
})
.collect_vec();
.map(ExprImpl::from_expr_proto)
.try_collect()
.unwrap();

let primary_to_secondary_mapping = index_item
let primary_to_secondary_mapping: BTreeMap<usize, usize> = index_item
.iter()
.enumerate()
.map(|(i, input_ref)| (input_ref.index, i))
.filter_map(|(i, expr)| match expr {
ExprImpl::InputRef(input_ref) => Some((input_ref.index, i)),
ExprImpl::FunctionCall(_) => None,
_ => unreachable!(),
})
.collect();

let secondary_to_primary_mapping = index_item
.iter()
.enumerate()
.map(|(i, input_ref)| (i, input_ref.index))
.collect();
let secondary_to_primary_mapping = BTreeMap::from_iter(
primary_to_secondary_mapping
.clone()
.into_iter()
.map(|(x, y)| (y, x)),
);

let original_columns = index_prost
.original_columns
Expand Down Expand Up @@ -147,7 +145,7 @@ impl IndexCatalog {
index_item: self
.index_item
.iter()
.map(InputRef::to_expr_proto)
.map(|expr| expr.to_expr_proto())
.collect_vec(),
original_columns: self.original_columns.iter().map(Into::into).collect_vec(),
}
Expand Down
Loading

0 comments on commit ced72de

Please sign in to comment.