From 8ba06053d37f473868509d970b22e853e44893ca Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Wed, 27 Sep 2023 02:27:54 +0800 Subject: [PATCH 01/36] fix(over window): fix error in using aggregate function result as window function argument rewrote the `partition by` and `group by` expressions (`input_ref`) in the window function to avoid out of index problems. --- .../src/optimizer/plan_node/logical_agg.rs | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index d3bf8d896f9ff..7f86551acb397 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -28,6 +28,7 @@ use super::{ }; use crate::expr::{ AggCall, Expr, ExprImpl, ExprRewriter, ExprType, FunctionCall, InputRef, Literal, OrderBy, + WindowFunction, }; use crate::optimizer::plan_node::generic::GenericPlanNode; use crate::optimizer::plan_node::stream::StreamPlanRef; @@ -731,6 +732,33 @@ impl ExprRewriter for LogicalAggBuilder { } } + /// When there is an `WindowFunction` (outside of agg call), it must refers to a group column. + /// Or all `InputRef`s appears in it must refer to a group column. + fn rewrite_window_function(&mut self, window_func: WindowFunction) -> ExprImpl { + let WindowFunction { + args, + partition_by, + order_by, + .. + } = window_func; + let args = args + .into_iter() + .map(|expr| self.rewrite_expr(expr)) + .collect(); + let partition_by = partition_by + .into_iter() + .map(|expr| self.rewrite_expr(expr)) + .collect(); + let order_by = order_by.rewrite_expr(self); + WindowFunction { + args, + partition_by, + order_by, + ..window_func + } + .into() + } + /// When there is an `InputRef` (outside of agg call), it must refers to a group column. fn rewrite_input_ref(&mut self, input_ref: InputRef) -> ExprImpl { let expr = input_ref.into(); From 0d7caa6e935cf8f5d684d32732faa8a374e514d4 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Fri, 29 Sep 2023 19:16:48 +0800 Subject: [PATCH 02/36] test: add planner_test agg with window function --- .../tests/testdata/input/agg.yaml | 13 ++++++++ .../tests/testdata/output/agg.yaml | 33 +++++++++++++++++++ 2 files changed, 46 insertions(+) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index b5b8e182703f8..09de5f42f3fd3 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -929,3 +929,16 @@ expected_outputs: - batch_plan - stream_plan + +- sql: | + CREATE TABLE t (a int, b int); + SELECT a, sum((sum(b))) OVER (PARTITION BY a ORDER BY a) FROM t GROUP BY a; + expected_outputs: + - batch_plan + - stream_plan +- sql: | + CREATE TABLE t (a int, b int); + SELECT a, row_number() OVER (PARTITION BY a ORDER BY a DESC) FROM t; + expected_outputs: + - batch_plan + - stream_plan diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index d62ce89d0ed3b..82e5d496e0343 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1649,3 +1649,36 @@ └─StreamProject { exprs: [count, t.id] } └─StreamHashAgg { group_key: [t.id], aggs: [count] } └─StreamTableScan { table: t, columns: [t.id], pk: [t.id], dist: UpstreamHashShard(t.id) } +- sql: | + CREATE TABLE t(a int, b int); + SELECT a, sum((sum(b))) OVER (PARTITION BY a ORDER BY a) FROM t GROUP BY a; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.a, sum] } + └─BatchOverWindow { window_functions: [sum(sum(t.b)) OVER(PARTITION BY t.a ORDER BY t.a ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchSort { order: [t.a ASC, t.a ASC] } + └─BatchHashAgg { group_key: [t.a], aggs: [sum(t.b)] } + └─BatchExchange { order: [], dist: HashShard(t.a) } + └─BatchScan { table: t, columns: [t.a, t.b], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [a, sum], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.a, sum] } + └─StreamOverWindow { window_functions: [sum(sum(t.b)) OVER(PARTITION BY t.a ORDER BY t.a ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamProject { exprs: [t.a, sum(t.b)] } + └─StreamHashAgg { group_key: [t.a], aggs: [sum(t.b), count] } + └─StreamExchange { dist: HashShard(t.a) } + └─StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } +- sql: | + CREATE TABLE t(a int, b int); + SELECT a, row_number() OVER (PARTITION BY a ORDER BY a DESC) FROM t; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY t.a ORDER BY t.a DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchExchange { order: [t.a ASC, t.a DESC], dist: HashShard(t.a) } + └─BatchSort { order: [t.a ASC, t.a DESC] } + └─BatchScan { table: t, columns: [t.a], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [a, t._row_id(hidden), row_number], stream_key: [t._row_id, a], pk_columns: [t._row_id, a], pk_conflict: NoCheck } + └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY t.a ORDER BY t.a DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(t.a) } + └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } From fc9f48bb80cb1b7e919737353d3bc60c3b893aa3 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Fri, 29 Sep 2023 19:17:30 +0800 Subject: [PATCH 03/36] test: add e2e_test agg with window function --- .../batch/aggregate/with_over_window.slt.part | 51 +++++++++++++++++++ 1 file changed, 51 insertions(+) create mode 100644 e2e_test/batch/aggregate/with_over_window.slt.part diff --git a/e2e_test/batch/aggregate/with_over_window.slt.part b/e2e_test/batch/aggregate/with_over_window.slt.part new file mode 100644 index 0000000000000..ea573b6f225be --- /dev/null +++ b/e2e_test/batch/aggregate/with_over_window.slt.part @@ -0,0 +1,51 @@ +statement ok +create table t (a int, b int); + +statement ok +insert into t values + (1, 39), + (1, 95), + (2, 48), + (3, 87), + (4, 19), + (4, 78), + (4, 53); + +query II +select + a, + sum((sum(b))) over (partition by a order by a) +from t +group by a; +---- +1 134 +2 48 +3 87 +4 150 + +query II +select + a, + row_number() over (partition by a order by a) +from t +group by a; +---- +1 1 +2 1 +3 1 +4 1 + +query II +select + a, + row_number() over (partition by a order by a desc) +from t +group by a; +---- +1 1 +2 1 +3 1 +4 1 + +statement ok +drop table t; From 593706c7e8f93cd6e83bdcf53e12440d949234c4 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Sun, 8 Oct 2023 12:16:25 +0800 Subject: [PATCH 04/36] test: add more complex agg with window e2e_test --- .../batch/aggregate/with_over_window.slt.part | 55 +++++++++++++++---- 1 file changed, 43 insertions(+), 12 deletions(-) diff --git a/e2e_test/batch/aggregate/with_over_window.slt.part b/e2e_test/batch/aggregate/with_over_window.slt.part index ea573b6f225be..f786e47dad0ea 100644 --- a/e2e_test/batch/aggregate/with_over_window.slt.part +++ b/e2e_test/batch/aggregate/with_over_window.slt.part @@ -1,15 +1,15 @@ statement ok -create table t (a int, b int); +create table t (a int, b int, c int, d int, e int); statement ok insert into t values - (1, 39), - (1, 95), - (2, 48), - (3, 87), - (4, 19), - (4, 78), - (4, 53); + (1, 23, 84, 11, 87), + (2, 34, 29, 22, 98), + (3, 45, 43, 33, 10), + (4, 56, 83, 44, 26), + (5, 68, 20, 55, 12), + (5, 68, 90, 66, 34), + (5, 68, 11, 77, 32); query II select @@ -18,10 +18,11 @@ select from t group by a; ---- -1 134 -2 48 -3 87 -4 150 +1 23 +2 34 +3 45 +4 56 +5 204 query II select @@ -34,6 +35,7 @@ group by a; 2 1 3 1 4 1 +5 1 query II select @@ -46,6 +48,35 @@ group by a; 2 1 3 1 4 1 +5 1 + +query III +select + a, + b, + sum(sum(c)) over (partition by a order by b) +from t +group by a, b; +---- +1 23 84 +2 34 29 +3 45 43 +4 56 83 +5 68 121 + +query III +select + a, + b, + sum(sum(c)) over (partition by a, avg(d) order by max(e), b) +from t +group by a, b; +---- +1 23 84 +2 34 29 +3 45 43 +4 56 83 +5 68 121 statement ok drop table t; From 7cd9457364e02916e6406b7df0987055dd5324df Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Sun, 8 Oct 2023 12:18:29 +0800 Subject: [PATCH 05/36] test: fix the agg planner_test query missed agg --- src/frontend/planner_test/tests/testdata/input/agg.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 09de5f42f3fd3..462d49ac8909d 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -938,7 +938,7 @@ - stream_plan - sql: | CREATE TABLE t (a int, b int); - SELECT a, row_number() OVER (PARTITION BY a ORDER BY a DESC) FROM t; + SELECT a, row_number() OVER (PARTITION BY a ORDER BY a DESC) FROM t GROUP BY a; expected_outputs: - batch_plan - stream_plan From 3b0645c137581ed5e631d388af573babe455033a Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Sun, 8 Oct 2023 12:35:13 +0800 Subject: [PATCH 06/36] test: more complex agg with window planner_test --- .../tests/testdata/input/agg.yaml | 16 +++++ .../tests/testdata/output/agg.yaml | 68 ++++++++++++++++--- 2 files changed, 75 insertions(+), 9 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 462d49ac8909d..99aa94ff773b9 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -942,3 +942,19 @@ expected_outputs: - batch_plan - stream_plan +- sql: | + CREATE TABLE t (a int, b int, c int); + SELECT a, b, sum(sum(c)) OVER (PARTITION BY a ORDER BY b) + FROM t + GROUP BY a, b; + expected_outputs: + - batch_plan + - stream_plan +- sql: | + CREATE TABLE t (a int, b int, c int, d int, e int); + SELECT a, b, sum(sum(c)) OVER (PARTITION BY a, avg(d) ORDER BY max(e), b) + FROM t + GROUP BY a, b; + expected_outputs: + - batch_plan + - stream_plan diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 82e5d496e0343..aefb4df98ef4e 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1650,7 +1650,7 @@ └─StreamHashAgg { group_key: [t.id], aggs: [count] } └─StreamTableScan { table: t, columns: [t.id], pk: [t.id], dist: UpstreamHashShard(t.id) } - sql: | - CREATE TABLE t(a int, b int); + CREATE TABLE t (a int, b int); SELECT a, sum((sum(b))) OVER (PARTITION BY a ORDER BY a) FROM t GROUP BY a; batch_plan: |- BatchExchange { order: [], dist: Single } @@ -1669,16 +1669,66 @@ └─StreamExchange { dist: HashShard(t.a) } └─StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | - CREATE TABLE t(a int, b int); - SELECT a, row_number() OVER (PARTITION BY a ORDER BY a DESC) FROM t; + CREATE TABLE t (a int, b int); + SELECT a, row_number() OVER (PARTITION BY a ORDER BY a DESC) FROM t GROUP BY a; batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY t.a ORDER BY t.a DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─BatchExchange { order: [t.a ASC, t.a DESC], dist: HashShard(t.a) } - └─BatchSort { order: [t.a ASC, t.a DESC] } - └─BatchScan { table: t, columns: [t.a], distribution: SomeShard } + └─BatchSort { order: [t.a ASC, t.a DESC] } + └─BatchHashAgg { group_key: [t.a], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(t.a) } + └─BatchScan { table: t, columns: [t.a], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [a, t._row_id(hidden), row_number], stream_key: [t._row_id, a], pk_columns: [t._row_id, a], pk_conflict: NoCheck } + StreamMaterialize { columns: [a, row_number], stream_key: [a], pk_columns: [a], pk_conflict: NoCheck } └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY t.a ORDER BY t.a DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(t.a) } - └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [t.a] } + └─StreamHashAgg { group_key: [t.a], aggs: [count] } + └─StreamExchange { dist: HashShard(t.a) } + └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } +- sql: | + CREATE TABLE t (a int, b int, c int); + SELECT a, b, sum(sum(c)) OVER (PARTITION BY a ORDER BY b) + FROM t + GROUP BY a, b; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.a, t.b, sum] } + └─BatchOverWindow { window_functions: [sum(sum(t.c)) OVER(PARTITION BY t.a ORDER BY t.b ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchExchange { order: [t.a ASC, t.b ASC], dist: HashShard(t.a) } + └─BatchSort { order: [t.a ASC, t.b ASC] } + └─BatchHashAgg { group_key: [t.a, t.b], aggs: [sum(t.c)] } + └─BatchExchange { order: [], dist: HashShard(t.a, t.b) } + └─BatchScan { table: t, columns: [t.a, t.b, t.c], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [a, b, sum], stream_key: [a, b], pk_columns: [a, b], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.a, t.b, sum] } + └─StreamOverWindow { window_functions: [sum(sum(t.c)) OVER(PARTITION BY t.a ORDER BY t.b ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(t.a) } + └─StreamProject { exprs: [t.a, t.b, sum(t.c)] } + └─StreamHashAgg { group_key: [t.a, t.b], aggs: [sum(t.c), count] } + └─StreamExchange { dist: HashShard(t.a, t.b) } + └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } +- sql: | + CREATE TABLE t (a int, b int, c int, d int, e int); + SELECT a, b, sum(sum(c)) OVER (PARTITION BY a, avg(d) ORDER BY max(e), b) + FROM t + GROUP BY a, b; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.a, t.b, sum] } + └─BatchOverWindow { window_functions: [sum(sum(t.c)) OVER(PARTITION BY t.a, $expr1 ORDER BY max(t.e) ASC, t.b ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchExchange { order: [t.a ASC, $expr1 ASC, max(t.e) ASC, t.b ASC], dist: HashShard(t.a, $expr1) } + └─BatchSort { order: [t.a ASC, $expr1 ASC, max(t.e) ASC, t.b ASC] } + └─BatchProject { exprs: [t.a, t.b, sum(t.c), max(t.e), (sum(t.d)::Decimal / count(t.d)::Decimal) as $expr1] } + └─BatchHashAgg { group_key: [t.a, t.b], aggs: [sum(t.c), sum(t.d), count(t.d), max(t.e)] } + └─BatchExchange { order: [], dist: HashShard(t.a, t.b) } + └─BatchScan { table: t, columns: [t.a, t.b, t.c, t.d, t.e], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [a, b, sum, $expr1(hidden)], stream_key: [a, b, $expr1], pk_columns: [a, b, $expr1], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.a, t.b, sum, $expr1] } + └─StreamOverWindow { window_functions: [sum(sum(t.c)) OVER(PARTITION BY t.a, $expr1 ORDER BY max(t.e) ASC, t.b ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(t.a, $expr1) } + └─StreamProject { exprs: [t.a, t.b, sum(t.c), max(t.e), (sum(t.d)::Decimal / count(t.d)::Decimal) as $expr1] } + └─StreamHashAgg { group_key: [t.a, t.b], aggs: [sum(t.c), sum(t.d), count(t.d), max(t.e), count] } + └─StreamExchange { dist: HashShard(t.a, t.b) } + └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t.d, t.e, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } From 5e18a8a98e0bc17c592bca149964498f26189d34 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Sun, 8 Oct 2023 15:07:19 +0800 Subject: [PATCH 07/36] test: add "over_window with agg" streaming e2e_test --- .../generated/batch/create.slt.part | 12 +++++++- .../over_window/generated/batch/drop.slt.part | 3 ++ .../over_window/generated/batch/mod.slt.part | 28 +++++++++++++++++++ .../generated/streaming/create.slt.part | 12 +++++++- .../generated/streaming/drop.slt.part | 3 ++ .../generated/streaming/mod.slt.part | 28 +++++++++++++++++++ .../over_window/templates/create.slt.part | 12 +++++++- e2e_test/over_window/templates/drop.slt.part | 3 ++ e2e_test/over_window/templates/mod.slt.part | 28 +++++++++++++++++++ 9 files changed, 126 insertions(+), 3 deletions(-) diff --git a/e2e_test/over_window/generated/batch/create.slt.part b/e2e_test/over_window/generated/batch/create.slt.part index 8e489c3dde0bd..5f4b5e1152804 100644 --- a/e2e_test/over_window/generated/batch/create.slt.part +++ b/e2e_test/over_window/generated/batch/create.slt.part @@ -49,6 +49,16 @@ select , row_number() over (partition by p1 order by p2 desc, id) as out11 from t; +# over + agg +statement ok +create view v_e as +select + p1, p2 + , row_number() over (partition by p1 order by p2) as out12 + , sum(sum(v2)) over (partition by p1, avg(time) order by max(v1), p2) as out13 +from t +group by p1, p2; + statement ok create view v_a_b as select @@ -103,4 +113,4 @@ select , first_value(v1) over (partition by p1, p2 order by time, id rows 3 preceding) as out3 , lag(v1 + 2, 0 + 1) over (partition by p1 - 1 order by id) as out4 , min(v1 * 2) over (partition by p1, p2 order by time + 1, id rows between current row and unbounded following) as out5 -from t; \ No newline at end of file +from t; diff --git a/e2e_test/over_window/generated/batch/drop.slt.part b/e2e_test/over_window/generated/batch/drop.slt.part index 8eaca578e1f4e..435ffd46433e7 100644 --- a/e2e_test/over_window/generated/batch/drop.slt.part +++ b/e2e_test/over_window/generated/batch/drop.slt.part @@ -12,6 +12,9 @@ drop view v_c; statement ok drop view v_d; +statement ok +drop view v_e; + statement ok drop view v_a_b; diff --git a/e2e_test/over_window/generated/batch/mod.slt.part b/e2e_test/over_window/generated/batch/mod.slt.part index ff46877de7ddf..2c7778fd46aff 100644 --- a/e2e_test/over_window/generated/batch/mod.slt.part +++ b/e2e_test/over_window/generated/batch/mod.slt.part @@ -41,6 +41,13 @@ select * from v_d order by id; 100003 100 208 2 723 807 3 1 100004 103 200 2 702 808 1 1 +query iiii +select * from v_e order by p1; +---- +100 200 1 1611 +100 208 2 807 +103 200 1 808 + include ./cross_check.slt.part statement ok @@ -88,6 +95,14 @@ select * from v_d order by id; 100005 100 200 3 717 810 4 4 100006 105 204 5 703 828 1 1 +query iiii +select * from v_e order by p1, p2; +---- +100 200 1 2421 +100 208 2 3228 +103 200 1 808 +105 204 1 828 + include ./cross_check.slt.part statement ok @@ -139,6 +154,13 @@ select * from v_d order by id; 100005 100 200 1 717 810 2 4 100006 105 204 5 703 828 1 1 +query iiiiiii +select * from v_e order by p1; +---- +100 200 1 3228 +103 200 1 808 +105 204 1 828 + query iiiiiiiiii select * from v_expr order by id; ---- @@ -182,6 +204,12 @@ select * from v_d order by id; 100005 100 200 1 717 810 2 2 100006 105 204 5 703 828 1 1 +query iiii +select * from v_e order by p1; +---- +100 200 1 1615 +105 204 1 828 + query iiiiiiiiii select * from v_expr order by id; ---- diff --git a/e2e_test/over_window/generated/streaming/create.slt.part b/e2e_test/over_window/generated/streaming/create.slt.part index 23a496ff0f315..4334fb1cdd30e 100644 --- a/e2e_test/over_window/generated/streaming/create.slt.part +++ b/e2e_test/over_window/generated/streaming/create.slt.part @@ -49,6 +49,16 @@ select , row_number() over (partition by p1 order by p2 desc, id) as out11 from t; +# over + agg +statement ok +create materialized view v_e as +select + p1, p2 + , row_number() over (partition by p1 order by p2) as out12 + , sum(sum(v2)) over (partition by p1, avg(time) order by max(v1), p2) as out13 +from t +group by p1, p2; + statement ok create materialized view v_a_b as select @@ -103,4 +113,4 @@ select , first_value(v1) over (partition by p1, p2 order by time, id rows 3 preceding) as out3 , lag(v1 + 2, 0 + 1) over (partition by p1 - 1 order by id) as out4 , min(v1 * 2) over (partition by p1, p2 order by time + 1, id rows between current row and unbounded following) as out5 -from t; \ No newline at end of file +from t; diff --git a/e2e_test/over_window/generated/streaming/drop.slt.part b/e2e_test/over_window/generated/streaming/drop.slt.part index d469282f41247..e6c4fcfaad244 100644 --- a/e2e_test/over_window/generated/streaming/drop.slt.part +++ b/e2e_test/over_window/generated/streaming/drop.slt.part @@ -12,6 +12,9 @@ drop materialized view v_c; statement ok drop materialized view v_d; +statement ok +drop materialized view v_e; + statement ok drop materialized view v_a_b; diff --git a/e2e_test/over_window/generated/streaming/mod.slt.part b/e2e_test/over_window/generated/streaming/mod.slt.part index ff46877de7ddf..2c7778fd46aff 100644 --- a/e2e_test/over_window/generated/streaming/mod.slt.part +++ b/e2e_test/over_window/generated/streaming/mod.slt.part @@ -41,6 +41,13 @@ select * from v_d order by id; 100003 100 208 2 723 807 3 1 100004 103 200 2 702 808 1 1 +query iiii +select * from v_e order by p1; +---- +100 200 1 1611 +100 208 2 807 +103 200 1 808 + include ./cross_check.slt.part statement ok @@ -88,6 +95,14 @@ select * from v_d order by id; 100005 100 200 3 717 810 4 4 100006 105 204 5 703 828 1 1 +query iiii +select * from v_e order by p1, p2; +---- +100 200 1 2421 +100 208 2 3228 +103 200 1 808 +105 204 1 828 + include ./cross_check.slt.part statement ok @@ -139,6 +154,13 @@ select * from v_d order by id; 100005 100 200 1 717 810 2 4 100006 105 204 5 703 828 1 1 +query iiiiiii +select * from v_e order by p1; +---- +100 200 1 3228 +103 200 1 808 +105 204 1 828 + query iiiiiiiiii select * from v_expr order by id; ---- @@ -182,6 +204,12 @@ select * from v_d order by id; 100005 100 200 1 717 810 2 2 100006 105 204 5 703 828 1 1 +query iiii +select * from v_e order by p1; +---- +100 200 1 1615 +105 204 1 828 + query iiiiiiiiii select * from v_expr order by id; ---- diff --git a/e2e_test/over_window/templates/create.slt.part b/e2e_test/over_window/templates/create.slt.part index 0d16b52fcdc86..7ac749e459b02 100644 --- a/e2e_test/over_window/templates/create.slt.part +++ b/e2e_test/over_window/templates/create.slt.part @@ -47,6 +47,16 @@ select , row_number() over (partition by p1 order by p2 desc, id) as out11 from t; +# over + agg +statement ok +create $view_type v_e as +select + p1, p2 + , row_number() over (partition by p1 order by p2) as out12 + , sum(sum(v2)) over (partition by p1, avg(time) order by max(v1), p2) as out13 +from t +group by p1, p2; + statement ok create $view_type v_a_b as select @@ -101,4 +111,4 @@ select , first_value(v1) over (partition by p1, p2 order by time, id rows 3 preceding) as out3 , lag(v1 + 2, 0 + 1) over (partition by p1 - 1 order by id) as out4 , min(v1 * 2) over (partition by p1, p2 order by time + 1, id rows between current row and unbounded following) as out5 -from t; \ No newline at end of file +from t; diff --git a/e2e_test/over_window/templates/drop.slt.part b/e2e_test/over_window/templates/drop.slt.part index 926305ee42699..def8e92379878 100644 --- a/e2e_test/over_window/templates/drop.slt.part +++ b/e2e_test/over_window/templates/drop.slt.part @@ -10,6 +10,9 @@ drop $view_type v_c; statement ok drop $view_type v_d; +statement ok +drop $view_type v_e; + statement ok drop $view_type v_a_b; diff --git a/e2e_test/over_window/templates/mod.slt.part b/e2e_test/over_window/templates/mod.slt.part index 3e48a52358701..1b1b86a0d40d3 100644 --- a/e2e_test/over_window/templates/mod.slt.part +++ b/e2e_test/over_window/templates/mod.slt.part @@ -39,6 +39,13 @@ select * from v_d order by id; 100003 100 208 2 723 807 3 1 100004 103 200 2 702 808 1 1 +query iiii +select * from v_e order by p1; +---- +100 200 1 1611 +100 208 2 807 +103 200 1 808 + include ./cross_check.slt.part statement ok @@ -86,6 +93,14 @@ select * from v_d order by id; 100005 100 200 3 717 810 4 4 100006 105 204 5 703 828 1 1 +query iiii +select * from v_e order by p1, p2; +---- +100 200 1 2421 +100 208 2 3228 +103 200 1 808 +105 204 1 828 + include ./cross_check.slt.part statement ok @@ -137,6 +152,13 @@ select * from v_d order by id; 100005 100 200 1 717 810 2 4 100006 105 204 5 703 828 1 1 +query iiiiiii +select * from v_e order by p1; +---- +100 200 1 3228 +103 200 1 808 +105 204 1 828 + query iiiiiiiiii select * from v_expr order by id; ---- @@ -180,6 +202,12 @@ select * from v_d order by id; 100005 100 200 1 717 810 2 2 100006 105 204 5 703 828 1 1 +query iiii +select * from v_e order by p1; +---- +100 200 1 1615 +105 204 1 828 + query iiiiiiiiii select * from v_expr order by id; ---- From 4147b8c33be508d1a3a95d5f111ba5da9c351101 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Mon, 9 Oct 2023 21:37:16 +0800 Subject: [PATCH 08/36] test: add `group by` to every agg + window query To ensure consistent result --- .../batch/aggregate/with_over_window.slt.part | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/e2e_test/batch/aggregate/with_over_window.slt.part b/e2e_test/batch/aggregate/with_over_window.slt.part index f786e47dad0ea..a450b13837791 100644 --- a/e2e_test/batch/aggregate/with_over_window.slt.part +++ b/e2e_test/batch/aggregate/with_over_window.slt.part @@ -16,7 +16,8 @@ select a, sum((sum(b))) over (partition by a order by a) from t -group by a; +group by a +order by a; ---- 1 23 2 34 @@ -29,7 +30,8 @@ select a, row_number() over (partition by a order by a) from t -group by a; +group by a +order by a; ---- 1 1 2 1 @@ -42,7 +44,8 @@ select a, row_number() over (partition by a order by a desc) from t -group by a; +group by a +order by a; ---- 1 1 2 1 @@ -56,7 +59,8 @@ select b, sum(sum(c)) over (partition by a order by b) from t -group by a, b; +group by a, b +order by a, b; ---- 1 23 84 2 34 29 @@ -70,7 +74,8 @@ select b, sum(sum(c)) over (partition by a, avg(d) order by max(e), b) from t -group by a, b; +group by a, b +order by a, b; ---- 1 23 84 2 34 29 From a7f3dc2201e4d2dd698aaea1de75d6c4906de663 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Sun, 15 Oct 2023 17:51:39 +0800 Subject: [PATCH 09/36] feat: add `comment on` clause support --- proto/catalog.proto | 22 ++- proto/ddl_service.proto | 12 ++ proto/plan_common.proto | 2 + proto/stream_plan.proto | 2 + src/common/src/catalog/column.rs | 8 + src/common/src/catalog/mod.rs | 2 + src/common/src/catalog/test_utils.rs | 1 + src/compute/tests/integration_tests.rs | 1 + src/connector/src/parser/avro/util.rs | 1 + src/connector/src/parser/protobuf/parser.rs | 1 + src/connector/src/sink/catalog/desc.rs | 4 + src/connector/src/sink/catalog/mod.rs | 4 + src/connector/src/source/manager.rs | 1 + src/frontend/src/binder/expr/mod.rs | 2 + src/frontend/src/catalog/catalog_service.rs | 22 +++ .../src/catalog/connection_catalog.rs | 2 + src/frontend/src/catalog/database_catalog.rs | 7 + src/frontend/src/catalog/function_catalog.rs | 2 + src/frontend/src/catalog/index_catalog.rs | 4 + src/frontend/src/catalog/root_catalog.rs | 8 + src/frontend/src/catalog/schema_catalog.rs | 4 + src/frontend/src/catalog/source_catalog.rs | 3 + .../src/catalog/system_catalog/mod.rs | 5 + .../catalog/system_catalog/rw_catalog/mod.rs | 2 + .../rw_catalog/rw_description.rs | 174 ++++++++++++++++++ src/frontend/src/catalog/table_catalog.rs | 7 + src/frontend/src/catalog/view_catalog.rs | 3 + .../src/expr/user_defined_function.rs | 1 + src/frontend/src/handler/comment.rs | 55 ++++++ src/frontend/src/handler/create_function.rs | 1 + src/frontend/src/handler/create_index.rs | 1 + src/frontend/src/handler/create_source.rs | 5 + src/frontend/src/handler/create_table.rs | 2 + src/frontend/src/handler/create_view.rs | 1 + src/frontend/src/handler/mod.rs | 6 + .../optimizer/plan_node/stream_materialize.rs | 1 + .../src/optimizer/plan_node/stream_sink.rs | 1 + src/frontend/src/optimizer/plan_node/utils.rs | 1 + src/frontend/src/test_utils.rs | 15 ++ src/meta/src/controller/mod.rs | 2 + src/meta/src/manager/catalog/mod.rs | 36 +++- src/meta/src/model/catalog.rs | 1 + src/meta/src/rpc/ddl_controller.rs | 8 +- src/meta/src/rpc/service/ddl_service.rs | 24 ++- src/rpc_client/src/meta_client.rs | 16 ++ src/source/src/source_desc.rs | 1 + src/storage/src/filter_key_extractor.rs | 1 + .../src/delete_range_runner.rs | 1 + src/utils/pgwire/src/pg_response.rs | 1 + 49 files changed, 481 insertions(+), 6 deletions(-) create mode 100644 src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs create mode 100644 src/frontend/src/handler/comment.rs diff --git a/proto/catalog.proto b/proto/catalog.proto index 2d4d51b5692b3..63f52e214d155 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -94,6 +94,7 @@ message Source { optional uint64 initialized_at_epoch = 15; optional uint64 created_at_epoch = 16; + optional string description = 17; // Per-source catalog version, used by schema change. uint64 version = 100; @@ -140,6 +141,7 @@ message Sink { string sink_from_name = 18; StreamJobStatus stream_job_status = 19; SinkFormatDesc format_desc = 20; + optional string description = 21; } message Connection { @@ -164,6 +166,7 @@ message Connection { PrivateLinkService private_link_service = 5; } uint32 owner = 6; + optional string description = 7; } message Index { @@ -182,6 +185,7 @@ message Index { optional uint64 initialized_at_epoch = 10; optional uint64 created_at_epoch = 11; StreamJobStatus stream_job_status = 12; + optional string description = 13; } message Function { @@ -195,11 +199,12 @@ message Function { string language = 7; string link = 8; string identifier = 10; + optional string description = 11; oneof kind { - ScalarFunction scalar = 11; - TableFunction table = 12; - AggregateFunction aggregate = 13; + ScalarFunction scalar = 12; + TableFunction table = 13; + AggregateFunction aggregate = 14; } message ScalarFunction {} message TableFunction {} @@ -280,6 +285,8 @@ message Table { CreateType create_type = 32; + optional string description = 33; + // Per-table catalog version, used by schema change. `None` for internal tables and tests. // Not to be confused with the global catalog version for notification service. TableVersion version = 100; @@ -303,6 +310,7 @@ message View { repeated uint32 dependent_relations = 8; // User-specified column names. repeated plan_common.Field columns = 9; + optional string description = 10; } message Schema { @@ -310,10 +318,18 @@ message Schema { uint32 database_id = 2; string name = 3; uint32 owner = 4; + optional string description = 5; } message Database { uint32 id = 1; string name = 2; uint32 owner = 3; + optional string description = 4; +} + +message Comment { + uint32 table_id = 1; + uint32 column_index = 2; + optional string description = 3; } diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 27c9f2ee82f83..29bd3e23260f8 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -314,6 +314,17 @@ message GetTablesResponse { map tables = 1; } +message CreateCommentRequest { + uint32 table_id = 1; + uint32 column_index = 2; + optional string comment = 3; +} + +message CreateCommentResponse { + common.Status status = 1; + uint64 version = 2; +} + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); @@ -342,5 +353,6 @@ service DdlService { rpc CreateConnection(CreateConnectionRequest) returns (CreateConnectionResponse); rpc ListConnections(ListConnectionsRequest) returns (ListConnectionsResponse); rpc DropConnection(DropConnectionRequest) returns (DropConnectionResponse); + rpc CreateComment(CreateCommentRequest) returns (CreateCommentResponse); rpc GetTables(GetTablesRequest) returns (GetTablesResponse); } diff --git a/proto/plan_common.proto b/proto/plan_common.proto index a88242a572693..1430bbea68588 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -37,6 +37,8 @@ message ColumnDesc { GeneratedColumnDesc generated_column = 6; DefaultColumnDesc default_column = 7; } + + optional string description = 8; } message ColumnCatalog { diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index a1026e0d36c1f..e33e674df7c04 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -181,6 +181,8 @@ message SinkDesc { // it is the name of the sink itself. string sink_from_name = 12; catalog.SinkFormatDesc format_desc = 13; + + optional string description = 14; } enum SinkLogStoreType { diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index cde16ef8d7652..9aaacf6e52df1 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -101,6 +101,7 @@ pub struct ColumnDesc { pub field_descs: Vec, pub type_name: String, pub generated_or_default_column: Option, + pub description: Option, } impl ColumnDesc { @@ -112,6 +113,7 @@ impl ColumnDesc { field_descs: vec![], type_name: String::new(), generated_or_default_column: None, + description: None, } } @@ -128,6 +130,7 @@ impl ColumnDesc { .map(|f| f.to_protobuf()) .collect_vec(), type_name: self.type_name.clone(), + description: self.description.clone(), generated_or_default_column: self.generated_or_default_column.clone(), } } @@ -171,6 +174,7 @@ impl ColumnDesc { name: name.to_string(), field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, } } @@ -191,6 +195,7 @@ impl ColumnDesc { name: name.to_string(), field_descs: fields, type_name: type_name.to_string(), + description: None, generated_or_default_column: None, } } @@ -206,6 +211,7 @@ impl ColumnDesc { .map(Self::from_field_without_column_id) .collect_vec(), type_name: field.type_name.clone(), + description: None, generated_or_default_column: None, } } @@ -242,6 +248,7 @@ impl From for ColumnDesc { name: prost.name, type_name: prost.type_name, field_descs, + description: prost.description.clone(), generated_or_default_column: prost.generated_or_default_column, } } @@ -261,6 +268,7 @@ impl From<&ColumnDesc> for PbColumnDesc { name: c.name.clone(), field_descs: c.field_descs.iter().map(ColumnDesc::to_protobuf).collect(), type_name: c.type_name.clone(), + description: c.description.clone(), generated_or_default_column: c.generated_or_default_column.clone(), } } diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index e83b4aa638907..36f2a9d01c18d 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -109,6 +109,7 @@ pub fn row_id_column_desc() -> ColumnDesc { name: row_id_column_name(), field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, } } @@ -130,6 +131,7 @@ pub fn offset_column_desc() -> ColumnDesc { name: offset_column_name(), field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, } } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index 2cce9b79b346e..e217b3b51ce9a 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -56,6 +56,7 @@ impl ColumnDescTestExt for ColumnDesc { name: name.to_string(), type_name: type_name.to_string(), field_descs: fields, + description: None, generated_or_default_column: None, } } diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index a43ae2e5762da..e47cb98d09807 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -158,6 +158,7 @@ async fn test_table_materialize() -> StreamResult<()> { name: field.name, field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, }) .collect_vec(); diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 917b23f1af6d0..78cd568792ff2 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -56,6 +56,7 @@ fn avro_field_to_column_desc( name: name.to_owned(), field_descs: vec_column, type_name: schema_name.to_string(), + description: None, generated_or_default_column: None, }) } diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 09763bcd6a49d..a0298d5577d01 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -196,6 +196,7 @@ impl ProtobufParserConfig { column_type: Some(field_type.to_protobuf()), field_descs, type_name: m.full_name().to_string(), + description: None, generated_or_default_column: None, }) } else { diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index 1fdd8b44959e3..6d1031ef6f7af 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -64,6 +64,8 @@ pub struct SinkDesc { /// Name of the "table" field for Debezium. If the sink is from table or mv, /// it is the name of table/mv. Otherwise, it is the name of the sink. pub sink_from_name: String, + + pub description: Option, } impl SinkDesc { @@ -95,6 +97,7 @@ impl SinkDesc { initialized_at_epoch: None, db_name: self.db_name, sink_from_name: self.sink_from_name, + description: None, } } @@ -116,6 +119,7 @@ impl SinkDesc { format_desc: self.format_desc.as_ref().map(|f| f.to_proto()), db_name: self.db_name.clone(), sink_from_name: self.sink_from_name.clone(), + description: self.description.clone(), } } } diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index c18dd7d10a92c..f81be598756a8 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -293,6 +293,8 @@ pub struct SinkCatalog { /// Name for the table info for Debezium sink pub sink_from_name: String, + + pub description: Option, } impl SinkCatalog { @@ -330,6 +332,7 @@ impl SinkCatalog { db_name: self.db_name.clone(), sink_from_name: self.sink_from_name.clone(), stream_job_status: PbStreamJobStatus::Creating.into(), + description: self.description.clone(), } } @@ -418,6 +421,7 @@ impl From for SinkCatalog { initialized_at_epoch: pb.initialized_at_epoch.map(Epoch::from), db_name: pb.db_name, sink_from_name: pb.sink_from_name, + description: pb.description.clone(), } } } diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index bdd72a090f9e8..0823d5fe81b34 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -122,6 +122,7 @@ impl From<&SourceColumnDesc> for ColumnDesc { name: s.name.clone(), field_descs: s.fields.clone(), type_name: "".to_string(), + description: None, generated_or_default_column: None, } } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index e9f10f572763a..658d3e77850c8 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -589,6 +589,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { name: f.name.real_value(), field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, }) }) @@ -602,6 +603,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { name: column_def.name.real_value(), field_descs, type_name: "".to_string(), + description: None, generated_or_default_column: None, }) } diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index 8eb6b9e3e4485..974824fe895ac 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -111,6 +111,13 @@ pub trait CatalogWriter: Send + Sync { connection: create_connection_request::Payload, ) -> Result<()>; + async fn create_comment( + &self, + table_id: TableId, + column_index: u32, + comment: Option, + ) -> Result<()>; + async fn drop_table( &self, source_id: Option, @@ -162,6 +169,7 @@ impl CatalogWriter for CatalogWriterImpl { name: db_name.to_string(), id: 0, owner, + description: None, }) .await?; self.wait_version(version).await @@ -180,6 +188,7 @@ impl CatalogWriter for CatalogWriterImpl { name: schema_name.to_string(), database_id: db_id, owner, + description: None, }) .await?; self.wait_version(version).await @@ -282,6 +291,19 @@ impl CatalogWriter for CatalogWriterImpl { self.wait_version(version).await } + async fn create_comment( + &self, + TableId { table_id }: TableId, + column_index: u32, + comment: Option, + ) -> Result<()> { + let version = self + .meta_client + .create_comment(table_id, column_index, comment) + .await?; + self.wait_version(version).await + } + async fn drop_table( &self, source_id: Option, diff --git a/src/frontend/src/catalog/connection_catalog.rs b/src/frontend/src/catalog/connection_catalog.rs index 7913d04379cd5..6b9cf7906e10a 100644 --- a/src/frontend/src/catalog/connection_catalog.rs +++ b/src/frontend/src/catalog/connection_catalog.rs @@ -32,6 +32,7 @@ pub struct ConnectionCatalog { pub name: String, pub info: connection::Info, pub owner: UserId, + pub description: Option, } impl ConnectionCatalog { @@ -55,6 +56,7 @@ impl From<&PbConnection> for ConnectionCatalog { name: prost.name.clone(), info: prost.info.clone().unwrap(), owner: prost.owner, + description: prost.description.clone(), } } } diff --git a/src/frontend/src/catalog/database_catalog.rs b/src/frontend/src/catalog/database_catalog.rs index 76cfc0cd359c1..81c510a6db8bd 100644 --- a/src/frontend/src/catalog/database_catalog.rs +++ b/src/frontend/src/catalog/database_catalog.rs @@ -28,6 +28,7 @@ pub struct DatabaseCatalog { schema_by_name: HashMap, schema_name_by_id: HashMap, owner: u32, + description: Option, } impl DatabaseCatalog { @@ -65,6 +66,7 @@ impl DatabaseCatalog { database_id: self.id, name: schema.name(), owner: schema.owner(), + description: None, }) .collect_vec() } @@ -108,6 +110,10 @@ impl DatabaseCatalog { pub fn owner(&self) -> u32 { self.owner } + + pub fn description(&self) -> Option<&str> { + self.description.as_deref() + } } impl From<&PbDatabase> for DatabaseCatalog { fn from(db: &PbDatabase) -> Self { @@ -117,6 +123,7 @@ impl From<&PbDatabase> for DatabaseCatalog { schema_by_name: HashMap::new(), schema_name_by_id: HashMap::new(), owner: db.owner, + description: db.description.clone(), } } } diff --git a/src/frontend/src/catalog/function_catalog.rs b/src/frontend/src/catalog/function_catalog.rs index e56a36d85f8c2..5c5f54131b959 100644 --- a/src/frontend/src/catalog/function_catalog.rs +++ b/src/frontend/src/catalog/function_catalog.rs @@ -31,6 +31,7 @@ pub struct FunctionCatalog { pub language: String, pub identifier: String, pub link: String, + pub description: Option, } #[derive(Clone, Display, PartialEq, Eq, Hash, Debug)] @@ -64,6 +65,7 @@ impl From<&PbFunction> for FunctionCatalog { language: prost.language.clone(), identifier: prost.identifier.clone(), link: prost.link.clone(), + description: prost.description.clone(), } } } diff --git a/src/frontend/src/catalog/index_catalog.rs b/src/frontend/src/catalog/index_catalog.rs index ca4b4036332d3..ed1bf2cf753da 100644 --- a/src/frontend/src/catalog/index_catalog.rs +++ b/src/frontend/src/catalog/index_catalog.rs @@ -63,6 +63,8 @@ pub struct IndexCatalog { pub created_at_epoch: Option, pub initialized_at_epoch: Option, + + pub description: Option, } impl IndexCatalog { @@ -124,6 +126,7 @@ impl IndexCatalog { original_columns, created_at_epoch: index_prost.created_at_epoch.map(Epoch::from), initialized_at_epoch: index_prost.initialized_at_epoch.map(Epoch::from), + description: index_prost.description.clone(), } } @@ -185,6 +188,7 @@ impl IndexCatalog { initialized_at_epoch: self.initialized_at_epoch.map(|e| e.0), created_at_epoch: self.created_at_epoch.map(|e| e.0), stream_job_status: PbStreamJobStatus::Creating.into(), + description: self.description.clone(), } } diff --git a/src/frontend/src/catalog/root_catalog.rs b/src/frontend/src/catalog/root_catalog.rs index f8fa09efa43e3..f5c1402c37083 100644 --- a/src/frontend/src/catalog/root_catalog.rs +++ b/src/frontend/src/catalog/root_catalog.rs @@ -370,6 +370,14 @@ impl Catalog { Ok(self.get_database_by_name(db_name)?.iter_schemas()) } + pub fn iter_schemas_except_rw_catalog( + &self, + db_name: &str, + ) -> CatalogResult> { + self.iter_schemas(db_name) + .map(|scs| scs.filter(|sc| !sc.is_rw_catalog())) + } + pub fn get_all_database_names(&self) -> Vec { self.database_by_name.keys().cloned().collect_vec() } diff --git a/src/frontend/src/catalog/schema_catalog.rs b/src/frontend/src/catalog/schema_catalog.rs index e42cbc639bba6..ab6533704e008 100644 --- a/src/frontend/src/catalog/schema_catalog.rs +++ b/src/frontend/src/catalog/schema_catalog.rs @@ -432,6 +432,10 @@ impl SchemaCatalog { self.system_table_by_name.values() } + pub fn is_rw_catalog(&self) -> bool { + !self.system_table_by_name.is_empty() + } + pub fn get_table_by_name(&self, table_name: &str) -> Option<&Arc> { self.table_by_name.get(table_name) } diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index ec35cfb7bde28..77eec6e01e339 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -43,6 +43,7 @@ pub struct SourceCatalog { pub connection_id: Option, pub created_at_epoch: Option, pub initialized_at_epoch: Option, + pub description: Option, pub version: SourceVersionId, } @@ -72,6 +73,7 @@ impl SourceCatalog { optional_associated_table_id: self .associated_table_id .map(|id| OptionalAssociatedTableId::AssociatedTableId(id.table_id)), + description: self.description.clone(), version: self.version, } } @@ -127,6 +129,7 @@ impl From<&PbSource> for SourceCatalog { connection_id, created_at_epoch: prost.created_at_epoch.map(Epoch::from), initialized_at_epoch: prost.initialized_at_epoch.map(Epoch::from), + description: prost.description.clone(), version, } } diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 4cd271f0495b9..0d68fda380da3 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -57,6 +57,8 @@ pub struct SystemTableCatalog { // owner of table, should always be default super user, keep it for compatibility. pub owner: u32, + + pub description: Option, } impl SystemTableCatalog { @@ -165,6 +167,7 @@ impl From<&BuiltinTable> for SystemTableCatalog { .collect(), pk: val.pk.to_vec(), owner: DEFAULT_SUPER_USER_ID, + description: None, } } } @@ -182,6 +185,7 @@ impl From<&BuiltinView> for ViewCatalog { sql: val.sql.to_string(), owner: DEFAULT_SUPER_USER_ID, properties: Default::default(), + description: None, } } } @@ -412,6 +416,7 @@ prepare_sys_catalog! { { BuiltinCatalog::Table(&RW_HUMMOCK_BRANCHED_OBJECTS), read_hummock_branched_objects await }, { BuiltinCatalog::Table(&RW_HUMMOCK_COMPACTION_GROUP_CONFIGS), read_hummock_compaction_group_configs await }, { BuiltinCatalog::Table(&RW_HUMMOCK_META_CONFIGS), read_hummock_meta_configs await}, + { BuiltinCatalog::Table(&RW_DESCRIPTION), read_rw_description }, } #[cfg(test)] diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs index 9f89c9eed5e81..517f40b460e94 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs @@ -17,6 +17,7 @@ mod rw_columns; mod rw_connections; mod rw_databases; mod rw_ddl_progress; +mod rw_description; mod rw_fragments; mod rw_functions; mod rw_hummock_branched_objects; @@ -50,6 +51,7 @@ pub use rw_columns::*; pub use rw_connections::*; pub use rw_databases::*; pub use rw_ddl_progress::*; +pub use rw_description::*; pub use rw_fragments::*; pub use rw_functions::*; pub use rw_hummock_branched_objects::*; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs new file mode 100644 index 0000000000000..3b6e52852bacc --- /dev/null +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -0,0 +1,174 @@ +use std::iter; + +use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; + +use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; + +pub const RW_DESCRIPTION: BuiltinTable = BuiltinTable { + name: "rw_description", + schema: RW_CATALOG_SCHEMA_NAME, + columns: &[ + (DataType::Int32, "objoid"), + (DataType::Int32, "classoid"), + (DataType::Int32, "objsubid"), + (DataType::Varchar, "description"), + ], + pk: &[0, 1, 2], +}; + +impl SysCatalogReaderImpl { + pub fn read_rw_description(&self) -> Result> { + let build_row = |table_id, catalog_id, description| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table_id)), + Some(ScalarImpl::Int32(catalog_id)), + Some(ScalarImpl::Int32(0)), + Some(ScalarImpl::Utf8(description)), + ]) + }; + let build_row_with_sub = |table_id, catalog_id, index, description| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table_id)), + Some(ScalarImpl::Int32(catalog_id)), + Some(ScalarImpl::Int32(index)), + Some(ScalarImpl::Utf8(description)), + ]) + }; + + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas_except_rw_catalog(&self.auth_context.database)?; + let rw_catalog = reader.get_schema_by_name(&self.auth_context.database, "rw_catalog")?; + + // XXX: is it shared object ?? + let database_desc = reader.iter_databases().map(|db| { + build_row( + db.id() as _, + rw_catalog + .get_system_table_by_name("rw_databases") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + db.description().unwrap_or_default().into(), + ) + }); + + Ok(schemas + .flat_map(|schema| { + let table_desc = schema.iter_table().flat_map(|table| { + iter::once(build_row( + table.id.table_id as _, + rw_catalog + .get_system_table_by_name("rw_tables") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + table.description.as_deref().unwrap_or_default().into(), + )) + .chain( + table + .columns + .iter() + .filter(|col| !col.is_hidden()) + .map(|col| { + build_row_with_sub( + table.id.table_id as _, + rw_catalog + .get_system_table_by_name("rw_tables") + .map(|st| st.id.table_id) + .unwrap_or_default() + as _, + col.column_id().get_id() as _, + col.column_desc + .description + .as_deref() + .unwrap_or_default() + .into(), + ) + }), + ) + }); + let mv_desc = schema.iter_mv().map(|mv| { + build_row( + mv.id.table_id as _, + rw_catalog + .get_system_table_by_name("rw_materialized_views") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + mv.description.as_deref().unwrap_or_default().into(), + ) + }); + let index_desc = schema.iter_index().map(|index| { + build_row( + index.id.index_id as _, + rw_catalog + .get_system_table_by_name("rw_indexes") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + index.description.as_deref().unwrap_or_default().into(), + ) + }); + let source_desc = schema.iter_source().map(|source| { + build_row( + source.id as _, + rw_catalog + .get_system_table_by_name("rw_sources") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + source.description.as_deref().unwrap_or_default().into(), + ) + }); + let sink_desc = schema.iter_sink().map(|sink| { + build_row( + sink.id.sink_id as _, + rw_catalog + .get_system_table_by_name("rw_sinks") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + sink.description.as_deref().unwrap_or_default().into(), + ) + }); + let view_desc = schema.iter_view().map(|view| { + build_row( + view.id as _, + rw_catalog + .get_system_table_by_name("rw_views") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + view.description.as_deref().unwrap_or_default().into(), + ) + }); + let function_desc = schema.iter_function().map(|function| { + build_row( + function.id.function_id() as _, + rw_catalog + .get_system_table_by_name("rw_functions") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + function.description.as_deref().unwrap_or_default().into(), + ) + }); + let connection_desc = schema.iter_connections().map(|connection| { + build_row( + connection.id as _, + rw_catalog + .get_system_table_by_name("rw_connetions") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + connection.description.as_deref().unwrap_or_default().into(), + ) + }); + + table_desc + .chain(mv_desc) + .chain(index_desc) + .chain(source_desc) + .chain(sink_desc) + .chain(view_desc) + .chain(function_desc) + .chain(connection_desc) + }) + .chain(database_desc) + .collect()) + } +} diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 0fadf44ad40cf..f30ea15ba57c8 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -149,6 +149,8 @@ pub struct TableCatalog { /// Indicate whether to use watermark cache for state table. pub cleaned_by_watermark: bool, + + pub description: Option, } #[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)] @@ -403,6 +405,7 @@ impl TableCatalog { cleaned_by_watermark: self.cleaned_by_watermark, stream_job_status: PbStreamJobStatus::Creating.into(), create_type: PbCreateType::Foreground.into(), + description: self.description.clone(), } } @@ -514,6 +517,7 @@ impl From for TableCatalog { created_at_epoch: tb.created_at_epoch.map(Epoch::from), initialized_at_epoch: tb.initialized_at_epoch.map(Epoch::from), cleaned_by_watermark: matches!(tb.cleaned_by_watermark, true), + description: tb.description, } } } @@ -606,6 +610,7 @@ mod tests { cleaned_by_watermark: false, stream_job_status: PbStreamJobStatus::Creating.into(), create_type: PbCreateType::Foreground.into(), + description: Some("description".to_string()), } .into(); @@ -631,6 +636,7 @@ mod tests { ColumnDesc::new_atomic(DataType::Varchar, "zipcode", 3), ], type_name: ".test.Country".to_string(), + description: None, generated_or_default_column: None, }, is_hidden: false @@ -660,6 +666,7 @@ mod tests { created_at_epoch: None, initialized_at_epoch: None, cleaned_by_watermark: false, + description: Some("description".to_string()) } ); assert_eq!(table, TableCatalog::from(table.to_prost(0, 0))); diff --git a/src/frontend/src/catalog/view_catalog.rs b/src/frontend/src/catalog/view_catalog.rs index b06d736d6d5a8..1ceb62d12b4b0 100644 --- a/src/frontend/src/catalog/view_catalog.rs +++ b/src/frontend/src/catalog/view_catalog.rs @@ -28,6 +28,8 @@ pub struct ViewCatalog { pub properties: WithOptions, pub sql: String, pub columns: Vec, + + pub description: Option, } impl From<&PbView> for ViewCatalog { @@ -39,6 +41,7 @@ impl From<&PbView> for ViewCatalog { properties: WithOptions::new(view.properties.clone()), sql: view.sql.clone(), columns: view.columns.iter().map(|f| f.into()).collect(), + description: view.description.clone(), } } } diff --git a/src/frontend/src/expr/user_defined_function.rs b/src/frontend/src/expr/user_defined_function.rs index 1c9d06320ba15..e725cfe009ad3 100644 --- a/src/frontend/src/expr/user_defined_function.rs +++ b/src/frontend/src/expr/user_defined_function.rs @@ -56,6 +56,7 @@ impl UserDefinedFunction { language: udf.get_language().clone(), identifier: udf.get_identifier().clone(), link: udf.get_link().clone(), + description: None, }; Ok(Self { diff --git a/src/frontend/src/handler/comment.rs b/src/frontend/src/handler/comment.rs new file mode 100644 index 0000000000000..47164d433ad46 --- /dev/null +++ b/src/frontend/src/handler/comment.rs @@ -0,0 +1,55 @@ +use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::error::{ErrorCode, Result}; +use risingwave_sqlparser::ast::{CommentObject, ObjectName}; + +use super::{HandlerArgs, RwPgResponse}; +use crate::Binder; + +pub async fn handle_comment( + handler_args: HandlerArgs, + object_type: CommentObject, + object_name: ObjectName, + comment: Option, +) -> Result { + let session = handler_args.session; + + let (table_id, column_index) = { + let mut binder = Binder::new_for_ddl(&session); + match object_type { + CommentObject::Column => { + // TODO: wait to ask: How to bind `t.col` + let [.., tab, col] = object_name.0.as_slice() else { + return Err(ErrorCode::BindError(format!( + "Invalid column: {}", + object_name.real_value() + )) + .into()); + }; + + let table = binder.bind_table(None, &tab.real_value(), None)?; + binder.bind_columns_to_context(col.real_value(), table.table_catalog.columns)?; + + let column = binder.bind_column(object_name.0.as_slice())?; + + ( + table.table_id, + column + .as_input_ref() + .map(|input_ref| input_ref.index + 1) // +1 since `_row_id` + .unwrap_or_default(), + ) + } + CommentObject::Table => { + let table = binder.bind_table(None, &object_name.real_value(), None)?; + (table.table_id, 0) + } + } + }; + + let catalog_writer = session.catalog_writer()?; + catalog_writer + .create_comment(table_id, column_index as _, comment) + .await?; + + Ok(PgResponse::empty_result(StatementType::COMMENT)) +} diff --git a/src/frontend/src/handler/create_function.rs b/src/frontend/src/handler/create_function.rs index 9d9db08204e49..a571498f2f4de 100644 --- a/src/frontend/src/handler/create_function.rs +++ b/src/frontend/src/handler/create_function.rs @@ -171,6 +171,7 @@ pub async fn handle_create_function( identifier, link, owner: session.user_id(), + description: None, }; let catalog_writer = session.catalog_writer()?; diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index a5a002d3b3d79..193f167de4e5b 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -243,6 +243,7 @@ pub(crate) fn gen_create_index_plan( initialized_at_epoch: None, created_at_epoch: None, stream_job_status: PbStreamJobStatus::Creating.into(), + description: None, }; let plan: PlanRef = materialize.into(); diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 7479348c4b80f..18547be0c3b06 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -647,6 +647,7 @@ pub(crate) async fn try_bind_columns_from_source( name: "_id".to_string(), field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, }, is_hidden: false, @@ -658,6 +659,7 @@ pub(crate) async fn try_bind_columns_from_source( name: "payload".to_string(), field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, }, is_hidden: false, @@ -792,6 +794,7 @@ fn check_and_add_timestamp_column( name: KAFKA_TIMESTAMP_COLUMN_NAME.to_string(), field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, }, @@ -809,6 +812,7 @@ fn add_upsert_default_key_column(columns: &mut Vec) { name: DEFAULT_KEY_COLUMN_NAME.to_string(), field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, }, is_hidden: true, @@ -1152,6 +1156,7 @@ pub async fn handle_create_source( initialized_at_epoch: None, created_at_epoch: None, optional_associated_table_id: None, + description: None, version: INITIAL_SOURCE_VERSION_ID, }; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 19d9a2f25c4b8..00e98e98c63ab 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -187,6 +187,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> name: name.real_value(), field_descs, type_name: "".to_string(), + description: None, generated_or_default_column: None, }, is_hidden: false, @@ -685,6 +686,7 @@ fn gen_table_plan_inner( optional_associated_table_id: Some(OptionalAssociatedTableId::AssociatedTableId( TableId::placeholder().table_id, )), + description: None, version: INITIAL_SOURCE_VERSION_ID, }); diff --git a/src/frontend/src/handler/create_view.rs b/src/frontend/src/handler/create_view.rs index 004c9a116f91a..9e6df885f4656 100644 --- a/src/frontend/src/handler/create_view.rs +++ b/src/frontend/src/handler/create_view.rs @@ -103,6 +103,7 @@ pub async fn handle_create_view( .collect_vec(), sql: format!("{}", query), columns: columns.into_iter().map(|f| f.to_prost()).collect(), + description: None, }; let catalog_writer = session.catalog_writer()?; diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 149f39bead330..ac0e36dec4b4d 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -39,6 +39,7 @@ mod alter_system; mod alter_table_column; pub mod alter_user; pub mod cancel_job; +mod comment; pub mod create_connection; mod create_database; pub mod create_function; @@ -523,6 +524,11 @@ pub async fn handle( session, } => transaction::handle_set(handler_args, modes, snapshot, session).await, Statement::CancelJobs(jobs) => handle_cancel(handler_args, jobs).await, + Statement::Comment { + object_type, + object_name, + comment, + } => comment::handle_comment(handler_args, object_type, object_name, comment).await, _ => Err( ErrorCode::NotImplemented(format!("Unhandled statement: {}", stmt), None.into()).into(), ), diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 128f1eeb24ed9..3758008cc30bb 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -232,6 +232,7 @@ impl StreamMaterialize { created_at_epoch: None, initialized_at_epoch: None, cleaned_by_watermark: false, + description: None, }) } diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index a51380d630331..59e501b54338a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -183,6 +183,7 @@ impl StreamSink { properties: properties.into_inner(), sink_type, format_desc, + description: None, }; Ok((input, sink_desc)) } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 475c5c0e32eb1..961e90a1e646e 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -177,6 +177,7 @@ impl TableCatalogBuilder { created_at_epoch: None, initialized_at_epoch: None, cleaned_by_watermark: false, + description: None, } } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 20eb252fc5053..1567154a2d902 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -204,6 +204,7 @@ impl CatalogWriter for MockCatalogWriter { name: db_name.to_string(), id: database_id, owner, + description: None, }); self.create_schema(database_id, DEFAULT_SCHEMA_NAME, owner) .await?; @@ -226,6 +227,7 @@ impl CatalogWriter for MockCatalogWriter { name: schema_name.to_string(), database_id: db_id, owner, + description: None, }); self.add_schema_id(id, db_id); Ok(()) @@ -318,6 +320,15 @@ impl CatalogWriter for MockCatalogWriter { unreachable!() } + async fn create_comment( + &self, + _table_id: TableId, + _column_index: u32, + _comment: Option, + ) -> Result<()> { + unreachable!() + } + async fn drop_table( &self, source_id: Option, @@ -497,24 +508,28 @@ impl MockCatalogWriter { id: 0, name: DEFAULT_DATABASE_NAME.to_string(), owner: DEFAULT_SUPER_USER_ID, + description: None, }); catalog.write().create_schema(&PbSchema { id: 1, name: DEFAULT_SCHEMA_NAME.to_string(), database_id: 0, owner: DEFAULT_SUPER_USER_ID, + description: None, }); catalog.write().create_schema(&PbSchema { id: 2, name: PG_CATALOG_SCHEMA_NAME.to_string(), database_id: 0, owner: DEFAULT_SUPER_USER_ID, + description: None, }); catalog.write().create_schema(&PbSchema { id: 3, name: RW_CATALOG_SCHEMA_NAME.to_string(), database_id: 0, owner: DEFAULT_SUPER_USER_ID, + description: None, }); let mut map: HashMap = HashMap::new(); map.insert(1_u32, 0_u32); diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 74f01497cc048..102efe87d79fd 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -63,6 +63,7 @@ impl From> for PbDatabase { id: value.0.database_id as _, name: value.0.name, owner: value.1.owner_id as _, + description: None, } } } @@ -93,6 +94,7 @@ impl From> for PbSchema { name: value.0.name, database_id: value.0.database_id as _, owner: value.1.owner_id as _, + description: None, } } } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 1c8f0c2f397c0..569316e32510d 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -34,7 +34,8 @@ use risingwave_common::catalog::{ use risingwave_common::{bail, ensure}; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ - Connection, Database, Function, Index, PbStreamJobStatus, Schema, Sink, Source, Table, View, + Comment, Connection, Database, Function, Index, PbStreamJobStatus, Schema, Sink, Source, Table, + View, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, Object}; @@ -217,6 +218,7 @@ impl CatalogManager { database_id: database.id, name: schema_name.to_string(), owner: database.owner, + description: None, }; schemas.insert(schema.id, schema.clone()); schemas_added.push(schema); @@ -2192,6 +2194,38 @@ impl CatalogManager { Ok(()) } + pub async fn create_comment(&self, comment: Comment) -> MetaResult { + let core = &mut *self.core.lock().await; + let database_core = &mut core.database; + database_core.ensure_table_view_or_source_id(&comment.table_id)?; + + let mut tables = BTreeMapTransaction::new(&mut database_core.tables); + + // TODO: dont unwrap + let mut table = tables.get_mut(comment.table_id).unwrap(); + let col_idx = comment.column_index; + if col_idx > 0 { + let column = table + .columns + .iter_mut() + .find(|col| col.column_desc.clone().unwrap().column_id == col_idx as i32) + .unwrap(); + column.column_desc.as_mut().unwrap().description = comment.description; + } else { + table.description = comment.description; + } + + let new_table = table.clone(); + + commit_meta!(self, tables)?; + + let version = self + .notify_frontend_relation_info(Operation::Update, RelationInfo::Table(new_table)) + .await; + + Ok(version) + } + pub async fn list_connections(&self) -> Vec { self.core.lock().await.database.list_connections() } diff --git a/src/meta/src/model/catalog.rs b/src/meta/src/model/catalog.rs index add886ae862da..c18c3844da96a 100644 --- a/src/meta/src/model/catalog.rs +++ b/src/meta/src/model/catalog.rs @@ -84,6 +84,7 @@ mod tests { id, name: format!("database_{}", id), owner: risingwave_common::catalog::DEFAULT_SUPER_USER_ID, + description: None, } } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 3ad5df55a997c..c2bfb7f0fe794 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -23,7 +23,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::epoch::Epoch; use risingwave_pb::catalog::connection::private_link_service::PbPrivateLinkProvider; use risingwave_pb::catalog::{ - connection, Connection, CreateType, Database, Function, Schema, Source, Table, View, + connection, Comment, Connection, CreateType, Database, Function, Schema, Source, Table, View, }; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::DdlProgress; @@ -100,6 +100,7 @@ pub enum DdlCommand { AlterSourceColumn(Source), CreateConnection(Connection), DropConnection(ConnectionId), + CreateComment(Comment), } #[derive(Clone)] @@ -257,6 +258,7 @@ impl DdlController { ctrl.drop_connection(connection_id).await } DdlCommand::AlterSourceColumn(source) => ctrl.alter_source_column(source).await, + DdlCommand::CreateComment(comment) => ctrl.create_comment(comment).await, } } .in_current_span(); @@ -1047,4 +1049,8 @@ impl DdlController { } } } + + async fn create_comment(&self, comment: Comment) -> MetaResult { + self.catalog_manager.create_comment(comment).await + } } diff --git a/src/meta/src/rpc/service/ddl_service.rs b/src/meta/src/rpc/service/ddl_service.rs index 935d398aeacb0..b355c9f983a75 100644 --- a/src/meta/src/rpc/service/ddl_service.rs +++ b/src/meta/src/rpc/service/ddl_service.rs @@ -25,7 +25,7 @@ use risingwave_pb::catalog::connection::private_link_service::{ use risingwave_pb::catalog::connection::PbPrivateLinkService; use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; -use risingwave_pb::catalog::{connection, Connection, CreateType, PbSource, PbTable}; +use risingwave_pb::catalog::{connection, Comment, Connection, CreateType, PbSource, PbTable}; use risingwave_pb::ddl_service::ddl_service_server::DdlService; use risingwave_pb::ddl_service::drop_table_request::PbSourceId; use risingwave_pb::ddl_service::*; @@ -674,6 +674,7 @@ impl DdlService for DdlServiceImpl { name: req.name, owner: req.owner_id, info: Some(connection::Info::PrivateLinkService(private_link_svc)), + description: None, }; // save private link info to catalog @@ -717,6 +718,27 @@ impl DdlService for DdlServiceImpl { })) } + async fn create_comment( + &self, + request: Request, + ) -> Result, Status> { + let req = request.into_inner(); + + let version = self + .ddl_controller + .run_command(DdlCommand::CreateComment(Comment { + table_id: req.table_id, + column_index: req.column_index, + description: req.comment, + })) + .await?; + + Ok(Response::new(CreateCommentResponse { + status: None, + version, + })) + } + #[cfg_attr(coverage, no_coverage)] async fn get_tables( &self, diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 74a80f8e9f3e6..eb9b37ca4fc3a 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -406,6 +406,21 @@ impl MetaClient { Ok((resp.table_id.into(), resp.version)) } + pub async fn create_comment( + &self, + table_id: u32, + column_index: u32, + comment: Option, + ) -> Result { + let request = CreateCommentRequest { + table_id, + column_index, + comment, + }; + let resp = self.inner.create_comment(request).await?; + Ok(resp.version) + } + pub async fn alter_relation_name( &self, relation: Relation, @@ -1700,6 +1715,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, create_connection, CreateConnectionRequest, CreateConnectionResponse } ,{ ddl_client, list_connections, ListConnectionsRequest, ListConnectionsResponse } ,{ ddl_client, drop_connection, DropConnectionRequest, DropConnectionResponse } + ,{ ddl_client, create_comment, CreateCommentRequest, CreateCommentResponse } ,{ ddl_client, get_tables, GetTablesRequest, GetTablesResponse } ,{ hummock_client, unpin_version_before, UnpinVersionBeforeRequest, UnpinVersionBeforeResponse } ,{ hummock_client, get_current_version, GetCurrentVersionRequest, GetCurrentVersionResponse } diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 4d4b9f9cb5b80..b9a71c9a1e645 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -249,6 +249,7 @@ pub mod test_utils { name: f.name.clone(), field_descs: vec![], type_name: "".to_string(), + description: None, generated_or_default_column: None, } .to_protobuf(), diff --git a/src/storage/src/filter_key_extractor.rs b/src/storage/src/filter_key_extractor.rs index b5a79a6f6b42f..6538042566537 100644 --- a/src/storage/src/filter_key_extractor.rs +++ b/src/storage/src/filter_key_extractor.rs @@ -551,6 +551,7 @@ mod tests { cleaned_by_watermark: false, stream_job_status: PbStreamJobStatus::Created.into(), create_type: PbCreateType::Foreground.into(), + description: None, } } diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 5963d7b33cdea..3b090cdf82f4e 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -154,6 +154,7 @@ async fn compaction_test( cleaned_by_watermark: false, stream_job_status: PbStreamJobStatus::Created.into(), create_type: PbCreateType::Foreground.into(), + description: None, }; let mut delete_range_table = delete_key_table.clone(); delete_range_table.id = 2; diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 29ea77f83b71b..b01fe3c99c071 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -53,6 +53,7 @@ pub enum StatementType { CREATE_INDEX, CREATE_FUNCTION, CREATE_CONNECTION, + COMMENT, DESCRIBE, GRANT_PRIVILEGE, DROP_TABLE, From 1a217e3ea91b2d3fe72afcf8b4ce7af85d154ddf Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Sun, 15 Oct 2023 18:08:08 +0800 Subject: [PATCH 10/36] fix: add license --- .../system_catalog/rw_catalog/rw_description.rs | 14 ++++++++++++++ src/frontend/src/handler/comment.rs | 14 ++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index 3b6e52852bacc..2303ab68dfae0 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -1,3 +1,17 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + use std::iter; use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; diff --git a/src/frontend/src/handler/comment.rs b/src/frontend/src/handler/comment.rs index 47164d433ad46..8c47b43069512 100644 --- a/src/frontend/src/handler/comment.rs +++ b/src/frontend/src/handler/comment.rs @@ -1,3 +1,17 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_sqlparser::ast::{CommentObject, ObjectName}; From 1e643881089f82bae145b4afc122f7c874b11f01 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Sun, 15 Oct 2023 18:20:34 +0800 Subject: [PATCH 11/36] fix(proto): revert tag order to avoid breaking change --- proto/catalog.proto | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index 63f52e214d155..4fd2c21cbfb73 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -199,16 +199,17 @@ message Function { string language = 7; string link = 8; string identifier = 10; - optional string description = 11; oneof kind { - ScalarFunction scalar = 12; - TableFunction table = 13; - AggregateFunction aggregate = 14; + ScalarFunction scalar = 11; + TableFunction table = 12; + AggregateFunction aggregate = 13; } message ScalarFunction {} message TableFunction {} message AggregateFunction {} + + optional string description = 14; } // See `TableCatalog` struct in frontend crate for more information. From fc7e404aca1cb2738423003fa3d232ed69e22f0b Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Wed, 18 Oct 2023 17:45:59 +0800 Subject: [PATCH 12/36] refactor: only support comment on `table` and `col` reduce the changes impact radius. --- proto/catalog.proto | 11 +- proto/ddl_service.proto | 8 +- proto/stream_plan.proto | 2 - src/connector/src/sink/catalog/desc.rs | 4 - src/connector/src/sink/catalog/mod.rs | 4 - src/frontend/src/catalog/catalog_service.rs | 12 +-- .../src/catalog/connection_catalog.rs | 2 - src/frontend/src/catalog/database_catalog.rs | 7 -- src/frontend/src/catalog/function_catalog.rs | 2 - src/frontend/src/catalog/index_catalog.rs | 4 - src/frontend/src/catalog/source_catalog.rs | 3 - .../src/catalog/system_catalog/mod.rs | 1 - .../rw_catalog/rw_description.rs | 100 +----------------- src/frontend/src/catalog/view_catalog.rs | 3 - .../src/expr/user_defined_function.rs | 1 - src/frontend/src/handler/comment.rs | 10 +- src/frontend/src/handler/create_function.rs | 1 - src/frontend/src/handler/create_index.rs | 1 - src/frontend/src/handler/create_source.rs | 1 - src/frontend/src/handler/create_table.rs | 1 - src/frontend/src/handler/create_view.rs | 1 - .../src/optimizer/plan_node/stream_sink.rs | 1 - src/frontend/src/test_utils.rs | 10 +- src/meta/src/controller/mod.rs | 2 - src/meta/src/manager/catalog/mod.rs | 12 +-- src/meta/src/model/catalog.rs | 1 - src/meta/src/rpc/ddl_controller.rs | 8 +- src/meta/src/rpc/service/ddl_service.rs | 11 +- src/rpc_client/src/meta_client.rs | 10 +- 29 files changed, 36 insertions(+), 198 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index 4fd2c21cbfb73..7023d553f2d01 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -94,7 +94,6 @@ message Source { optional uint64 initialized_at_epoch = 15; optional uint64 created_at_epoch = 16; - optional string description = 17; // Per-source catalog version, used by schema change. uint64 version = 100; @@ -141,7 +140,6 @@ message Sink { string sink_from_name = 18; StreamJobStatus stream_job_status = 19; SinkFormatDesc format_desc = 20; - optional string description = 21; } message Connection { @@ -166,7 +164,6 @@ message Connection { PrivateLinkService private_link_service = 5; } uint32 owner = 6; - optional string description = 7; } message Index { @@ -185,7 +182,6 @@ message Index { optional uint64 initialized_at_epoch = 10; optional uint64 created_at_epoch = 11; StreamJobStatus stream_job_status = 12; - optional string description = 13; } message Function { @@ -208,8 +204,6 @@ message Function { message ScalarFunction {} message TableFunction {} message AggregateFunction {} - - optional string description = 14; } // See `TableCatalog` struct in frontend crate for more information. @@ -311,7 +305,6 @@ message View { repeated uint32 dependent_relations = 8; // User-specified column names. repeated plan_common.Field columns = 9; - optional string description = 10; } message Schema { @@ -319,18 +312,16 @@ message Schema { uint32 database_id = 2; string name = 3; uint32 owner = 4; - optional string description = 5; } message Database { uint32 id = 1; string name = 2; uint32 owner = 3; - optional string description = 4; } message Comment { uint32 table_id = 1; - uint32 column_index = 2; + optional uint32 column_index = 2; optional string description = 3; } diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 29bd3e23260f8..05792623f408a 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -314,13 +314,13 @@ message GetTablesResponse { map tables = 1; } -message CreateCommentRequest { +message CommentOnRequest { uint32 table_id = 1; - uint32 column_index = 2; + optional uint32 column_index = 2; optional string comment = 3; } -message CreateCommentResponse { +message CommentOnResponse { common.Status status = 1; uint64 version = 2; } @@ -353,6 +353,6 @@ service DdlService { rpc CreateConnection(CreateConnectionRequest) returns (CreateConnectionResponse); rpc ListConnections(ListConnectionsRequest) returns (ListConnectionsResponse); rpc DropConnection(DropConnectionRequest) returns (DropConnectionResponse); - rpc CreateComment(CreateCommentRequest) returns (CreateCommentResponse); + rpc CommentOn(CommentOnRequest) returns (CommentOnResponse); rpc GetTables(GetTablesRequest) returns (GetTablesResponse); } diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index e33e674df7c04..a1026e0d36c1f 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -181,8 +181,6 @@ message SinkDesc { // it is the name of the sink itself. string sink_from_name = 12; catalog.SinkFormatDesc format_desc = 13; - - optional string description = 14; } enum SinkLogStoreType { diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index 6d1031ef6f7af..1fdd8b44959e3 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -64,8 +64,6 @@ pub struct SinkDesc { /// Name of the "table" field for Debezium. If the sink is from table or mv, /// it is the name of table/mv. Otherwise, it is the name of the sink. pub sink_from_name: String, - - pub description: Option, } impl SinkDesc { @@ -97,7 +95,6 @@ impl SinkDesc { initialized_at_epoch: None, db_name: self.db_name, sink_from_name: self.sink_from_name, - description: None, } } @@ -119,7 +116,6 @@ impl SinkDesc { format_desc: self.format_desc.as_ref().map(|f| f.to_proto()), db_name: self.db_name.clone(), sink_from_name: self.sink_from_name.clone(), - description: self.description.clone(), } } } diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index f81be598756a8..c18dd7d10a92c 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -293,8 +293,6 @@ pub struct SinkCatalog { /// Name for the table info for Debezium sink pub sink_from_name: String, - - pub description: Option, } impl SinkCatalog { @@ -332,7 +330,6 @@ impl SinkCatalog { db_name: self.db_name.clone(), sink_from_name: self.sink_from_name.clone(), stream_job_status: PbStreamJobStatus::Creating.into(), - description: self.description.clone(), } } @@ -421,7 +418,6 @@ impl From for SinkCatalog { initialized_at_epoch: pb.initialized_at_epoch.map(Epoch::from), db_name: pb.db_name, sink_from_name: pb.sink_from_name, - description: pb.description.clone(), } } } diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index 974824fe895ac..c1be43f260710 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -111,10 +111,10 @@ pub trait CatalogWriter: Send + Sync { connection: create_connection_request::Payload, ) -> Result<()>; - async fn create_comment( + async fn comment_on( &self, table_id: TableId, - column_index: u32, + column_index: Option, comment: Option, ) -> Result<()>; @@ -169,7 +169,6 @@ impl CatalogWriter for CatalogWriterImpl { name: db_name.to_string(), id: 0, owner, - description: None, }) .await?; self.wait_version(version).await @@ -188,7 +187,6 @@ impl CatalogWriter for CatalogWriterImpl { name: schema_name.to_string(), database_id: db_id, owner, - description: None, }) .await?; self.wait_version(version).await @@ -291,15 +289,15 @@ impl CatalogWriter for CatalogWriterImpl { self.wait_version(version).await } - async fn create_comment( + async fn comment_on( &self, TableId { table_id }: TableId, - column_index: u32, + column_index: Option, comment: Option, ) -> Result<()> { let version = self .meta_client - .create_comment(table_id, column_index, comment) + .comment_on(table_id, column_index, comment) .await?; self.wait_version(version).await } diff --git a/src/frontend/src/catalog/connection_catalog.rs b/src/frontend/src/catalog/connection_catalog.rs index 6b9cf7906e10a..7913d04379cd5 100644 --- a/src/frontend/src/catalog/connection_catalog.rs +++ b/src/frontend/src/catalog/connection_catalog.rs @@ -32,7 +32,6 @@ pub struct ConnectionCatalog { pub name: String, pub info: connection::Info, pub owner: UserId, - pub description: Option, } impl ConnectionCatalog { @@ -56,7 +55,6 @@ impl From<&PbConnection> for ConnectionCatalog { name: prost.name.clone(), info: prost.info.clone().unwrap(), owner: prost.owner, - description: prost.description.clone(), } } } diff --git a/src/frontend/src/catalog/database_catalog.rs b/src/frontend/src/catalog/database_catalog.rs index 81c510a6db8bd..76cfc0cd359c1 100644 --- a/src/frontend/src/catalog/database_catalog.rs +++ b/src/frontend/src/catalog/database_catalog.rs @@ -28,7 +28,6 @@ pub struct DatabaseCatalog { schema_by_name: HashMap, schema_name_by_id: HashMap, owner: u32, - description: Option, } impl DatabaseCatalog { @@ -66,7 +65,6 @@ impl DatabaseCatalog { database_id: self.id, name: schema.name(), owner: schema.owner(), - description: None, }) .collect_vec() } @@ -110,10 +108,6 @@ impl DatabaseCatalog { pub fn owner(&self) -> u32 { self.owner } - - pub fn description(&self) -> Option<&str> { - self.description.as_deref() - } } impl From<&PbDatabase> for DatabaseCatalog { fn from(db: &PbDatabase) -> Self { @@ -123,7 +117,6 @@ impl From<&PbDatabase> for DatabaseCatalog { schema_by_name: HashMap::new(), schema_name_by_id: HashMap::new(), owner: db.owner, - description: db.description.clone(), } } } diff --git a/src/frontend/src/catalog/function_catalog.rs b/src/frontend/src/catalog/function_catalog.rs index 5c5f54131b959..e56a36d85f8c2 100644 --- a/src/frontend/src/catalog/function_catalog.rs +++ b/src/frontend/src/catalog/function_catalog.rs @@ -31,7 +31,6 @@ pub struct FunctionCatalog { pub language: String, pub identifier: String, pub link: String, - pub description: Option, } #[derive(Clone, Display, PartialEq, Eq, Hash, Debug)] @@ -65,7 +64,6 @@ impl From<&PbFunction> for FunctionCatalog { language: prost.language.clone(), identifier: prost.identifier.clone(), link: prost.link.clone(), - description: prost.description.clone(), } } } diff --git a/src/frontend/src/catalog/index_catalog.rs b/src/frontend/src/catalog/index_catalog.rs index ed1bf2cf753da..ca4b4036332d3 100644 --- a/src/frontend/src/catalog/index_catalog.rs +++ b/src/frontend/src/catalog/index_catalog.rs @@ -63,8 +63,6 @@ pub struct IndexCatalog { pub created_at_epoch: Option, pub initialized_at_epoch: Option, - - pub description: Option, } impl IndexCatalog { @@ -126,7 +124,6 @@ impl IndexCatalog { original_columns, created_at_epoch: index_prost.created_at_epoch.map(Epoch::from), initialized_at_epoch: index_prost.initialized_at_epoch.map(Epoch::from), - description: index_prost.description.clone(), } } @@ -188,7 +185,6 @@ impl IndexCatalog { initialized_at_epoch: self.initialized_at_epoch.map(|e| e.0), created_at_epoch: self.created_at_epoch.map(|e| e.0), stream_job_status: PbStreamJobStatus::Creating.into(), - description: self.description.clone(), } } diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index 77eec6e01e339..ec35cfb7bde28 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -43,7 +43,6 @@ pub struct SourceCatalog { pub connection_id: Option, pub created_at_epoch: Option, pub initialized_at_epoch: Option, - pub description: Option, pub version: SourceVersionId, } @@ -73,7 +72,6 @@ impl SourceCatalog { optional_associated_table_id: self .associated_table_id .map(|id| OptionalAssociatedTableId::AssociatedTableId(id.table_id)), - description: self.description.clone(), version: self.version, } } @@ -129,7 +127,6 @@ impl From<&PbSource> for SourceCatalog { connection_id, created_at_epoch: prost.created_at_epoch.map(Epoch::from), initialized_at_epoch: prost.initialized_at_epoch.map(Epoch::from), - description: prost.description.clone(), version, } } diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 0d68fda380da3..334f7e7f5cc7b 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -185,7 +185,6 @@ impl From<&BuiltinView> for ViewCatalog { sql: val.sql.to_string(), owner: DEFAULT_SUPER_USER_ID, properties: Default::default(), - description: None, } } } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index 2303ab68dfae0..7d8bb7c6864dc 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -39,7 +39,7 @@ impl SysCatalogReaderImpl { OwnedRow::new(vec![ Some(ScalarImpl::Int32(table_id)), Some(ScalarImpl::Int32(catalog_id)), - Some(ScalarImpl::Int32(0)), + None, Some(ScalarImpl::Utf8(description)), ]) }; @@ -56,21 +56,9 @@ impl SysCatalogReaderImpl { let schemas = reader.iter_schemas_except_rw_catalog(&self.auth_context.database)?; let rw_catalog = reader.get_schema_by_name(&self.auth_context.database, "rw_catalog")?; - // XXX: is it shared object ?? - let database_desc = reader.iter_databases().map(|db| { - build_row( - db.id() as _, - rw_catalog - .get_system_table_by_name("rw_databases") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, - db.description().unwrap_or_default().into(), - ) - }); - Ok(schemas .flat_map(|schema| { - let table_desc = schema.iter_table().flat_map(|table| { + schema.iter_table().flat_map(|table| { iter::once(build_row( table.id.table_id as _, rw_catalog @@ -83,7 +71,7 @@ impl SysCatalogReaderImpl { table .columns .iter() - .filter(|col| !col.is_hidden()) + // .filter(|col| !col.is_hidden()) .map(|col| { build_row_with_sub( table.id.table_id as _, @@ -101,88 +89,8 @@ impl SysCatalogReaderImpl { ) }), ) - }); - let mv_desc = schema.iter_mv().map(|mv| { - build_row( - mv.id.table_id as _, - rw_catalog - .get_system_table_by_name("rw_materialized_views") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, - mv.description.as_deref().unwrap_or_default().into(), - ) - }); - let index_desc = schema.iter_index().map(|index| { - build_row( - index.id.index_id as _, - rw_catalog - .get_system_table_by_name("rw_indexes") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, - index.description.as_deref().unwrap_or_default().into(), - ) - }); - let source_desc = schema.iter_source().map(|source| { - build_row( - source.id as _, - rw_catalog - .get_system_table_by_name("rw_sources") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, - source.description.as_deref().unwrap_or_default().into(), - ) - }); - let sink_desc = schema.iter_sink().map(|sink| { - build_row( - sink.id.sink_id as _, - rw_catalog - .get_system_table_by_name("rw_sinks") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, - sink.description.as_deref().unwrap_or_default().into(), - ) - }); - let view_desc = schema.iter_view().map(|view| { - build_row( - view.id as _, - rw_catalog - .get_system_table_by_name("rw_views") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, - view.description.as_deref().unwrap_or_default().into(), - ) - }); - let function_desc = schema.iter_function().map(|function| { - build_row( - function.id.function_id() as _, - rw_catalog - .get_system_table_by_name("rw_functions") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, - function.description.as_deref().unwrap_or_default().into(), - ) - }); - let connection_desc = schema.iter_connections().map(|connection| { - build_row( - connection.id as _, - rw_catalog - .get_system_table_by_name("rw_connetions") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, - connection.description.as_deref().unwrap_or_default().into(), - ) - }); - - table_desc - .chain(mv_desc) - .chain(index_desc) - .chain(source_desc) - .chain(sink_desc) - .chain(view_desc) - .chain(function_desc) - .chain(connection_desc) + }) }) - .chain(database_desc) .collect()) } } diff --git a/src/frontend/src/catalog/view_catalog.rs b/src/frontend/src/catalog/view_catalog.rs index 1ceb62d12b4b0..b06d736d6d5a8 100644 --- a/src/frontend/src/catalog/view_catalog.rs +++ b/src/frontend/src/catalog/view_catalog.rs @@ -28,8 +28,6 @@ pub struct ViewCatalog { pub properties: WithOptions, pub sql: String, pub columns: Vec, - - pub description: Option, } impl From<&PbView> for ViewCatalog { @@ -41,7 +39,6 @@ impl From<&PbView> for ViewCatalog { properties: WithOptions::new(view.properties.clone()), sql: view.sql.clone(), columns: view.columns.iter().map(|f| f.into()).collect(), - description: view.description.clone(), } } } diff --git a/src/frontend/src/expr/user_defined_function.rs b/src/frontend/src/expr/user_defined_function.rs index e725cfe009ad3..1c9d06320ba15 100644 --- a/src/frontend/src/expr/user_defined_function.rs +++ b/src/frontend/src/expr/user_defined_function.rs @@ -56,7 +56,6 @@ impl UserDefinedFunction { language: udf.get_language().clone(), identifier: udf.get_identifier().clone(), link: udf.get_link().clone(), - description: None, }; Ok(Self { diff --git a/src/frontend/src/handler/comment.rs b/src/frontend/src/handler/comment.rs index 8c47b43069512..95716140e71d7 100644 --- a/src/frontend/src/handler/comment.rs +++ b/src/frontend/src/handler/comment.rs @@ -31,7 +31,6 @@ pub async fn handle_comment( let mut binder = Binder::new_for_ddl(&session); match object_type { CommentObject::Column => { - // TODO: wait to ask: How to bind `t.col` let [.., tab, col] = object_name.0.as_slice() else { return Err(ErrorCode::BindError(format!( "Invalid column: {}", @@ -47,22 +46,19 @@ pub async fn handle_comment( ( table.table_id, - column - .as_input_ref() - .map(|input_ref| input_ref.index + 1) // +1 since `_row_id` - .unwrap_or_default(), + column.as_input_ref().map(|input_ref| input_ref.index as _), ) } CommentObject::Table => { let table = binder.bind_table(None, &object_name.real_value(), None)?; - (table.table_id, 0) + (table.table_id, None) } } }; let catalog_writer = session.catalog_writer()?; catalog_writer - .create_comment(table_id, column_index as _, comment) + .comment_on(table_id, column_index, comment) .await?; Ok(PgResponse::empty_result(StatementType::COMMENT)) diff --git a/src/frontend/src/handler/create_function.rs b/src/frontend/src/handler/create_function.rs index a571498f2f4de..9d9db08204e49 100644 --- a/src/frontend/src/handler/create_function.rs +++ b/src/frontend/src/handler/create_function.rs @@ -171,7 +171,6 @@ pub async fn handle_create_function( identifier, link, owner: session.user_id(), - description: None, }; let catalog_writer = session.catalog_writer()?; diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 193f167de4e5b..a5a002d3b3d79 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -243,7 +243,6 @@ pub(crate) fn gen_create_index_plan( initialized_at_epoch: None, created_at_epoch: None, stream_job_status: PbStreamJobStatus::Creating.into(), - description: None, }; let plan: PlanRef = materialize.into(); diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 18547be0c3b06..29c38993a8f50 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1156,7 +1156,6 @@ pub async fn handle_create_source( initialized_at_epoch: None, created_at_epoch: None, optional_associated_table_id: None, - description: None, version: INITIAL_SOURCE_VERSION_ID, }; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 00e98e98c63ab..6168c2500ce5b 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -686,7 +686,6 @@ fn gen_table_plan_inner( optional_associated_table_id: Some(OptionalAssociatedTableId::AssociatedTableId( TableId::placeholder().table_id, )), - description: None, version: INITIAL_SOURCE_VERSION_ID, }); diff --git a/src/frontend/src/handler/create_view.rs b/src/frontend/src/handler/create_view.rs index 9e6df885f4656..004c9a116f91a 100644 --- a/src/frontend/src/handler/create_view.rs +++ b/src/frontend/src/handler/create_view.rs @@ -103,7 +103,6 @@ pub async fn handle_create_view( .collect_vec(), sql: format!("{}", query), columns: columns.into_iter().map(|f| f.to_prost()).collect(), - description: None, }; let catalog_writer = session.catalog_writer()?; diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 59e501b54338a..a51380d630331 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -183,7 +183,6 @@ impl StreamSink { properties: properties.into_inner(), sink_type, format_desc, - description: None, }; Ok((input, sink_desc)) } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 1567154a2d902..71921ddd4c594 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -204,7 +204,6 @@ impl CatalogWriter for MockCatalogWriter { name: db_name.to_string(), id: database_id, owner, - description: None, }); self.create_schema(database_id, DEFAULT_SCHEMA_NAME, owner) .await?; @@ -227,7 +226,6 @@ impl CatalogWriter for MockCatalogWriter { name: schema_name.to_string(), database_id: db_id, owner, - description: None, }); self.add_schema_id(id, db_id); Ok(()) @@ -320,10 +318,10 @@ impl CatalogWriter for MockCatalogWriter { unreachable!() } - async fn create_comment( + async fn comment_on( &self, _table_id: TableId, - _column_index: u32, + _column_index: Option, _comment: Option, ) -> Result<()> { unreachable!() @@ -508,28 +506,24 @@ impl MockCatalogWriter { id: 0, name: DEFAULT_DATABASE_NAME.to_string(), owner: DEFAULT_SUPER_USER_ID, - description: None, }); catalog.write().create_schema(&PbSchema { id: 1, name: DEFAULT_SCHEMA_NAME.to_string(), database_id: 0, owner: DEFAULT_SUPER_USER_ID, - description: None, }); catalog.write().create_schema(&PbSchema { id: 2, name: PG_CATALOG_SCHEMA_NAME.to_string(), database_id: 0, owner: DEFAULT_SUPER_USER_ID, - description: None, }); catalog.write().create_schema(&PbSchema { id: 3, name: RW_CATALOG_SCHEMA_NAME.to_string(), database_id: 0, owner: DEFAULT_SUPER_USER_ID, - description: None, }); let mut map: HashMap = HashMap::new(); map.insert(1_u32, 0_u32); diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 102efe87d79fd..74f01497cc048 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -63,7 +63,6 @@ impl From> for PbDatabase { id: value.0.database_id as _, name: value.0.name, owner: value.1.owner_id as _, - description: None, } } } @@ -94,7 +93,6 @@ impl From> for PbSchema { name: value.0.name, database_id: value.0.database_id as _, owner: value.1.owner_id as _, - description: None, } } } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 569316e32510d..ac84f7e4afa57 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -218,7 +218,6 @@ impl CatalogManager { database_id: database.id, name: schema_name.to_string(), owner: database.owner, - description: None, }; schemas.insert(schema.id, schema.clone()); schemas_added.push(schema); @@ -2194,7 +2193,7 @@ impl CatalogManager { Ok(()) } - pub async fn create_comment(&self, comment: Comment) -> MetaResult { + pub async fn comment_on(&self, comment: Comment) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; database_core.ensure_table_view_or_source_id(&comment.table_id)?; @@ -2203,13 +2202,8 @@ impl CatalogManager { // TODO: dont unwrap let mut table = tables.get_mut(comment.table_id).unwrap(); - let col_idx = comment.column_index; - if col_idx > 0 { - let column = table - .columns - .iter_mut() - .find(|col| col.column_desc.clone().unwrap().column_id == col_idx as i32) - .unwrap(); + if let Some(col_idx) = comment.column_index { + let column = &mut table.columns[col_idx as usize]; column.column_desc.as_mut().unwrap().description = comment.description; } else { table.description = comment.description; diff --git a/src/meta/src/model/catalog.rs b/src/meta/src/model/catalog.rs index c18c3844da96a..add886ae862da 100644 --- a/src/meta/src/model/catalog.rs +++ b/src/meta/src/model/catalog.rs @@ -84,7 +84,6 @@ mod tests { id, name: format!("database_{}", id), owner: risingwave_common::catalog::DEFAULT_SUPER_USER_ID, - description: None, } } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index c2bfb7f0fe794..cad6fc69b0aa8 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -100,7 +100,7 @@ pub enum DdlCommand { AlterSourceColumn(Source), CreateConnection(Connection), DropConnection(ConnectionId), - CreateComment(Comment), + CommentOn(Comment), } #[derive(Clone)] @@ -258,7 +258,7 @@ impl DdlController { ctrl.drop_connection(connection_id).await } DdlCommand::AlterSourceColumn(source) => ctrl.alter_source_column(source).await, - DdlCommand::CreateComment(comment) => ctrl.create_comment(comment).await, + DdlCommand::CommentOn(comment) => ctrl.comment_on(comment).await, } } .in_current_span(); @@ -1050,7 +1050,7 @@ impl DdlController { } } - async fn create_comment(&self, comment: Comment) -> MetaResult { - self.catalog_manager.create_comment(comment).await + async fn comment_on(&self, comment: Comment) -> MetaResult { + self.catalog_manager.comment_on(comment).await } } diff --git a/src/meta/src/rpc/service/ddl_service.rs b/src/meta/src/rpc/service/ddl_service.rs index b355c9f983a75..73e3189a0c65a 100644 --- a/src/meta/src/rpc/service/ddl_service.rs +++ b/src/meta/src/rpc/service/ddl_service.rs @@ -674,7 +674,6 @@ impl DdlService for DdlServiceImpl { name: req.name, owner: req.owner_id, info: Some(connection::Info::PrivateLinkService(private_link_svc)), - description: None, }; // save private link info to catalog @@ -718,22 +717,22 @@ impl DdlService for DdlServiceImpl { })) } - async fn create_comment( + async fn comment_on( &self, - request: Request, - ) -> Result, Status> { + request: Request, + ) -> Result, Status> { let req = request.into_inner(); let version = self .ddl_controller - .run_command(DdlCommand::CreateComment(Comment { + .run_command(DdlCommand::CommentOn(Comment { table_id: req.table_id, column_index: req.column_index, description: req.comment, })) .await?; - Ok(Response::new(CreateCommentResponse { + Ok(Response::new(CommentOnResponse { status: None, version, })) diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index eb9b37ca4fc3a..c5a48c16d22ff 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -406,18 +406,18 @@ impl MetaClient { Ok((resp.table_id.into(), resp.version)) } - pub async fn create_comment( + pub async fn comment_on( &self, table_id: u32, - column_index: u32, + column_index: Option, comment: Option, ) -> Result { - let request = CreateCommentRequest { + let request = CommentOnRequest { table_id, column_index, comment, }; - let resp = self.inner.create_comment(request).await?; + let resp = self.inner.comment_on(request).await?; Ok(resp.version) } @@ -1715,7 +1715,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, create_connection, CreateConnectionRequest, CreateConnectionResponse } ,{ ddl_client, list_connections, ListConnectionsRequest, ListConnectionsResponse } ,{ ddl_client, drop_connection, DropConnectionRequest, DropConnectionResponse } - ,{ ddl_client, create_comment, CreateCommentRequest, CreateCommentResponse } + ,{ ddl_client, comment_on, CommentOnRequest, CommentOnResponse } ,{ ddl_client, get_tables, GetTablesRequest, GetTablesResponse } ,{ hummock_client, unpin_version_before, UnpinVersionBeforeRequest, UnpinVersionBeforeResponse } ,{ hummock_client, get_current_version, GetCurrentVersionRequest, GetCurrentVersionResponse } From 3772fe6c2d074ea4a2737ee800f41b69c23fd6d6 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Thu, 19 Oct 2023 02:04:04 +0800 Subject: [PATCH 13/36] feat: new `description` field in `describe` result only support column description. --- src/frontend/src/handler/describe.rs | 36 ++++++++++++++++++++++------ src/frontend/src/handler/util.rs | 6 ++++- 2 files changed, 34 insertions(+), 8 deletions(-) diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index f2fb89d02a7dc..7e570948788f4 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -100,15 +100,28 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res // Convert primary key to rows if !pk_columns.is_empty() { + let names = format!( + "{}", + display_comma_separated(&pk_columns.iter().map(|x| &x.name).collect_vec()) + ); + let descs = format!( + "{}", + display_comma_separated( + &pk_columns + .iter() + .map(|x| x.description.as_deref().unwrap_or_default()) + .collect_vec() + ) + ); + rows.push(Row::new(vec![ Some("primary key".into()), - Some( - format!( - "{}", - display_comma_separated(&pk_columns.into_iter().map(|x| x.name).collect_vec()), - ) - .into(), - ), + Some(names.into()), + if descs.is_empty() { + None + } else { + Some(descs.into()) + }, ])); } @@ -138,9 +151,13 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res .into(), ) }, + None, + // TODO: index description + // index.description.map(Into::into), ]) })); + // TODO: table name and description as title of response // TODO: recover the original user statement Ok(PgResponse::builder(StatementType::DESCRIBE) .values( @@ -156,6 +173,11 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res DataType::Varchar.to_oid(), DataType::Varchar.type_len(), ), + PgFieldDescriptor::new( + "Description".to_owned(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), ], ) .into()) diff --git a/src/frontend/src/handler/util.rs b/src/frontend/src/handler/util.rs index a5645b38b1c48..62e3b196e2920 100644 --- a/src/frontend/src/handler/util.rs +++ b/src/frontend/src/handler/util.rs @@ -182,7 +182,11 @@ pub fn col_descs_to_rows(columns: Vec) -> Vec { } else { c.data_type.to_string() }; - Row::new(vec![Some(c.name.into()), Some(type_name.into())]) + Row::new(vec![ + Some(c.name.into()), + Some(type_name.into()), + c.description.map(Into::into), + ]) }) .collect_vec() }) From d32503c00a3f3608a1685bb910ee1728de3144bd Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Thu, 19 Oct 2023 02:05:55 +0800 Subject: [PATCH 14/36] test: update `describe` test --- e2e_test/ddl/show.slt | 28 +++++++++++++++++----------- e2e_test/extended_mode/basic.slt | 10 +++++----- 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/e2e_test/ddl/show.slt b/e2e_test/ddl/show.slt index 787aacddb0330..ce798269a6174 100644 --- a/e2e_test/ddl/show.slt +++ b/e2e_test/ddl/show.slt @@ -7,13 +7,19 @@ create materialized view mv3 as select sum(v1) as sum_v1 from t3; statement ok create view v3 as select sum(v2) as sum_v2 from t3; -query TT +statement ok +comment on column t3.v2 is 'column desc'; + +statement ok +comment on column t3._row_id is '_row_id desc'; + +query TTT describe t3; ---- -v1 integer -v2 integer -v3 integer -primary key _row_id +v1 integer NULL +v2 integer column desc +v3 integer NULL +primary key _row_id _row_id desc query TT show columns from t3; @@ -30,14 +36,14 @@ show indexes from t3; ---- idx1 t3 v1 ASC, v2 ASC v3 v1 -query TT +query TTT describe t3; ---- -v1 integer -v2 integer -v3 integer -primary key _row_id -idx1 index(v1 ASC, v2 ASC) include(v3) distributed by(v1) +v1 integer NULL +v2 integer column desc +v3 integer NULL +primary key _row_id _row_id desc +idx1 index(v1 ASC, v2 ASC) include(v3) distributed by(v1) NULL query TT show create index idx1; diff --git a/e2e_test/extended_mode/basic.slt b/e2e_test/extended_mode/basic.slt index 51513a444ec79..5b43a10ac2248 100644 --- a/e2e_test/extended_mode/basic.slt +++ b/e2e_test/extended_mode/basic.slt @@ -39,13 +39,13 @@ values(round(42.4382)); statement ok create table t3 (v1 int, v2 int, v3 int); -query TT +query TTT describe t3; ---- -v1 integer -v2 integer -v3 integer -primary key _row_id +v1 integer NULL +v2 integer NULL +v3 integer NULL +primary key _row_id NULL query TT show columns from t3; From 7999c4c6e4504b3e7f06a53ef43c3e788b1f32d0 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Thu, 19 Oct 2023 11:56:51 +0800 Subject: [PATCH 15/36] chore: add some comment --- .../src/catalog/system_catalog/rw_catalog/rw_description.rs | 3 +++ src/frontend/src/handler/comment.rs | 1 + 2 files changed, 4 insertions(+) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index 7d8bb7c6864dc..9af6c6000e82d 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -25,8 +25,11 @@ pub const RW_DESCRIPTION: BuiltinTable = BuiltinTable { name: "rw_description", schema: RW_CATALOG_SCHEMA_NAME, columns: &[ + // table_id, view_id, function_id, etc. (DataType::Int32, "objoid"), + // rw_tables, rw_views, rw_functions, etc. (DataType::Int32, "classoid"), + // If it is not None, it means column number. In this case, objoid should be table_id. (DataType::Int32, "objsubid"), (DataType::Varchar, "description"), ], diff --git a/src/frontend/src/handler/comment.rs b/src/frontend/src/handler/comment.rs index 95716140e71d7..61b62cd6a6f01 100644 --- a/src/frontend/src/handler/comment.rs +++ b/src/frontend/src/handler/comment.rs @@ -29,6 +29,7 @@ pub async fn handle_comment( let (table_id, column_index) = { let mut binder = Binder::new_for_ddl(&session); + // only `Column` and `Table` object are now supported match object_type { CommentObject::Column => { let [.., tab, col] = object_name.0.as_slice() else { From d6b316a009d73904902e2ad4463944e99f50df9b Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Thu, 19 Oct 2023 12:28:54 +0800 Subject: [PATCH 16/36] refactor: use ok_or_else instead of unwrap --- src/frontend/src/handler/describe.rs | 16 ++++++++++------ src/meta/src/manager/catalog/mod.rs | 25 +++++++++++++++++++++---- 2 files changed, 31 insertions(+), 10 deletions(-) diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 03f957499572c..b8a7e7b22b2a7 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -100,7 +100,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res // Convert all column descs to rows let mut rows = col_descs_to_rows(columns); - fn fmt_col(columns: &Vec, cb: C) -> String + fn fmt_col(columns: &[ColumnDesc], cb: C) -> String where T: Display + ?Sized, C: FnMut(&ColumnDesc) -> &T, @@ -114,7 +114,9 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res // Convert primary key to rows if !pk_columns.is_empty() { let names = fmt_col(&pk_columns, |x| &x.name); - let descs = fmt_col(&pk_columns, |x| x.description.as_deref().unwrap_or_default()); + let descs = fmt_col(&pk_columns, |x| { + x.description.as_deref().unwrap_or_default() + }); rows.push(Row::new(vec![ Some("primary key".into()), @@ -131,7 +133,9 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res // Convert distribution keys to rows if !dist_columns.is_empty() { let names = fmt_col(&dist_columns, |x| &x.name); - let descs = fmt_col(&dist_columns, |x| x.description.as_deref().unwrap_or_default()); + let descs = fmt_col(&dist_columns, |x| { + x.description.as_deref().unwrap_or_default() + }); rows.push(Row::new(vec![ Some("distribution key".into()), @@ -172,9 +176,9 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res ) }, None, // Is Hidden - None, // Description - // TODO: index description - // index.description.map(Into::into), + None, /* Description + * TODO: index description + * index.description.map(Into::into), */ ]) })); diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 83e1dfbbb9e6f..2ff90929d6b70 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -2201,11 +2201,28 @@ impl CatalogManager { let mut tables = BTreeMapTransaction::new(&mut database_core.tables); - // TODO: dont unwrap - let mut table = tables.get_mut(comment.table_id).unwrap(); + let mut table = tables + .get_mut(comment.table_id) + .ok_or_else(|| anyhow!("table id {} not found", comment.table_id))?; if let Some(col_idx) = comment.column_index { - let column = &mut table.columns[col_idx as usize]; - column.column_desc.as_mut().unwrap().description = comment.description; + let column = table.columns.get_mut(col_idx as usize).ok_or_else(|| { + anyhow!( + "column index {} for table id {} not found", + col_idx, + comment.table_id + ) + })?; + column + .column_desc + .as_mut() + .ok_or_else(|| { + anyhow!( + "column_desc at index {} for table id {} not found", + col_idx, + comment.table_id + ) + })? + .description = comment.description; } else { table.description = comment.description; } From 6a0f9ddb93628bf152e5b5169777c1cd39828d85 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Thu, 19 Oct 2023 17:43:22 +0800 Subject: [PATCH 17/36] feat: leave the `describe` description of pk and dk blank --- e2e_test/ddl/show.slt | 8 ++++---- e2e_test/extended_mode/basic.slt | 12 ++++++------ src/frontend/src/handler/describe.rs | 28 +++++----------------------- 3 files changed, 15 insertions(+), 33 deletions(-) diff --git a/e2e_test/ddl/show.slt b/e2e_test/ddl/show.slt index b2c19b2b7a4a0..32b9dc48f6579 100644 --- a/e2e_test/ddl/show.slt +++ b/e2e_test/ddl/show.slt @@ -20,8 +20,8 @@ v1 integer false NULL v2 integer false Lorem ipsum dolor sit amet v3 integer false NULL _row_id serial true consectetur adipiscing elit -primary key _row_id NULL consectetur adipiscing elit -distribution key _row_id NULL consectetur adipiscing elit +primary key _row_id NULL NULL +distribution key _row_id NULL NULL query TTT show columns from t3; @@ -49,8 +49,8 @@ v1 integer false NULL v2 integer false Nemo enim ipsam v3 integer false NULL _row_id serial true consectetur adipiscing elit -primary key _row_id NULL consectetur adipiscing elit -distribution key _row_id NULL consectetur adipiscing elit +primary key _row_id NULL NULL +distribution key _row_id NULL NULL idx1 index(v1 ASC, v2 ASC) include(v3) distributed by(v1) NULL NULL query TT diff --git a/e2e_test/extended_mode/basic.slt b/e2e_test/extended_mode/basic.slt index 09cbcf1e43bb7..3c3ad0c43e649 100644 --- a/e2e_test/extended_mode/basic.slt +++ b/e2e_test/extended_mode/basic.slt @@ -42,12 +42,12 @@ create table t3 (v1 int, v2 int, v3 int); query TTTT describe t3; ---- -v1 integer false NULL -v2 integer false NULL -v3 integer false NULL -_row_id serial true NULL -primary key _row_id NULL -distribution key _row_id NULL +v1 integer false NULL +v2 integer false NULL +v3 integer false NULL +_row_id serial true NULL +primary key _row_id NULL NULL +distribution key _row_id NULL NULL query TTT show columns from t3; diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index b8a7e7b22b2a7..f8c41825853fc 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -100,7 +100,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res // Convert all column descs to rows let mut rows = col_descs_to_rows(columns); - fn fmt_col(columns: &[ColumnDesc], cb: C) -> String + fn fmt_col_project(columns: &[ColumnDesc], cb: C) -> String where T: Display + ?Sized, C: FnMut(&ColumnDesc) -> &T, @@ -113,39 +113,21 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res // Convert primary key to rows if !pk_columns.is_empty() { - let names = fmt_col(&pk_columns, |x| &x.name); - let descs = fmt_col(&pk_columns, |x| { - x.description.as_deref().unwrap_or_default() - }); - rows.push(Row::new(vec![ Some("primary key".into()), - Some(names.into()), + Some(fmt_col_project(&pk_columns, |x| &x.name).into()), None, // Is Hidden - if descs.is_empty() { - None - } else { - Some(descs.into()) - }, + None, // Description ])); } // Convert distribution keys to rows if !dist_columns.is_empty() { - let names = fmt_col(&dist_columns, |x| &x.name); - let descs = fmt_col(&dist_columns, |x| { - x.description.as_deref().unwrap_or_default() - }); - rows.push(Row::new(vec![ Some("distribution key".into()), - Some(names.into()), + Some(fmt_col_project(&dist_columns, |x| &x.name).into()), None, // Is Hidden - if descs.is_empty() { - None - } else { - Some(descs.into()) - }, + None, // Description ])); } From 66c736e85c55a2321bd660c25d9f174fe33afa3c Mon Sep 17 00:00:00 2001 From: jinser Date: Thu, 19 Oct 2023 21:39:17 +0800 Subject: [PATCH 18/36] Apply suggestions from code review Keep the field order during construction consistent with that at definition Co-authored-by: Richard Chien --- src/common/src/catalog/column.rs | 2 +- src/frontend/src/binder/expr/mod.rs | 4 ++-- src/frontend/src/handler/create_table.rs | 2 +- src/frontend/src/handler/describe.rs | 9 +++++---- src/meta/src/manager/catalog/mod.rs | 19 ++++++++----------- 5 files changed, 17 insertions(+), 19 deletions(-) diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index 9aaacf6e52df1..e728d8c2e664d 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -130,8 +130,8 @@ impl ColumnDesc { .map(|f| f.to_protobuf()) .collect_vec(), type_name: self.type_name.clone(), - description: self.description.clone(), generated_or_default_column: self.generated_or_default_column.clone(), + description: self.description.clone(), } } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index fa87ffc714f1f..221056f3a4822 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -575,8 +575,8 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { name: f.name.real_value(), field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, }) }) .collect::>>()? @@ -589,8 +589,8 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { name: column_def.name.real_value(), field_descs, type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, }) } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 6168c2500ce5b..bb02797c21395 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -187,8 +187,8 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> name: name.real_value(), field_descs, type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, }, is_hidden: false, }); diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index f8c41825853fc..8408c0c743ff6 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -157,10 +157,11 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res .into(), ) }, - None, // Is Hidden - None, /* Description - * TODO: index description - * index.description.map(Into::into), */ + // Is Hidden + None, + // Description + // TODO: index description + None, ]) })); diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 2ff90929d6b70..34f8b474fff9d 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -2212,17 +2212,14 @@ impl CatalogManager { comment.table_id ) })?; - column - .column_desc - .as_mut() - .ok_or_else(|| { - anyhow!( - "column_desc at index {} for table id {} not found", - col_idx, - comment.table_id - ) - })? - .description = comment.description; + let column_desc = column.column_desc.as_mut().ok_or_else(|| { + anyhow!( + "column desc at index {} for table id {} not found", + col_idx, + comment.table_id + ) + })?; + column_desc.description = comment.description; } else { table.description = comment.description; } From a37fbde2acfaa7bd880b4c5d392858bf235039a1 Mon Sep 17 00:00:00 2001 From: jinser Date: Thu, 19 Oct 2023 21:43:38 +0800 Subject: [PATCH 19/36] Update src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs Co-authored-by: Richard Chien --- .../src/catalog/system_catalog/rw_catalog/rw_description.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index 9af6c6000e82d..68ba1454c0fc0 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -57,7 +57,7 @@ impl SysCatalogReaderImpl { let reader = self.catalog_reader.read_guard(); let schemas = reader.iter_schemas_except_rw_catalog(&self.auth_context.database)?; - let rw_catalog = reader.get_schema_by_name(&self.auth_context.database, "rw_catalog")?; + let rw_catalog = reader.get_schema_by_name(&self.auth_context.database, RW_CATALOG_SCHEMA_NAME)?; Ok(schemas .flat_map(|schema| { From a9b64a9c0bb627664726bd19be548f41fcd8b6d5 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Thu, 19 Oct 2023 22:05:36 +0800 Subject: [PATCH 20/36] chore: update comment --- proto/catalog.proto | 1 + proto/ddl_service.proto | 2 +- proto/plan_common.proto | 1 + .../src/catalog/system_catalog/mod.rs | 1 + .../rw_catalog/rw_description.rs | 42 ++++++++----------- src/frontend/src/catalog/table_catalog.rs | 1 + 6 files changed, 23 insertions(+), 25 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index 7023d553f2d01..aad173bf71c8a 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -280,6 +280,7 @@ message Table { CreateType create_type = 32; + // This field is used to store the description set by the `comment on` clause. optional string description = 33; // Per-table catalog version, used by schema change. `None` for internal tables and tests. diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 05792623f408a..328880d191d5a 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -353,6 +353,6 @@ service DdlService { rpc CreateConnection(CreateConnectionRequest) returns (CreateConnectionResponse); rpc ListConnections(ListConnectionsRequest) returns (ListConnectionsResponse); rpc DropConnection(DropConnectionRequest) returns (DropConnectionResponse); - rpc CommentOn(CommentOnRequest) returns (CommentOnResponse); rpc GetTables(GetTablesRequest) returns (GetTablesResponse); + rpc CommentOn(CommentOnRequest) returns (CommentOnResponse); } diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 1430bbea68588..149a379a78d92 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -38,6 +38,7 @@ message ColumnDesc { DefaultColumnDesc default_column = 7; } + // This field is used to store the description set by the `comment on` clause. optional string description = 8; } diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 334f7e7f5cc7b..d64db79b8ced1 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -58,6 +58,7 @@ pub struct SystemTableCatalog { // owner of table, should always be default super user, keep it for compatibility. pub owner: u32, + /// description of table, set by `comment on`. pub description: Option, } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index 68ba1454c0fc0..179aab0d955b0 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -29,7 +29,7 @@ pub const RW_DESCRIPTION: BuiltinTable = BuiltinTable { (DataType::Int32, "objoid"), // rw_tables, rw_views, rw_functions, etc. (DataType::Int32, "classoid"), - // If it is not None, it means column number. In this case, objoid should be table_id. + // If `objoid` is `table_id`, then non-null `objsubid` is column number. (DataType::Int32, "objsubid"), (DataType::Varchar, "description"), ], @@ -57,7 +57,8 @@ impl SysCatalogReaderImpl { let reader = self.catalog_reader.read_guard(); let schemas = reader.iter_schemas_except_rw_catalog(&self.auth_context.database)?; - let rw_catalog = reader.get_schema_by_name(&self.auth_context.database, RW_CATALOG_SCHEMA_NAME)?; + let rw_catalog = + reader.get_schema_by_name(&self.auth_context.database, RW_CATALOG_SCHEMA_NAME)?; Ok(schemas .flat_map(|schema| { @@ -70,28 +71,21 @@ impl SysCatalogReaderImpl { .unwrap_or_default() as _, table.description.as_deref().unwrap_or_default().into(), )) - .chain( - table - .columns - .iter() - // .filter(|col| !col.is_hidden()) - .map(|col| { - build_row_with_sub( - table.id.table_id as _, - rw_catalog - .get_system_table_by_name("rw_tables") - .map(|st| st.id.table_id) - .unwrap_or_default() - as _, - col.column_id().get_id() as _, - col.column_desc - .description - .as_deref() - .unwrap_or_default() - .into(), - ) - }), - ) + .chain(table.columns.iter().map(|col| { + build_row_with_sub( + table.id.table_id as _, + rw_catalog + .get_system_table_by_name("rw_tables") + .map(|st| st.id.table_id) + .unwrap_or_default() as _, + col.column_id().get_id() as _, + col.column_desc + .description + .as_deref() + .unwrap_or_default() + .into(), + ) + })) }) }) .collect()) diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index f30ea15ba57c8..f12ef7728032f 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -150,6 +150,7 @@ pub struct TableCatalog { /// Indicate whether to use watermark cache for state table. pub cleaned_by_watermark: bool, + /// description of table, set by `comment on`. pub description: Option, } From effe791cc2ced5eec8bafc02bf1e96f7bfb836b9 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Thu, 19 Oct 2023 22:18:35 +0800 Subject: [PATCH 21/36] refactor(frontend): remove unnecessary pub methods in `schema_catalog` and `root_catalog`. --- src/frontend/src/catalog/root_catalog.rs | 8 -------- src/frontend/src/catalog/schema_catalog.rs | 4 ---- .../catalog/system_catalog/rw_catalog/rw_description.rs | 4 +++- 3 files changed, 3 insertions(+), 13 deletions(-) diff --git a/src/frontend/src/catalog/root_catalog.rs b/src/frontend/src/catalog/root_catalog.rs index f5c1402c37083..f8fa09efa43e3 100644 --- a/src/frontend/src/catalog/root_catalog.rs +++ b/src/frontend/src/catalog/root_catalog.rs @@ -370,14 +370,6 @@ impl Catalog { Ok(self.get_database_by_name(db_name)?.iter_schemas()) } - pub fn iter_schemas_except_rw_catalog( - &self, - db_name: &str, - ) -> CatalogResult> { - self.iter_schemas(db_name) - .map(|scs| scs.filter(|sc| !sc.is_rw_catalog())) - } - pub fn get_all_database_names(&self) -> Vec { self.database_by_name.keys().cloned().collect_vec() } diff --git a/src/frontend/src/catalog/schema_catalog.rs b/src/frontend/src/catalog/schema_catalog.rs index ab6533704e008..e42cbc639bba6 100644 --- a/src/frontend/src/catalog/schema_catalog.rs +++ b/src/frontend/src/catalog/schema_catalog.rs @@ -432,10 +432,6 @@ impl SchemaCatalog { self.system_table_by_name.values() } - pub fn is_rw_catalog(&self) -> bool { - !self.system_table_by_name.is_empty() - } - pub fn get_table_by_name(&self, table_name: &str) -> Option<&Arc> { self.table_by_name.get(table_name) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index 179aab0d955b0..b2b32ab5d4e2c 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -56,9 +56,11 @@ impl SysCatalogReaderImpl { }; let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas_except_rw_catalog(&self.auth_context.database)?; let rw_catalog = reader.get_schema_by_name(&self.auth_context.database, RW_CATALOG_SCHEMA_NAME)?; + let schemas = reader + .iter_schemas(&self.auth_context.database)? + .filter(|schema| schema.id() != rw_catalog.id()); Ok(schemas .flat_map(|schema| { From 223f5fe05888d2497ba07221f304080687b3f248 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Thu, 19 Oct 2023 22:31:08 +0800 Subject: [PATCH 22/36] refactor(frontend): simplify the code --- .../rw_catalog/rw_description.rs | 32 +++++++------------ 1 file changed, 12 insertions(+), 20 deletions(-) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index b2b32ab5d4e2c..c49f8ce1242f1 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -38,19 +38,11 @@ pub const RW_DESCRIPTION: BuiltinTable = BuiltinTable { impl SysCatalogReaderImpl { pub fn read_rw_description(&self) -> Result> { - let build_row = |table_id, catalog_id, description| { + let build_row = |table_id, catalog_id, index: Option, description| { OwnedRow::new(vec![ Some(ScalarImpl::Int32(table_id)), Some(ScalarImpl::Int32(catalog_id)), - None, - Some(ScalarImpl::Utf8(description)), - ]) - }; - let build_row_with_sub = |table_id, catalog_id, index, description| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table_id)), - Some(ScalarImpl::Int32(catalog_id)), - Some(ScalarImpl::Int32(index)), + index.map(ScalarImpl::Int32), Some(ScalarImpl::Utf8(description)), ]) }; @@ -62,25 +54,25 @@ impl SysCatalogReaderImpl { .iter_schemas(&self.auth_context.database)? .filter(|schema| schema.id() != rw_catalog.id()); + let rw_tables_id: i32 = rw_catalog + .get_system_table_by_name("rw_tables") + .map(|st| st.id.table_id) + .unwrap_or_default() as _; + Ok(schemas .flat_map(|schema| { schema.iter_table().flat_map(|table| { iter::once(build_row( table.id.table_id as _, - rw_catalog - .get_system_table_by_name("rw_tables") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, + rw_tables_id, + None, table.description.as_deref().unwrap_or_default().into(), )) .chain(table.columns.iter().map(|col| { - build_row_with_sub( + build_row( table.id.table_id as _, - rw_catalog - .get_system_table_by_name("rw_tables") - .map(|st| st.id.table_id) - .unwrap_or_default() as _, - col.column_id().get_id() as _, + rw_tables_id, + Some(col.column_id().get_id() as _), col.column_desc .description .as_deref() From e72854c77fe31b84200a8b74099657a1263aeb9d Mon Sep 17 00:00:00 2001 From: jinser Date: Mon, 23 Oct 2023 16:57:22 +0800 Subject: [PATCH 23/36] Update src/meta/src/manager/catalog/mod.rs Only ensure table id can be comment in meta node. Co-authored-by: August --- src/meta/src/manager/catalog/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 34f8b474fff9d..0b82afdc0d9d1 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -2197,7 +2197,7 @@ impl CatalogManager { pub async fn comment_on(&self, comment: Comment) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; - database_core.ensure_table_view_or_source_id(&comment.table_id)?; + database_core.ensure_table_id(&comment.table_id)?; let mut tables = BTreeMapTransaction::new(&mut database_core.tables); From 7e151af8dbb877781506c859fadee0f01a302dc6 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Tue, 24 Oct 2023 02:34:01 +0800 Subject: [PATCH 24/36] feat: resolve schema when `comment on` --- proto/catalog.proto | 6 ++- proto/ddl_service.proto | 4 +- src/frontend/src/catalog/catalog_service.rs | 22 +++-------- src/frontend/src/handler/comment.rs | 44 +++++++++++++++------ src/frontend/src/test_utils.rs | 9 +---- src/meta/service/src/ddl_service.rs | 9 +++-- src/meta/src/manager/catalog/mod.rs | 5 ++- src/rpc_client/src/meta_client.rs | 14 ++----- 8 files changed, 58 insertions(+), 55 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index aad173bf71c8a..c966b7bbe5eb0 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -323,6 +323,8 @@ message Database { message Comment { uint32 table_id = 1; - optional uint32 column_index = 2; - optional string description = 3; + uint32 schema_id = 2; + uint32 database_id = 3; + optional uint32 column_index = 4; + optional string description = 5; } diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 328880d191d5a..0b6ba89afd325 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -315,9 +315,7 @@ message GetTablesResponse { } message CommentOnRequest { - uint32 table_id = 1; - optional uint32 column_index = 2; - optional string comment = 3; + catalog.Comment comment = 1; } message CommentOnResponse { diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index c1be43f260710..be85293acd27f 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -21,7 +21,8 @@ use risingwave_common::error::ErrorCode::InternalError; use risingwave_common::error::{Result, RwError}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::catalog::{ - PbCreateType, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, + PbComment, PbCreateType, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, + PbView, }; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::create_connection_request; @@ -111,12 +112,7 @@ pub trait CatalogWriter: Send + Sync { connection: create_connection_request::Payload, ) -> Result<()>; - async fn comment_on( - &self, - table_id: TableId, - column_index: Option, - comment: Option, - ) -> Result<()>; + async fn comment_on(&self, comment: PbComment) -> Result<()>; async fn drop_table( &self, @@ -289,16 +285,8 @@ impl CatalogWriter for CatalogWriterImpl { self.wait_version(version).await } - async fn comment_on( - &self, - TableId { table_id }: TableId, - column_index: Option, - comment: Option, - ) -> Result<()> { - let version = self - .meta_client - .comment_on(table_id, column_index, comment) - .await?; + async fn comment_on(&self, comment: PbComment) -> Result<()> { + let version = self.meta_client.comment_on(comment).await?; self.wait_version(version).await } diff --git a/src/frontend/src/handler/comment.rs b/src/frontend/src/handler/comment.rs index 61b62cd6a6f01..1522e94fc740c 100644 --- a/src/frontend/src/handler/comment.rs +++ b/src/frontend/src/handler/comment.rs @@ -14,6 +14,7 @@ use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::error::{ErrorCode, Result}; +use risingwave_pb::catalog::PbComment; use risingwave_sqlparser::ast::{CommentObject, ObjectName}; use super::{HandlerArgs, RwPgResponse}; @@ -27,12 +28,12 @@ pub async fn handle_comment( ) -> Result { let session = handler_args.session; - let (table_id, column_index) = { + let comment = { let mut binder = Binder::new_for_ddl(&session); // only `Column` and `Table` object are now supported match object_type { CommentObject::Column => { - let [.., tab, col] = object_name.0.as_slice() else { + let [tab @ .., col] = object_name.0.as_slice() else { return Err(ErrorCode::BindError(format!( "Invalid column: {}", object_name.real_value() @@ -40,27 +41,46 @@ pub async fn handle_comment( .into()); }; - let table = binder.bind_table(None, &tab.real_value(), None)?; + let (schema, table) = Binder::resolve_schema_qualified_name( + session.database(), + ObjectName(tab.to_vec()), + )?; + + let (database_id, schema_id) = + session.get_database_and_schema_id_for_create(schema)?; + let table = binder.bind_table(None, &table, None)?; binder.bind_columns_to_context(col.real_value(), table.table_catalog.columns)?; let column = binder.bind_column(object_name.0.as_slice())?; - ( - table.table_id, - column.as_input_ref().map(|input_ref| input_ref.index as _), - ) + PbComment { + table_id: table.table_id.into(), + schema_id, + database_id, + column_index: column.as_input_ref().map(|input_ref| input_ref.index as _), + description: comment, + } } CommentObject::Table => { - let table = binder.bind_table(None, &object_name.real_value(), None)?; - (table.table_id, None) + let (schema, table) = + Binder::resolve_schema_qualified_name(session.database(), object_name)?; + let (database_id, schema_id) = + session.get_database_and_schema_id_for_create(schema)?; + let table = binder.bind_table(None, &table, None)?; + + PbComment { + table_id: table.table_id.into(), + schema_id, + database_id, + column_index: None, + description: comment, + } } } }; let catalog_writer = session.catalog_writer()?; - catalog_writer - .comment_on(table_id, column_index, comment) - .await?; + catalog_writer.comment_on(comment).await?; Ok(PgResponse::empty_result(StatementType::COMMENT)) } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 71921ddd4c594..902d2d5f99ce0 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -32,7 +32,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::backup_service::MetaSnapshotMetadata; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ - PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, + PbComment, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, }; use risingwave_pb::ddl_service::{create_connection_request, DdlProgress}; use risingwave_pb::hummock::write_limits::WriteLimit; @@ -318,12 +318,7 @@ impl CatalogWriter for MockCatalogWriter { unreachable!() } - async fn comment_on( - &self, - _table_id: TableId, - _column_index: Option, - _comment: Option, - ) -> Result<()> { + async fn comment_on(&self, _comment: PbComment) -> Result<()> { unreachable!() } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 73e3189a0c65a..97fefb1aaa867 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -722,13 +722,16 @@ impl DdlService for DdlServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); + let comment = req.get_comment()?.clone(); let version = self .ddl_controller .run_command(DdlCommand::CommentOn(Comment { - table_id: req.table_id, - column_index: req.column_index, - description: req.comment, + table_id: comment.table_id, + schema_id: comment.schema_id, + database_id: comment.database_id, + column_index: comment.column_index, + description: comment.description, })) .await?; diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 0b82afdc0d9d1..88bf1c1ae49db 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -2197,7 +2197,10 @@ impl CatalogManager { pub async fn comment_on(&self, comment: Comment) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; - database_core.ensure_table_id(&comment.table_id)?; + + database_core.ensure_database_id(comment.database_id)?; + database_core.ensure_schema_id(comment.schema_id)?; + database_core.ensure_table_id(comment.table_id)?; let mut tables = BTreeMapTransaction::new(&mut database_core.tables); diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 583689e3e67bb..c23e6c850a886 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -40,7 +40,8 @@ use risingwave_hummock_sdk::{ use risingwave_pb::backup_service::backup_service_client::BackupServiceClient; use risingwave_pb::backup_service::*; use risingwave_pb::catalog::{ - Connection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, + Connection, PbComment, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, + PbView, Table, }; use risingwave_pb::cloud_service::cloud_service_client::CloudServiceClient; use risingwave_pb::cloud_service::*; @@ -406,16 +407,9 @@ impl MetaClient { Ok((resp.table_id.into(), resp.version)) } - pub async fn comment_on( - &self, - table_id: u32, - column_index: Option, - comment: Option, - ) -> Result { + pub async fn comment_on(&self, comment: PbComment) -> Result { let request = CommentOnRequest { - table_id, - column_index, - comment, + comment: Some(comment), }; let resp = self.inner.comment_on(request).await?; Ok(resp.version) From d61a60a0774fb9cb727c7c5a1f97c9eea3c9f87d Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Tue, 24 Oct 2023 02:37:21 +0800 Subject: [PATCH 25/36] fix: add `PgFieldDescriptor` in `infer` --- src/frontend/src/session.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 67eac0df34d05..0385bd52690ef 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -1148,6 +1148,11 @@ fn infer(bound: Option, stmt: Statement) -> Result Ok(vec![PgFieldDescriptor::new( "QUERY PLAN".to_owned(), From d980bcd366589c14d821e3b0b1b93e6963dd7852 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Tue, 24 Oct 2023 02:48:53 +0800 Subject: [PATCH 26/36] refactor: better error in catalog comment_on --- src/meta/src/manager/catalog/mod.rs | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 88bf1c1ae49db..7a8e684d2391c 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -2204,17 +2204,13 @@ impl CatalogManager { let mut tables = BTreeMapTransaction::new(&mut database_core.tables); - let mut table = tables - .get_mut(comment.table_id) - .ok_or_else(|| anyhow!("table id {} not found", comment.table_id))?; + // unwrap is safe because the table id was ensured before + let mut table = tables.get_mut(comment.table_id).unwrap(); if let Some(col_idx) = comment.column_index { - let column = table.columns.get_mut(col_idx as usize).ok_or_else(|| { - anyhow!( - "column index {} for table id {} not found", - col_idx, - comment.table_id - ) - })?; + let column = table + .columns + .get_mut(col_idx as usize) + .ok_or_else(|| MetaError::catalog_id_not_found("column", col_idx))?; let column_desc = column.column_desc.as_mut().ok_or_else(|| { anyhow!( "column desc at index {} for table id {} not found", From 9f9da21b194f1bba70a5d289fec8d92650c97aae Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Tue, 24 Oct 2023 03:26:53 +0800 Subject: [PATCH 27/36] feat: support `pg_description` upon `rw_description` --- .../pg_catalog/pg_description.rs | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs index 8bc91f4a10bb9..62f8ffdc7dbce 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs @@ -21,22 +21,24 @@ use crate::catalog::system_catalog::BuiltinView; /// The catalog `pg_description` stores description. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-description.html`] -pub static PG_DESCRIPTION: LazyLock = LazyLock::new(|| { - BuiltinView { - name: "pg_description", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "objoid"), - (DataType::Int32, "classoid"), - (DataType::Int32, "objsubid"), - (DataType::Varchar, "description"), - ], - sql: "SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_tables \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_materialized_views \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_views \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_indexes \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_sources \ - UNION ALL SELECT id AS objoid, NULL::integer AS classoid, 0 AS objsubid, NULL AS description FROM rw_catalog.rw_system_tables\ - ".into(), - } +pub static PG_DESCRIPTION: LazyLock = LazyLock::new(|| BuiltinView { + name: "pg_description", + schema: PG_CATALOG_SCHEMA_NAME, + columns: &[ + (DataType::Int32, "objoid"), + (DataType::Int32, "classoid"), + (DataType::Int32, "objsubid"), + (DataType::Varchar, "description"), + ], + // objsubid = 0 => _row_id (hidden column) + // objsubid is NULL => table self + sql: "SELECT objoid, \ + classoid, \ + CASE \ + WHEN objsubid = 0 THEN -1 \ + WHEN objsubid IS NULL THEN 0 \ + ELSE objsubid \ + END AS objsubid, \ + description FROM rw_catalog.rw_description;" + .into(), }); From 3b1aeda53bfc6132e18a26f3fc17ca2f9f187507 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Tue, 24 Oct 2023 15:18:58 +0800 Subject: [PATCH 28/36] refactor: unify order of order `description` --- src/common/src/catalog/column.rs | 8 ++++---- src/common/src/catalog/mod.rs | 4 ++-- src/common/src/catalog/test_utils.rs | 2 +- src/compute/tests/integration_tests.rs | 2 +- src/connector/src/parser/avro/util.rs | 2 +- src/connector/src/parser/protobuf/parser.rs | 2 +- src/connector/src/source/manager.rs | 2 +- src/frontend/src/handler/create_source.rs | 8 ++++---- src/source/src/source_desc.rs | 2 +- 9 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index e728d8c2e664d..b70084fbf864a 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -174,8 +174,8 @@ impl ColumnDesc { name: name.to_string(), field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, } } @@ -195,8 +195,8 @@ impl ColumnDesc { name: name.to_string(), field_descs: fields, type_name: type_name.to_string(), - description: None, generated_or_default_column: None, + description: None, } } @@ -248,8 +248,8 @@ impl From for ColumnDesc { name: prost.name, type_name: prost.type_name, field_descs, - description: prost.description.clone(), generated_or_default_column: prost.generated_or_default_column, + description: prost.description.clone(), } } } @@ -268,8 +268,8 @@ impl From<&ColumnDesc> for PbColumnDesc { name: c.name.clone(), field_descs: c.field_descs.iter().map(ColumnDesc::to_protobuf).collect(), type_name: c.type_name.clone(), - description: c.description.clone(), generated_or_default_column: c.generated_or_default_column.clone(), + description: c.description.clone(), } } } diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 36f2a9d01c18d..1a46cdcf4057a 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -109,8 +109,8 @@ pub fn row_id_column_desc() -> ColumnDesc { name: row_id_column_name(), field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, } } @@ -131,8 +131,8 @@ pub fn offset_column_desc() -> ColumnDesc { name: offset_column_name(), field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, } } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index e217b3b51ce9a..6b524edb92430 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -56,8 +56,8 @@ impl ColumnDescTestExt for ColumnDesc { name: name.to_string(), type_name: type_name.to_string(), field_descs: fields, - description: None, generated_or_default_column: None, + description: None, } } } diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index e47cb98d09807..4b105cca07fe1 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -158,8 +158,8 @@ async fn test_table_materialize() -> StreamResult<()> { name: field.name, field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, }) .collect_vec(); let (barrier_tx, barrier_rx) = unbounded_channel(); diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 78cd568792ff2..e1b63962bf23c 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -56,8 +56,8 @@ fn avro_field_to_column_desc( name: name.to_owned(), field_descs: vec_column, type_name: schema_name.to_string(), - description: None, generated_or_default_column: None, + description: None, }) } _ => { diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index a0298d5577d01..82dde062cdd11 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -196,8 +196,8 @@ impl ProtobufParserConfig { column_type: Some(field_type.to_protobuf()), field_descs, type_name: m.full_name().to_string(), - description: None, generated_or_default_column: None, + description: None, }) } else { *index += 1; diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 0823d5fe81b34..8624550274299 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -122,8 +122,8 @@ impl From<&SourceColumnDesc> for ColumnDesc { name: s.name.clone(), field_descs: s.fields.clone(), type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, } } } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index f0a2e15a939d9..c496c3ac2f9b1 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -647,8 +647,8 @@ pub(crate) async fn try_bind_columns_from_source( name: "_id".to_string(), field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, }, is_hidden: false, }, @@ -659,8 +659,8 @@ pub(crate) async fn try_bind_columns_from_source( name: "payload".to_string(), field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, }, is_hidden: false, }, @@ -794,8 +794,8 @@ fn check_and_add_timestamp_column( name: KAFKA_TIMESTAMP_COLUMN_NAME.to_string(), field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, }, is_hidden: true, @@ -812,8 +812,8 @@ fn add_upsert_default_key_column(columns: &mut Vec) { name: DEFAULT_KEY_COLUMN_NAME.to_string(), field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, }, is_hidden: true, }; diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 86d9b429ba8f3..838ad6d24d14a 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -259,8 +259,8 @@ pub mod test_utils { name: f.name.clone(), field_descs: vec![], type_name: "".to_string(), - description: None, generated_or_default_column: None, + description: None, } .to_protobuf(), ), From f4ad632b39c2b2dbdafc6bb4f5af5dc346c764d5 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Tue, 24 Oct 2023 15:30:06 +0800 Subject: [PATCH 29/36] refactor: more specified function `concat` --- src/frontend/src/handler/describe.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 8408c0c743ff6..398879243bbe4 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -100,14 +100,13 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res // Convert all column descs to rows let mut rows = col_descs_to_rows(columns); - fn fmt_col_project(columns: &[ColumnDesc], cb: C) -> String + fn concat(display_elems: impl IntoIterator) -> String where - T: Display + ?Sized, - C: FnMut(&ColumnDesc) -> &T, + T: Display, { format!( "{}", - display_comma_separated(&columns.iter().map(cb).collect::>()) + display_comma_separated(&display_elems.into_iter().collect::>()) ) } @@ -115,7 +114,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res if !pk_columns.is_empty() { rows.push(Row::new(vec![ Some("primary key".into()), - Some(fmt_col_project(&pk_columns, |x| &x.name).into()), + Some(concat(pk_columns.iter().map(|x| &x.name)).into()), None, // Is Hidden None, // Description ])); @@ -125,7 +124,7 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res if !dist_columns.is_empty() { rows.push(Row::new(vec![ Some("distribution key".into()), - Some(fmt_col_project(&dist_columns, |x| &x.name).into()), + Some(concat(dist_columns.iter().map(|x| &x.name)).into()), None, // Is Hidden None, // Description ])); From 53ac31056515f7fc10d8590fe29caa80e88cdba0 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Tue, 24 Oct 2023 23:06:45 +0800 Subject: [PATCH 30/36] feat: add table description in `describe` --- src/frontend/src/handler/describe.rs | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 398879243bbe4..ec2d464929ffd 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -35,11 +35,13 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res let mut binder = Binder::new_for_system(&session); let relation = binder.bind_relation_by_name(table_name.clone(), None, false)?; // For Source, it doesn't have table catalog so use get source to get column descs. - let (columns, pk_columns, dist_columns, indices): ( + let (columns, pk_columns, dist_columns, indices, relname, description): ( Vec, Vec, Vec, Vec>, + String, + Option, ) = match relation { Relation::Source(s) => { let pk_column_catalogs = s @@ -56,7 +58,14 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res .unwrap() }) .collect_vec(); - (s.catalog.columns, pk_column_catalogs, vec![], vec![]) + ( + s.catalog.columns, + pk_column_catalogs, + vec![], + vec![], + s.catalog.name, + None, // Description + ) } Relation::BaseTable(t) => { let pk_column_catalogs = t @@ -76,6 +85,8 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res pk_column_catalogs, dist_columns, t.table_indexes, + t.table_catalog.name, + t.table_catalog.description, ) } Relation::SystemTable(t) => { @@ -90,6 +101,8 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res pk_column_catalogs, vec![], vec![], + t.sys_table_catalog.name.clone(), + None, // Description ) } _ => { @@ -164,6 +177,13 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res ]) })); + rows.push(Row::new(vec![ + Some("table description".into()), + Some(relname.into()), + None, // Is Hidden + description.map(Into::into), // Description + ])); + // TODO: table name and description as title of response // TODO: recover the original user statement Ok(PgResponse::builder(StatementType::DESCRIBE) From ccd8a91908e353d871ad42cdce6a58c3ba07797d Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Wed, 25 Oct 2023 00:36:52 +0800 Subject: [PATCH 31/36] feat: more consistent comment behavior with pgsql - `pg_description` only contains non-empty `description` row. - `comment on ... is '';` can be used to remove description. --- .../pg_catalog/pg_description.rs | 3 ++- .../rw_catalog/rw_description.rs | 25 ++++++++----------- src/frontend/src/handler/comment.rs | 1 + 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs index 62f8ffdc7dbce..593522ceda705 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs @@ -39,6 +39,7 @@ pub static PG_DESCRIPTION: LazyLock = LazyLock::new(|| BuiltinView WHEN objsubid IS NULL THEN 0 \ ELSE objsubid \ END AS objsubid, \ - description FROM rw_catalog.rw_description;" + description FROM rw_catalog.rw_description \ + WHERE description IS NOT NULL;" .into(), }); diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index c49f8ce1242f1..370dec33a2a2a 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -38,14 +38,15 @@ pub const RW_DESCRIPTION: BuiltinTable = BuiltinTable { impl SysCatalogReaderImpl { pub fn read_rw_description(&self) -> Result> { - let build_row = |table_id, catalog_id, index: Option, description| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table_id)), - Some(ScalarImpl::Int32(catalog_id)), - index.map(ScalarImpl::Int32), - Some(ScalarImpl::Utf8(description)), - ]) - }; + let build_row = + |table_id, catalog_id, index: Option, description: Option>| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table_id)), + Some(ScalarImpl::Int32(catalog_id)), + index.map(ScalarImpl::Int32), + description.map(ScalarImpl::Utf8), + ]) + }; let reader = self.catalog_reader.read_guard(); let rw_catalog = @@ -66,18 +67,14 @@ impl SysCatalogReaderImpl { table.id.table_id as _, rw_tables_id, None, - table.description.as_deref().unwrap_or_default().into(), + table.description.as_deref().map(Into::into), )) .chain(table.columns.iter().map(|col| { build_row( table.id.table_id as _, rw_tables_id, Some(col.column_id().get_id() as _), - col.column_desc - .description - .as_deref() - .unwrap_or_default() - .into(), + col.column_desc.description.as_deref().map(Into::into), ) })) }) diff --git a/src/frontend/src/handler/comment.rs b/src/frontend/src/handler/comment.rs index 1522e94fc740c..b0ff42a790346 100644 --- a/src/frontend/src/handler/comment.rs +++ b/src/frontend/src/handler/comment.rs @@ -27,6 +27,7 @@ pub async fn handle_comment( comment: Option, ) -> Result { let session = handler_args.session; + let comment = comment.filter(|s| !s.is_empty()); let comment = { let mut binder = Binder::new_for_ddl(&session); From 90ec94c9d8626409e64d3e61289c4d50e4f0930f Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Wed, 25 Oct 2023 00:59:28 +0800 Subject: [PATCH 32/36] test: update `pg_description` e2e_test --- .../batch/catalog/pg_description.slt.part | 81 +++++++++++++++---- 1 file changed, 64 insertions(+), 17 deletions(-) diff --git a/e2e_test/batch/catalog/pg_description.slt.part b/e2e_test/batch/catalog/pg_description.slt.part index 41ade5ffbb9b3..44ba006ea4367 100644 --- a/e2e_test/batch/catalog/pg_description.slt.part +++ b/e2e_test/batch/catalog/pg_description.slt.part @@ -1,18 +1,65 @@ -query IIITT -SELECT d.*, c.relname FROM pg_catalog.pg_description d join pg_catalog.pg_class c on d.objoid = c.oid ORDER BY d.objoid limit 15; +statement ok +create table t(a int, b text, c date); + +statement ok +comment on table t is 'Lorem ipsum'; + +statement ok +comment on column t.a is 'Praesent elementum'; + +statement ok +comment on column public.t.c is 'Nullam ultricies'; + +statement ok +comment on column public.t._row_id is 'facilisis enim'; + +query TIIIT +SELECT + c.relname, + ( + SELECT relname FROM pg_catalog.pg_class WHERE oid = d.classoid + ) AS classoid, + d.objsubid, + d.description +FROM + pg_catalog.pg_description d + JOIN pg_catalog.pg_class c + ON d.objoid = c.oid +ORDER BY d.objsubid; ---- -1 NULL 0 NULL pg_type -2 NULL 0 NULL pg_namespace -3 NULL 0 NULL pg_cast -4 NULL 0 NULL pg_matviews -5 NULL 0 NULL pg_user -6 NULL 0 NULL pg_class -7 NULL 0 NULL pg_index -8 NULL 0 NULL pg_opclass -9 NULL 0 NULL pg_collation -10 NULL 0 NULL pg_am -11 NULL 0 NULL pg_operator -12 NULL 0 NULL pg_views -13 NULL 0 NULL pg_attribute -14 NULL 0 NULL pg_database -15 NULL 0 NULL pg_description \ No newline at end of file +t rw_tables -1 facilisis enim +t rw_tables 0 Lorem ipsum +t rw_tables 1 Praesent elementum +t rw_tables 3 Nullam ultricies + +statement ok +comment on table public.t is NULL; + +statement ok +comment on column t._row_id is NULL; + +statement ok +comment on column t.c is ''; + +statement ok +comment on column public.t.b is 'Vivamus fermentum'; + +query TIIIT +SELECT + c.relname, + ( + SELECT relname FROM pg_catalog.pg_class WHERE oid = d.classoid + ) AS classoid, + d.objsubid, + d.description +FROM + pg_catalog.pg_description d + JOIN pg_catalog.pg_class c + ON d.objoid = c.oid +ORDER BY d.objsubid; +---- +t rw_tables 1 Praesent elementum +t rw_tables 2 Vivamus fermentum + +statement ok +drop table t; From 05fb23b04194996c2560a47acf8a188d57f0f692 Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Wed, 25 Oct 2023 01:08:07 +0800 Subject: [PATCH 33/36] test: add more `comment on` & `describe` e2e_test - comment on table - comment on column - re-comment on to override description - remove comment by set empty string or NULL --- e2e_test/ddl/show.slt | 47 +++++++++++++++++++++++++++++-------------- 1 file changed, 32 insertions(+), 15 deletions(-) diff --git a/e2e_test/ddl/show.slt b/e2e_test/ddl/show.slt index 32b9dc48f6579..9586731207fa7 100644 --- a/e2e_test/ddl/show.slt +++ b/e2e_test/ddl/show.slt @@ -7,21 +7,28 @@ create materialized view mv3 as select sum(v1) as sum_v1 from t3; statement ok create view v3 as select sum(v2) as sum_v2 from t3; +statement ok +comment on table t3 is 'volutpat vitae'; + +statement ok +comment on column t3.v1 is 'turpis vehicula'; + statement ok comment on column t3.v2 is 'Lorem ipsum dolor sit amet'; statement ok -comment on column t3._row_id is 'consectetur adipiscing elit'; +comment on column public.t3._row_id is 'consectetur adipiscing elit'; query TTTT describe t3; ---- -v1 integer false NULL -v2 integer false Lorem ipsum dolor sit amet -v3 integer false NULL -_row_id serial true consectetur adipiscing elit -primary key _row_id NULL NULL -distribution key _row_id NULL NULL +v1 integer false turpis vehicula +v2 integer false Lorem ipsum dolor sit amet +v3 integer false NULL +_row_id serial true consectetur adipiscing elit +primary key _row_id NULL NULL +distribution key _row_id NULL NULL +table description t3 NULL volutpat vitae query TTT show columns from t3; @@ -40,18 +47,28 @@ show indexes from t3; idx1 t3 v1 ASC, v2 ASC v3 v1 statement ok -comment on column t3.v2 is 'Nemo enim ipsam'; +comment on table public.t3 is 'consectetur turpis'; + +statement ok +comment on column t3.v1 is 'Nemo enim ipsam'; + +statement ok +comment on column t3.v2 is ''; + +statement ok +comment on column t3._row_id is NULL; query TTTT describe t3; ---- -v1 integer false NULL -v2 integer false Nemo enim ipsam -v3 integer false NULL -_row_id serial true consectetur adipiscing elit -primary key _row_id NULL NULL -distribution key _row_id NULL NULL -idx1 index(v1 ASC, v2 ASC) include(v3) distributed by(v1) NULL NULL +v1 integer false Nemo enim ipsam +v2 integer false NULL +v3 integer false NULL +_row_id serial true NULL +primary key _row_id NULL NULL +distribution key _row_id NULL NULL +idx1 index(v1 ASC, v2 ASC) include(v3) distributed by(v1) NULL NULL +table description t3 NULL consectetur turpis query TT show create index idx1; From 2bf7dd9a4422aea4d8d3759ff17ec87acf28c04f Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Wed, 25 Oct 2023 01:30:36 +0800 Subject: [PATCH 34/36] refactor: avoid `type_complexity` clippy warning --- src/frontend/src/handler/describe.rs | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index ec2d464929ffd..10c4f32d26c26 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -13,20 +13,18 @@ // limitations under the License. use std::fmt::Display; -use std::sync::Arc; use itertools::Itertools; use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::types::Row; -use risingwave_common::catalog::{ColumnCatalog, ColumnDesc}; use risingwave_common::error::Result; use risingwave_common::types::DataType; use risingwave_sqlparser::ast::{display_comma_separated, ObjectName}; use super::RwPgResponse; use crate::binder::{Binder, Relation}; -use crate::catalog::{CatalogError, IndexCatalog}; +use crate::catalog::CatalogError; use crate::handler::util::col_descs_to_rows; use crate::handler::HandlerArgs; @@ -35,14 +33,9 @@ pub fn handle_describe(handler_args: HandlerArgs, table_name: ObjectName) -> Res let mut binder = Binder::new_for_system(&session); let relation = binder.bind_relation_by_name(table_name.clone(), None, false)?; // For Source, it doesn't have table catalog so use get source to get column descs. - let (columns, pk_columns, dist_columns, indices, relname, description): ( - Vec, - Vec, - Vec, - Vec>, - String, - Option, - ) = match relation { + + // Vec, Vec, Vec, Vec>, String, Option + let (columns, pk_columns, dist_columns, indices, relname, description) = match relation { Relation::Source(s) => { let pk_column_catalogs = s .catalog From 426bc1a48e9a4331284169441077536d822e82dc Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Wed, 25 Oct 2023 08:35:32 +0800 Subject: [PATCH 35/36] test: up to date `describe` --- e2e_test/extended_mode/basic.slt | 1 + src/frontend/src/handler/describe.rs | 1 + 2 files changed, 2 insertions(+) diff --git a/e2e_test/extended_mode/basic.slt b/e2e_test/extended_mode/basic.slt index 3c3ad0c43e649..4d7e51940473b 100644 --- a/e2e_test/extended_mode/basic.slt +++ b/e2e_test/extended_mode/basic.slt @@ -48,6 +48,7 @@ v3 integer false NULL _row_id serial true NULL primary key _row_id NULL NULL distribution key _row_id NULL NULL +table description t3 NULL NUll query TTT show columns from t3; diff --git a/src/frontend/src/handler/describe.rs b/src/frontend/src/handler/describe.rs index 10c4f32d26c26..cef7af9dbd324 100644 --- a/src/frontend/src/handler/describe.rs +++ b/src/frontend/src/handler/describe.rs @@ -257,6 +257,7 @@ mod tests { "primary key".into() => "v3".into(), "distribution key".into() => "v3".into(), "idx1".into() => "index(v1 DESC, v2 ASC, v3 ASC) include(v4) distributed by(v1)".into(), + "table description".into() => "t".into(), }; assert_eq!(columns, expected_columns); From e2872baeab105a2a14b75223cfb724479610d2ca Mon Sep 17 00:00:00 2001 From: Jinser Kafka Date: Wed, 25 Oct 2023 12:19:29 +0800 Subject: [PATCH 36/36] test: typo fix `NULL` --- e2e_test/extended_mode/basic.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/extended_mode/basic.slt b/e2e_test/extended_mode/basic.slt index 4d7e51940473b..054dae2f6a234 100644 --- a/e2e_test/extended_mode/basic.slt +++ b/e2e_test/extended_mode/basic.slt @@ -48,7 +48,7 @@ v3 integer false NULL _row_id serial true NULL primary key _row_id NULL NULL distribution key _row_id NULL NULL -table description t3 NULL NUll +table description t3 NULL NULL query TTT show columns from t3;