diff --git a/Cargo.lock b/Cargo.lock index d009e627e3165..3116b378bc045 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -385,9 +385,9 @@ checksum = "ecc7ab41815b3c653ccd2978ec3255c81349336702dfdf62ee6f7069b12a3aae" [[package]] name = "async-trait" -version = "0.1.68" +version = "0.1.72" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9ccdd8f2a161be9bd5c023df56f1b2a0bd1d83872ae53b71a84a12c9bf6e842" +checksum = "cc6dde6e4ed435a4c1ee4e73592f5ba9da2151af10076cc04858746af9352d09" dependencies = [ "proc-macro2", "quote", @@ -835,9 +835,9 @@ dependencies = [ [[package]] name = "axum" -version = "0.6.18" +version = "0.6.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8175979259124331c1d7bf6586ee7e0da434155e4b2d48ec2c8386281d8df39" +checksum = "a6a1de45611fdb535bfde7b7de4fd54f4fd2b17b1737c0a59b69bf9b92074b8c" dependencies = [ "async-trait", "axum-core", @@ -5248,9 +5248,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.31" +version = "1.0.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fe8a65d69dd0808184ebb5f836ab526bb259db23c657efa38711b1072ee47f0" +checksum = "50f3b39ccfb720540debaa0164757101c08ecb8d326b15358ce76a62c7e85965" dependencies = [ "proc-macro2", ] @@ -5545,7 +5545,7 @@ dependencies = [ "reqwest", "serde", "serde_json", - "serde_with 2.3.3", + "serde_with 3.1.0", "serde_yaml", "tempfile", "tracing", @@ -5789,7 +5789,7 @@ dependencies = [ "serde_bytes", "serde_default", "serde_json", - "serde_with 2.3.3", + "serde_with 3.1.0", "smallbitset", "speedate", "static_assertions", @@ -5980,7 +5980,7 @@ dependencies = [ "serde", "serde_derive", "serde_json", - "serde_with 2.3.3", + "serde_with 3.1.0", "simd-json", "tempfile", "thiserror", @@ -6374,7 +6374,7 @@ dependencies = [ "risingwave_frontend", "risingwave_sqlparser", "serde", - "serde_with 2.3.3", + "serde_with 3.1.0", "serde_yaml", "tempfile", "walkdir", @@ -6534,7 +6534,7 @@ dependencies = [ "madsim-tokio", "risingwave_sqlparser", "serde", - "serde_with 2.3.3", + "serde_with 3.1.0", "serde_yaml", "walkdir", "workspace-hack", @@ -6578,7 +6578,7 @@ dependencies = [ "regex", "risingwave_rt", "serde", - "serde_with 2.3.3", + "serde_with 3.1.0", "tokio-postgres", "tokio-stream", "toml 0.7.3", @@ -7198,6 +7198,22 @@ dependencies = [ "time 0.3.23", ] +[[package]] +name = "serde_with" +version = "3.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "21e47d95bc83ed33b2ecf84f4187ad1ab9685d18ff28db000c99deac8ce180e3" +dependencies = [ + "base64 0.21.0", + "chrono", + "hex", + "indexmap", + "serde", + "serde_json", + "serde_with_macros 3.1.0", + "time 0.3.23", +] + [[package]] name = "serde_with_macros" version = "1.5.2" @@ -7222,6 +7238,18 @@ dependencies = [ "syn 2.0.26", ] +[[package]] +name = "serde_with_macros" +version = "3.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ea3cee93715c2e266b9338b7544da68a9f24e227722ba482bd1c024367c77c65" +dependencies = [ + "darling 0.20.1", + "proc-macro2", + "quote", + "syn 2.0.26", +] + [[package]] name = "serde_yaml" version = "0.9.21" @@ -9022,7 +9050,7 @@ dependencies = [ "scopeguard", "serde", "serde_json", - "serde_with 2.3.3", + "serde_with 3.1.0", "smallvec", "subtle", "syn 1.0.109", diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 0fd44ec518436..6732f4c9e7281 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -116,6 +116,16 @@ echo "--- Kill cluster" cargo make ci-kill pkill -f connector-node +echo "--- e2e, ci-1cn-1fe, protobuf schema registry" +RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ +cargo make ci-start ci-1cn-1fe +python3 -m pip install requests protobuf confluent-kafka +python3 e2e_test/schema_registry/pb.py "message_queue:29092" "http://message_queue:8081" "sr_pb_test" 20 +sqllogictest -p 4566 -d dev './e2e_test/schema_registry/pb.slt' + +echo "--- Kill cluster" +cargo make ci-kill + echo "--- e2e, ci-kafka-plus-pubsub, kafka and pubsub source" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ cargo make ci-start ci-pubsub diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index bf01147512038..1417d7d537b0c 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -270,6 +270,7 @@ steps: timeout_in_minutes: 60 - label: "upload micro-benchmark" + if: build.branch == "main" || build.pull_request.labels includes "ci/upload-micro-benchmark" command: - "BUILDKITE_BUILD_NUMBER=$BUILDKITE_BUILD_NUMBER ci/scripts/upload-micro-bench-results.sh" depends_on: "run-micro-benchmarks" diff --git a/e2e_test/schema_registry/pb.py b/e2e_test/schema_registry/pb.py new file mode 100644 index 0000000000000..8b5e850a82796 --- /dev/null +++ b/e2e_test/schema_registry/pb.py @@ -0,0 +1,66 @@ +from protobuf import user_pb2 +import sys +from confluent_kafka import Producer +from confluent_kafka.serialization import ( + SerializationContext, + MessageField, +) +from confluent_kafka.schema_registry import SchemaRegistryClient +from confluent_kafka.schema_registry.protobuf import ProtobufSerializer + + +def delivery_report(err, msg): + if err is not None: + print("Delivery failed for User record {}: {}".format(msg.value(), err)) + + +def get_user(i): + return user_pb2.User( + id=i, + name="User_{}".format(i), + address="Address_{}".format(i), + city="City_{}".format(i), + gender=user_pb2.MALE if i % 2 == 0 else user_pb2.FEMALE, + ) + + +def send_to_kafka(producer_conf, schema_registry_conf, topic, num_records): + schema_registry_client = SchemaRegistryClient(schema_registry_conf) + serializer = ProtobufSerializer( + user_pb2.User, + schema_registry_client, + {"use.deprecated.format": False}, + ) + + producer = Producer(producer_conf) + for i in range(num_records): + user = get_user(i) + + producer.produce( + topic=topic, + partition=0, + value=serializer(user, SerializationContext(topic, MessageField.VALUE)), + on_delivery=delivery_report, + ) + producer.flush() + print("Send {} records to kafka\n".format(num_records)) + + +if __name__ == "__main__": + if len(sys.argv) < 4: + print("pb.py ") + exit(1) + + broker_list = sys.argv[1] + schema_registry_url = sys.argv[2] + topic = sys.argv[3] + num_records = int(sys.argv[4]) + + schema_registry_conf = {"url": schema_registry_url} + producer_conf = {"bootstrap.servers": broker_list} + + try: + send_to_kafka(producer_conf, schema_registry_conf, topic, num_records) + except Exception as e: + print("Send Protobuf data to schema registry and kafka failed {}", e) + exit(1) diff --git a/e2e_test/schema_registry/pb.slt b/e2e_test/schema_registry/pb.slt new file mode 100644 index 0000000000000..a48f7f77cbd86 --- /dev/null +++ b/e2e_test/schema_registry/pb.slt @@ -0,0 +1,36 @@ +# Before running this test, seed data into kafka: +# python3 e2e_test/schema_registry/pb.py + +# Create a table. +statement ok +create table sr_pb_test with ( + connector = 'kafka', + topic = 'sr_pb_test', + properties.bootstrap.server = 'message_queue:29092', + scan.startup.mode = 'earliest', + message = 'test.User') +FORMAT plain ENCODE protobuf( + schema.registry = 'http://message_queue:8081', + message = 'test.User' + ); + +# Wait for source +sleep 10s + +# Flush into storage +statement ok +flush; + +query I +select count(*) from sr_pb_test; +---- +20 + +query II +select min(id), max(id) from sr_pb_test; +---- +0 19 + + +statement ok +drop table sr_pb_test; \ No newline at end of file diff --git a/e2e_test/schema_registry/protobuf/user.proto b/e2e_test/schema_registry/protobuf/user.proto new file mode 100644 index 0000000000000..79245a3ba4ff9 --- /dev/null +++ b/e2e_test/schema_registry/protobuf/user.proto @@ -0,0 +1,16 @@ +syntax = "proto3"; + +package test; + +message User { + int32 id = 1; + string name = 2; + string address = 3; + string city = 4; + Gender gender = 5; +} + +enum Gender { + MALE = 0; + FEMALE = 1; +} \ No newline at end of file diff --git a/e2e_test/schema_registry/protobuf/user_pb2.py b/e2e_test/schema_registry/protobuf/user_pb2.py new file mode 100644 index 0000000000000..1258739d4ae2f --- /dev/null +++ b/e2e_test/schema_registry/protobuf/user_pb2.py @@ -0,0 +1,28 @@ +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# source: user.proto +"""Generated protocol buffer code.""" +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import symbol_database as _symbol_database +from google.protobuf.internal import builder as _builder +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\nuser.proto\x12\x04test\"]\n\x04User\x12\n\n\x02id\x18\x01 \x01(\x05\x12\x0c\n\x04name\x18\x02 \x01(\t\x12\x0f\n\x07\x61\x64\x64ress\x18\x03 \x01(\t\x12\x0c\n\x04\x63ity\x18\x04 \x01(\t\x12\x1c\n\x06gender\x18\x05 \x01(\x0e\x32\x0c.test.Gender*\x1e\n\x06Gender\x12\x08\n\x04MALE\x10\x00\x12\n\n\x06\x46\x45MALE\x10\x01\x62\x06proto3') + +_globals = globals() +_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, 'user_pb2', _globals) +if _descriptor._USE_C_DESCRIPTORS == False: + + DESCRIPTOR._options = None + _globals['_GENDER']._serialized_start=115 + _globals['_GENDER']._serialized_end=145 + _globals['_USER']._serialized_start=20 + _globals['_USER']._serialized_end=113 +# @@protoc_insertion_point(module_scope) diff --git a/proto/catalog.proto b/proto/catalog.proto index 68cc782522560..cd1cf35756779 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -64,6 +64,9 @@ message Source { } string definition = 13; optional uint32 connection_id = 14; + + optional uint64 initialized_at_epoch = 15; + optional uint64 created_at_epoch = 16; } enum SinkType { @@ -94,6 +97,8 @@ message Sink { map properties = 12; string definition = 13; optional uint32 connection_id = 14; + optional uint64 initialized_at_epoch = 15; + optional uint64 created_at_epoch = 16; } message Connection { @@ -132,6 +137,9 @@ message Index { // The index of `InputRef` is the column index of the primary table. repeated expr.ExprNode index_item = 8; repeated int32 original_columns = 9; + + optional uint64 initialized_at_epoch = 10; + optional uint64 created_at_epoch = 11; } message Function { @@ -216,6 +224,10 @@ message Table { // The range of row count of the table. // This field is not always present due to backward compatibility. Use `Cardinality::unknown` in this case. plan_common.Cardinality cardinality = 27; + + optional uint64 initialized_at_epoch = 28; + optional uint64 created_at_epoch = 29; + // 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; diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index abf147536f932..24b0612eed62a 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -72,7 +72,7 @@ serde = { version = "1", features = ["derive"] } serde_bytes = "0.11" serde_default = "0.1" serde_json = "1" -serde_with = "2" +serde_with = "3" smallbitset = "0.6.1" speedate = "0.7.0" static_assertions = "1" diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 74a69109ef328..17e579a0944df 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -77,7 +77,7 @@ rust_decimal = "1" serde = { version = "1", features = ["derive", "rc"] } serde_derive = "1" serde_json = "1" -serde_with = { version = "2", features = ["json"] } +serde_with = { version = "3", features = ["json"] } simd-json = "0.9.1" tempfile = "3" thiserror = "1" diff --git a/src/connector/src/aws_auth.rs b/src/connector/src/aws_auth.rs index a2f7af64ccdb7..a004fbaafd12b 100644 --- a/src/connector/src/aws_auth.rs +++ b/src/connector/src/aws_auth.rs @@ -116,9 +116,14 @@ impl AwsAuthProps { let credentials_provider = self .with_role_provider(self.build_credential_provider()) .await?; - let config_loader = aws_config::from_env() + let mut config_loader = aws_config::from_env() .region(region) .credentials_provider(credentials_provider); + + if let Some(endpoint) = self.endpoint.as_ref() { + config_loader = config_loader.endpoint_url(endpoint); + } + Ok(config_loader.load().await) } } diff --git a/src/connector/src/aws_utils.rs b/src/connector/src/aws_utils.rs index b5f9a937341c2..785a4396bacba 100644 --- a/src/connector/src/aws_utils.rs +++ b/src/connector/src/aws_utils.rs @@ -105,6 +105,7 @@ pub fn s3_client( s3_config::Builder::from(&sdk_config.clone()) .retry_config(retry_conf) .timeout_config(timeout_conf) + .force_path_style(true) .build() } else { s3_config::Config::new(sdk_config) diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index e6fb73336e16c..4faee51cd2ef7 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -333,6 +333,8 @@ mod tests { ColumnDesc::new_atomic(DataType::Varchar, "username", 9), ], ), + ColumnDesc::new_atomic(DataType::Varchar, "I64CastToVarchar", 10), + ColumnDesc::new_atomic(DataType::Int64, "VarcharCastToI64", 11), ] .iter() .map(SourceColumnDesc::from) @@ -352,7 +354,9 @@ mod tests { "id": "7772634297", "name": "Lily Frami yet", "username": "Dooley5659" - } + }, + "I64CastToVarchar": 1598197865760800768, + "VarcharCastToI64": "1598197865760800768" } "#.to_vec(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 1); @@ -381,7 +385,9 @@ mod tests { Some(ScalarImpl::Utf8("7772634297".into())), Some(ScalarImpl::Utf8("Lily Frami yet".into())), Some(ScalarImpl::Utf8("Dooley5659".into())), - ]) )) + ]) )), + Some(ScalarImpl::Utf8("1598197865760800768".into())), + Some(ScalarImpl::Int64(1598197865760800768)), ]; assert_eq!(row, expected.into()); } diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index 27c298bc54f2d..1f1b5a33b4141 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -68,6 +68,16 @@ pub enum BooleanHandling { }, } +#[derive(Clone, Debug)] +pub enum VarcharHandling { + // do not allow other types cast to varchar + Strict, + // allow Json Value (Null, Bool, I64, I128, U64, U128, F64) cast to varchar + OnlyPrimaryTypes, + // allow all type cast to varchar (inc. Array, Object) + AllTypes, +} + #[derive(Clone, Debug)] pub struct JsonParseOptions { pub bytea_handling: ByteaHandling, @@ -75,6 +85,7 @@ pub struct JsonParseOptions { pub json_value_handling: JsonValueHandling, pub numeric_handling: NumericHandling, pub boolean_handing: BooleanHandling, + pub varchar_handing: VarcharHandling, pub ignoring_keycase: bool, } @@ -96,6 +107,7 @@ impl JsonParseOptions { string_parsing: true, string_integer_parsing: true, }, + varchar_handing: VarcharHandling::Strict, ignoring_keycase: true, }; pub const DEBEZIUM: JsonParseOptions = JsonParseOptions { @@ -109,6 +121,7 @@ impl JsonParseOptions { string_parsing: false, string_integer_parsing: false, }, + varchar_handing: VarcharHandling::Strict, ignoring_keycase: true, }; pub const DEFAULT: JsonParseOptions = JsonParseOptions { @@ -119,6 +132,7 @@ impl JsonParseOptions { string_parsing: true, }, boolean_handing: BooleanHandling::Strict, + varchar_handing: VarcharHandling::OnlyPrimaryTypes, ignoring_keycase: true, }; @@ -336,6 +350,30 @@ impl JsonParseOptions { .into(), // ---- Varchar ----- (Some(DataType::Varchar) | None, ValueType::String) => value.as_str().unwrap().into(), + ( + Some(DataType::Varchar), + ValueType::Bool + | ValueType::I64 + | ValueType::I128 + | ValueType::U64 + | ValueType::U128 + | ValueType::F64, + ) if matches!(self.varchar_handing, VarcharHandling::OnlyPrimaryTypes) => { + value.to_string().into() + } + ( + Some(DataType::Varchar), + ValueType::Bool + | ValueType::I64 + | ValueType::I128 + | ValueType::U64 + | ValueType::U128 + | ValueType::F64 + | ValueType::Array + | ValueType::Object, + ) if matches!(self.varchar_handing, VarcharHandling::AllTypes) => { + value.to_string().into() + } // ---- Time ----- (Some(DataType::Time), ValueType::String) => str_to_time(value.as_str().unwrap()) .map_err(|_| create_error())? diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index 5ace2746dcd19..1e8aa6a068739 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -80,6 +80,8 @@ impl SinkDesc { properties: self.properties.into_iter().collect(), sink_type: self.sink_type, connection_id, + created_at_epoch: None, + initialized_at_epoch: None, } } diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index d3f0079ced7b0..5cd291dc0ae75 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -20,6 +20,7 @@ use itertools::Itertools; use risingwave_common::catalog::{ ColumnCatalog, ConnectionId, DatabaseId, Field, Schema, SchemaId, TableId, UserId, }; +use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::{PbSink, PbSinkType}; @@ -145,6 +146,10 @@ pub struct SinkCatalog { /// Sink may use a privatelink connection to connect to the downstream system. pub connection_id: Option, + + pub created_at_epoch: Option, + + pub initialized_at_epoch: Option, } impl SinkCatalog { @@ -176,6 +181,8 @@ impl SinkCatalog { properties: self.properties.clone(), sink_type: self.sink_type.to_proto() as i32, connection_id: self.connection_id.map(|id| id.into()), + initialized_at_epoch: self.initialized_at_epoch.map(|e| e.0), + created_at_epoch: self.created_at_epoch.map(|e| e.0), } } @@ -248,6 +255,8 @@ impl From for SinkCatalog { .collect_vec(), sink_type: SinkType::from_proto(sink_type), connection_id: pb.connection_id.map(ConnectionId), + created_at_epoch: pb.created_at_epoch.map(Epoch::from), + initialized_at_epoch: pb.initialized_at_epoch.map(Epoch::from), } } } diff --git a/src/ctl/src/cmd_impl/hummock/list_version.rs b/src/ctl/src/cmd_impl/hummock/list_version.rs index 4da34dc4310d8..3cc082671273d 100644 --- a/src/ctl/src/cmd_impl/hummock/list_version.rs +++ b/src/ctl/src/cmd_impl/hummock/list_version.rs @@ -17,11 +17,36 @@ use risingwave_rpc_client::HummockMetaClient; use crate::CtlContext; -pub async fn list_version(context: &CtlContext, verbose: bool) -> anyhow::Result<()> { +pub async fn list_version( + context: &CtlContext, + verbose: bool, + verbose_key_range: bool, +) -> anyhow::Result<()> { let meta_client = context.meta_client().await?; - let version = meta_client.get_current_version().await?; + let mut version = meta_client.get_current_version().await?; + + if verbose && verbose_key_range { + println!("{:#?}", version); + } else if verbose { + version.levels.iter_mut().for_each(|(_cg_id, levels)| { + // l0 + if levels.l0.is_some() { + let l0 = levels.l0.as_mut().unwrap(); + for sub_level in &mut l0.sub_levels { + for t in &mut sub_level.table_infos { + t.key_range = None; + } + } + } + + // l1 ~ lmax + for level in &mut levels.levels { + for t in &mut level.table_infos { + t.key_range = None; + } + } + }); - if verbose { println!("{:#?}", version); } else { println!( diff --git a/src/ctl/src/cmd_impl/hummock/trigger_manual_compaction.rs b/src/ctl/src/cmd_impl/hummock/trigger_manual_compaction.rs index 961a44662bafa..1ca9826651097 100644 --- a/src/ctl/src/cmd_impl/hummock/trigger_manual_compaction.rs +++ b/src/ctl/src/cmd_impl/hummock/trigger_manual_compaction.rs @@ -21,10 +21,11 @@ pub async fn trigger_manual_compaction( compaction_group_id: u64, table_id: u32, level: u32, + sst_ids: Vec, ) -> anyhow::Result<()> { let meta_client = context.meta_client().await?; let result = meta_client - .trigger_manual_compaction(compaction_group_id, table_id, level) + .trigger_manual_compaction(compaction_group_id, table_id, level, sst_ids) .await; println!("{:#?}", result); Ok(()) diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index c2e6d2cdff3c2..9c9a3fb3e235b 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -139,6 +139,9 @@ enum HummockCommands { ListVersion { #[clap(short, long = "verbose", default_value_t = false)] verbose: bool, + + #[clap(long = "verbose_key_range", default_value_t = false)] + verbose_key_range: bool, }, /// list hummock version deltas in the meta store @@ -173,6 +176,9 @@ enum HummockCommands { #[clap(short, long = "level", default_value_t = 1)] level: u32, + + #[clap(short, long = "sst-ids")] + sst_ids: Vec, }, /// trigger a full GC for SSTs that is not in version and with timestamp <= now - /// sst_retention_time_sec. @@ -417,8 +423,11 @@ pub async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { Commands::Hummock(HummockCommands::DisableCommitEpoch) => { cmd_impl::hummock::disable_commit_epoch(context).await? } - Commands::Hummock(HummockCommands::ListVersion { verbose }) => { - cmd_impl::hummock::list_version(context, verbose).await?; + Commands::Hummock(HummockCommands::ListVersion { + verbose, + verbose_key_range, + }) => { + cmd_impl::hummock::list_version(context, verbose, verbose_key_range).await?; } Commands::Hummock(HummockCommands::ListVersionDeltas { start_id, @@ -440,12 +449,14 @@ pub async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { compaction_group_id, table_id, level, + sst_ids, }) => { cmd_impl::hummock::trigger_manual_compaction( context, compaction_group_id, table_id, level, + sst_ids, ) .await? } diff --git a/src/expr/macro/src/gen.rs b/src/expr/macro/src/gen.rs index a2f62c0e04efa..80a04092cecfe 100644 --- a/src/expr/macro/src/gen.rs +++ b/src/expr/macro/src/gen.rs @@ -70,6 +70,7 @@ impl FunctionAttr { } else { self.generate_build_fn()? }; + let deprecated = self.deprecated; Ok(quote! { #[ctor::ctor] fn #ctor_name() { @@ -79,6 +80,7 @@ impl FunctionAttr { inputs_type: &[#(#args),*], ret_type: #ret, build: #build_fn, + deprecated: #deprecated, }) }; } }) diff --git a/src/expr/macro/src/lib.rs b/src/expr/macro/src/lib.rs index 384adbc1349e5..384bb26a5d25e 100644 --- a/src/expr/macro/src/lib.rs +++ b/src/expr/macro/src/lib.rs @@ -422,6 +422,7 @@ struct FunctionAttr { init_state: Option, prebuild: Option, type_infer: Option, + deprecated: bool, user_fn: UserFunctionAttr, } diff --git a/src/expr/macro/src/parse.rs b/src/expr/macro/src/parse.rs index cbae5b6c5dc71..7aa8871a258c6 100644 --- a/src/expr/macro/src/parse.rs +++ b/src/expr/macro/src/parse.rs @@ -62,6 +62,7 @@ impl FunctionAttr { init_state: find_argument(attr, "init_state"), prebuild: find_argument(attr, "prebuild"), type_infer: find_argument(attr, "type_infer"), + deprecated: find_name(attr, "deprecated"), user_fn, }) } @@ -179,3 +180,11 @@ fn find_argument(attr: &syn::AttributeArgs, name: &str) -> Option { Some(lit_str.value()) }) } + +/// Find name `#[xxx(.., name)]`. +fn find_name(attr: &syn::AttributeArgs, name: &str) -> bool { + attr.iter().any(|n| { + let syn::NestedMeta::Meta(syn::Meta::Path(path)) = n else { return false }; + path.is_ident(name) + }) +} diff --git a/src/expr/src/agg/mod.rs b/src/expr/src/agg/mod.rs index b0d7722d240e8..8c2159ae858b4 100644 --- a/src/expr/src/agg/mod.rs +++ b/src/expr/src/agg/mod.rs @@ -93,7 +93,8 @@ pub fn build(agg: AggCall) -> Result { func: agg.kind, inputs_type: &args, ret_type, - set_returning: false + set_returning: false, + deprecated: false, } )) })?; diff --git a/src/expr/src/expr/build.rs b/src/expr/src/expr/build.rs index 28fe7bfe61bbe..5b6a337578075 100644 --- a/src/expr/src/expr/build.rs +++ b/src/expr/src/expr/build.rs @@ -31,7 +31,6 @@ use super::expr_some_all::SomeAllExpression; use super::expr_udf::UdfExpression; use super::expr_vnode::VnodeExpression; use crate::expr::expr_proctime::ProcTimeExpression; -use crate::expr::expr_to_timestamp_const_tmpl::build_to_timestamp_expr_legacy; use crate::expr::{ BoxedExpression, Expression, InputRefExpression, LiteralExpression, TryFromExprNodeBoxed, }; @@ -81,11 +80,6 @@ pub fn build_from_prost(prost: &ExprNode) -> Result { .map(build_from_prost) .try_collect()?; - // deprecated exprs not in signature map just for backward compatibility - if func_type == E::ToTimestamp1 && ret_type == DataType::Timestamp { - return build_to_timestamp_expr_legacy(ret_type, children); - } - build_func(func_type, ret_type, children) } } @@ -111,6 +105,7 @@ pub fn build_func( inputs_type: &args, ret_type: (&ret_type).into(), set_returning: false, + deprecated: false, } )) })?; diff --git a/src/expr/src/expr/expr_to_timestamp_const_tmpl.rs b/src/expr/src/expr/expr_to_timestamp_const_tmpl.rs index 014aab60f6338..cdbf75f6c0df2 100644 --- a/src/expr/src/expr/expr_to_timestamp_const_tmpl.rs +++ b/src/expr/src/expr/expr_to_timestamp_const_tmpl.rs @@ -121,6 +121,7 @@ fn build_to_timestamp_expr( } /// Support building the variant returning timestamp without time zone for backward compatibility. +#[build_function("to_timestamp1(varchar, varchar) -> timestamp", deprecated)] pub fn build_to_timestamp_expr_legacy( return_type: DataType, children: Vec, diff --git a/src/expr/src/sig/agg.rs b/src/expr/src/sig/agg.rs index 343710066b8e7..9a914e0cde64e 100644 --- a/src/expr/src/sig/agg.rs +++ b/src/expr/src/sig/agg.rs @@ -47,6 +47,7 @@ impl fmt::Debug for AggFuncSig { inputs_type: self.inputs_type, ret_type: self.ret_type, set_returning: false, + deprecated: false, } .fmt(f) } diff --git a/src/expr/src/sig/func.rs b/src/expr/src/sig/func.rs index c933fce200993..d84a065ad095a 100644 --- a/src/expr/src/sig/func.rs +++ b/src/expr/src/sig/func.rs @@ -16,7 +16,6 @@ use std::collections::HashMap; use std::fmt; -use std::ops::Deref; use std::sync::LazyLock; use risingwave_common::types::{DataType, DataTypeName}; @@ -53,6 +52,7 @@ impl FuncSigMap { } /// Returns a function signature with the same type, argument types and return type. + /// Deprecated functions are included. pub fn get(&self, ty: PbType, args: &[DataTypeName], ret: DataTypeName) -> Option<&FuncSign> { let v = self.0.get(&(ty, args.len()))?; v.iter() @@ -60,8 +60,12 @@ impl FuncSigMap { } /// Returns all function signatures with the same type and number of arguments. - pub fn get_with_arg_nums(&self, ty: PbType, nargs: usize) -> &[FuncSign] { - self.0.get(&(ty, nargs)).map_or(&[], Deref::deref) + /// Deprecated functions are excluded. + pub fn get_with_arg_nums(&self, ty: PbType, nargs: usize) -> Vec<&FuncSign> { + match self.0.get(&(ty, nargs)) { + Some(v) => v.iter().filter(|d| !d.deprecated).collect(), + None => vec![], + } } } @@ -72,6 +76,9 @@ pub struct FuncSign { pub inputs_type: &'static [DataTypeName], pub ret_type: DataTypeName, pub build: fn(return_type: DataType, children: Vec) -> Result, + /// Whether the function is deprecated and should not be used in the frontend. + /// For backward compatibility, it is still available in the backend. + pub deprecated: bool, } impl fmt::Debug for FuncSign { @@ -81,6 +88,7 @@ impl fmt::Debug for FuncSign { inputs_type: self.inputs_type, ret_type: self.ret_type, set_returning: false, + deprecated: self.deprecated, } .fmt(f) } @@ -124,6 +132,10 @@ mod tests { // validate the FUNC_SIG_MAP is consistent assert_eq!(func, &sig.func); assert_eq!(num_args, &sig.inputs_type.len()); + // exclude deprecated functions + if sig.deprecated { + continue; + } new_map .entry(*func) @@ -187,12 +199,6 @@ mod tests { ArrayAccess: [ "array_access(list, int32) -> boolean/int16/int32/int64/int256/float32/float64/decimal/serial/date/time/timestamp/timestamptz/interval/varchar/bytea/jsonb/list/struct", ], - ArrayLength: [ - "array_length(list) -> int64/int32", - ], - Cardinality: [ - "cardinality(list) -> int64/int32", - ], } "#]]; expected.assert_debug_eq(&duplicated); diff --git a/src/expr/src/sig/mod.rs b/src/expr/src/sig/mod.rs index d34699d461d55..cea417a3ca4ee 100644 --- a/src/expr/src/sig/mod.rs +++ b/src/expr/src/sig/mod.rs @@ -28,16 +28,18 @@ pub(crate) struct FuncSigDebug<'a, T> { pub inputs_type: &'a [DataTypeName], pub ret_type: DataTypeName, pub set_returning: bool, + pub deprecated: bool, } impl<'a, T: std::fmt::Display> std::fmt::Debug for FuncSigDebug<'a, T> { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { let s = format!( - "{}({:?}) -> {}{:?}", + "{}({:?}) -> {}{:?}{}", self.func, self.inputs_type.iter().format(", "), if self.set_returning { "setof " } else { "" }, - self.ret_type + self.ret_type, + if self.deprecated { " [deprecated]" } else { "" }, ) .to_ascii_lowercase(); diff --git a/src/expr/src/sig/table_function.rs b/src/expr/src/sig/table_function.rs index 4d96769e6b0e5..a8ebce5e378bd 100644 --- a/src/expr/src/sig/table_function.rs +++ b/src/expr/src/sig/table_function.rs @@ -90,6 +90,7 @@ impl fmt::Debug for FuncSign { inputs_type: self.inputs_type, ret_type: self.ret_type, set_returning: true, + deprecated: false, } .fmt(f) } diff --git a/src/expr/src/table_function/mod.rs b/src/expr/src/table_function/mod.rs index 0b3f172c63a08..f7b6920e11e6e 100644 --- a/src/expr/src/table_function/mod.rs +++ b/src/expr/src/table_function/mod.rs @@ -146,6 +146,7 @@ pub fn build( inputs_type: &args, ret_type: (&return_type).into(), set_returning: true, + deprecated: false, } )) })?; diff --git a/src/expr/src/vector_op/array_length.rs b/src/expr/src/vector_op/array_length.rs index 81357bef924c9..4b44b2ac4d9b4 100644 --- a/src/expr/src/vector_op/array_length.rs +++ b/src/expr/src/vector_op/array_length.rs @@ -56,11 +56,11 @@ use crate::ExprError; /// ---- /// 1 /// -/// query error type unknown +/// query error Cannot implicitly cast /// select array_length(null); /// ``` #[function("array_length(list) -> int32")] -#[function("array_length(list) -> int64")] // for compatibility with plans from old version +#[function("array_length(list) -> int64", deprecated)] fn array_length>(array: ListRef<'_>) -> Result { array .len() diff --git a/src/expr/src/vector_op/cardinality.rs b/src/expr/src/vector_op/cardinality.rs index 3fc8470295d7e..d482da3b30d39 100644 --- a/src/expr/src/vector_op/cardinality.rs +++ b/src/expr/src/vector_op/cardinality.rs @@ -56,11 +56,11 @@ use crate::ExprError; /// ---- /// 1 /// -/// query error type unknown +/// query error Cannot implicitly cast /// select cardinality(null); /// ``` #[function("cardinality(list) -> int32")] -#[function("cardinality(list) -> int64")] // for compatibility with plans from old version +#[function("cardinality(list) -> int64", deprecated)] fn cardinality>(array: ListRef<'_>) -> Result { array .flatten() diff --git a/src/expr/src/vector_op/position.rs b/src/expr/src/vector_op/position.rs index 1d434aa2170a9..0700c4e847e97 100644 --- a/src/expr/src/vector_op/position.rs +++ b/src/expr/src/vector_op/position.rs @@ -45,7 +45,7 @@ use risingwave_expr_macro::function; /// ---- /// 4 /// ``` -#[function("strpos(varchar, varchar) -> int32")] // backward compatibility with old proto +#[function("strpos(varchar, varchar) -> int32", deprecated)] #[function("position(varchar, varchar) -> int32")] pub fn position(str: &str, sub_str: &str) -> i32 { match str.find(sub_str) { diff --git a/src/expr/src/window_function/state/mod.rs b/src/expr/src/window_function/state/mod.rs index 59f066f70e2a7..becf633107df6 100644 --- a/src/expr/src/window_function/state/mod.rs +++ b/src/expr/src/window_function/state/mod.rs @@ -126,7 +126,8 @@ pub fn create_window_state(call: &WindowFuncCall) -> Result, pub original_columns: Vec, + + pub created_at_epoch: Option, + + pub initialized_at_epoch: Option, } impl IndexCatalog { @@ -117,6 +122,8 @@ impl IndexCatalog { secondary_to_primary_mapping, function_mapping, original_columns, + created_at_epoch: index_prost.created_at_epoch.map(Epoch::from), + initialized_at_epoch: index_prost.initialized_at_epoch.map(Epoch::from), } } @@ -175,6 +182,8 @@ impl IndexCatalog { .map(|expr| expr.to_expr_proto()) .collect_vec(), original_columns: self.original_columns.iter().map(Into::into).collect_vec(), + initialized_at_epoch: self.initialized_at_epoch.map(|e| e.0), + created_at_epoch: self.created_at_epoch.map(|e| e.0), } } diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index e32b374c5974e..dfcbc3bd9cdd2 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -15,6 +15,7 @@ use std::collections::BTreeMap; use risingwave_common::catalog::ColumnCatalog; +use risingwave_common::util::epoch::Epoch; use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::{PbSource, StreamSourceInfo, WatermarkDesc}; @@ -40,6 +41,8 @@ pub struct SourceCatalog { pub associated_table_id: Option, pub definition: String, pub connection_id: Option, + pub created_at_epoch: Option, + pub initialized_at_epoch: Option, } impl SourceCatalog { @@ -91,6 +94,8 @@ impl From<&PbSource> for SourceCatalog { associated_table_id: associated_table_id.map(|x| x.into()), definition: prost.definition.clone(), connection_id, + created_at_epoch: prost.created_at_epoch.map(Epoch::from), + initialized_at_epoch: prost.initialized_at_epoch.map(Epoch::from), } } } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs index 9a76534e7d7b3..662538e6555bd 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs @@ -43,7 +43,6 @@ pub mod pg_type; pub mod pg_user; pub mod pg_views; -use itertools::Itertools; pub use pg_am::*; pub use pg_attrdef::*; pub use pg_attribute::*; @@ -74,597 +73,3 @@ pub use pg_tablespace::*; pub use pg_type::*; pub use pg_user::*; pub use pg_views::*; -use risingwave_common::array::ListValue; -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::error::{ErrorCode, Result}; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::ScalarImpl; -use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_pb::user::grant_privilege::Object; -use serde_json::json; - -use super::SysCatalogReaderImpl; -use crate::catalog::schema_catalog::SchemaCatalog; -use crate::catalog::system_catalog::get_acl_items; -use crate::user::user_authentication::encrypted_raw_password; - -impl SysCatalogReaderImpl { - pub(super) fn read_types(&self) -> Result> { - let schema_id = self - .catalog_reader - .read_guard() - .get_schema_by_name(&self.auth_context.database, PG_CATALOG_SCHEMA_NAME)? - .id(); - Ok(get_pg_type_data(schema_id)) - } - - pub(super) fn read_cast(&self) -> Result> { - Ok(PG_CAST_DATA_ROWS.clone()) - } - - pub(super) fn read_namespace(&self) -> Result> { - let schemas = self - .catalog_reader - .read_guard() - .get_all_schema_info(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); - Ok(schemas - .iter() - .map(|schema| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(schema.id as i32)), - Some(ScalarImpl::Utf8(schema.name.clone().into())), - Some(ScalarImpl::Int32(schema.owner as i32)), - Some(ScalarImpl::Utf8( - get_acl_items(&Object::SchemaId(schema.id), &users, username_map).into(), - )), - ]) - }) - .collect_vec()) - } - - pub(super) fn read_user_info(&self) -> Result> { - let reader = self.user_info_reader.read_guard(); - let users = reader.get_all_users(); - Ok(users - .iter() - .map(|user| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(user.id as i32)), - Some(ScalarImpl::Utf8(user.name.clone().into())), - Some(ScalarImpl::Bool(user.can_create_db)), - Some(ScalarImpl::Bool(user.is_super)), - // compatible with PG. - Some(ScalarImpl::Utf8("********".into())), - ]) - }) - .collect_vec()) - } - - pub(super) fn read_user_info_shadow(&self) -> Result> { - let reader = self.user_info_reader.read_guard(); - // Since this catalog contains passwords, it must not be publicly readable. - match reader.get_user_by_name(&self.auth_context.user_name) { - None => { - return Err(ErrorCode::CatalogError( - format!("user {} not found", self.auth_context.user_name).into(), - ) - .into()); - } - Some(user) => { - if !user.is_super { - return Err(ErrorCode::PermissionDenied( - "permission denied for table pg_shadow".to_string(), - ) - .into()); - } - } - } - - let users = reader.get_all_users(); - Ok(users - .iter() - .map(|user| { - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(user.name.clone().into())), - Some(ScalarImpl::Int32(user.id as i32)), - Some(ScalarImpl::Bool(user.can_create_db)), - Some(ScalarImpl::Bool(user.is_super)), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Bool(false)), - user.auth_info - .as_ref() - .map(|info| ScalarImpl::Utf8(encrypted_raw_password(info).into())), - None, - None, - ]) - }) - .collect_vec()) - } - - // FIXME(noel): Tracked by - pub(super) fn read_opclass_info(&self) -> Result> { - Ok(vec![]) - } - - // FIXME(noel): Tracked by - pub(super) fn read_operator_info(&self) -> Result> { - Ok(vec![]) - } - - // FIXME(noel): Tracked by - pub(super) fn read_am_info(&self) -> Result> { - Ok(vec![]) - } - - // FIXME(noel): Tracked by - pub(super) fn read_collation_info(&self) -> Result> { - Ok(vec![]) - } - - pub(super) fn read_attrdef_info(&self) -> Result> { - Ok(vec![]) - } - - pub(crate) fn read_shdescription_info(&self) -> Result> { - Ok(vec![]) - } - - pub(crate) fn read_enum_info(&self) -> Result> { - Ok(vec![]) - } - - pub(super) fn read_roles_info(&self) -> Result> { - let reader = self.user_info_reader.read_guard(); - let users = reader.get_all_users(); - Ok(users - .iter() - .map(|user| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(user.id as i32)), - Some(ScalarImpl::Utf8(user.name.clone().into())), - Some(ScalarImpl::Bool(user.is_super)), - Some(ScalarImpl::Bool(true)), - Some(ScalarImpl::Bool(user.can_create_user)), - Some(ScalarImpl::Bool(user.can_create_db)), - Some(ScalarImpl::Bool(user.can_login)), - Some(ScalarImpl::Utf8("********".into())), - ]) - }) - .collect_vec()) - } - - pub(super) fn read_class_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let schema_infos = reader.get_all_schema_info(&self.auth_context.database)?; - - Ok(schemas - .zip_eq_debug(schema_infos.iter()) - .flat_map(|(schema, schema_info)| { - // !!! If we need to add more class types, remember to update - // Catalog::get_id_by_class_name_inner accordingly. - - let rows = schema - .iter_table() - .map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id() as i32)), - Some(ScalarImpl::Utf8(table.name.clone().into())), - Some(ScalarImpl::Int32(schema_info.id as i32)), - Some(ScalarImpl::Int32(table.owner as i32)), - Some(ScalarImpl::Utf8("r".into())), - Some(ScalarImpl::Int32(0)), - Some(ScalarImpl::Int32(0)), - ]) - }) - .collect_vec(); - - let mvs = schema - .iter_mv() - .map(|mv| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(mv.id.table_id() as i32)), - Some(ScalarImpl::Utf8(mv.name.clone().into())), - Some(ScalarImpl::Int32(schema_info.id as i32)), - Some(ScalarImpl::Int32(mv.owner as i32)), - Some(ScalarImpl::Utf8("m".into())), - Some(ScalarImpl::Int32(0)), - Some(ScalarImpl::Int32(0)), - ]) - }) - .collect_vec(); - - let indexes = schema - .iter_index() - .map(|index| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(index.index_table.id.table_id as i32)), - Some(ScalarImpl::Utf8(index.name.clone().into())), - Some(ScalarImpl::Int32(schema_info.id as i32)), - Some(ScalarImpl::Int32(index.index_table.owner as i32)), - Some(ScalarImpl::Utf8("i".into())), - Some(ScalarImpl::Int32(0)), - Some(ScalarImpl::Int32(0)), - ]) - }) - .collect_vec(); - - let sources = schema - .iter_source() - .map(|source| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(source.id as i32)), - Some(ScalarImpl::Utf8(source.name.clone().into())), - Some(ScalarImpl::Int32(schema_info.id as i32)), - Some(ScalarImpl::Int32(source.owner as i32)), - Some(ScalarImpl::Utf8("x".into())), - Some(ScalarImpl::Int32(0)), - Some(ScalarImpl::Int32(0)), - ]) - }) - .collect_vec(); - - let sys_tables = schema - .iter_system_tables() - .map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id() as i32)), - Some(ScalarImpl::Utf8(table.name.clone().into())), - Some(ScalarImpl::Int32(schema_info.id as i32)), - Some(ScalarImpl::Int32(table.owner as i32)), - Some(ScalarImpl::Utf8("r".into())), - Some(ScalarImpl::Int32(0)), - Some(ScalarImpl::Int32(0)), - ]) - }) - .collect_vec(); - - let views = schema - .iter_view() - .map(|view| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(view.id as i32)), - Some(ScalarImpl::Utf8(view.name().into())), - Some(ScalarImpl::Int32(schema_info.id as i32)), - Some(ScalarImpl::Int32(view.owner as i32)), - Some(ScalarImpl::Utf8("v".into())), - Some(ScalarImpl::Int32(0)), - Some(ScalarImpl::Int32(0)), - ]) - }) - .collect_vec(); - - let internal_tables = schema - .iter_internal_table() - .map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id() as i32)), - Some(ScalarImpl::Utf8(table.name.clone().into())), - Some(ScalarImpl::Int32(schema_info.id as i32)), - Some(ScalarImpl::Int32(table.owner as i32)), - Some(ScalarImpl::Utf8("n".into())), - Some(ScalarImpl::Int32(0)), - Some(ScalarImpl::Int32(0)), - ]) - }) - .collect_vec(); - - rows.into_iter() - .chain(mvs.into_iter()) - .chain(indexes.into_iter()) - .chain(sources.into_iter()) - .chain(sys_tables.into_iter()) - .chain(views.into_iter()) - .chain(internal_tables.into_iter()) - .collect_vec() - }) - .collect_vec()) - } - - pub(super) fn read_index_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - - Ok(schemas - .flat_map(|schema| { - schema.iter_index().map(|index| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(index.id.index_id() as i32)), - Some(ScalarImpl::Int32(index.primary_table.id.table_id() as i32)), - Some(ScalarImpl::Int16(index.original_columns.len() as i16)), - Some(ScalarImpl::List(ListValue::new( - index - .original_columns - .iter() - .map(|index| Some(ScalarImpl::Int16(index.get_id() as i16 + 1))) - .collect_vec(), - ))), - None, - None, - ]) - }) - }) - .collect_vec()) - } - - pub(super) async fn read_mviews_info(&self) -> Result> { - let mut table_ids = Vec::new(); - { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.get_all_schema_names(&self.auth_context.database)?; - for schema in &schemas { - reader - .get_schema_by_name(&self.auth_context.database, schema)? - .iter_mv() - .for_each(|t| { - table_ids.push(t.id.table_id); - }); - } - } - - let table_fragments = self.meta_client.list_table_fragments(&table_ids).await?; - let mut rows = Vec::new(); - let reader = self.catalog_reader.read_guard(); - let schemas = reader.get_all_schema_names(&self.auth_context.database)?; - for schema in &schemas { - reader - .get_schema_by_name(&self.auth_context.database, schema)? - .iter_mv() - .for_each(|t| { - if let Some(fragments) = table_fragments.get(&t.id.table_id) { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.owner as i32)), - Some(ScalarImpl::Utf8(t.definition.clone().into())), - Some(ScalarImpl::Int32(t.id.table_id as i32)), - Some(ScalarImpl::Utf8( - fragments.get_env().unwrap().get_timezone().clone().into(), - )), - Some(ScalarImpl::Utf8( - json!(fragments.get_fragments()).to_string().into(), - )), - ])); - } - }); - } - - Ok(rows) - } - - pub(super) fn read_views_info(&self) -> Result> { - // TODO(zehua): solve the deadlock problem. - // Get two read locks. The order must be the same as - // `FrontendObserverNode::handle_initialization_notification`. - let catalog_reader = self.catalog_reader.read_guard(); - let user_info_reader = self.user_info_reader.read_guard(); - let schemas = catalog_reader.iter_schemas(&self.auth_context.database)?; - - Ok(schemas - .flat_map(|schema| { - schema.iter_view().map(|view| { - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.name().into())), - Some(ScalarImpl::Utf8(view.name().into())), - Some(ScalarImpl::Utf8( - user_info_reader - .get_user_name_by_id(view.owner) - .unwrap() - .into(), - )), - // TODO(zehua): may be not same as postgresql's "definition" column. - Some(ScalarImpl::Utf8(view.sql.clone().into())), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_indexes_info(&self) -> Result> { - let catalog_reader = self.catalog_reader.read_guard(); - let schemas = catalog_reader.iter_schemas(&self.auth_context.database)?; - - Ok(schemas - .flat_map(|schema: &SchemaCatalog| { - schema.iter_index().map(|index| { - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.name().into())), - Some(ScalarImpl::Utf8(index.primary_table.name.clone().into())), - Some(ScalarImpl::Utf8(index.index_table.name.clone().into())), - None, - Some(ScalarImpl::Utf8(index.index_table.create_sql().into())), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_pg_attribute(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - - Ok(schemas - .flat_map(|schema| { - let view_rows = schema.iter_view().flat_map(|view| { - view.columns.iter().enumerate().map(|(index, column)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(view.id as i32)), - Some(ScalarImpl::Utf8(column.name.clone().into())), - Some(ScalarImpl::Int32(column.data_type().to_oid())), - Some(ScalarImpl::Int16(column.data_type().type_len())), - Some(ScalarImpl::Int16(index as i16 + 1)), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Utf8("".into())), - Some(ScalarImpl::Utf8("".into())), - // From https://www.postgresql.org/docs/current/catalog-pg-attribute.html - // The value will generally be -1 for types that do not need - // `atttypmod`. - Some(ScalarImpl::Int32(-1)), - ]) - }) - }); - - schema - .iter_valid_table() - .flat_map(|table| { - table - .columns() - .iter() - .enumerate() - .filter(|(_, column)| !column.is_hidden()) - .map(|(index, column)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id() as i32)), - Some(ScalarImpl::Utf8(column.name().into())), - Some(ScalarImpl::Int32(column.data_type().to_oid())), - Some(ScalarImpl::Int16(column.data_type().type_len())), - Some(ScalarImpl::Int16(index as i16 + 1)), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Utf8("".into())), - Some(ScalarImpl::Utf8("".into())), - // From https://www.postgresql.org/docs/current/catalog-pg-attribute.html - // The value will generally be -1 for types that do not need - // `atttypmod`. - Some(ScalarImpl::Int32(-1)), - ]) - }) - }) - .chain(view_rows) - }) - .collect_vec()) - } - - pub(super) fn read_database_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let databases = reader.get_all_database_names(); - - Ok(databases - .iter() - .map(|db| new_pg_database_row(reader.get_database_by_name(db).unwrap().id(), db)) - .collect_vec()) - } - - pub(super) fn read_description_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - - Ok(schemas - .flat_map(|schema| { - let rows = schema - .iter_table() - .map(|table| new_pg_description_row(table.id().table_id)) - .collect_vec(); - - let mvs = schema - .iter_mv() - .map(|mv| new_pg_description_row(mv.id().table_id)) - .collect_vec(); - - let indexes = schema - .iter_index() - .map(|index| new_pg_description_row(index.id.index_id())) - .collect_vec(); - - let sources = schema - .iter_source() - .map(|source| new_pg_description_row(source.id)) - .collect_vec(); - - let sys_tables = schema - .iter_system_tables() - .map(|table| new_pg_description_row(table.id().table_id)) - .collect_vec(); - - let views = schema - .iter_view() - .map(|view| new_pg_description_row(view.id)) - .collect_vec(); - - rows.into_iter() - .chain(mvs.into_iter()) - .chain(indexes.into_iter()) - .chain(sources.into_iter()) - .chain(sys_tables.into_iter()) - .chain(views.into_iter()) - .collect_vec() - }) - .collect_vec()) - } - - pub(super) fn read_settings_info(&self) -> Result> { - Ok(PG_SETTINGS_DATA_ROWS.clone()) - } - - pub(super) fn read_keywords_info(&self) -> Result> { - Ok(PG_KEYWORDS_DATA_ROWS.clone()) - } - - pub(super) fn read_tablespace_info(&self) -> Result> { - Ok(PG_TABLESPACE_DATA_ROWS.clone()) - } - - pub(crate) fn read_conversion_info(&self) -> Result> { - Ok(vec![]) - } - - pub(super) fn read_stat_activity(&self) -> Result> { - Ok(vec![]) - } - - pub(super) fn read_inherits_info(&self) -> Result> { - Ok(PG_INHERITS_DATA_ROWS.clone()) - } - - pub(super) fn read_constraint_info(&self) -> Result> { - Ok(PG_CONSTRAINT_DATA_ROWS.clone()) - } - - pub(crate) fn read_pg_proc_info(&self) -> Result> { - Ok(PG_PROC_DATA_ROWS.clone()) - } - - pub(crate) fn read_pg_tables_info(&self) -> Result> { - // TODO: avoid acquire two read locks here. The order is the same as in `read_views_info`. - let reader = self.catalog_reader.read_guard(); - let user_info_reader = self.user_info_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - - Ok(schemas - .flat_map(|schema| { - schema - .iter_table() - .map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.name().into())), - Some(ScalarImpl::Utf8(table.name().into())), - Some(ScalarImpl::Utf8( - user_info_reader - .get_user_name_by_id(table.owner) - .unwrap() - .into(), - )), - None, - ]) - }) - .chain(schema.iter_system_tables().map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.name().into())), - Some(ScalarImpl::Utf8(table.name().into())), - Some(ScalarImpl::Utf8( - user_info_reader - .get_user_name_by_id(table.owner) - .unwrap() - .into(), - )), - None, - ]) - })) - }) - .collect_vec()) - } -} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_am.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_am.rs index 03fab1031d4e5..c91ba685babee 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_am.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_am.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// Stores information about relation access methods. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-am.html`] @@ -25,3 +27,9 @@ pub const PG_AM_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "amhandler"), (DataType::Varchar, "amtype"), ]; + +impl SysCatalogReaderImpl { + pub fn read_am_info(&self) -> Result> { + Ok(vec![]) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attrdef.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attrdef.rs index 6dbf42f9305cf..1865c2f142d67 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attrdef.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attrdef.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_attrdef` stores column default values. The main information about columns is /// stored in `pg_attribute`. Only columns for which a default value has been explicitly set will @@ -28,3 +30,9 @@ pub const PG_ATTRDEF_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ // expression. We don't have `pg_node_tree` type yet, so we use `text` instead. (DataType::Varchar, "adbin"), ]; + +impl SysCatalogReaderImpl { + pub fn read_attrdef_info(&self) -> Result> { + Ok(vec![]) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs index 4c9eb1a4c995e..8d1bf240c2cd6 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_attribute` stores information about table columns. There will be exactly one /// `pg_attribute` row for every column in every table in the database. (There will also be @@ -36,3 +39,62 @@ pub const PG_ATTRIBUTE_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Varchar, "attgenerated"), (DataType::Int32, "atttypmod"), ]; + +impl SysCatalogReaderImpl { + pub fn read_pg_attribute(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + + Ok(schemas + .flat_map(|schema| { + let view_rows = schema.iter_view().flat_map(|view| { + view.columns.iter().enumerate().map(|(index, column)| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(view.id as i32)), + Some(ScalarImpl::Utf8(column.name.clone().into())), + Some(ScalarImpl::Int32(column.data_type().to_oid())), + Some(ScalarImpl::Int16(column.data_type().type_len())), + Some(ScalarImpl::Int16(index as i16 + 1)), + Some(ScalarImpl::Bool(false)), + Some(ScalarImpl::Bool(false)), + Some(ScalarImpl::Utf8("".into())), + Some(ScalarImpl::Utf8("".into())), + // From https://www.postgresql.org/docs/current/catalog-pg-attribute.html + // The value will generally be -1 for types that do not need + // `atttypmod`. + Some(ScalarImpl::Int32(-1)), + ]) + }) + }); + + schema + .iter_valid_table() + .flat_map(|table| { + table + .columns() + .iter() + .enumerate() + .filter(|(_, column)| !column.is_hidden()) + .map(|(index, column)| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table.id.table_id() as i32)), + Some(ScalarImpl::Utf8(column.name().into())), + Some(ScalarImpl::Int32(column.data_type().to_oid())), + Some(ScalarImpl::Int16(column.data_type().type_len())), + Some(ScalarImpl::Int16(index as i16 + 1)), + Some(ScalarImpl::Bool(false)), + Some(ScalarImpl::Bool(false)), + Some(ScalarImpl::Utf8("".into())), + Some(ScalarImpl::Utf8("".into())), + // From https://www.postgresql.org/docs/current/catalog-pg-attribute.html + // The value will generally be -1 for types that do not need + // `atttypmod`. + Some(ScalarImpl::Int32(-1)), + ]) + }) + }) + .chain(view_rows) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs index 762584c99e968..44affbcbaeeba 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs @@ -15,10 +15,11 @@ use std::sync::LazyLock; use itertools::Itertools; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; use crate::expr::cast_map_array; /// The catalog `pg_cast` stores data type conversion paths. @@ -47,3 +48,9 @@ pub static PG_CAST_DATA_ROWS: LazyLock> = LazyLock::new(|| { }) .collect_vec() }); + +impl SysCatalogReaderImpl { + pub fn read_cast(&self) -> Result> { + Ok(PG_CAST_DATA_ROWS.clone()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs index 7961ab41c9fbf..9d1d1b5154e67 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs @@ -12,9 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::util::iter_util::ZipEqDebug; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_class` catalogs tables and most everything else that has columns or is otherwise /// similar to a table. Ref: [`https://www.postgresql.org/docs/current/catalog-pg-class.html`] @@ -30,3 +34,133 @@ pub const PG_CLASS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ // 0 (DataType::Int32, "reltablespace"), ]; + +impl SysCatalogReaderImpl { + pub fn read_class_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + let schema_infos = reader.get_all_schema_info(&self.auth_context.database)?; + + Ok(schemas + .zip_eq_debug(schema_infos.iter()) + .flat_map(|(schema, schema_info)| { + // !!! If we need to add more class types, remember to update + // Catalog::get_id_by_class_name_inner accordingly. + + let rows = schema + .iter_table() + .map(|table| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table.id.table_id() as i32)), + Some(ScalarImpl::Utf8(table.name.clone().into())), + Some(ScalarImpl::Int32(schema_info.id as i32)), + Some(ScalarImpl::Int32(table.owner as i32)), + Some(ScalarImpl::Utf8("r".into())), + Some(ScalarImpl::Int32(0)), + Some(ScalarImpl::Int32(0)), + ]) + }) + .collect_vec(); + + let mvs = schema + .iter_mv() + .map(|mv| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(mv.id.table_id() as i32)), + Some(ScalarImpl::Utf8(mv.name.clone().into())), + Some(ScalarImpl::Int32(schema_info.id as i32)), + Some(ScalarImpl::Int32(mv.owner as i32)), + Some(ScalarImpl::Utf8("m".into())), + Some(ScalarImpl::Int32(0)), + Some(ScalarImpl::Int32(0)), + ]) + }) + .collect_vec(); + + let indexes = schema + .iter_index() + .map(|index| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(index.index_table.id.table_id as i32)), + Some(ScalarImpl::Utf8(index.name.clone().into())), + Some(ScalarImpl::Int32(schema_info.id as i32)), + Some(ScalarImpl::Int32(index.index_table.owner as i32)), + Some(ScalarImpl::Utf8("i".into())), + Some(ScalarImpl::Int32(0)), + Some(ScalarImpl::Int32(0)), + ]) + }) + .collect_vec(); + + let sources = schema + .iter_source() + .map(|source| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(source.id as i32)), + Some(ScalarImpl::Utf8(source.name.clone().into())), + Some(ScalarImpl::Int32(schema_info.id as i32)), + Some(ScalarImpl::Int32(source.owner as i32)), + Some(ScalarImpl::Utf8("x".into())), + Some(ScalarImpl::Int32(0)), + Some(ScalarImpl::Int32(0)), + ]) + }) + .collect_vec(); + + let sys_tables = schema + .iter_system_tables() + .map(|table| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table.id.table_id() as i32)), + Some(ScalarImpl::Utf8(table.name.clone().into())), + Some(ScalarImpl::Int32(schema_info.id as i32)), + Some(ScalarImpl::Int32(table.owner as i32)), + Some(ScalarImpl::Utf8("r".into())), + Some(ScalarImpl::Int32(0)), + Some(ScalarImpl::Int32(0)), + ]) + }) + .collect_vec(); + + let views = schema + .iter_view() + .map(|view| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(view.id as i32)), + Some(ScalarImpl::Utf8(view.name().into())), + Some(ScalarImpl::Int32(schema_info.id as i32)), + Some(ScalarImpl::Int32(view.owner as i32)), + Some(ScalarImpl::Utf8("v".into())), + Some(ScalarImpl::Int32(0)), + Some(ScalarImpl::Int32(0)), + ]) + }) + .collect_vec(); + + let internal_tables = schema + .iter_internal_table() + .map(|table| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table.id.table_id() as i32)), + Some(ScalarImpl::Utf8(table.name.clone().into())), + Some(ScalarImpl::Int32(schema_info.id as i32)), + Some(ScalarImpl::Int32(table.owner as i32)), + Some(ScalarImpl::Utf8("n".into())), + Some(ScalarImpl::Int32(0)), + Some(ScalarImpl::Int32(0)), + ]) + }) + .collect_vec(); + + rows.into_iter() + .chain(mvs.into_iter()) + .chain(indexes.into_iter()) + .chain(sources.into_iter()) + .chain(sys_tables.into_iter()) + .chain(views.into_iter()) + .chain(internal_tables.into_iter()) + .collect_vec() + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_collation.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_collation.rs index bf9aef2afa0d5..1bef96afe58f8 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_collation.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_collation.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// Mapping from sql name to system locale groups. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-collation.html`]. @@ -32,3 +34,9 @@ pub const PG_COLLATION_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Varchar, "colliculocale"), (DataType::Varchar, "collversion"), ]; + +impl SysCatalogReaderImpl { + pub fn read_collation_info(&self) -> Result> { + Ok(vec![]) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs index c1fba384548cf..411a13dbbbc65 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs @@ -14,10 +14,11 @@ use std::sync::LazyLock; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_constraint` records information about table and index inheritance hierarchies. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-constraint.html`] @@ -54,3 +55,9 @@ pub static PG_CONSTRAINT_COLUMNS: LazyLock>> = }); pub static PG_CONSTRAINT_DATA_ROWS: LazyLock> = LazyLock::new(Vec::new); + +impl SysCatalogReaderImpl { + pub fn read_constraint_info(&self) -> Result> { + Ok(PG_CONSTRAINT_DATA_ROWS.clone()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_conversion.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_conversion.rs index d8d73f934d442..974df3eac2921 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_conversion.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_conversion.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_conversion` describes encoding conversion functions. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-conversion.html`] @@ -29,3 +31,9 @@ pub const PG_CONVERSION_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "conproc"), (DataType::Boolean, "condefault"), ]; + +impl SysCatalogReaderImpl { + pub(crate) fn read_conversion_info(&self) -> Result> { + Ok(vec![]) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs index f6f54b7eac303..1098b953ccde5 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs @@ -12,10 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use itertools::Itertools; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_database` stores database. /// @@ -74,3 +76,15 @@ pub fn new_pg_database_row(id: u32, name: &str) -> OwnedRow { None, ]) } + +impl SysCatalogReaderImpl { + pub fn read_database_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let databases = reader.get_all_database_names(); + + Ok(databases + .iter() + .map(|db| new_pg_database_row(reader.get_database_by_name(db).unwrap().id(), db)) + .collect_vec()) + } +} 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 120a05e12f385..cc9afb6f550a9 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 @@ -12,10 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use itertools::Itertools; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_description` stores description. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-description.html`] @@ -38,3 +40,52 @@ pub fn new_pg_description_row(id: u32) -> OwnedRow { None, ]) } + +impl SysCatalogReaderImpl { + pub fn read_description_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + + Ok(schemas + .flat_map(|schema| { + let rows = schema + .iter_table() + .map(|table| new_pg_description_row(table.id().table_id)) + .collect_vec(); + + let mvs = schema + .iter_mv() + .map(|mv| new_pg_description_row(mv.id().table_id)) + .collect_vec(); + + let indexes = schema + .iter_index() + .map(|index| new_pg_description_row(index.id.index_id())) + .collect_vec(); + + let sources = schema + .iter_source() + .map(|source| new_pg_description_row(source.id)) + .collect_vec(); + + let sys_tables = schema + .iter_system_tables() + .map(|table| new_pg_description_row(table.id().table_id)) + .collect_vec(); + + let views = schema + .iter_view() + .map(|view| new_pg_description_row(view.id)) + .collect_vec(); + + rows.into_iter() + .chain(mvs.into_iter()) + .chain(indexes.into_iter()) + .chain(sources.into_iter()) + .chain(sys_tables.into_iter()) + .chain(views.into_iter()) + .collect_vec() + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_enum.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_enum.rs index 4c12c051dd4a6..3cc466f594df0 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_enum.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_enum.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The `pg_enum` catalog contains entries showing the values and labels for each enum type. /// The internal representation of a given enum value is actually the OID of its associated row in @@ -26,3 +28,9 @@ pub const PG_ENUM_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Float32, "enumsortorder"), (DataType::Varchar, "enumlabel"), ]; + +impl SysCatalogReaderImpl { + pub fn read_enum_info(&self) -> Result> { + Ok(vec![]) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs index 23bded4c99e0f..100bfc5f106dd 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs @@ -14,9 +14,13 @@ use std::sync::LazyLock; -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::array::ListValue; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_index` contains part of the information about indexes. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-index.html`] @@ -33,3 +37,31 @@ pub static PG_INDEX_COLUMNS: LazyLock>> = LazyLo (DataType::Varchar, "indpred"), ] }); + +impl SysCatalogReaderImpl { + pub fn read_index_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + + Ok(schemas + .flat_map(|schema| { + schema.iter_index().map(|index| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(index.id.index_id() as i32)), + Some(ScalarImpl::Int32(index.primary_table.id.table_id() as i32)), + Some(ScalarImpl::Int16(index.original_columns.len() as i16)), + Some(ScalarImpl::List(ListValue::new( + index + .original_columns + .iter() + .map(|index| Some(ScalarImpl::Int16(index.get_id() as i16 + 1))) + .collect_vec(), + ))), + None, + None, + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs index 310167391f86d..58d9b86ce8e55 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs @@ -12,9 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::schema_catalog::SchemaCatalog; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The view `pg_indexes` provides access to useful information about each index in the database. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-indexes.html`] @@ -26,3 +30,24 @@ pub const PG_INDEXES_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Varchar, "tablespace"), (DataType::Varchar, "indexdef"), ]; + +impl SysCatalogReaderImpl { + pub fn read_indexes_info(&self) -> Result> { + let catalog_reader = self.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&self.auth_context.database)?; + + Ok(schemas + .flat_map(|schema: &SchemaCatalog| { + schema.iter_index().map(|index| { + OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.name().into())), + Some(ScalarImpl::Utf8(index.primary_table.name.clone().into())), + Some(ScalarImpl::Utf8(index.index_table.name.clone().into())), + None, + Some(ScalarImpl::Utf8(index.index_table.create_sql().into())), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_inherits.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_inherits.rs index 0eb2a4546f2d6..8dde3392e3220 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_inherits.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_inherits.rs @@ -14,10 +14,11 @@ use std::sync::LazyLock; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_inherits` records information about table and index inheritance hierarchies. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-inherits.html`] @@ -31,3 +32,9 @@ pub const PG_INHERITS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ ]; pub static PG_INHERITS_DATA_ROWS: LazyLock> = LazyLock::new(Vec::new); + +impl SysCatalogReaderImpl { + pub fn read_inherits_info(&self) -> Result> { + Ok(PG_INHERITS_DATA_ROWS.clone()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_keywords.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_keywords.rs index 17e22243a939e..a16d5d31907a1 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_keywords.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_keywords.rs @@ -14,10 +14,11 @@ use std::sync::LazyLock; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_keywords` stores keywords. `pg_get_keywords` returns the content of this table. /// Ref: [`https://www.postgresql.org/docs/15/functions-info.html`] @@ -31,3 +32,9 @@ pub const PG_KEYWORDS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ // TODO: set reserved keywords here pub static PG_KEYWORDS_DATA_ROWS: LazyLock> = LazyLock::new(Vec::new); + +impl SysCatalogReaderImpl { + pub fn read_keywords_info(&self) -> Result> { + Ok(PG_KEYWORDS_DATA_ROWS.clone()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_matviews.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_matviews.rs index 3f7f700afdd43..6f2afd40fbd2f 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_matviews.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_matviews.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use serde_json::json; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The view `pg_matviews` provides access to useful information about each materialized view in the /// database. @@ -33,3 +36,50 @@ pub const PG_MATVIEWS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ * dates/timestamps as tstz */ (DataType::Varchar, "matviewgraph"), // materialized view graph is json encoded fragment infos. ]; + +impl SysCatalogReaderImpl { + pub async fn read_mviews_info(&self) -> Result> { + let mut table_ids = Vec::new(); + { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.get_all_schema_names(&self.auth_context.database)?; + for schema in &schemas { + reader + .get_schema_by_name(&self.auth_context.database, schema)? + .iter_mv() + .for_each(|t| { + table_ids.push(t.id.table_id); + }); + } + } + + let table_fragments = self.meta_client.list_table_fragments(&table_ids).await?; + let mut rows = Vec::new(); + let reader = self.catalog_reader.read_guard(); + let schemas = reader.get_all_schema_names(&self.auth_context.database)?; + for schema in &schemas { + reader + .get_schema_by_name(&self.auth_context.database, schema)? + .iter_mv() + .for_each(|t| { + if let Some(fragments) = table_fragments.get(&t.id.table_id) { + rows.push(OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.clone().into())), + Some(ScalarImpl::Utf8(t.name.clone().into())), + Some(ScalarImpl::Int32(t.owner as i32)), + Some(ScalarImpl::Utf8(t.definition.clone().into())), + Some(ScalarImpl::Int32(t.id.table_id as i32)), + Some(ScalarImpl::Utf8( + fragments.get_env().unwrap().get_timezone().clone().into(), + )), + Some(ScalarImpl::Utf8( + json!(fragments.get_fragments()).to_string().into(), + )), + ])); + } + }); + } + + Ok(rows) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs index ea87470d287f5..22f00a3522357 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs @@ -12,9 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{ + get_acl_items, SysCatalogReaderImpl, SystemCatalogColumnsDef, +}; /// The catalog `pg_namespace` stores namespaces. A namespace is the structure underlying SQL /// schemas: each namespace can have a separate collection of relations, types, etc. without name @@ -26,3 +32,28 @@ pub const PG_NAMESPACE_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "nspowner"), (DataType::Varchar, "nspacl"), ]; + +impl SysCatalogReaderImpl { + pub fn read_namespace(&self) -> Result> { + let schemas = self + .catalog_reader + .read_guard() + .get_all_schema_info(&self.auth_context.database)?; + let user_reader = self.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); + Ok(schemas + .iter() + .map(|schema| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(schema.id as i32)), + Some(ScalarImpl::Utf8(schema.name.clone().into())), + Some(ScalarImpl::Int32(schema.owner as i32)), + Some(ScalarImpl::Utf8( + get_acl_items(&Object::SchemaId(schema.id), &users, username_map).into(), + )), + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_opclass.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_opclass.rs index 600efa8322a4c..76d69b240cd23 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_opclass.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_opclass.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_opclass` defines index access method operator classes. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-opclass.html`]. @@ -30,3 +32,9 @@ pub const PG_OPCLASS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "opcdefault"), (DataType::Int32, "opckeytype"), ]; + +impl SysCatalogReaderImpl { + pub fn read_opclass_info(&self) -> Result> { + Ok(vec![]) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_operator.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_operator.rs index a1e48fe1b23c3..cc8bc3e0cb5e7 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_operator.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_operator.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_operator` stores operator info. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-operator.html`] @@ -36,3 +38,9 @@ pub const PG_OPERATOR_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "oprrest"), (DataType::Int32, "oprjoin"), ]; + +impl SysCatalogReaderImpl { + pub fn read_operator_info(&self) -> Result> { + Ok(vec![]) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_proc.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_proc.rs index 32935c742ace3..ac004ebee8458 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_proc.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_proc.rs @@ -14,10 +14,11 @@ use std::sync::LazyLock; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_proc` stores information about functions, procedures, aggregate functions, and /// window functions (collectively also known as routines). @@ -35,3 +36,9 @@ pub const PG_PROC_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ // TODO: read real data including oid etc in rw, currently there are no such data in rw. // more details can be found here: https://github.com/postgres/postgres/blob/master/src/include/catalog/pg_proc.dat pub static PG_PROC_DATA_ROWS: LazyLock> = LazyLock::new(Vec::new); + +impl SysCatalogReaderImpl { + pub fn read_pg_proc_info(&self) -> Result> { + Ok(PG_PROC_DATA_ROWS.clone()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_roles.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_roles.rs index 48a770a0b2336..abbd62b5577a6 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_roles.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_roles.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_roles` provides access to information about database roles. This is simply a /// publicly readable view of `pg_authid` that blanks out the password field. @@ -30,3 +33,25 @@ pub const PG_ROLES_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Boolean, "rolcanlogin"), (DataType::Varchar, "rolpassword"), ]; + +impl SysCatalogReaderImpl { + pub fn read_roles_info(&self) -> Result> { + let reader = self.user_info_reader.read_guard(); + let users = reader.get_all_users(); + Ok(users + .iter() + .map(|user| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(user.id as i32)), + Some(ScalarImpl::Utf8(user.name.clone().into())), + Some(ScalarImpl::Bool(user.is_super)), + Some(ScalarImpl::Bool(true)), + Some(ScalarImpl::Bool(user.can_create_user)), + Some(ScalarImpl::Bool(user.can_create_db)), + Some(ScalarImpl::Bool(user.can_login)), + Some(ScalarImpl::Utf8("********".into())), + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_settings.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_settings.rs index 6e335d4c44869..773e8c998e27e 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_settings.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_settings.rs @@ -14,10 +14,11 @@ use std::sync::LazyLock; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_settings` stores settings. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-settings.html`] @@ -26,3 +27,9 @@ pub const PG_SETTINGS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[(DataType::Varchar, "name"), (DataType::Varchar, "setting")]; pub static PG_SETTINGS_DATA_ROWS: LazyLock> = LazyLock::new(Vec::new); + +impl SysCatalogReaderImpl { + pub fn read_settings_info(&self) -> Result> { + Ok(PG_SETTINGS_DATA_ROWS.clone()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shadow.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shadow.rs index dd0da796bce56..4044fe70aae1b 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shadow.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shadow.rs @@ -14,9 +14,13 @@ use std::sync::LazyLock; -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; +use crate::user::user_authentication::encrypted_raw_password; /// The view `pg_shadow` exists for backwards compatibility: it emulates a catalog that existed in /// PostgreSQL before version 8.1. It shows properties of all roles that are marked as rolcanlogin @@ -39,3 +43,46 @@ pub static PG_SHADOW_COLUMNS: LazyLock>> = LazyL (DataType::List(Box::new(DataType::Varchar)), "useconfig"), ] }); + +impl SysCatalogReaderImpl { + pub fn read_user_info_shadow(&self) -> Result> { + let reader = self.user_info_reader.read_guard(); + // Since this catalog contains passwords, it must not be publicly readable. + match reader.get_user_by_name(&self.auth_context.user_name) { + None => { + return Err(ErrorCode::CatalogError( + format!("user {} not found", self.auth_context.user_name).into(), + ) + .into()); + } + Some(user) => { + if !user.is_super { + return Err(ErrorCode::PermissionDenied( + "permission denied for table pg_shadow".to_string(), + ) + .into()); + } + } + } + + let users = reader.get_all_users(); + Ok(users + .iter() + .map(|user| { + OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(user.name.clone().into())), + Some(ScalarImpl::Int32(user.id as i32)), + Some(ScalarImpl::Bool(user.can_create_db)), + Some(ScalarImpl::Bool(user.is_super)), + Some(ScalarImpl::Bool(false)), + Some(ScalarImpl::Bool(false)), + user.auth_info + .as_ref() + .map(|info| ScalarImpl::Utf8(encrypted_raw_password(info).into())), + None, + None, + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shdescription.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shdescription.rs index 43c8fa75956ab..f572fa1ce3430 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shdescription.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shdescription.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_shdescription` stores optional descriptions (comments) for shared database /// objects. Ref: [`https://www.postgresql.org/docs/current/catalog-pg-shdescription.html`] @@ -24,3 +26,9 @@ pub const PG_SHDESCRIPTION_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "classoid"), (DataType::Varchar, "description"), ]; + +impl SysCatalogReaderImpl { + pub fn read_shdescription_info(&self) -> Result> { + Ok(vec![]) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_stat_activity.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_stat_activity.rs index 68a261df628d8..f9be898b01ae0 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_stat_activity.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_stat_activity.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The `pg_stat_activity` view will have one row per server process, showing information related to /// the current activity of that process. Ref: [`https://www.postgresql.org/docs/current/monitoring-stats.html#MONITORING-PG-STAT-ACTIVITY-VIEW`] @@ -38,3 +40,9 @@ pub const PG_STAT_ACTIVITY_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ * communication with this backend, or -1 if a Unix socket * is used. */ ]; + +impl SysCatalogReaderImpl { + pub fn read_stat_activity(&self) -> Result> { + Ok(vec![]) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tables.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tables.rs index 3420eb887be61..e0132aeb0dba9 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tables.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tables.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The view `pg_tables` provides access to useful information about each table in the database. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-tables.html`] @@ -27,3 +30,45 @@ pub const PG_TABLES_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Varchar, "tablespace"), /* Since we don't have any concept of tablespace, we will * set this to null. */ ]; + +impl SysCatalogReaderImpl { + pub fn read_pg_tables_info(&self) -> Result> { + // TODO: avoid acquire two read locks here. The order is the same as in `read_views_info`. + let reader = self.catalog_reader.read_guard(); + let user_info_reader = self.user_info_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + + Ok(schemas + .flat_map(|schema| { + schema + .iter_table() + .map(|table| { + OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.name().into())), + Some(ScalarImpl::Utf8(table.name().into())), + Some(ScalarImpl::Utf8( + user_info_reader + .get_user_name_by_id(table.owner) + .unwrap() + .into(), + )), + None, + ]) + }) + .chain(schema.iter_system_tables().map(|table| { + OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.name().into())), + Some(ScalarImpl::Utf8(table.name().into())), + Some(ScalarImpl::Utf8( + user_info_reader + .get_user_name_by_id(table.owner) + .unwrap() + .into(), + )), + None, + ]) + })) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs index 87f97622515f0..e94eb09b0ca05 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs @@ -14,10 +14,11 @@ use std::sync::LazyLock; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_tablespace` stores information about the available tablespaces. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-tablespace.html`] @@ -32,3 +33,9 @@ pub const PG_TABLESPACE_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ ]; pub static PG_TABLESPACE_DATA_ROWS: LazyLock> = LazyLock::new(Vec::new); + +impl SysCatalogReaderImpl { + pub fn read_tablespace_info(&self) -> Result> { + Ok(PG_TABLESPACE_DATA_ROWS.clone()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs index 06142e626768d..e200b6951a71d 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs @@ -13,10 +13,12 @@ // limitations under the License. use itertools::Itertools; +use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; +use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_type` stores information about data types. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-type.html`] @@ -97,3 +99,14 @@ pub fn get_pg_type_data(namespace_id: u32) -> Vec { }) .collect_vec() } + +impl SysCatalogReaderImpl { + pub fn read_types(&self) -> Result> { + let schema_id = self + .catalog_reader + .read_guard() + .get_schema_by_name(&self.auth_context.database, PG_CATALOG_SCHEMA_NAME)? + .id(); + Ok(get_pg_type_data(schema_id)) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_user.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_user.rs index 9d88a4ceacf61..0cc98e92d2aa1 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_user.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_user.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The catalog `pg_user` provides access to information about database users. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-user.html`] @@ -29,3 +32,23 @@ pub const PG_USER_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Boolean, "usesuper"), (DataType::Varchar, "passwd"), ]; + +impl SysCatalogReaderImpl { + pub fn read_user_info(&self) -> Result> { + let reader = self.user_info_reader.read_guard(); + let users = reader.get_all_users(); + Ok(users + .iter() + .map(|user| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(user.id as i32)), + Some(ScalarImpl::Utf8(user.name.clone().into())), + Some(ScalarImpl::Bool(user.can_create_db)), + Some(ScalarImpl::Bool(user.is_super)), + // compatible with PG. + Some(ScalarImpl::Utf8("********".into())), + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_views.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_views.rs index ab792e9da2867..dd10628278f69 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_views.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_views.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// The view `pg_views` provides access to useful information about each view in the database. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-views.html`] @@ -27,3 +30,33 @@ pub const PG_VIEWS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Varchar, "viewowner"), (DataType::Varchar, "definition"), ]; + +impl SysCatalogReaderImpl { + pub fn read_views_info(&self) -> Result> { + // TODO(zehua): solve the deadlock problem. + // Get two read locks. The order must be the same as + // `FrontendObserverNode::handle_initialization_notification`. + let catalog_reader = self.catalog_reader.read_guard(); + let user_info_reader = self.user_info_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&self.auth_context.database)?; + + Ok(schemas + .flat_map(|schema| { + schema.iter_view().map(|view| { + OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.name().into())), + Some(ScalarImpl::Utf8(view.name().into())), + Some(ScalarImpl::Utf8( + user_info_reader + .get_user_name_by_id(view.owner) + .unwrap() + .into(), + )), + // TODO(zehua): may be not same as postgresql's "definition" column. + Some(ScalarImpl::Utf8(view.sql.clone().into())), + ]) + }) + }) + .collect_vec()) + } +} 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 574326cfb7857..d7988107ccd60 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs @@ -33,14 +33,6 @@ mod rw_users; mod rw_views; mod rw_worker_nodes; -use itertools::Itertools; -use risingwave_common::array::ListValue; -use risingwave_common::error::Result; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{ScalarImpl, Timestamp}; -use risingwave_common::util::epoch::Epoch; -use risingwave_pb::stream_plan::FragmentTypeFlag; -use risingwave_pb::user::grant_privilege::Object; pub use rw_actors::*; pub use rw_connections::*; pub use rw_databases::*; @@ -61,660 +53,3 @@ pub use rw_tables::*; pub use rw_users::*; pub use rw_views::*; pub use rw_worker_nodes::*; -use serde_json::json; - -use super::SysCatalogReaderImpl; -use crate::catalog::system_catalog::get_acl_items; -use crate::handler::create_source::UPSTREAM_SOURCE_KEY; - -impl SysCatalogReaderImpl { - pub(super) async fn read_meta_snapshot(&self) -> Result> { - let try_get_date_time = |epoch: u64| { - if epoch == 0 { - return None; - } - let time_millis = Epoch::from(epoch).as_unix_millis(); - Timestamp::with_secs_nsecs( - (time_millis / 1000) as i64, - (time_millis % 1000 * 1_000_000) as u32, - ) - .map(ScalarImpl::Timestamp) - .ok() - }; - let meta_snapshots = self - .meta_client - .list_meta_snapshots() - .await? - .into_iter() - .map(|s| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int64(s.id as i64)), - Some(ScalarImpl::Int64(s.hummock_version_id as i64)), - Some(ScalarImpl::Int64(s.safe_epoch as i64)), - try_get_date_time(s.safe_epoch), - Some(ScalarImpl::Int64(s.max_committed_epoch as i64)), - try_get_date_time(s.max_committed_epoch), - ]) - }) - .collect_vec(); - Ok(meta_snapshots) - } - - pub(super) async fn read_ddl_progress(&self) -> Result> { - let ddl_grogress = self - .meta_client - .list_ddl_progress() - .await? - .into_iter() - .map(|s| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int64(s.id as i64)), - Some(ScalarImpl::Utf8(s.statement.into())), - Some(ScalarImpl::Utf8(s.progress.into())), - ]) - }) - .collect_vec(); - Ok(ddl_grogress) - } - - pub(super) async fn read_relation_info(&self) -> Result> { - let mut table_ids = Vec::new(); - { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.get_all_schema_names(&self.auth_context.database)?; - for schema in &schemas { - let schema_catalog = - reader.get_schema_by_name(&self.auth_context.database, schema)?; - - schema_catalog.iter_mv().for_each(|t| { - table_ids.push(t.id.table_id); - }); - - schema_catalog.iter_table().for_each(|t| { - table_ids.push(t.id.table_id); - }); - - schema_catalog.iter_sink().for_each(|t| { - table_ids.push(t.id.sink_id); - }); - - schema_catalog.iter_index().for_each(|t| { - table_ids.push(t.index_table.id.table_id); - }); - } - } - - let table_fragments = self.meta_client.list_table_fragments(&table_ids).await?; - let mut rows = Vec::new(); - let reader = self.catalog_reader.read_guard(); - let schemas = reader.get_all_schema_names(&self.auth_context.database)?; - for schema in &schemas { - let schema_catalog = reader.get_schema_by_name(&self.auth_context.database, schema)?; - schema_catalog.iter_mv().for_each(|t| { - if let Some(fragments) = table_fragments.get(&t.id.table_id) { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.owner as i32)), - Some(ScalarImpl::Utf8(t.definition.clone().into())), - Some(ScalarImpl::Utf8("MATERIALIZED VIEW".into())), - Some(ScalarImpl::Int32(t.id.table_id as i32)), - Some(ScalarImpl::Utf8( - fragments.get_env().unwrap().get_timezone().clone().into(), - )), - Some(ScalarImpl::Utf8( - json!(fragments.get_fragments()).to_string().into(), - )), - ])); - } - }); - - schema_catalog.iter_table().for_each(|t| { - if let Some(fragments) = table_fragments.get(&t.id.table_id) { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.owner as i32)), - Some(ScalarImpl::Utf8(t.definition.clone().into())), - Some(ScalarImpl::Utf8("TABLE".into())), - Some(ScalarImpl::Int32(t.id.table_id as i32)), - Some(ScalarImpl::Utf8( - fragments.get_env().unwrap().get_timezone().clone().into(), - )), - Some(ScalarImpl::Utf8( - json!(fragments.get_fragments()).to_string().into(), - )), - ])); - } - }); - - schema_catalog.iter_sink().for_each(|t| { - if let Some(fragments) = table_fragments.get(&t.id.sink_id) { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.owner.user_id as i32)), - Some(ScalarImpl::Utf8(t.definition.clone().into())), - Some(ScalarImpl::Utf8("SINK".into())), - Some(ScalarImpl::Int32(t.id.sink_id as i32)), - Some(ScalarImpl::Utf8( - fragments.get_env().unwrap().get_timezone().clone().into(), - )), - Some(ScalarImpl::Utf8( - json!(fragments.get_fragments()).to_string().into(), - )), - ])); - } - }); - - schema_catalog.iter_index().for_each(|t| { - if let Some(fragments) = table_fragments.get(&t.index_table.id.table_id) { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.index_table.owner as i32)), - Some(ScalarImpl::Utf8(t.index_table.definition.clone().into())), - Some(ScalarImpl::Utf8("INDEX".into())), - Some(ScalarImpl::Int32(t.index_table.id.table_id as i32)), - Some(ScalarImpl::Utf8( - fragments.get_env().unwrap().get_timezone().clone().into(), - )), - Some(ScalarImpl::Utf8( - json!(fragments.get_fragments()).to_string().into(), - )), - ])); - } - }); - - // Sources have no fragments. - schema_catalog.iter_source().for_each(|t| { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.owner as i32)), - Some(ScalarImpl::Utf8(t.definition.clone().into())), - Some(ScalarImpl::Utf8("SOURCE".into())), - Some(ScalarImpl::Int32(t.id as i32)), - Some(ScalarImpl::Utf8("".into())), - None, - ])); - }); - } - - Ok(rows) - } - - pub(super) fn read_rw_database_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); - - Ok(reader - .iter_databases() - .map(|db| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(db.id() as i32)), - Some(ScalarImpl::Utf8(db.name().into())), - Some(ScalarImpl::Int32(db.owner() as i32)), - Some(ScalarImpl::Utf8( - get_acl_items(&Object::DatabaseId(db.id()), &users, username_map).into(), - )), - ]) - }) - .collect_vec()) - } - - pub(super) fn read_rw_schema_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); - - Ok(schemas - .map(|schema| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Utf8(schema.name().into())), - Some(ScalarImpl::Int32(schema.owner() as i32)), - Some(ScalarImpl::Utf8( - get_acl_items(&Object::SchemaId(schema.id()), &users, username_map).into(), - )), - ]) - }) - .collect_vec()) - } - - pub(super) fn read_rw_user_info(&self) -> Result> { - let reader = self.user_info_reader.read_guard(); - let users = reader.get_all_users(); - - Ok(users - .into_iter() - .map(|user| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(user.id as i32)), - Some(ScalarImpl::Utf8(user.name.into())), - Some(ScalarImpl::Bool(user.is_super)), - Some(ScalarImpl::Bool(user.can_create_db)), - Some(ScalarImpl::Bool(user.can_create_user)), - Some(ScalarImpl::Bool(user.can_login)), - ]) - }) - .collect_vec()) - } - - pub(super) fn read_rw_table_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); - - Ok(schemas - .flat_map(|schema| { - schema.iter_table().map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id as i32)), - Some(ScalarImpl::Utf8(table.name().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(table.owner as i32)), - Some(ScalarImpl::Utf8(table.create_sql().into())), - Some(ScalarImpl::Utf8( - get_acl_items( - &Object::TableId(table.id.table_id), - &users, - username_map, - ) - .into(), - )), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_rw_mview_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); - - Ok(schemas - .flat_map(|schema| { - schema.iter_mv().map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id as i32)), - Some(ScalarImpl::Utf8(table.name().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(table.owner as i32)), - Some(ScalarImpl::Utf8(table.create_sql().into())), - Some(ScalarImpl::Utf8( - get_acl_items( - &Object::TableId(table.id.table_id), - &users, - username_map, - ) - .into(), - )), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_rw_indexes_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - - Ok(schemas - .flat_map(|schema| { - schema.iter_index().map(|index| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(index.id.index_id as i32)), - Some(ScalarImpl::Utf8(index.name.clone().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(index.index_table.owner as i32)), - Some(ScalarImpl::Utf8(index.index_table.create_sql().into())), - Some(ScalarImpl::Utf8("".into())), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_rw_views_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); - - Ok(schemas - .flat_map(|schema| { - schema.iter_view().map(|view| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(view.id as i32)), - Some(ScalarImpl::Utf8(view.name().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(view.owner as i32)), - Some(ScalarImpl::Utf8(view.create_sql().into())), - Some(ScalarImpl::Utf8( - get_acl_items(&Object::ViewId(view.id), &users, username_map).into(), - )), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_rw_sources_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); - - Ok(schemas - .flat_map(|schema| { - schema - .iter_source() - .filter(|s| s.associated_table_id.is_none()) - .map(|source| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(source.id as i32)), - Some(ScalarImpl::Utf8(source.name.clone().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(source.owner as i32)), - Some(ScalarImpl::Utf8( - source - .properties - .get(UPSTREAM_SOURCE_KEY) - .cloned() - .unwrap_or("".to_string()) - .to_uppercase() - .into(), - )), - Some(ScalarImpl::List(ListValue::new( - source - .columns - .iter() - .map(|c| Some(ScalarImpl::Utf8(c.name().into()))) - .collect_vec(), - ))), - Some(ScalarImpl::Utf8( - source.info.get_row_format().unwrap().as_str_name().into(), - )), - Some(ScalarImpl::Bool(source.append_only)), - source.connection_id.map(|id| ScalarImpl::Int32(id as i32)), - Some(ScalarImpl::Utf8(source.create_sql().into())), - Some( - get_acl_items(&Object::SourceId(source.id), &users, username_map) - .into(), - ), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_rw_sinks_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); - - Ok(schemas - .flat_map(|schema| { - schema.iter_sink().map(|sink| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(sink.id.sink_id as i32)), - Some(ScalarImpl::Utf8(sink.name.clone().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(sink.owner.user_id as i32)), - Some(ScalarImpl::Utf8( - sink.properties - .get(UPSTREAM_SOURCE_KEY) - .cloned() - .unwrap_or("".to_string()) - .to_uppercase() - .into(), - )), - Some(ScalarImpl::Utf8( - sink.sink_type.to_proto().as_str_name().into(), - )), - sink.connection_id - .map(|id| ScalarImpl::Int32(id.connection_id() as i32)), - Some(ScalarImpl::Utf8(sink.create_sql().into())), - Some( - get_acl_items(&Object::SinkId(sink.id.sink_id), &users, username_map) - .into(), - ), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_rw_connections_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - - Ok(schemas - .flat_map(|schema| { - schema.iter_connections().map(|conn| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(conn.id as i32)), - Some(ScalarImpl::Utf8(conn.name.clone().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(conn.owner as i32)), - Some(ScalarImpl::Utf8(conn.connection_type().into())), - Some(ScalarImpl::Utf8(conn.provider().into())), - Some(ScalarImpl::Utf8("".into())), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_rw_functions_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); - - Ok(schemas - .flat_map(|schema| { - schema.iter_function().map(|function| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(function.id.function_id() as i32)), - Some(ScalarImpl::Utf8(function.name.clone().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(function.owner as i32)), - Some(ScalarImpl::Utf8(function.kind.to_string().into())), - Some(ScalarImpl::List(ListValue::new( - function - .arg_types - .iter() - .map(|t| Some(ScalarImpl::Int32(t.to_oid()))) - .collect_vec(), - ))), - Some(ScalarImpl::Int32(function.return_type.to_oid())), - Some(ScalarImpl::Utf8(function.language.clone().into())), - Some(ScalarImpl::Utf8(function.link.clone().into())), - Some(ScalarImpl::Utf8( - get_acl_items( - &Object::FunctionId(function.id.function_id()), - &users, - username_map, - ) - .into(), - )), - ]) - }) - }) - .collect_vec()) - } - - pub(super) fn read_rw_worker_nodes_info(&self) -> Result> { - let workers = self.worker_node_manager.list_worker_nodes(); - - Ok(workers - .into_iter() - .map(|worker| { - let host = worker.host.as_ref().unwrap(); - let property = worker.property.as_ref().unwrap(); - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(worker.id as i32)), - Some(ScalarImpl::Utf8(host.host.clone().into())), - Some(ScalarImpl::Utf8(host.port.to_string().into())), - Some(ScalarImpl::Utf8( - worker.get_type().unwrap().as_str_name().into(), - )), - Some(ScalarImpl::Utf8( - worker.get_state().unwrap().as_str_name().into(), - )), - Some(ScalarImpl::Int32(worker.parallel_units.len() as i32)), - Some(ScalarImpl::Bool(property.is_streaming)), - Some(ScalarImpl::Bool(property.is_serving)), - Some(ScalarImpl::Bool(property.is_unschedulable)), - ]) - }) - .collect_vec()) - } - - pub(super) fn read_rw_parallel_units_info(&self) -> Result> { - let workers = self.worker_node_manager.list_worker_nodes(); - - Ok(workers - .into_iter() - .flat_map(|worker| { - worker.parallel_units.into_iter().map(|unit| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(unit.id as i32)), - Some(ScalarImpl::Int32(unit.worker_node_id as i32)), - ]) - }) - }) - .collect_vec()) - } - - /// FIXME: we need to introduce revision snapshot read on meta to avoid any inconsistency when - /// we are trying to join any table fragments related system tables. - pub(super) async fn read_rw_table_fragments_info(&self) -> Result> { - let states = self.meta_client.list_table_fragment_states().await?; - - Ok(states - .into_iter() - .map(|state| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(state.table_id as i32)), - Some(ScalarImpl::Utf8(state.state().as_str_name().into())), - ]) - }) - .collect_vec()) - } - - pub(super) async fn read_rw_fragment_distributions_info(&self) -> Result> { - let distributions = self.meta_client.list_fragment_distribution().await?; - - Ok(distributions - .into_iter() - .map(|distribution| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(distribution.fragment_id as i32)), - Some(ScalarImpl::Int32(distribution.table_id as i32)), - Some(ScalarImpl::Utf8( - distribution.distribution_type().as_str_name().into(), - )), - Some(ScalarImpl::List(ListValue::new( - distribution - .state_table_ids - .into_iter() - .map(|id| Some(ScalarImpl::Int32(id as i32))) - .collect_vec(), - ))), - Some(ScalarImpl::List(ListValue::new( - distribution - .upstream_fragment_ids - .into_iter() - .map(|id| Some(ScalarImpl::Int32(id as i32))) - .collect_vec(), - ))), - Some(ScalarImpl::List(ListValue::new( - Self::extract_fragment_type_flag(distribution.fragment_type_mask) - .into_iter() - .flat_map(|t| t.as_str_name().strip_prefix("FRAGMENT_TYPE_FLAG_")) - .map(|t| Some(ScalarImpl::Utf8(t.into()))) - .collect_vec(), - ))), - ]) - }) - .collect_vec()) - } - - pub(super) async fn read_rw_actor_states_info(&self) -> Result> { - let states = self.meta_client.list_actor_states().await?; - - Ok(states - .into_iter() - .map(|state| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(state.actor_id as i32)), - Some(ScalarImpl::Int32(state.fragment_id as i32)), - Some(ScalarImpl::Int32(state.parallel_unit_id as i32)), - Some(ScalarImpl::Utf8(state.state().as_str_name().into())), - ]) - }) - .collect_vec()) - } - - pub(super) fn read_table_stats(&self) -> Result> { - let catalog = self.catalog_reader.read_guard(); - let table_stats = catalog.table_stats(); - let mut rows = vec![]; - for (id, stats) in &table_stats.table_stats { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Int32(*id as i32)), - Some(ScalarImpl::Int64(stats.total_key_count)), - Some(ScalarImpl::Int64(stats.total_key_size)), - Some(ScalarImpl::Int64(stats.total_value_size)), - ])); - } - Ok(rows) - } - - fn extract_fragment_type_flag(mask: u32) -> Vec { - let mut result = vec![]; - for i in 0..32 { - let bit = 1 << i; - if mask & bit != 0 { - match FragmentTypeFlag::from_i32(bit as i32) { - None => continue, - Some(flag) => result.push(flag), - }; - } - } - result - } -} - -#[cfg(test)] -mod tests { - use risingwave_pb::stream_plan::FragmentTypeFlag; - - use crate::catalog::system_catalog::SysCatalogReaderImpl; - - #[test] - fn test_extract_mask() { - let mask = (FragmentTypeFlag::Source as u32) | (FragmentTypeFlag::ChainNode as u32); - let result = SysCatalogReaderImpl::extract_fragment_type_flag(mask); - assert_eq!(result.len(), 2); - assert!(result.contains(&FragmentTypeFlag::Source)); - assert!(result.contains(&FragmentTypeFlag::ChainNode)) - } -} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs index 651506b888d8b..9a3ef97d012a8 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_ACTORS_TABLE_NAME: &str = "rw_actors"; @@ -24,3 +27,21 @@ pub const RW_ACTORS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "parallel_unit_id"), (DataType::Varchar, "status"), ]; + +impl SysCatalogReaderImpl { + pub async fn read_rw_actor_states_info(&self) -> Result> { + let states = self.meta_client.list_actor_states().await?; + + Ok(states + .into_iter() + .map(|state| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(state.actor_id as i32)), + Some(ScalarImpl::Int32(state.fragment_id as i32)), + Some(ScalarImpl::Int32(state.parallel_unit_id as i32)), + Some(ScalarImpl::Utf8(state.state().as_str_name().into())), + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs index caff1bf44e1eb..6081126f74199 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_CONNECTIONS_TABLE_NAME: &str = "rw_connections"; @@ -27,3 +30,26 @@ pub const RW_CONNECTIONS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Varchar, "provider"), (DataType::Varchar, "acl"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_connections_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + + Ok(schemas + .flat_map(|schema| { + schema.iter_connections().map(|conn| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(conn.id as i32)), + Some(ScalarImpl::Utf8(conn.name.clone().into())), + Some(ScalarImpl::Int32(schema.id() as i32)), + Some(ScalarImpl::Int32(conn.owner as i32)), + Some(ScalarImpl::Utf8(conn.connection_type().into())), + Some(ScalarImpl::Utf8(conn.provider().into())), + Some(ScalarImpl::Utf8("".into())), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs index 13766a52a95f5..e8d288b73d7d5 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs @@ -12,9 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{ + get_acl_items, SysCatalogReaderImpl, SystemCatalogColumnsDef, +}; pub const RW_DATABASES_TABLE_NAME: &str = "rw_databases"; @@ -24,3 +30,26 @@ pub const RW_DATABASES_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "owner"), (DataType::Varchar, "acl"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_database_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let user_reader = self.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); + + Ok(reader + .iter_databases() + .map(|db| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(db.id() as i32)), + Some(ScalarImpl::Utf8(db.name().into())), + Some(ScalarImpl::Int32(db.owner() as i32)), + Some(ScalarImpl::Utf8( + get_acl_items(&Object::DatabaseId(db.id()), &users, username_map).into(), + )), + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs index dc13832e5f8b8..5e9f9a4b029ea 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_DDL_PROGRESS_TABLE_NAME: &str = "rw_ddl_progress"; @@ -23,3 +26,22 @@ pub const RW_DDL_PROGRESS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Varchar, "ddl_statement"), (DataType::Varchar, "progress"), ]; + +impl SysCatalogReaderImpl { + pub async fn read_ddl_progress(&self) -> Result> { + let ddl_progress = self + .meta_client + .list_ddl_progress() + .await? + .into_iter() + .map(|s| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int64(s.id as i64)), + Some(ScalarImpl::Utf8(s.statement.into())), + Some(ScalarImpl::Utf8(s.progress.into())), + ]) + }) + .collect_vec(); + Ok(ddl_progress) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs index bcdd61956b1fa..57e8136a8839a 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs @@ -14,9 +14,14 @@ use std::sync::LazyLock; -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::array::ListValue; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::stream_plan::FragmentTypeFlag; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_FRAGMENTS_TABLE_NAME: &str = "rw_fragments"; @@ -33,3 +38,73 @@ pub static RW_FRAGMENTS_COLUMNS: LazyLock>> = La (DataType::List(Box::new(DataType::Varchar)), "flags"), ] }); + +impl SysCatalogReaderImpl { + fn extract_fragment_type_flag(mask: u32) -> Vec { + let mut result = vec![]; + for i in 0..32 { + let bit = 1 << i; + if mask & bit != 0 { + match FragmentTypeFlag::from_i32(bit as i32) { + None => continue, + Some(flag) => result.push(flag), + }; + } + } + result + } + + pub async fn read_rw_fragment_distributions_info(&self) -> Result> { + let distributions = self.meta_client.list_fragment_distribution().await?; + + Ok(distributions + .into_iter() + .map(|distribution| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(distribution.fragment_id as i32)), + Some(ScalarImpl::Int32(distribution.table_id as i32)), + Some(ScalarImpl::Utf8( + distribution.distribution_type().as_str_name().into(), + )), + Some(ScalarImpl::List(ListValue::new( + distribution + .state_table_ids + .into_iter() + .map(|id| Some(ScalarImpl::Int32(id as i32))) + .collect_vec(), + ))), + Some(ScalarImpl::List(ListValue::new( + distribution + .upstream_fragment_ids + .into_iter() + .map(|id| Some(ScalarImpl::Int32(id as i32))) + .collect_vec(), + ))), + Some(ScalarImpl::List(ListValue::new( + Self::extract_fragment_type_flag(distribution.fragment_type_mask) + .into_iter() + .flat_map(|t| t.as_str_name().strip_prefix("FRAGMENT_TYPE_FLAG_")) + .map(|t| Some(ScalarImpl::Utf8(t.into()))) + .collect_vec(), + ))), + ]) + }) + .collect_vec()) + } +} + +#[cfg(test)] +mod tests { + use risingwave_pb::stream_plan::FragmentTypeFlag; + + use crate::catalog::system_catalog::SysCatalogReaderImpl; + + #[test] + fn test_extract_mask() { + let mask = (FragmentTypeFlag::Source as u32) | (FragmentTypeFlag::ChainNode as u32); + let result = SysCatalogReaderImpl::extract_fragment_type_flag(mask); + assert_eq!(result.len(), 2); + assert!(result.contains(&FragmentTypeFlag::Source)); + assert!(result.contains(&FragmentTypeFlag::ChainNode)) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs index f6da82559e1cf..ff943e5f757c1 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs @@ -14,9 +14,16 @@ use std::sync::LazyLock; -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::array::ListValue; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{ + get_acl_items, SysCatalogReaderImpl, SystemCatalogColumnsDef, +}; pub const RW_FUNCTIONS_TABLE_NAME: &str = "rw_functions"; @@ -36,3 +43,45 @@ pub static RW_FUNCTIONS_COLUMNS: LazyLock>> = La (DataType::Varchar, "acl"), ] }); + +impl SysCatalogReaderImpl { + pub fn read_rw_functions_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + let user_reader = self.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); + + Ok(schemas + .flat_map(|schema| { + schema.iter_function().map(|function| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(function.id.function_id() as i32)), + Some(ScalarImpl::Utf8(function.name.clone().into())), + Some(ScalarImpl::Int32(schema.id() as i32)), + Some(ScalarImpl::Int32(function.owner as i32)), + Some(ScalarImpl::Utf8(function.kind.to_string().into())), + Some(ScalarImpl::List(ListValue::new( + function + .arg_types + .iter() + .map(|t| Some(ScalarImpl::Int32(t.to_oid()))) + .collect_vec(), + ))), + Some(ScalarImpl::Int32(function.return_type.to_oid())), + Some(ScalarImpl::Utf8(function.language.clone().into())), + Some(ScalarImpl::Utf8(function.link.clone().into())), + Some(ScalarImpl::Utf8( + get_acl_items( + &Object::FunctionId(function.id.function_id()), + &users, + username_map, + ) + .into(), + )), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs index 545411633f30c..f8daf1eac5bbd 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_INDEXES_TABLE_NAME: &str = "rw_indexes"; @@ -25,4 +28,30 @@ pub const RW_INDEXES_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "owner"), (DataType::Varchar, "definition"), (DataType::Varchar, "acl"), + (DataType::Timestamptz, "initialized_at"), + (DataType::Timestamptz, "created_at"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_indexes_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + + Ok(schemas + .flat_map(|schema| { + schema.iter_index().map(|index| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(index.id.index_id as i32)), + Some(ScalarImpl::Utf8(index.name.clone().into())), + Some(ScalarImpl::Int32(schema.id() as i32)), + Some(ScalarImpl::Int32(index.index_table.owner as i32)), + Some(ScalarImpl::Utf8(index.index_table.create_sql().into())), + Some(ScalarImpl::Utf8("".into())), + index.initialized_at_epoch.map(|e| e.as_scalar()), + index.created_at_epoch.map(|e| e.as_scalar()), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs index eef446a41326d..d181aaa4cb5e0 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs @@ -12,9 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{ + get_acl_items, SysCatalogReaderImpl, SystemCatalogColumnsDef, +}; pub const RW_MATERIALIZED_VIEWS_TABLE_NAME: &str = "rw_materialized_views"; @@ -25,4 +31,40 @@ pub const RW_MATERIALIZED_VIEWS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "owner"), (DataType::Varchar, "definition"), (DataType::Varchar, "acl"), + (DataType::Timestamptz, "initialized_at"), + (DataType::Timestamptz, "created_at"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_mview_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + let user_reader = self.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); + + Ok(schemas + .flat_map(|schema| { + schema.iter_mv().map(|table| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table.id.table_id as i32)), + Some(ScalarImpl::Utf8(table.name().into())), + Some(ScalarImpl::Int32(schema.id() as i32)), + Some(ScalarImpl::Int32(table.owner as i32)), + Some(ScalarImpl::Utf8(table.create_sql().into())), + Some(ScalarImpl::Utf8( + get_acl_items( + &Object::TableId(table.id.table_id), + &users, + username_map, + ) + .into(), + )), + table.initialized_at_epoch.map(|e| e.as_scalar()), + table.created_at_epoch.map(|e| e.as_scalar()), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs index bf30607dd666c..306f7eb200860 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs @@ -12,9 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl, Timestamp}; +use risingwave_common::util::epoch::Epoch; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_META_SNAPSHOT_TABLE_NAME: &str = "rw_meta_snapshot"; @@ -30,3 +34,37 @@ pub const RW_META_SNAPSHOT_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ // human-readable timestamp of max_committed_epoch (DataType::Timestamp, "max_committed_epoch_ts"), ]; + +impl SysCatalogReaderImpl { + pub async fn read_meta_snapshot(&self) -> Result> { + let try_get_date_time = |epoch: u64| { + if epoch == 0 { + return None; + } + let time_millis = Epoch::from(epoch).as_unix_millis(); + Timestamp::with_secs_nsecs( + (time_millis / 1000) as i64, + (time_millis % 1000 * 1_000_000) as u32, + ) + .map(ScalarImpl::Timestamp) + .ok() + }; + let meta_snapshots = self + .meta_client + .list_meta_snapshots() + .await? + .into_iter() + .map(|s| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int64(s.id as i64)), + Some(ScalarImpl::Int64(s.hummock_version_id as i64)), + Some(ScalarImpl::Int64(s.safe_epoch as i64)), + try_get_date_time(s.safe_epoch), + Some(ScalarImpl::Int64(s.max_committed_epoch as i64)), + try_get_date_time(s.max_committed_epoch), + ]) + }) + .collect_vec(); + Ok(meta_snapshots) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs index 09bfe0c322a3c..cd7c9c3a42906 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs @@ -12,11 +12,32 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_PARALLEL_UNITS_TABLE_NAME: &str = "rw_parallel_units"; pub const RW_PARALLEL_UNITS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[(DataType::Int32, "id"), (DataType::Int32, "worker_id")]; + +impl SysCatalogReaderImpl { + pub fn read_rw_parallel_units_info(&self) -> Result> { + let workers = self.worker_node_manager.list_worker_nodes(); + + Ok(workers + .into_iter() + .flat_map(|worker| { + worker.parallel_units.into_iter().map(|unit| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(unit.id as i32)), + Some(ScalarImpl::Int32(unit.worker_node_id as i32)), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs index 4de1836bb4f9f..7c73414a2b97e 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use serde_json::json; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_RELATION_INFO_TABLE_NAME: &str = "rw_relation_info"; @@ -28,4 +31,145 @@ pub const RW_RELATION_INFO_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Varchar, "relationtimezone"), /* The timezone used to interpret ambiguous * dates/timestamps as tstz */ (DataType::Varchar, "fragments"), // fragments is json encoded fragment infos. + (DataType::Timestamptz, "initialized_at"), + (DataType::Timestamptz, "created_at"), ]; + +impl SysCatalogReaderImpl { + pub async fn read_relation_info(&self) -> Result> { + let mut table_ids = Vec::new(); + { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.get_all_schema_names(&self.auth_context.database)?; + for schema in &schemas { + let schema_catalog = + reader.get_schema_by_name(&self.auth_context.database, schema)?; + + schema_catalog.iter_mv().for_each(|t| { + table_ids.push(t.id.table_id); + }); + + schema_catalog.iter_table().for_each(|t| { + table_ids.push(t.id.table_id); + }); + + schema_catalog.iter_sink().for_each(|t| { + table_ids.push(t.id.sink_id); + }); + + schema_catalog.iter_index().for_each(|t| { + table_ids.push(t.index_table.id.table_id); + }); + } + } + + let table_fragments = self.meta_client.list_table_fragments(&table_ids).await?; + let mut rows = Vec::new(); + let reader = self.catalog_reader.read_guard(); + let schemas = reader.get_all_schema_names(&self.auth_context.database)?; + for schema in &schemas { + let schema_catalog = reader.get_schema_by_name(&self.auth_context.database, schema)?; + schema_catalog.iter_mv().for_each(|t| { + if let Some(fragments) = table_fragments.get(&t.id.table_id) { + rows.push(OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.clone().into())), + Some(ScalarImpl::Utf8(t.name.clone().into())), + Some(ScalarImpl::Int32(t.owner as i32)), + Some(ScalarImpl::Utf8(t.definition.clone().into())), + Some(ScalarImpl::Utf8("MATERIALIZED VIEW".into())), + Some(ScalarImpl::Int32(t.id.table_id as i32)), + Some(ScalarImpl::Utf8( + fragments.get_env().unwrap().get_timezone().clone().into(), + )), + Some(ScalarImpl::Utf8( + json!(fragments.get_fragments()).to_string().into(), + )), + t.initialized_at_epoch.map(|e| e.as_scalar()), + t.created_at_epoch.map(|e| e.as_scalar()), + ])); + } + }); + + schema_catalog.iter_table().for_each(|t| { + if let Some(fragments) = table_fragments.get(&t.id.table_id) { + rows.push(OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.clone().into())), + Some(ScalarImpl::Utf8(t.name.clone().into())), + Some(ScalarImpl::Int32(t.owner as i32)), + Some(ScalarImpl::Utf8(t.definition.clone().into())), + Some(ScalarImpl::Utf8("TABLE".into())), + Some(ScalarImpl::Int32(t.id.table_id as i32)), + Some(ScalarImpl::Utf8( + fragments.get_env().unwrap().get_timezone().clone().into(), + )), + Some(ScalarImpl::Utf8( + json!(fragments.get_fragments()).to_string().into(), + )), + t.initialized_at_epoch.map(|e| e.as_scalar()), + t.created_at_epoch.map(|e| e.as_scalar()), + ])); + } + }); + + schema_catalog.iter_sink().for_each(|t| { + if let Some(fragments) = table_fragments.get(&t.id.sink_id) { + rows.push(OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.clone().into())), + Some(ScalarImpl::Utf8(t.name.clone().into())), + Some(ScalarImpl::Int32(t.owner.user_id as i32)), + Some(ScalarImpl::Utf8(t.definition.clone().into())), + Some(ScalarImpl::Utf8("SINK".into())), + Some(ScalarImpl::Int32(t.id.sink_id as i32)), + Some(ScalarImpl::Utf8( + fragments.get_env().unwrap().get_timezone().clone().into(), + )), + Some(ScalarImpl::Utf8( + json!(fragments.get_fragments()).to_string().into(), + )), + t.initialized_at_epoch.map(|e| e.as_scalar()), + t.created_at_epoch.map(|e| e.as_scalar()), + ])); + } + }); + + schema_catalog.iter_index().for_each(|t| { + if let Some(fragments) = table_fragments.get(&t.index_table.id.table_id) { + rows.push(OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.clone().into())), + Some(ScalarImpl::Utf8(t.name.clone().into())), + Some(ScalarImpl::Int32(t.index_table.owner as i32)), + Some(ScalarImpl::Utf8(t.index_table.definition.clone().into())), + Some(ScalarImpl::Utf8("INDEX".into())), + Some(ScalarImpl::Int32(t.index_table.id.table_id as i32)), + Some(ScalarImpl::Utf8( + fragments.get_env().unwrap().get_timezone().clone().into(), + )), + Some(ScalarImpl::Utf8( + json!(fragments.get_fragments()).to_string().into(), + )), + t.initialized_at_epoch.map(|e| e.as_scalar()), + t.created_at_epoch.map(|e| e.as_scalar()), + ])); + } + }); + + // Sources have no fragments. + schema_catalog.iter_source().for_each(|t| { + rows.push(OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(schema.clone().into())), + Some(ScalarImpl::Utf8(t.name.clone().into())), + Some(ScalarImpl::Int32(t.owner as i32)), + Some(ScalarImpl::Utf8(t.definition.clone().into())), + Some(ScalarImpl::Utf8("SOURCE".into())), + Some(ScalarImpl::Int32(t.id as i32)), + Some(ScalarImpl::Utf8("".into())), + None, + t.initialized_at_epoch.map(|e| e.as_scalar()), + t.created_at_epoch.map(|e| e.as_scalar()), + ])); + }); + } + + Ok(rows) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs index 839ae1b797bb9..c89baf7f8dc05 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs @@ -12,9 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{ + get_acl_items, SysCatalogReaderImpl, SystemCatalogColumnsDef, +}; pub const RW_SCHEMAS_TABLE_NAME: &str = "rw_schemas"; @@ -24,3 +30,26 @@ pub const RW_SCHEMAS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "owner"), (DataType::Varchar, "acl"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_schema_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + let user_reader = self.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); + + Ok(schemas + .map(|schema| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(schema.id() as i32)), + Some(ScalarImpl::Utf8(schema.name().into())), + Some(ScalarImpl::Int32(schema.owner() as i32)), + Some(ScalarImpl::Utf8( + get_acl_items(&Object::SchemaId(schema.id()), &users, username_map).into(), + )), + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs index a72acb8b4ab58..ccbf11ae2636e 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs @@ -12,9 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{ + get_acl_items, SysCatalogReaderImpl, SystemCatalogColumnsDef, +}; +use crate::handler::create_source::UPSTREAM_SOURCE_KEY; pub const RW_SINKS_TABLE_NAME: &str = "rw_sinks"; @@ -28,4 +35,49 @@ pub const RW_SINKS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "connection_id"), (DataType::Varchar, "definition"), (DataType::Varchar, "acl"), + (DataType::Timestamptz, "initialized_at"), + (DataType::Timestamptz, "created_at"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_sinks_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + let user_reader = self.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); + + Ok(schemas + .flat_map(|schema| { + schema.iter_sink().map(|sink| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(sink.id.sink_id as i32)), + Some(ScalarImpl::Utf8(sink.name.clone().into())), + Some(ScalarImpl::Int32(schema.id() as i32)), + Some(ScalarImpl::Int32(sink.owner.user_id as i32)), + Some(ScalarImpl::Utf8( + sink.properties + .get(UPSTREAM_SOURCE_KEY) + .cloned() + .unwrap_or("".to_string()) + .to_uppercase() + .into(), + )), + Some(ScalarImpl::Utf8( + sink.sink_type.to_proto().as_str_name().into(), + )), + sink.connection_id + .map(|id| ScalarImpl::Int32(id.connection_id() as i32)), + Some(ScalarImpl::Utf8(sink.create_sql().into())), + Some( + get_acl_items(&Object::SinkId(sink.id.sink_id), &users, username_map) + .into(), + ), + sink.initialized_at_epoch.map(|e| e.as_scalar()), + sink.created_at_epoch.map(|e| e.as_scalar()), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs index 3084bf78737d9..b83eeb7a7e437 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs @@ -14,9 +14,17 @@ use std::sync::LazyLock; -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::array::ListValue; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{ + get_acl_items, SysCatalogReaderImpl, SystemCatalogColumnsDef, +}; +use crate::handler::create_source::UPSTREAM_SOURCE_KEY; pub const RW_SOURCES_TABLE_NAME: &str = "rw_sources"; @@ -34,5 +42,61 @@ pub static RW_SOURCES_COLUMNS: LazyLock>> = Lazy (DataType::Int32, "connection_id"), (DataType::Varchar, "definition"), (DataType::Varchar, "acl"), + (DataType::Timestamptz, "initialized_at"), + (DataType::Timestamptz, "created_at"), ] }); + +impl SysCatalogReaderImpl { + pub fn read_rw_sources_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + let user_reader = self.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); + + Ok(schemas + .flat_map(|schema| { + schema + .iter_source() + .filter(|s| s.associated_table_id.is_none()) + .map(|source| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(source.id as i32)), + Some(ScalarImpl::Utf8(source.name.clone().into())), + Some(ScalarImpl::Int32(schema.id() as i32)), + Some(ScalarImpl::Int32(source.owner as i32)), + Some(ScalarImpl::Utf8( + source + .properties + .get(UPSTREAM_SOURCE_KEY) + .cloned() + .unwrap_or("".to_string()) + .to_uppercase() + .into(), + )), + Some(ScalarImpl::List(ListValue::new( + source + .columns + .iter() + .map(|c| Some(ScalarImpl::Utf8(c.name().into()))) + .collect_vec(), + ))), + Some(ScalarImpl::Utf8( + source.info.get_row_format().unwrap().as_str_name().into(), + )), + Some(ScalarImpl::Bool(source.append_only)), + source.connection_id.map(|id| ScalarImpl::Int32(id as i32)), + Some(ScalarImpl::Utf8(source.create_sql().into())), + Some( + get_acl_items(&Object::SourceId(source.id), &users, username_map) + .into(), + ), + source.initialized_at_epoch.map(|e| e.as_scalar()), + source.created_at_epoch.map(|e| e.as_scalar()), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs index 04c5242a849b1..63a96f25f51b7 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs @@ -12,11 +12,30 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_TABLE_FRAGMENTS_TABLE_NAME: &str = "rw_table_fragments"; pub const RW_TABLE_FRAGMENTS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[(DataType::Int32, "table_id"), (DataType::Varchar, "status")]; + +impl SysCatalogReaderImpl { + pub async fn read_rw_table_fragments_info(&self) -> Result> { + let states = self.meta_client.list_table_fragment_states().await?; + + Ok(states + .into_iter() + .map(|state| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(state.table_id as i32)), + Some(ScalarImpl::Utf8(state.state().as_str_name().into())), + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs index e426fdc8372f6..fbac5bc3849ba 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_TABLE_STATS_TABLE_NAME: &str = "rw_table_stats"; pub const RW_TABLE_STATS_TABLE_ID_INDEX: usize = 0; @@ -27,3 +29,20 @@ pub const RW_TABLE_STATS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int64, "total_key_size"), (DataType::Int64, "total_value_size"), ]; + +impl SysCatalogReaderImpl { + pub fn read_table_stats(&self) -> Result> { + let catalog = self.catalog_reader.read_guard(); + let table_stats = catalog.table_stats(); + let mut rows = vec![]; + for (id, stats) in &table_stats.table_stats { + rows.push(OwnedRow::new(vec![ + Some(ScalarImpl::Int32(*id as i32)), + Some(ScalarImpl::Int64(stats.total_key_count)), + Some(ScalarImpl::Int64(stats.total_key_size)), + Some(ScalarImpl::Int64(stats.total_value_size)), + ])); + } + Ok(rows) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs index ff4f6cb816389..b3ce4f713c3bd 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs @@ -12,9 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{ + get_acl_items, SysCatalogReaderImpl, SystemCatalogColumnsDef, +}; pub const RW_TABLES_TABLE_NAME: &str = "rw_tables"; @@ -25,4 +31,40 @@ pub const RW_TABLES_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Int32, "owner"), (DataType::Varchar, "definition"), (DataType::Varchar, "acl"), + (DataType::Timestamptz, "initialized_at"), + (DataType::Timestamptz, "created_at"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_table_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + let user_reader = self.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); + + Ok(schemas + .flat_map(|schema| { + schema.iter_table().map(|table| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(table.id.table_id as i32)), + Some(ScalarImpl::Utf8(table.name().into())), + Some(ScalarImpl::Int32(schema.id() as i32)), + Some(ScalarImpl::Int32(table.owner as i32)), + Some(ScalarImpl::Utf8(table.create_sql().into())), + Some(ScalarImpl::Utf8( + get_acl_items( + &Object::TableId(table.id.table_id), + &users, + username_map, + ) + .into(), + )), + table.initialized_at_epoch.map(|e| e.as_scalar()), + table.created_at_epoch.map(|e| e.as_scalar()), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs index 56cdcf8007d13..9558ca3ece856 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; pub const RW_USERS_TABLE_NAME: &str = "rw_users"; @@ -26,3 +29,24 @@ pub const RW_USERS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Boolean, "create_user"), (DataType::Boolean, "can_login"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_user_info(&self) -> Result> { + let reader = self.user_info_reader.read_guard(); + let users = reader.get_all_users(); + + Ok(users + .into_iter() + .map(|user| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(user.id as i32)), + Some(ScalarImpl::Utf8(user.name.into())), + Some(ScalarImpl::Bool(user.is_super)), + Some(ScalarImpl::Bool(user.can_create_db)), + Some(ScalarImpl::Bool(user.can_create_user)), + Some(ScalarImpl::Bool(user.can_login)), + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs index 5849808eee4ab..12926a2924ce8 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs @@ -12,9 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{ + get_acl_items, SysCatalogReaderImpl, SystemCatalogColumnsDef, +}; pub const RW_VIEWS_TABLE_NAME: &str = "rw_views"; @@ -26,3 +32,30 @@ pub const RW_VIEWS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Varchar, "definition"), (DataType::Varchar, "acl"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_views_info(&self) -> Result> { + let reader = self.catalog_reader.read_guard(); + let schemas = reader.iter_schemas(&self.auth_context.database)?; + let user_reader = self.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); + + Ok(schemas + .flat_map(|schema| { + schema.iter_view().map(|view| { + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(view.id as i32)), + Some(ScalarImpl::Utf8(view.name().into())), + Some(ScalarImpl::Int32(schema.id() as i32)), + Some(ScalarImpl::Int32(view.owner as i32)), + Some(ScalarImpl::Utf8(view.create_sql().into())), + Some(ScalarImpl::Utf8( + get_acl_items(&Object::ViewId(view.id), &users, username_map).into(), + )), + ]) + }) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs index 92c3fa370f778..5760a0f796a57 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs @@ -12,9 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::DataType; +use itertools::Itertools; +use risingwave_common::error::Result; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, ScalarImpl}; -use crate::catalog::system_catalog::SystemCatalogColumnsDef; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemCatalogColumnsDef}; /// `rw_worker_nodes` contains all information about the compute nodes in the cluster. /// TODO: Add other type of nodes if necessary in the future. @@ -31,3 +34,32 @@ pub const RW_WORKER_NODES_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ (DataType::Boolean, "is_serving"), (DataType::Boolean, "is_unschedulable"), ]; + +impl SysCatalogReaderImpl { + pub fn read_rw_worker_nodes_info(&self) -> Result> { + let workers = self.worker_node_manager.list_worker_nodes(); + + Ok(workers + .into_iter() + .map(|worker| { + let host = worker.host.as_ref().unwrap(); + let property = worker.property.as_ref().unwrap(); + OwnedRow::new(vec![ + Some(ScalarImpl::Int32(worker.id as i32)), + Some(ScalarImpl::Utf8(host.host.clone().into())), + Some(ScalarImpl::Utf8(host.port.to_string().into())), + Some(ScalarImpl::Utf8( + worker.get_type().unwrap().as_str_name().into(), + )), + Some(ScalarImpl::Utf8( + worker.get_state().unwrap().as_str_name().into(), + )), + Some(ScalarImpl::Int32(worker.parallel_units.len() as i32)), + Some(ScalarImpl::Bool(property.is_streaming)), + Some(ScalarImpl::Bool(property.is_serving)), + Some(ScalarImpl::Bool(property.is_unschedulable)), + ]) + }) + .collect_vec()) + } +} diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 2a8726c5a1dad..a828bb430591a 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -21,6 +21,7 @@ use risingwave_common::catalog::{ }; use risingwave_common::constants::hummock::TABLE_OPTION_DUMMY_RETENTION_SECOND; use risingwave_common::error::{ErrorCode, RwError}; +use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::table::{OptionalAssociatedSourceId, PbTableType, PbTableVersion}; use risingwave_pb::catalog::PbTable; @@ -141,6 +142,10 @@ pub struct TableCatalog { /// Optional field specifies the distribution key indices in pk. /// See https://github.com/risingwavelabs/risingwave/issues/8377 for more information. pub dist_key_in_pk: Vec, + + pub created_at_epoch: Option, + + pub initialized_at_epoch: Option, } #[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)] @@ -385,6 +390,8 @@ impl TableCatalog { dist_key_in_pk: self.dist_key_in_pk.iter().map(|x| *x as _).collect(), handle_pk_conflict_behavior: self.conflict_behavior.to_protobuf().into(), cardinality: Some(self.cardinality.to_protobuf()), + initialized_at_epoch: self.initialized_at_epoch.map(|epoch| epoch.0), + created_at_epoch: self.created_at_epoch.map(|epoch| epoch.0), } } @@ -488,6 +495,8 @@ impl From for TableCatalog { .cardinality .map(|c| Cardinality::from_protobuf(&c)) .unwrap_or_else(Cardinality::unknown), + created_at_epoch: tb.created_at_epoch.map(Epoch::from), + initialized_at_epoch: tb.initialized_at_epoch.map(Epoch::from), } } } @@ -562,6 +571,7 @@ mod tests { )]), fragment_id: 0, dml_fragment_id: None, + initialized_at_epoch: None, value_indices: vec![0], definition: "".into(), read_prefix_len_hint: 0, @@ -575,6 +585,7 @@ mod tests { handle_pk_conflict_behavior: 3, dist_key_in_pk: vec![], cardinality: None, + created_at_epoch: None, } .into(); @@ -626,6 +637,8 @@ mod tests { watermark_columns: FixedBitSet::with_capacity(2), dist_key_in_pk: vec![], cardinality: Cardinality::unknown(), + created_at_epoch: None, + initialized_at_epoch: None, } ); assert_eq!(table, TableCatalog::from(table.to_prost(0, 0))); diff --git a/src/frontend/src/expr/type_inference/func.rs b/src/frontend/src/expr/type_inference/func.rs index dd4ce14494479..72363fc39dd85 100644 --- a/src/frontend/src/expr/type_inference/func.rs +++ b/src/frontend/src/expr/type_inference/func.rs @@ -596,16 +596,6 @@ fn infer_type_for_special( } Ok(Some(DataType::Varchar)) } - ExprType::ArrayLength => { - ensure_arity!("array_length", 1 <= | inputs | <= 2); - inputs[0].ensure_array_type()?; - - if let Some(arg1) = inputs.get_mut(1) { - arg1.cast_implicit_mut(DataType::Int32)?; - } - - Ok(Some(DataType::Int32)) - } ExprType::StringToArray => { ensure_arity!("string_to_array", 2 <= | inputs | <= 3); @@ -615,12 +605,6 @@ fn infer_type_for_special( Ok(Some(DataType::List(Box::new(DataType::Varchar)))) } - ExprType::Cardinality => { - ensure_arity!("cardinality", | inputs | == 1); - inputs[0].ensure_array_type()?; - - Ok(Some(DataType::Int32)) - } ExprType::TrimArray => { ensure_arity!("trim_array", | inputs | == 2); inputs[0].ensure_array_type()?; @@ -683,7 +667,7 @@ fn infer_type_name<'a>( } } - let mut candidates = top_matches(candidates, inputs); + let mut candidates = top_matches(&candidates, inputs); if candidates.is_empty() { return Err(ErrorCode::NotImplemented( @@ -706,7 +690,7 @@ fn infer_type_name<'a>( match &candidates[..] { [] => unreachable!(), - [sig] => Ok(sig), + [sig] => Ok(*sig), _ => Err(ErrorCode::BindError(format!( "function {:?}{:?} is not unique\nHINT: Could not choose a best candidate function. You might need to add explicit type casts.", func_type, @@ -763,7 +747,7 @@ fn implicit_ok(source: DataTypeName, target: DataTypeName, eq_ok: bool) -> bool /// [rule 4c src]: https://github.com/postgres/postgres/blob/86a4dc1e6f29d1992a2afa3fac1a0b0a6e84568c/src/backend/parser/parse_func.c#L1062-L1104 /// [rule 4d src]: https://github.com/postgres/postgres/blob/86a4dc1e6f29d1992a2afa3fac1a0b0a6e84568c/src/backend/parser/parse_func.c#L1106-L1153 fn top_matches<'a>( - candidates: &'a [FuncSign], + candidates: &[&'a FuncSign], inputs: &[Option], ) -> Vec<&'a FuncSign> { let mut best_exact = 0; @@ -795,7 +779,7 @@ fn top_matches<'a>( best_candidates.clear(); } if n_exact == best_exact && n_preferred == best_preferred { - best_candidates.push(sig); + best_candidates.push(*sig); } } best_candidates @@ -1226,6 +1210,7 @@ mod tests { inputs_type: formals, ret_type: DUMMY_RET, build: |_, _| unreachable!(), + deprecated: false, }); } let result = infer_type_name(&sig_map, DUMMY_FUNC, inputs); diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 9778c687e9895..a847a30a20bd5 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -240,6 +240,8 @@ pub(crate) fn gen_create_index_plan( primary_table_id: table.id.table_id, index_item, original_columns, + initialized_at_epoch: None, + created_at_epoch: 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 b76b6cfbdfa9a..78ba6e32b7460 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -943,6 +943,8 @@ pub async fn handle_create_source( watermark_descs, definition, connection_id, + initialized_at_epoch: None, + created_at_epoch: None, optional_associated_table_id: None, }; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 204bf062cf81e..e51eacb3b1c51 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -593,6 +593,8 @@ fn gen_table_plan_inner( watermark_descs: watermark_descs.clone(), definition: "".to_string(), connection_id, + initialized_at_epoch: None, + created_at_epoch: None, optional_associated_table_id: Some(OptionalAssociatedTableId::AssociatedTableId( TableId::placeholder().table_id, )), diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 1bf3f84d39dcd..5e5789e2b0de2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -219,6 +219,8 @@ impl StreamMaterialize { watermark_columns, dist_key_in_pk: vec![], cardinality, + created_at_epoch: None, + initialized_at_epoch: None, }) } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index e95ccc2b9da02..e3ec3ef50e538 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -174,6 +174,8 @@ impl TableCatalogBuilder { watermark_columns, dist_key_in_pk: self.dist_key_in_pk.unwrap_or(vec![]), cardinality: Cardinality::unknown(), // TODO(card): cardinality of internal table + created_at_epoch: None, + initialized_at_epoch: None, } } diff --git a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs index edf84a427d0ea..35e0538cb76f7 100644 --- a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs @@ -801,6 +801,55 @@ pub mod tests { ); } + #[test] + fn test_issue_11154() { + let mut local_stats = LocalPickerStatistic::default(); + let mut l0 = generate_l0_overlapping_sublevels(vec![ + vec![ + generate_table(4, 1, 1, 200, 1), + generate_table(5, 1, 400, 600, 1), + ], + vec![ + generate_table(6, 1, 1, 200, 1), + generate_table(7, 1, 400, 600, 1), + ], + vec![ + generate_table(8, 1, 1, 200, 1), + generate_table(9, 1, 400, 600, 1), + ], + vec![generate_table(10, 1, 1, 600, 1)], + ]); + // We can set level_type only because the input above is valid. + for s in &mut l0.sub_levels { + s.level_type = LevelType::Nonoverlapping as i32; + } + let levels = Levels { + l0: Some(l0), + levels: vec![generate_level(1, vec![generate_table(3, 1, 0, 100000, 1)])], + member_table_ids: vec![1], + ..Default::default() + }; + let levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; + + // Pick with large max_compaction_bytes results all sub levels included in input. + let config = Arc::new( + CompactionConfigBuilder::new() + .max_compaction_bytes(800) + .sub_level_max_compaction_bytes(50000) + .max_bytes_for_level_base(500000) + .level0_sub_level_compact_level_count(1) + .build(), + ); + // Only include sub-level 0 results will violate MAX_WRITE_AMPLIFICATION. + // So all sub-levels are included to make write amplification < MAX_WRITE_AMPLIFICATION. + let mut picker = LevelCompactionPicker::new(1, config); + let ret = picker + .pick_compaction(&levels, &levels_handler, &mut local_stats) + .unwrap(); + // avoid add sst_10 and cause a big task + assert_eq!(3, ret.input_levels.len()); + } + #[test] fn test_l0_to_l1_break_on_pending_sub_level() { let l0 = generate_l0_nonoverlapping_multi_sublevels(vec![ diff --git a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs index b03fd56699a7e..3f426c44fa14b 100644 --- a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs @@ -205,12 +205,6 @@ impl NonOverlapSubLevelPicker { break; } - if ret.total_file_size >= self.max_compaction_bytes - || ret.total_file_count >= self.max_file_count as usize - { - break; - } - let mut overlap_files_range = overlap_info.check_multiple_include(&target_level.table_infos); if overlap_files_range.is_empty() { @@ -290,6 +284,15 @@ impl NonOverlapSubLevelPicker { .map(|(_, files)| files.len()) .sum::(); + // more than 1 sub_level + if ret.total_file_count > 1 + && ret.total_file_size + (add_files_size + current_level_size) + >= self.max_compaction_bytes + || ret.total_file_count + add_files_count >= self.max_file_count as usize + { + break; + } + ret.total_file_count += add_files_count; ret.total_file_size += add_files_size + current_level_size; if !overlap_files_range.is_empty() { diff --git a/src/meta/src/hummock/manager/checkpoint.rs b/src/meta/src/hummock/manager/checkpoint.rs index d78e24fb8cac6..bf899e9a96b59 100644 --- a/src/meta/src/hummock/manager/checkpoint.rs +++ b/src/meta/src/hummock/manager/checkpoint.rs @@ -39,27 +39,33 @@ impl HummockManager where S: MetaStore, { - pub(crate) async fn read_checkpoint(&self) -> Result> { + /// # Panics + /// if checkpoint is not found. + pub(crate) async fn read_checkpoint(&self) -> Result { // We `list` then `read`. Because from `read`'s error, we cannot tell whether it's "object // not found" or other kind of error. use prost::Message; + let dir = version_checkpoint_dir(&self.version_checkpoint_path); let metadata = self .object_store - .list(&version_checkpoint_dir(&self.version_checkpoint_path)) + .list(&dir) .await? .into_iter() .filter(|o| o.key == self.version_checkpoint_path) .collect_vec(); assert!(metadata.len() <= 1); if metadata.is_empty() { - return Ok(None); + panic!( + "Hummock version checkpoints do not exist in object store, prefix: {}", + dir + ); } let data = self .object_store .read(&self.version_checkpoint_path, None) .await?; let ckpt = HummockVersionCheckpoint::decode(data).map_err(|e| anyhow::anyhow!(e))?; - Ok(Some(ckpt)) + Ok(ckpt) } pub(super) async fn write_checkpoint( diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 0e440c4b5c4c3..0bd943680720a 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -470,7 +470,7 @@ where checkpoint_version } else { // Read checkpoint from object store. - versioning_guard.checkpoint = self.read_checkpoint().await?.expect("checkpoint exists"); + versioning_guard.checkpoint = self.read_checkpoint().await?; versioning_guard .checkpoint .version diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 6514cd174c42e..dc95378b3a2d2 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -181,6 +181,7 @@ impl HummockMetaClient for MockHummockMetaClient { _compaction_group_id: u64, _table_id: u32, _level: u32, + _sst_ids: Vec, ) -> Result<()> { todo!() } diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 544d1e2eb3a2c..bbe49fd74f59d 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -104,6 +104,7 @@ macro_rules! commit_meta { } use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_common::util::epoch::Epoch; use risingwave_pb::meta::relation::RelationInfo; use risingwave_pb::meta::{CreatingJobInfo, Relation, RelationGroup}; pub(crate) use {commit_meta, commit_meta_with_trx}; @@ -761,7 +762,7 @@ where pub async fn finish_create_table_procedure( &self, internal_tables: Vec, - table: &Table, + table: Table, ) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; @@ -1413,7 +1414,7 @@ where pub async fn finish_create_source_procedure( &self, - source: &Source, + mut source: Source, ) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; @@ -1425,6 +1426,8 @@ where "source must be in creating procedure" ); database_core.in_progress_creation_tracker.remove(&key); + + source.created_at_epoch = Some(Epoch::now().0); sources.insert(source.id, source.clone()); commit_meta!(self, sources)?; @@ -1527,8 +1530,8 @@ where pub async fn finish_create_table_procedure_with_source( &self, - source: &Source, - mview: &Table, + source: Source, + mview: Table, internal_tables: Vec
, ) -> MetaResult { let core = &mut *self.core.lock().await; @@ -1864,8 +1867,8 @@ where pub async fn finish_create_index_procedure( &self, internal_tables: Vec
, - index: &Index, - table: &Table, + index: Index, + table: Table, ) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; @@ -1885,6 +1888,7 @@ where .remove(&table.id); indexes.insert(index.id, index.clone()); + tables.insert(table.id, table.clone()); for table in &internal_tables { tables.insert(table.id, table.clone()); @@ -1948,7 +1952,7 @@ where pub async fn finish_create_sink_procedure( &self, internal_tables: Vec
, - sink: &Sink, + sink: Sink, ) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 147e2ef2d359c..cf233f8a44ec5 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -113,8 +113,8 @@ where .worker_node .property .as_ref() - .unwrap() - .is_unschedulable; + .map(|p| p.is_unschedulable) + .unwrap_or_default(); } worker.update_ttl(self.max_heartbeat_interval); diff --git a/src/meta/src/manager/streaming_job.rs b/src/meta/src/manager/streaming_job.rs index 6dd6524239425..6b3e71fe20092 100644 --- a/src/meta/src/manager/streaming_job.rs +++ b/src/meta/src/manager/streaming_job.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; use risingwave_common::catalog::TableVersionId; +use risingwave_common::util::epoch::Epoch; use risingwave_pb::catalog::{Index, Sink, Source, Table}; use crate::model::FragmentId; @@ -29,6 +30,44 @@ pub enum StreamingJob { Index(Index, Table), } +impl StreamingJob { + pub(crate) fn mark_created(&mut self) { + let created_at_epoch = Some(Epoch::now().0); + match self { + StreamingJob::MaterializedView(table) => table.created_at_epoch = created_at_epoch, + StreamingJob::Sink(table) => table.created_at_epoch = created_at_epoch, + StreamingJob::Table(source, table) => { + table.created_at_epoch = created_at_epoch; + if let Some(source) = source { + source.created_at_epoch = created_at_epoch; + } + } + StreamingJob::Index(index, _) => { + index.created_at_epoch = created_at_epoch; + } + } + } + + pub(crate) fn mark_initialized(&mut self) { + let initialized_at_epoch = Some(Epoch::now().0); + match self { + StreamingJob::MaterializedView(table) => { + table.initialized_at_epoch = initialized_at_epoch + } + StreamingJob::Sink(table) => table.initialized_at_epoch = initialized_at_epoch, + StreamingJob::Table(source, table) => { + table.initialized_at_epoch = initialized_at_epoch; + if let Some(source) = source { + source.initialized_at_epoch = initialized_at_epoch; + } + } + StreamingJob::Index(index, _) => { + index.initialized_at_epoch = initialized_at_epoch; + } + } + } +} + impl StreamingJob { pub fn set_id(&mut self, id: u32) { match self { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index c7f0c7b15f9df..5453d99529f35 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_common::config::DefaultParallelism; 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, Database, Function, Schema, Source, Table, View, @@ -208,7 +209,10 @@ where self.catalog_manager.drop_schema(schema_id).await } - async fn create_source(&self, source: Source) -> MetaResult { + async fn create_source(&self, mut source: Source) -> MetaResult { + // set the initialized_at_epoch to the current epoch. + source.initialized_at_epoch = Some(Epoch::now().0); + self.catalog_manager .start_create_source_procedure(&source) .await?; @@ -221,7 +225,7 @@ where } self.catalog_manager - .finish_create_source_procedure(&source) + .finish_create_source_procedure(source) .await } @@ -290,6 +294,9 @@ where .prepare_stream_job(&mut stream_job, fragment_graph) .await?; + // Update the corresponding 'initiated_at' field. + stream_job.mark_initialized(); + let mut internal_tables = vec![]; let result = try { let (ctx, table_fragments) = self @@ -316,7 +323,7 @@ where }; match result { - Ok(_) => self.finish_stream_job(&stream_job, internal_tables).await, + Ok(_) => self.finish_stream_job(stream_job, internal_tables).await, Err(err) => { self.cancel_stream_job(&stream_job, internal_tables).await; Err(err) @@ -547,11 +554,15 @@ where /// `finish_stream_job` finishes a stream job and clean some states. async fn finish_stream_job( &self, - stream_job: &StreamingJob, + mut stream_job: StreamingJob, internal_tables: Vec
, ) -> MetaResult { // 1. finish procedure. let mut creating_internal_table_ids = internal_tables.iter().map(|t| t.id).collect_vec(); + + // Update the corresponding 'created_at' field. + stream_job.mark_created(); + let version = match stream_job { StreamingJob::MaterializedView(table) => { creating_internal_table_ids.push(table.id); diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 8f177102a20b5..7fb35a449be03 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -856,7 +856,7 @@ mod tests { .create_streaming_job(table_fragments, ctx) .await?; self.catalog_manager - .finish_create_table_procedure(vec![], &table) + .finish_create_table_procedure(vec![], table) .await?; Ok(()) } diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index ed02e26dfe5e4..da999a1953fa4 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -29,7 +29,7 @@ regex = "1" reqwest = "0.11" serde = { version = "1", features = ["derive"] } serde_json = "1" -serde_with = "2" +serde_with = "3" serde_yaml = "0.9" tempfile = "3" tokio = { version = "0.2", package = "madsim-tokio", features = [ diff --git a/src/rpc_client/src/hummock_meta_client.rs b/src/rpc_client/src/hummock_meta_client.rs index a5db0812daf92..678795f03c49b 100644 --- a/src/rpc_client/src/hummock_meta_client.rs +++ b/src/rpc_client/src/hummock_meta_client.rs @@ -49,6 +49,7 @@ pub trait HummockMetaClient: Send + Sync + 'static { compaction_group_id: u64, table_id: u32, level: u32, + sst_ids: Vec, ) -> Result<()>; async fn report_full_scan_task(&self, object_ids: Vec) -> Result<()>; async fn trigger_full_gc(&self, sst_retention_time_sec: u64) -> Result<()>; diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 1947ecf70eb30..35b04ef599c38 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -1126,6 +1126,7 @@ impl HummockMetaClient for MetaClient { compaction_group_id: u64, table_id: u32, level: u32, + sst_ids: Vec, ) -> Result<()> { // TODO: support key_range parameter let req = TriggerManualCompactionRequest { @@ -1134,6 +1135,7 @@ impl HummockMetaClient for MetaClient { // if table_id not exist, manual_compaction will include all the sst // without check internal_table_id level, + sst_ids, ..Default::default() }; diff --git a/src/sqlparser/test_runner/Cargo.toml b/src/sqlparser/test_runner/Cargo.toml index d1cd39edf1a93..f0c62dcfc3b7d 100644 --- a/src/sqlparser/test_runner/Cargo.toml +++ b/src/sqlparser/test_runner/Cargo.toml @@ -15,7 +15,7 @@ console = "0.15" futures = { version = "0.3", default-features = false, features = ["alloc"] } risingwave_sqlparser = { path = "../" } serde = { version = "1", features = ["derive"] } -serde_with = "2" +serde_with = "3" serde_yaml = "0.9" tokio = { version = "0.2", package = "madsim-tokio", features = [ "rt", diff --git a/src/storage/src/filter_key_extractor.rs b/src/storage/src/filter_key_extractor.rs index a7bb97faa9d80..f11f9d8d7c3f1 100644 --- a/src/storage/src/filter_key_extractor.rs +++ b/src/storage/src/filter_key_extractor.rs @@ -493,6 +493,7 @@ mod tests { )]), fragment_id: 0, dml_fragment_id: None, + initialized_at_epoch: None, vnode_col_index: None, row_id_index: Some(0), value_indices: vec![0], @@ -503,6 +504,7 @@ mod tests { watermark_indices: vec![], dist_key_in_pk: vec![], cardinality: None, + created_at_epoch: None, } } diff --git a/src/storage/src/hummock/compactor/compaction_utils.rs b/src/storage/src/hummock/compactor/compaction_utils.rs index 101b4d8656e79..e208dfd6281db 100644 --- a/src/storage/src/hummock/compactor/compaction_utils.rs +++ b/src/storage/src/hummock/compactor/compaction_utils.rs @@ -231,20 +231,34 @@ pub async fn generate_splits( Ok(vec![]) } -pub fn estimate_task_memory_capacity(context: Arc, task: &CompactTask) -> usize { +pub fn estimate_task_memory_capacity( + context: Arc, + task: &CompactTask, +) -> (usize, usize, usize) { let max_target_file_size = context.storage_opts.sstable_size_mb as usize * (1 << 20); - let total_file_size = task + let total_input_file_size = task .input_ssts .iter() .flat_map(|level| level.table_infos.iter()) .map(|table| table.file_size) .sum::(); + let total_input_uncompressed_file_size = task + .input_ssts + .iter() + .flat_map(|level| level.table_infos.iter()) + .map(|table| table.uncompressed_file_size) + .sum::(); let capacity = std::cmp::min(task.target_file_size as usize, max_target_file_size); - let total_file_size = (total_file_size as f64 * 1.2).round() as usize; + let total_file_size = (total_input_file_size as f64 * 1.2).round() as usize; - match task.compression_algorithm { + let c = match task.compression_algorithm { 0 => std::cmp::min(capacity, total_file_size), _ => capacity, - } + }; + ( + c, + total_input_file_size as usize, + total_input_uncompressed_file_size as usize, + ) } diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 6e5a058b9d9e0..527edf104c27f 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -52,7 +52,7 @@ impl CompactorRunner { 1 => CompressionAlgorithm::Lz4, _ => CompressionAlgorithm::Zstd, }; - options.capacity = estimate_task_memory_capacity(context.clone(), &task); + options.capacity = estimate_task_memory_capacity(context.clone(), &task).0; let key_range = KeyRange { left: Bytes::copy_from_slice(task.splits[split_index].get_left()), diff --git a/src/storage/src/hummock/compactor/iterator.rs b/src/storage/src/hummock/compactor/iterator.rs index b15ecbf4b846c..f784dc3cb18b1 100644 --- a/src/storage/src/hummock/compactor/iterator.rs +++ b/src/storage/src/hummock/compactor/iterator.rs @@ -222,6 +222,8 @@ impl Drop for SstableStreamIterator { /// Iterates over the KV-pairs of a given list of SSTs. The key-ranges of these SSTs are assumed to /// be consecutive and non-overlapping. pub struct ConcatSstableIterator { + /// **CAUTION:** `key_range` is used for optimization. It doesn't guarantee value returned by + /// the iterator is in this range. key_range: KeyRange, /// The iterator of the current table. diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index 800a9ecd6b179..d359dc020b8d6 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -176,9 +176,6 @@ impl Compactor { ]) .start_timer(); - let (need_quota, total_file_count, total_key_count) = - estimate_state_for_compaction(&compact_task); - let mut multi_filter = build_multi_compaction_filter(&compact_task); let mut compact_table_ids = compact_task @@ -256,6 +253,9 @@ impl Compactor { return Self::compact_done(compact_task, context.clone(), vec![], task_status); } } + + let (need_quota, total_file_count, total_key_count) = + estimate_state_for_compaction(&compact_task); // Number of splits (key ranges) is equal to number of compaction tasks let parallelism = compact_task.splits.len(); assert_ne!(parallelism, 0, "splits cannot be empty"); @@ -280,14 +280,17 @@ impl Compactor { } }; - let task_memory_capacity_with_parallelism = - estimate_task_memory_capacity(context.clone(), &compact_task) * parallelism; + let (capacity, total_file_size, total_file_size_uncompressed) = + estimate_task_memory_capacity(context.clone(), &compact_task); + let task_memory_capacity_with_parallelism = capacity * parallelism; tracing::info!( - "Ready to handle compaction task: {} need memory: {} input_file_counts {} total_key_count {} target_level {} compression_algorithm {:?} parallelism {} task_memory_capacity_with_parallelism {}", + "Ready to handle compaction task: {} need memory: {} input_file_counts {} input_file_size {} input_file_size_uncompressed {} total_key_count {} target_level {} compression_algorithm {:?} parallelism {} task_memory_capacity_with_parallelism {}", compact_task.task_id, need_quota, total_file_count, + total_file_size, + total_file_size_uncompressed, total_key_count, compact_task.target_level, compact_task.compression_algorithm, diff --git a/src/storage/src/hummock/hummock_meta_client.rs b/src/storage/src/hummock/hummock_meta_client.rs index 068ebfdbd4199..c7ed6f203903a 100644 --- a/src/storage/src/hummock/hummock_meta_client.rs +++ b/src/storage/src/hummock/hummock_meta_client.rs @@ -113,9 +113,10 @@ impl HummockMetaClient for MonitoredHummockMetaClient { compaction_group_id: u64, table_id: u32, level: u32, + sst_ids: Vec, ) -> Result<()> { self.meta_client - .trigger_manual_compaction(compaction_group_id, table_id, level) + .trigger_manual_compaction(compaction_group_id, table_id, level, sst_ids) .await } diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 12a1ff2d710d0..a0e08c4ae832f 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -123,6 +123,7 @@ async fn compaction_test( )]), fragment_id: 0, dml_fragment_id: None, + initialized_at_epoch: None, vnode_col_index: None, value_indices: vec![], definition: "".to_string(), @@ -136,6 +137,7 @@ async fn compaction_test( watermark_indices: vec![], dist_key_in_pk: vec![], cardinality: None, + created_at_epoch: None, }; let mut delete_range_table = delete_key_table.clone(); delete_range_table.id = 2; diff --git a/src/tests/sqlsmith/src/sql_gen/types.rs b/src/tests/sqlsmith/src/sql_gen/types.rs index 2b19aa2c2e1de..939d869744ea5 100644 --- a/src/tests/sqlsmith/src/sql_gen/types.rs +++ b/src/tests/sqlsmith/src/sql_gen/types.rs @@ -24,7 +24,6 @@ use risingwave_expr::sig::agg::{agg_func_sigs, AggFuncSig as RwAggFuncSig}; use risingwave_expr::sig::cast::{cast_sigs, CastContext, CastSig as RwCastSig}; use risingwave_expr::sig::func::{func_sigs, FuncSign as RwFuncSig}; use risingwave_frontend::expr::ExprType; -use risingwave_pb::expr::expr_node::PbType; use risingwave_sqlparser::ast::{BinaryOperator, DataType as AstDataType, StructField}; pub(super) fn data_type_to_ast_data_type(data_type: &DataType) -> AstDataType { @@ -187,12 +186,7 @@ pub(crate) static FUNC_TABLE: LazyLock>> = LazyLo .iter() .all(|t| *t != DataTypeName::Timestamptz) && !FUNC_BAN_LIST.contains(&func.func) - && (func.func != PbType::Cardinality - || !(func.inputs_type[0] == DataTypeName::List - && func.ret_type == DataTypeName::Int64)) - && (func.func != PbType::ArrayLength - || !(func.inputs_type[0] == DataTypeName::List - && func.ret_type == DataTypeName::Int64)) + && !func.deprecated // deprecated functions are not accepted by frontend }) .filter_map(|func| func.try_into().ok()) .for_each(|func: FuncSig| funcs.entry(func.ret_type.clone()).or_default().push(func)); diff --git a/src/tests/state_cleaning_test/Cargo.toml b/src/tests/state_cleaning_test/Cargo.toml index d655f0d2ecba0..8bf784fc982a1 100644 --- a/src/tests/state_cleaning_test/Cargo.toml +++ b/src/tests/state_cleaning_test/Cargo.toml @@ -21,7 +21,7 @@ prometheus = { version = "0.13" } regex = "1" risingwave_rt = { path = "../../utils/runtime" } serde = { version = "1", features = ["derive"] } -serde_with = "2" +serde_with = "3" tokio = { version = "0.2", package = "madsim-tokio" } tokio-postgres = "0.7" tokio-stream = { version = "0.1", features = ["fs"] } diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 3a8b443ddf1d7..277843e3c23d9 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -84,7 +84,7 @@ rust_decimal = { version = "1", features = ["db-postgres", "maths"] } scopeguard = { version = "1" } serde = { version = "1", features = ["alloc", "derive", "rc"] } serde_json = { version = "1", features = ["alloc"] } -serde_with = { version = "2", features = ["json"] } +serde_with = { version = "3", features = ["json"] } smallvec = { version = "1", default-features = false, features = ["serde"] } subtle = { version = "2" } time = { version = "0.3", features = ["formatting", "local-offset", "macros", "parsing"] } @@ -174,7 +174,7 @@ rust_decimal = { version = "1", features = ["db-postgres", "maths"] } scopeguard = { version = "1" } serde = { version = "1", features = ["alloc", "derive", "rc"] } serde_json = { version = "1", features = ["alloc"] } -serde_with = { version = "2", features = ["json"] } +serde_with = { version = "3", features = ["json"] } smallvec = { version = "1", default-features = false, features = ["serde"] } subtle = { version = "2" } syn-dff4ba8e3ae991db = { package = "syn", version = "1", features = ["extra-traits", "full", "visit", "visit-mut"] }