mirror of
https://github.com/GreptimeTeam/greptimedb.git
synced 2026-01-14 01:02:55 +00:00
feat: support pg_namespace, pg_class and related psql command (#4428)
* feat: add function 'pg_catalog.pg_table_is_visible'q * feat: add 'pg_class' and 'pg_namespace', now we can run '\d' and '\dt'! * refactor: move memory_table::tables to utils::tables * refactor: move out predicate to system_schema to reuse it * feat: predicates pushdown * test: add pg_namespace, pg_class related sqlness test * fix: typos and license header * fix: sqlness test * refactor: use `expect` instead of `unwrap` here * refactor: remove the `information_schema::utils` mod * doc: make the comment in pg_get_userbyid more precise * doc: add TODO and comment in pg_catalog * fix: typo * fix: sqlness * doc: change to comment on PGClassBuilder to TODO
This commit is contained in:
@@ -15,6 +15,8 @@
|
||||
pub mod information_schema;
|
||||
mod memory_table;
|
||||
pub mod pg_catalog;
|
||||
mod predicate;
|
||||
mod utils;
|
||||
|
||||
use std::collections::HashMap;
|
||||
use std::sync::Arc;
|
||||
|
||||
@@ -18,14 +18,12 @@ pub mod flows;
|
||||
mod information_memory_table;
|
||||
pub mod key_column_usage;
|
||||
mod partitions;
|
||||
mod predicate;
|
||||
mod region_peers;
|
||||
mod runtime_metrics;
|
||||
pub mod schemata;
|
||||
mod table_constraints;
|
||||
mod table_names;
|
||||
pub mod tables;
|
||||
pub(crate) mod utils;
|
||||
mod views;
|
||||
|
||||
use std::collections::HashMap;
|
||||
@@ -37,7 +35,6 @@ use common_recordbatch::SendableRecordBatchStream;
|
||||
use datatypes::schema::SchemaRef;
|
||||
use lazy_static::lazy_static;
|
||||
use paste::paste;
|
||||
pub(crate) use predicate::Predicates;
|
||||
use store_api::storage::{ScanRequest, TableId};
|
||||
use table::metadata::TableType;
|
||||
use table::TableRef;
|
||||
@@ -58,6 +55,7 @@ use crate::system_schema::information_schema::schemata::InformationSchemaSchemat
|
||||
use crate::system_schema::information_schema::table_constraints::InformationSchemaTableConstraints;
|
||||
use crate::system_schema::information_schema::tables::InformationSchemaTables;
|
||||
use crate::system_schema::memory_table::MemoryTable;
|
||||
pub(crate) use crate::system_schema::predicate::Predicates;
|
||||
use crate::system_schema::SystemSchemaProvider;
|
||||
use crate::CatalogManager;
|
||||
|
||||
|
||||
@@ -41,7 +41,8 @@ use store_api::storage::{ScanRequest, TableId};
|
||||
|
||||
use super::CLUSTER_INFO;
|
||||
use crate::error::{CreateRecordBatchSnafu, InternalSnafu, ListNodesSnafu, Result};
|
||||
use crate::system_schema::information_schema::{utils, InformationTable, Predicates};
|
||||
use crate::system_schema::information_schema::{InformationTable, Predicates};
|
||||
use crate::system_schema::utils;
|
||||
use crate::CatalogManager;
|
||||
|
||||
const PEER_ID: &str = "peer_id";
|
||||
|
||||
@@ -19,7 +19,7 @@ use datatypes::schema::{Schema, SchemaRef};
|
||||
use datatypes::vectors::{Int64Vector, StringVector, VectorRef};
|
||||
|
||||
use super::table_names::*;
|
||||
use crate::system_schema::memory_table::tables::{
|
||||
use crate::system_schema::utils::tables::{
|
||||
bigint_column, datetime_column, string_column, string_columns,
|
||||
};
|
||||
|
||||
|
||||
@@ -36,7 +36,8 @@ use crate::error::{
|
||||
CreateRecordBatchSnafu, InternalSnafu, Result, TableMetadataManagerSnafu,
|
||||
UpgradeWeakCatalogManagerRefSnafu,
|
||||
};
|
||||
use crate::system_schema::information_schema::{utils, InformationTable, Predicates};
|
||||
use crate::system_schema::information_schema::{InformationTable, Predicates};
|
||||
use crate::system_schema::utils;
|
||||
use crate::CatalogManager;
|
||||
|
||||
pub const CATALOG_NAME: &str = "catalog_name";
|
||||
|
||||
@@ -13,7 +13,6 @@
|
||||
// limitations under the License.
|
||||
|
||||
mod table_columns;
|
||||
pub mod tables;
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
|
||||
@@ -13,6 +13,8 @@
|
||||
// limitations under the License.
|
||||
|
||||
mod pg_catalog_memory_table;
|
||||
mod pg_class;
|
||||
mod pg_namespace;
|
||||
mod table_names;
|
||||
|
||||
use std::collections::HashMap;
|
||||
@@ -23,11 +25,13 @@ use datatypes::schema::ColumnSchema;
|
||||
use lazy_static::lazy_static;
|
||||
use paste::paste;
|
||||
use pg_catalog_memory_table::get_schema_columns;
|
||||
use pg_class::PGClass;
|
||||
use pg_namespace::PGNamespace;
|
||||
use table::TableRef;
|
||||
pub use table_names::*;
|
||||
|
||||
use super::memory_table::tables::u32_column;
|
||||
use super::memory_table::MemoryTable;
|
||||
use super::utils::tables::u32_column;
|
||||
use super::{SystemSchemaProvider, SystemSchemaProviderInner, SystemTableRef};
|
||||
use crate::CatalogManager;
|
||||
|
||||
@@ -46,7 +50,7 @@ fn oid_column() -> ColumnSchema {
|
||||
/// [`PGCatalogProvider`] is the provider for a schema named `pg_catalog`, it is not a catalog.
|
||||
pub struct PGCatalogProvider {
|
||||
catalog_name: String,
|
||||
_catalog_manager: Weak<dyn CatalogManager>,
|
||||
catalog_manager: Weak<dyn CatalogManager>,
|
||||
tables: HashMap<String, TableRef>,
|
||||
}
|
||||
|
||||
@@ -79,7 +83,7 @@ impl PGCatalogProvider {
|
||||
pub fn new(catalog_name: String, catalog_manager: Weak<dyn CatalogManager>) -> Self {
|
||||
let mut provider = Self {
|
||||
catalog_name,
|
||||
_catalog_manager: catalog_manager,
|
||||
catalog_manager,
|
||||
tables: HashMap::new(),
|
||||
};
|
||||
provider.build_tables();
|
||||
@@ -90,9 +94,19 @@ impl PGCatalogProvider {
|
||||
// SECURITY NOTE:
|
||||
// Must follow the same security rules as [`InformationSchemaProvider::build_tables`].
|
||||
let mut tables = HashMap::new();
|
||||
// TODO(J0HN50N133): modeling the table_name as a enum type to get rid of expect/unwrap here
|
||||
// It's safe to unwrap here because we are sure that the constants have been handle correctly inside system_table.
|
||||
for name in MEMORY_TABLES.iter() {
|
||||
tables.insert(name.to_string(), self.build_table(name).expect(name));
|
||||
}
|
||||
tables.insert(
|
||||
PG_NAMESPACE.to_string(),
|
||||
self.build_table(PG_NAMESPACE).expect(PG_NAMESPACE),
|
||||
);
|
||||
tables.insert(
|
||||
PG_CLASS.to_string(),
|
||||
self.build_table(PG_CLASS).expect(PG_NAMESPACE),
|
||||
);
|
||||
self.tables = tables;
|
||||
}
|
||||
}
|
||||
@@ -105,6 +119,14 @@ impl SystemSchemaProviderInner for PGCatalogProvider {
|
||||
fn system_table(&self, name: &str) -> Option<SystemTableRef> {
|
||||
match name {
|
||||
table_names::PG_TYPE => setup_memory_table!(PG_TYPE),
|
||||
table_names::PG_NAMESPACE => Some(Arc::new(PGNamespace::new(
|
||||
self.catalog_name.clone(),
|
||||
self.catalog_manager.clone(),
|
||||
))),
|
||||
table_names::PG_CLASS => Some(Arc::new(PGClass::new(
|
||||
self.catalog_name.clone(),
|
||||
self.catalog_manager.clone(),
|
||||
))),
|
||||
_ => None,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,7 +20,7 @@ use datatypes::vectors::{Int16Vector, StringVector, UInt32Vector, VectorRef};
|
||||
use super::oid_column;
|
||||
use super::table_names::PG_TYPE;
|
||||
use crate::memory_table_cols;
|
||||
use crate::system_schema::memory_table::tables::{i16_column, string_column};
|
||||
use crate::system_schema::utils::tables::{i16_column, string_column};
|
||||
|
||||
fn pg_type_schema_columns() -> (Vec<ColumnSchema>, Vec<VectorRef>) {
|
||||
// TODO(j0hn50n133): acquire this information from `DataType` instead of hardcoding it to avoid regression.
|
||||
|
||||
249
src/catalog/src/system_schema/pg_catalog/pg_class.rs
Normal file
249
src/catalog/src/system_schema/pg_catalog/pg_class.rs
Normal file
@@ -0,0 +1,249 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::sync::{Arc, Weak};
|
||||
|
||||
use arrow_schema::SchemaRef as ArrowSchemaRef;
|
||||
use common_catalog::consts::PG_CATALOG_PG_CLASS_TABLE_ID;
|
||||
use common_error::ext::BoxedError;
|
||||
use common_recordbatch::adapter::RecordBatchStreamAdapter;
|
||||
use common_recordbatch::{DfSendableRecordBatchStream, RecordBatch};
|
||||
use datafusion::execution::TaskContext;
|
||||
use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter;
|
||||
use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream;
|
||||
use datatypes::scalars::ScalarVectorBuilder;
|
||||
use datatypes::schema::{Schema, SchemaRef};
|
||||
use datatypes::value::Value;
|
||||
use datatypes::vectors::{StringVectorBuilder, UInt32VectorBuilder, VectorRef};
|
||||
use futures::TryStreamExt;
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use store_api::storage::ScanRequest;
|
||||
use table::metadata::TableType;
|
||||
|
||||
use super::{OID_COLUMN_NAME, PG_CLASS};
|
||||
use crate::error::{
|
||||
CreateRecordBatchSnafu, InternalSnafu, Result, UpgradeWeakCatalogManagerRefSnafu,
|
||||
};
|
||||
use crate::information_schema::Predicates;
|
||||
use crate::system_schema::utils::tables::{string_column, u32_column};
|
||||
use crate::system_schema::SystemTable;
|
||||
use crate::CatalogManager;
|
||||
|
||||
// === column name ===
|
||||
pub const RELNAME: &str = "relname";
|
||||
pub const RELNAMESPACE: &str = "relnamespace";
|
||||
pub const RELKIND: &str = "relkind";
|
||||
pub const RELOWNER: &str = "relowner";
|
||||
|
||||
// === enum value of relkind ===
|
||||
pub const RELKIND_TABLE: &str = "r";
|
||||
pub const RELKIND_VIEW: &str = "v";
|
||||
|
||||
/// The initial capacity of the vector builders.
|
||||
const INIT_CAPACITY: usize = 42;
|
||||
/// The dummy owner id for the namespace.
|
||||
const DUMMY_OWNER_ID: u32 = 0;
|
||||
|
||||
/// The `pg_catalog.pg_class` table implementation.
|
||||
pub(super) struct PGClass {
|
||||
schema: SchemaRef,
|
||||
catalog_name: String,
|
||||
catalog_manager: Weak<dyn CatalogManager>,
|
||||
}
|
||||
|
||||
impl PGClass {
|
||||
pub(super) fn new(catalog_name: String, catalog_manager: Weak<dyn CatalogManager>) -> Self {
|
||||
Self {
|
||||
schema: Self::schema(),
|
||||
catalog_name,
|
||||
catalog_manager,
|
||||
}
|
||||
}
|
||||
|
||||
fn schema() -> SchemaRef {
|
||||
Arc::new(Schema::new(vec![
|
||||
u32_column(OID_COLUMN_NAME),
|
||||
string_column(RELNAME),
|
||||
string_column(RELNAMESPACE),
|
||||
string_column(RELKIND),
|
||||
u32_column(RELOWNER),
|
||||
]))
|
||||
}
|
||||
|
||||
fn builder(&self) -> PGClassBuilder {
|
||||
PGClassBuilder::new(
|
||||
self.schema.clone(),
|
||||
self.catalog_name.clone(),
|
||||
self.catalog_manager.clone(),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
impl SystemTable for PGClass {
|
||||
fn table_id(&self) -> table::metadata::TableId {
|
||||
PG_CATALOG_PG_CLASS_TABLE_ID
|
||||
}
|
||||
|
||||
fn table_name(&self) -> &'static str {
|
||||
PG_CLASS
|
||||
}
|
||||
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
|
||||
fn to_stream(
|
||||
&self,
|
||||
request: ScanRequest,
|
||||
) -> Result<common_recordbatch::SendableRecordBatchStream> {
|
||||
let schema = self.schema.arrow_schema().clone();
|
||||
let mut builder = self.builder();
|
||||
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
|
||||
schema,
|
||||
futures::stream::once(async move {
|
||||
builder
|
||||
.make_class(Some(request))
|
||||
.await
|
||||
.map(|x| x.into_df_record_batch())
|
||||
.map_err(Into::into)
|
||||
}),
|
||||
));
|
||||
Ok(Box::pin(
|
||||
RecordBatchStreamAdapter::try_new(stream)
|
||||
.map_err(BoxedError::new)
|
||||
.context(InternalSnafu)?,
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
impl DfPartitionStream for PGClass {
|
||||
fn schema(&self) -> &ArrowSchemaRef {
|
||||
self.schema.arrow_schema()
|
||||
}
|
||||
|
||||
fn execute(&self, _: Arc<TaskContext>) -> DfSendableRecordBatchStream {
|
||||
let schema = self.schema.arrow_schema().clone();
|
||||
let mut builder = self.builder();
|
||||
Box::pin(DfRecordBatchStreamAdapter::new(
|
||||
schema,
|
||||
futures::stream::once(async move {
|
||||
builder
|
||||
.make_class(None)
|
||||
.await
|
||||
.map(|x| x.into_df_record_batch())
|
||||
.map_err(Into::into)
|
||||
}),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
/// Builds the `pg_catalog.pg_class` table row by row
|
||||
/// TODO(J0HN50N133): `relowner` is always the [`DUMMY_OWNER_ID`] cuz we don't have user.
|
||||
/// Once we have user system, make it the actual owner of the table.
|
||||
struct PGClassBuilder {
|
||||
schema: SchemaRef,
|
||||
catalog_name: String,
|
||||
catalog_manager: Weak<dyn CatalogManager>,
|
||||
|
||||
oid: UInt32VectorBuilder,
|
||||
relname: StringVectorBuilder,
|
||||
relnamespace: StringVectorBuilder,
|
||||
relkind: StringVectorBuilder,
|
||||
relowner: UInt32VectorBuilder,
|
||||
}
|
||||
|
||||
impl PGClassBuilder {
|
||||
fn new(
|
||||
schema: SchemaRef,
|
||||
catalog_name: String,
|
||||
catalog_manager: Weak<dyn CatalogManager>,
|
||||
) -> Self {
|
||||
Self {
|
||||
schema,
|
||||
catalog_name,
|
||||
catalog_manager,
|
||||
|
||||
oid: UInt32VectorBuilder::with_capacity(INIT_CAPACITY),
|
||||
relname: StringVectorBuilder::with_capacity(INIT_CAPACITY),
|
||||
relnamespace: StringVectorBuilder::with_capacity(INIT_CAPACITY),
|
||||
relkind: StringVectorBuilder::with_capacity(INIT_CAPACITY),
|
||||
relowner: UInt32VectorBuilder::with_capacity(INIT_CAPACITY),
|
||||
}
|
||||
}
|
||||
|
||||
async fn make_class(&mut self, request: Option<ScanRequest>) -> Result<RecordBatch> {
|
||||
let catalog_name = self.catalog_name.clone();
|
||||
let catalog_manager = self
|
||||
.catalog_manager
|
||||
.upgrade()
|
||||
.context(UpgradeWeakCatalogManagerRefSnafu)?;
|
||||
let predicates = Predicates::from_scan_request(&request);
|
||||
for schema_name in catalog_manager.schema_names(&catalog_name).await? {
|
||||
let mut stream = catalog_manager.tables(&catalog_name, &schema_name);
|
||||
while let Some(table) = stream.try_next().await? {
|
||||
let table_info = table.table_info();
|
||||
self.add_class(
|
||||
&predicates,
|
||||
table_info.table_id(),
|
||||
&schema_name,
|
||||
&table_info.name,
|
||||
if table_info.table_type == TableType::View {
|
||||
RELKIND_VIEW
|
||||
} else {
|
||||
RELKIND_TABLE
|
||||
},
|
||||
);
|
||||
}
|
||||
}
|
||||
self.finish()
|
||||
}
|
||||
|
||||
fn add_class(
|
||||
&mut self,
|
||||
predicates: &Predicates,
|
||||
oid: u32,
|
||||
schema: &str,
|
||||
table: &str,
|
||||
kind: &str,
|
||||
) {
|
||||
let row = [
|
||||
(OID_COLUMN_NAME, &Value::from(oid)),
|
||||
(RELNAMESPACE, &Value::from(schema)),
|
||||
(RELNAME, &Value::from(table)),
|
||||
(RELKIND, &Value::from(kind)),
|
||||
(RELOWNER, &Value::from(DUMMY_OWNER_ID)),
|
||||
];
|
||||
|
||||
if !predicates.eval(&row) {
|
||||
return;
|
||||
}
|
||||
|
||||
self.oid.push(Some(oid));
|
||||
self.relnamespace.push(Some(schema));
|
||||
self.relname.push(Some(table));
|
||||
self.relkind.push(Some(kind));
|
||||
self.relowner.push(Some(DUMMY_OWNER_ID));
|
||||
}
|
||||
|
||||
fn finish(&mut self) -> Result<RecordBatch> {
|
||||
let columns: Vec<VectorRef> = vec![
|
||||
Arc::new(self.oid.finish()),
|
||||
Arc::new(self.relname.finish()),
|
||||
Arc::new(self.relnamespace.finish()),
|
||||
Arc::new(self.relkind.finish()),
|
||||
Arc::new(self.relowner.finish()),
|
||||
];
|
||||
RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu)
|
||||
}
|
||||
}
|
||||
191
src/catalog/src/system_schema/pg_catalog/pg_namespace.rs
Normal file
191
src/catalog/src/system_schema/pg_catalog/pg_namespace.rs
Normal file
@@ -0,0 +1,191 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::sync::{Arc, Weak};
|
||||
|
||||
use arrow_schema::SchemaRef as ArrowSchemaRef;
|
||||
use common_catalog::consts::PG_CATALOG_PG_NAMESPACE_TABLE_ID;
|
||||
use common_error::ext::BoxedError;
|
||||
use common_recordbatch::adapter::RecordBatchStreamAdapter;
|
||||
use common_recordbatch::{DfSendableRecordBatchStream, RecordBatch, SendableRecordBatchStream};
|
||||
use datafusion::execution::TaskContext;
|
||||
use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter;
|
||||
use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream;
|
||||
use datatypes::scalars::ScalarVectorBuilder;
|
||||
use datatypes::schema::{Schema, SchemaRef};
|
||||
use datatypes::value::Value;
|
||||
use datatypes::vectors::{StringVectorBuilder, VectorRef};
|
||||
use snafu::{OptionExt, ResultExt};
|
||||
use store_api::storage::ScanRequest;
|
||||
|
||||
use super::{OID_COLUMN_NAME, PG_NAMESPACE};
|
||||
use crate::error::{
|
||||
CreateRecordBatchSnafu, InternalSnafu, Result, UpgradeWeakCatalogManagerRefSnafu,
|
||||
};
|
||||
use crate::information_schema::Predicates;
|
||||
use crate::system_schema::utils::tables::string_column;
|
||||
use crate::system_schema::SystemTable;
|
||||
use crate::CatalogManager;
|
||||
|
||||
/// The `pg_catalog.pg_namespace` table implementation.
|
||||
/// namespace is a schema in greptime
|
||||
|
||||
const NSPNAME: &str = "nspname";
|
||||
const INIT_CAPACITY: usize = 42;
|
||||
|
||||
pub(super) struct PGNamespace {
|
||||
schema: SchemaRef,
|
||||
catalog_name: String,
|
||||
catalog_manager: Weak<dyn CatalogManager>,
|
||||
}
|
||||
|
||||
impl PGNamespace {
|
||||
pub(super) fn new(catalog_name: String, catalog_manager: Weak<dyn CatalogManager>) -> Self {
|
||||
Self {
|
||||
schema: Self::schema(),
|
||||
catalog_name,
|
||||
catalog_manager,
|
||||
}
|
||||
}
|
||||
|
||||
fn schema() -> SchemaRef {
|
||||
Arc::new(Schema::new(vec![
|
||||
// TODO(J0HN50N133): we do not have a numeric schema id, use schema name as a workaround. Use a proper schema id once we have it.
|
||||
string_column(OID_COLUMN_NAME),
|
||||
string_column(NSPNAME),
|
||||
]))
|
||||
}
|
||||
|
||||
fn builder(&self) -> PGNamespaceBuilder {
|
||||
PGNamespaceBuilder::new(
|
||||
self.schema.clone(),
|
||||
self.catalog_name.clone(),
|
||||
self.catalog_manager.clone(),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
impl SystemTable for PGNamespace {
|
||||
fn schema(&self) -> SchemaRef {
|
||||
self.schema.clone()
|
||||
}
|
||||
|
||||
fn table_id(&self) -> table::metadata::TableId {
|
||||
PG_CATALOG_PG_NAMESPACE_TABLE_ID
|
||||
}
|
||||
|
||||
fn table_name(&self) -> &'static str {
|
||||
PG_NAMESPACE
|
||||
}
|
||||
|
||||
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
|
||||
let schema = self.schema.arrow_schema().clone();
|
||||
let mut builder = self.builder();
|
||||
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
|
||||
schema,
|
||||
futures::stream::once(async move {
|
||||
builder
|
||||
.make_namespace(Some(request))
|
||||
.await
|
||||
.map(|x| x.into_df_record_batch())
|
||||
.map_err(Into::into)
|
||||
}),
|
||||
));
|
||||
Ok(Box::pin(
|
||||
RecordBatchStreamAdapter::try_new(stream)
|
||||
.map_err(BoxedError::new)
|
||||
.context(InternalSnafu)?,
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
impl DfPartitionStream for PGNamespace {
|
||||
fn schema(&self) -> &ArrowSchemaRef {
|
||||
self.schema.arrow_schema()
|
||||
}
|
||||
|
||||
fn execute(&self, _: Arc<TaskContext>) -> DfSendableRecordBatchStream {
|
||||
let schema = self.schema.arrow_schema().clone();
|
||||
let mut builder = self.builder();
|
||||
Box::pin(DfRecordBatchStreamAdapter::new(
|
||||
schema,
|
||||
futures::stream::once(async move {
|
||||
builder
|
||||
.make_namespace(None)
|
||||
.await
|
||||
.map(|x| x.into_df_record_batch())
|
||||
.map_err(Into::into)
|
||||
}),
|
||||
))
|
||||
}
|
||||
}
|
||||
|
||||
/// Builds the `pg_catalog.pg_namespace` table row by row
|
||||
/// `oid` use schema name as a workaround since we don't have numeric schema id.
|
||||
/// `nspname` is the schema name.
|
||||
struct PGNamespaceBuilder {
|
||||
schema: SchemaRef,
|
||||
catalog_name: String,
|
||||
catalog_manager: Weak<dyn CatalogManager>,
|
||||
|
||||
oid: StringVectorBuilder,
|
||||
nspname: StringVectorBuilder,
|
||||
}
|
||||
|
||||
impl PGNamespaceBuilder {
|
||||
fn new(
|
||||
schema: SchemaRef,
|
||||
catalog_name: String,
|
||||
catalog_manager: Weak<dyn CatalogManager>,
|
||||
) -> Self {
|
||||
Self {
|
||||
schema,
|
||||
catalog_name,
|
||||
catalog_manager,
|
||||
oid: StringVectorBuilder::with_capacity(INIT_CAPACITY),
|
||||
nspname: StringVectorBuilder::with_capacity(INIT_CAPACITY),
|
||||
}
|
||||
}
|
||||
|
||||
/// Construct the `pg_catalog.pg_namespace` virtual table
|
||||
async fn make_namespace(&mut self, request: Option<ScanRequest>) -> Result<RecordBatch> {
|
||||
let catalog_name = self.catalog_name.clone();
|
||||
let catalog_manager = self
|
||||
.catalog_manager
|
||||
.upgrade()
|
||||
.context(UpgradeWeakCatalogManagerRefSnafu)?;
|
||||
let predicates = Predicates::from_scan_request(&request);
|
||||
for schema_name in catalog_manager.schema_names(&catalog_name).await? {
|
||||
self.add_namespace(&predicates, &schema_name);
|
||||
}
|
||||
self.finish()
|
||||
}
|
||||
fn finish(&mut self) -> Result<RecordBatch> {
|
||||
let columns: Vec<VectorRef> =
|
||||
vec![Arc::new(self.oid.finish()), Arc::new(self.nspname.finish())];
|
||||
RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu)
|
||||
}
|
||||
|
||||
fn add_namespace(&mut self, predicates: &Predicates, schema_name: &str) {
|
||||
let row = [
|
||||
(OID_COLUMN_NAME, &Value::from(schema_name)),
|
||||
(NSPNAME, &Value::from(schema_name)),
|
||||
];
|
||||
if !predicates.eval(&row) {
|
||||
return;
|
||||
}
|
||||
self.oid.push(Some(schema_name));
|
||||
self.nspname.push(Some(schema_name));
|
||||
}
|
||||
}
|
||||
@@ -25,7 +25,7 @@ type ColumnName = String;
|
||||
/// we only support these simple predicates currently.
|
||||
/// TODO(dennis): supports more predicate types.
|
||||
#[derive(Clone, PartialEq, Eq, Debug)]
|
||||
enum Predicate {
|
||||
pub(crate) enum Predicate {
|
||||
Eq(ColumnName, Value),
|
||||
Like(ColumnName, String, bool),
|
||||
NotEq(ColumnName, Value),
|
||||
@@ -12,6 +12,8 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
pub mod tables;
|
||||
|
||||
use std::sync::{Arc, Weak};
|
||||
|
||||
use common_config::Mode;
|
||||
@@ -103,6 +103,7 @@ pub const INFORMATION_SCHEMA_FLOW_TABLE_ID: u32 = 33;
|
||||
/// ----- Begin of pg_catalog tables -----
|
||||
pub const PG_CATALOG_PG_CLASS_TABLE_ID: u32 = 256;
|
||||
pub const PG_CATALOG_PG_TYPE_TABLE_ID: u32 = 257;
|
||||
pub const PG_CATALOG_PG_NAMESPACE_TABLE_ID: u32 = 258;
|
||||
|
||||
/// ----- End of pg_catalog tables -----
|
||||
pub const MITO_ENGINE: &str = "mito";
|
||||
|
||||
@@ -19,8 +19,6 @@ use snafu::ResultExt;
|
||||
|
||||
use crate::scalars::expression::ctx::EvalContext;
|
||||
|
||||
/// TODO: remove the allow_unused when it's used.
|
||||
#[allow(unused)]
|
||||
pub fn scalar_unary_op<L: Scalar, O: Scalar, F>(
|
||||
l: &VectorRef,
|
||||
f: F,
|
||||
|
||||
@@ -14,6 +14,7 @@
|
||||
|
||||
mod build;
|
||||
mod database;
|
||||
mod pg_catalog;
|
||||
mod procedure_state;
|
||||
mod timezone;
|
||||
mod version;
|
||||
@@ -22,6 +23,7 @@ use std::sync::Arc;
|
||||
|
||||
use build::BuildFunction;
|
||||
use database::DatabaseFunction;
|
||||
use pg_catalog::PGCatalogFunction;
|
||||
use procedure_state::ProcedureStateFunction;
|
||||
use timezone::TimezoneFunction;
|
||||
use version::VersionFunction;
|
||||
@@ -37,5 +39,6 @@ impl SystemFunction {
|
||||
registry.register(Arc::new(DatabaseFunction));
|
||||
registry.register(Arc::new(TimezoneFunction));
|
||||
registry.register(Arc::new(ProcedureStateFunction));
|
||||
PGCatalogFunction::register(registry);
|
||||
}
|
||||
}
|
||||
|
||||
39
src/common/function/src/system/pg_catalog.rs
Normal file
39
src/common/function/src/system/pg_catalog.rs
Normal file
@@ -0,0 +1,39 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
mod pg_get_userbyid;
|
||||
mod table_is_visible;
|
||||
|
||||
use std::sync::Arc;
|
||||
|
||||
use pg_get_userbyid::PGGetUserByIdFunction;
|
||||
use table_is_visible::PGTableIsVisibleFunction;
|
||||
|
||||
use crate::function_registry::FunctionRegistry;
|
||||
|
||||
#[macro_export]
|
||||
macro_rules! pg_catalog_func_fullname {
|
||||
($name:literal) => {
|
||||
concat!("pg_catalog.", $name)
|
||||
};
|
||||
}
|
||||
|
||||
pub(super) struct PGCatalogFunction;
|
||||
|
||||
impl PGCatalogFunction {
|
||||
pub fn register(registry: &FunctionRegistry) {
|
||||
registry.register(Arc::new(PGTableIsVisibleFunction));
|
||||
registry.register(Arc::new(PGGetUserByIdFunction));
|
||||
}
|
||||
}
|
||||
72
src/common/function/src/system/pg_catalog/pg_get_userbyid.rs
Normal file
72
src/common/function/src/system/pg_catalog/pg_get_userbyid.rs
Normal file
@@ -0,0 +1,72 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::fmt::{self};
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_query::error::Result;
|
||||
use common_query::prelude::{Signature, Volatility};
|
||||
use datatypes::prelude::{ConcreteDataType, DataType, VectorRef};
|
||||
use datatypes::types::LogicalPrimitiveType;
|
||||
use datatypes::with_match_primitive_type_id;
|
||||
use num_traits::AsPrimitive;
|
||||
|
||||
use crate::function::{Function, FunctionContext};
|
||||
use crate::scalars::expression::{scalar_unary_op, EvalContext};
|
||||
|
||||
#[derive(Clone, Debug, Default)]
|
||||
pub struct PGGetUserByIdFunction;
|
||||
|
||||
const NAME: &str = crate::pg_catalog_func_fullname!("pg_get_userbyid");
|
||||
|
||||
impl fmt::Display for PGGetUserByIdFunction {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
write!(f, crate::pg_catalog_func_fullname!("PG_GET_USERBYID"))
|
||||
}
|
||||
}
|
||||
|
||||
impl Function for PGGetUserByIdFunction {
|
||||
fn name(&self) -> &str {
|
||||
NAME
|
||||
}
|
||||
|
||||
fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result<ConcreteDataType> {
|
||||
Ok(ConcreteDataType::string_datatype())
|
||||
}
|
||||
|
||||
fn signature(&self) -> Signature {
|
||||
Signature::uniform(
|
||||
1,
|
||||
vec![ConcreteDataType::uint32_datatype()],
|
||||
Volatility::Immutable,
|
||||
)
|
||||
}
|
||||
|
||||
fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result<VectorRef> {
|
||||
with_match_primitive_type_id!(columns[0].data_type().logical_type_id(), |$T| {
|
||||
let col = scalar_unary_op::<<$T as LogicalPrimitiveType>::Native, String, _>(&columns[0], pg_get_user_by_id, &mut EvalContext::default())?;
|
||||
Ok(Arc::new(col))
|
||||
}, {
|
||||
unreachable!()
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
fn pg_get_user_by_id<I>(table_oid: Option<I>, _ctx: &mut EvalContext) -> Option<String>
|
||||
where
|
||||
I: AsPrimitive<u32>,
|
||||
{
|
||||
// TODO(J0HN50N133): We lack way to get the user_info by a numeric value. Once we have it, we can implement this function.
|
||||
table_oid.map(|_| "".to_string())
|
||||
}
|
||||
@@ -0,0 +1,72 @@
|
||||
// Copyright 2023 Greptime Team
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
use std::fmt::{self};
|
||||
use std::sync::Arc;
|
||||
|
||||
use common_query::error::Result;
|
||||
use common_query::prelude::{Signature, Volatility};
|
||||
use datatypes::prelude::{ConcreteDataType, DataType, VectorRef};
|
||||
use datatypes::types::LogicalPrimitiveType;
|
||||
use datatypes::with_match_primitive_type_id;
|
||||
use num_traits::AsPrimitive;
|
||||
|
||||
use crate::function::{Function, FunctionContext};
|
||||
use crate::scalars::expression::{scalar_unary_op, EvalContext};
|
||||
|
||||
#[derive(Clone, Debug, Default)]
|
||||
pub struct PGTableIsVisibleFunction;
|
||||
|
||||
const NAME: &str = crate::pg_catalog_func_fullname!("pg_table_is_visible");
|
||||
|
||||
impl fmt::Display for PGTableIsVisibleFunction {
|
||||
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
|
||||
write!(f, crate::pg_catalog_func_fullname!("PG_TABLE_IS_VISIBLE"))
|
||||
}
|
||||
}
|
||||
|
||||
impl Function for PGTableIsVisibleFunction {
|
||||
fn name(&self) -> &str {
|
||||
NAME
|
||||
}
|
||||
|
||||
fn return_type(&self, _input_types: &[ConcreteDataType]) -> Result<ConcreteDataType> {
|
||||
Ok(ConcreteDataType::boolean_datatype())
|
||||
}
|
||||
|
||||
fn signature(&self) -> Signature {
|
||||
Signature::uniform(
|
||||
1,
|
||||
vec![ConcreteDataType::uint32_datatype()],
|
||||
Volatility::Immutable,
|
||||
)
|
||||
}
|
||||
|
||||
fn eval(&self, _func_ctx: FunctionContext, columns: &[VectorRef]) -> Result<VectorRef> {
|
||||
with_match_primitive_type_id!(columns[0].data_type().logical_type_id(), |$T| {
|
||||
let col = scalar_unary_op::<<$T as LogicalPrimitiveType>::Native, bool, _>(&columns[0], pg_table_is_visible, &mut EvalContext::default())?;
|
||||
Ok(Arc::new(col))
|
||||
}, {
|
||||
unreachable!()
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
fn pg_table_is_visible<I>(table_oid: Option<I>, _ctx: &mut EvalContext) -> Option<bool>
|
||||
where
|
||||
I: AsPrimitive<u32>,
|
||||
{
|
||||
// There is no table visibility in greptime, so we always return true
|
||||
table_oid.map(|_| true)
|
||||
}
|
||||
@@ -24,14 +24,16 @@ Affected Rows: 0
|
||||
|
||||
select table_catalog, table_schema, table_name from information_schema.tables where table_schema != 'information_schema';
|
||||
|
||||
+---------------+--------------+------------+
|
||||
| table_catalog | table_schema | table_name |
|
||||
+---------------+--------------+------------+
|
||||
| greptime | abc | t |
|
||||
| greptime | abcde | t |
|
||||
| greptime | pg_catalog | pg_type |
|
||||
| greptime | public | numbers |
|
||||
+---------------+--------------+------------+
|
||||
+---------------+--------------+--------------+
|
||||
| table_catalog | table_schema | table_name |
|
||||
+---------------+--------------+--------------+
|
||||
| greptime | abc | t |
|
||||
| greptime | abcde | t |
|
||||
| greptime | pg_catalog | pg_class |
|
||||
| greptime | pg_catalog | pg_type |
|
||||
| greptime | pg_catalog | pg_namespace |
|
||||
| greptime | public | numbers |
|
||||
+---------------+--------------+--------------+
|
||||
|
||||
use public;
|
||||
|
||||
|
||||
@@ -45,6 +45,8 @@ order by table_schema, table_name;
|
||||
|greptime|information_schema|tables|LOCALTEMPORARY|3|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|
||||
|greptime|information_schema|triggers|LOCALTEMPORARY|24|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|
||||
|greptime|information_schema|views|LOCALTEMPORARY|32|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|
||||
|greptime|pg_catalog|pg_class|LOCALTEMPORARY|256|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|
||||
|greptime|pg_catalog|pg_namespace|LOCALTEMPORARY|258|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|
||||
|greptime|pg_catalog|pg_type|LOCALTEMPORARY|257|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|
||||
|greptime|public|numbers|LOCALTEMPORARY|2|0|0|0|0|0|test_engine|11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|
||||
+++++++++++++++++++++++++
|
||||
@@ -411,6 +413,13 @@ select * from information_schema.columns order by table_schema, table_name, colu
|
||||
| greptime | information_schema | views | table_name | 3 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
|
||||
| greptime | information_schema | views | table_schema | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
|
||||
| greptime | information_schema | views | view_definition | 4 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
|
||||
| greptime | pg_catalog | pg_class | oid | 1 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
|
||||
| greptime | pg_catalog | pg_class | relkind | 4 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
|
||||
| greptime | pg_catalog | pg_class | relname | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
|
||||
| greptime | pg_catalog | pg_class | relnamespace | 3 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
|
||||
| greptime | pg_catalog | pg_class | relowner | 5 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
|
||||
| greptime | pg_catalog | pg_namespace | nspname | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
|
||||
| greptime | pg_catalog | pg_namespace | oid | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
|
||||
| greptime | pg_catalog | pg_type | oid | 1 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
|
||||
| greptime | pg_catalog | pg_type | typlen | 3 | | | 5 | 0 | | | | | | select,insert | | Int16 | smallint | FIELD | | No | smallint | | |
|
||||
| greptime | pg_catalog | pg_type | typname | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
|
||||
|
||||
@@ -30,3 +30,185 @@ select * from pg_catalog.pg_type order by oid;
|
||||
| 20 | List | -1 |
|
||||
+-----+-----------+--------+
|
||||
|
||||
-- \d
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('r','p','v','m','S','f','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
+--------+---------+-------+-------+
|
||||
| Schema | Name | Type | Owner |
|
||||
+--------+---------+-------+-------+
|
||||
| public | numbers | table | |
|
||||
+--------+---------+-------+-------+
|
||||
|
||||
-- \dt
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('r','p','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
+--------+---------+-------+-------+
|
||||
| Schema | Name | Type | Owner |
|
||||
+--------+---------+-------+-------+
|
||||
| public | numbers | table | |
|
||||
+--------+---------+-------+-------+
|
||||
|
||||
create
|
||||
database my_db;
|
||||
|
||||
Affected Rows: 1
|
||||
|
||||
use my_db;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
create table foo
|
||||
(
|
||||
ts TIMESTAMP TIME INDEX
|
||||
);
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
-- show tables in `my_db`
|
||||
select relname
|
||||
from pg_catalog.pg_class
|
||||
where relnamespace = (
|
||||
select oid
|
||||
from pg_catalog.pg_namespace
|
||||
where nspname = 'my_db'
|
||||
);
|
||||
|
||||
+---------+
|
||||
| relname |
|
||||
+---------+
|
||||
| foo |
|
||||
+---------+
|
||||
|
||||
-- \dt
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('r','p','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
+--------+---------+-------+-------+
|
||||
| Schema | Name | Type | Owner |
|
||||
+--------+---------+-------+-------+
|
||||
| my_db | foo | table | |
|
||||
| public | numbers | table | |
|
||||
+--------+---------+-------+-------+
|
||||
|
||||
-- show tables in `my_db`, `public`
|
||||
select relname
|
||||
from pg_catalog.pg_class
|
||||
where relnamespace in (
|
||||
select oid
|
||||
from pg_catalog.pg_namespace
|
||||
where nspname = 'my_db' or nspname = 'public'
|
||||
)
|
||||
order by relname;
|
||||
|
||||
+---------+
|
||||
| relname |
|
||||
+---------+
|
||||
| foo |
|
||||
| numbers |
|
||||
+---------+
|
||||
|
||||
select relname
|
||||
from pg_catalog.pg_class
|
||||
where relnamespace in (
|
||||
select oid
|
||||
from pg_catalog.pg_namespace
|
||||
where nspname like 'my%'
|
||||
);
|
||||
|
||||
+---------+
|
||||
| relname |
|
||||
+---------+
|
||||
| foo |
|
||||
+---------+
|
||||
|
||||
select relnamespace, relname, relkind
|
||||
from pg_catalog.pg_class
|
||||
where relnamespace in (
|
||||
select oid
|
||||
from pg_catalog.pg_namespace
|
||||
where nspname <> 'public'
|
||||
and nspname <> 'information_schema'
|
||||
and nspname <> 'pg_catalog'
|
||||
)
|
||||
order by relnamespace, relname;
|
||||
|
||||
+--------------+---------+---------+
|
||||
| relnamespace | relname | relkind |
|
||||
+--------------+---------+---------+
|
||||
| my_db | foo | r |
|
||||
+--------------+---------+---------+
|
||||
|
||||
use public;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
drop schema my_db;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
use pg_catalog;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
-- pg_class
|
||||
desc table pg_class;
|
||||
|
||||
+--------------+--------+-----+------+---------+---------------+
|
||||
| Column | Type | Key | Null | Default | Semantic Type |
|
||||
+--------------+--------+-----+------+---------+---------------+
|
||||
| oid | UInt32 | | NO | | FIELD |
|
||||
| relname | String | | NO | | FIELD |
|
||||
| relnamespace | String | | NO | | FIELD |
|
||||
| relkind | String | | NO | | FIELD |
|
||||
| relowner | UInt32 | | NO | | FIELD |
|
||||
+--------------+--------+-----+------+---------+---------------+
|
||||
|
||||
desc table pg_namespace;
|
||||
|
||||
+---------+--------+-----+------+---------+---------------+
|
||||
| Column | Type | Key | Null | Default | Semantic Type |
|
||||
+---------+--------+-----+------+---------+---------------+
|
||||
| oid | String | | NO | | FIELD |
|
||||
| nspname | String | | NO | | FIELD |
|
||||
+---------+--------+-----+------+---------+---------------+
|
||||
|
||||
drop table my_db.foo;
|
||||
|
||||
Error: 4001(TableNotFound), Table not found: greptime.my_db.foo
|
||||
|
||||
use public;
|
||||
|
||||
Affected Rows: 0
|
||||
|
||||
|
||||
@@ -1,4 +1,110 @@
|
||||
-- should not able to create pg_catalog
|
||||
create database pg_catalog;
|
||||
|
||||
select * from pg_catalog.pg_type order by oid;
|
||||
select * from pg_catalog.pg_type order by oid;
|
||||
|
||||
-- \d
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('r','p','v','m','S','f','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
-- \dt
|
||||
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('r','p','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
create
|
||||
database my_db;
|
||||
|
||||
use my_db;
|
||||
|
||||
create table foo
|
||||
(
|
||||
ts TIMESTAMP TIME INDEX
|
||||
);
|
||||
|
||||
-- show tables in `my_db`
|
||||
select relname
|
||||
from pg_catalog.pg_class
|
||||
where relnamespace = (
|
||||
select oid
|
||||
from pg_catalog.pg_namespace
|
||||
where nspname = 'my_db'
|
||||
);
|
||||
|
||||
-- \dt
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('r','p','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
-- show tables in `my_db`, `public`
|
||||
select relname
|
||||
from pg_catalog.pg_class
|
||||
where relnamespace in (
|
||||
select oid
|
||||
from pg_catalog.pg_namespace
|
||||
where nspname = 'my_db' or nspname = 'public'
|
||||
)
|
||||
order by relname;
|
||||
|
||||
select relname
|
||||
from pg_catalog.pg_class
|
||||
where relnamespace in (
|
||||
select oid
|
||||
from pg_catalog.pg_namespace
|
||||
where nspname like 'my%'
|
||||
);
|
||||
|
||||
select relnamespace, relname, relkind
|
||||
from pg_catalog.pg_class
|
||||
where relnamespace in (
|
||||
select oid
|
||||
from pg_catalog.pg_namespace
|
||||
where nspname <> 'public'
|
||||
and nspname <> 'information_schema'
|
||||
and nspname <> 'pg_catalog'
|
||||
)
|
||||
order by relnamespace, relname;
|
||||
|
||||
use public;
|
||||
|
||||
drop schema my_db;
|
||||
|
||||
use pg_catalog;
|
||||
|
||||
-- pg_class
|
||||
desc table pg_class;
|
||||
|
||||
desc table pg_namespace;
|
||||
|
||||
drop table my_db.foo;
|
||||
|
||||
use public;
|
||||
|
||||
@@ -63,6 +63,26 @@ SHOW FULL TABLES;
|
||||
| test_view | VIEW |
|
||||
+------------+-----------------+
|
||||
|
||||
-- psql: \dv
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('v','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
+--------+-----------+------+-------+
|
||||
| Schema | Name | Type | Owner |
|
||||
+--------+-----------+------+-------+
|
||||
| public | test_view | view | |
|
||||
+--------+-----------+------+-------+
|
||||
|
||||
-- SQLNESS REPLACE (\s\d+\s) ID
|
||||
-- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) DATETIME
|
||||
-- SQLNESS REPLACE [\u0020\-]+
|
||||
@@ -90,6 +110,8 @@ SELECT * FROM INFORMATION_SCHEMA.TABLES ORDER BY TABLE_NAME, TABLE_TYPE;
|
||||
|greptime|information_schema|optimizer_trace|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|
||||
|greptime|information_schema|parameters|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|
||||
|greptime|information_schema|partitions|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|
||||
|greptime|pg_catalog|pg_class|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|
||||
|greptime|pg_catalog|pg_namespace|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|
||||
|greptime|pg_catalog|pg_type|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|
||||
|greptime|information_schema|profiling|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|
||||
|greptime|information_schema|referential_constraints|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|
||||
@@ -169,3 +191,20 @@ SHOW TABLES;
|
||||
| numbers |
|
||||
+---------+
|
||||
|
||||
-- psql: \dv
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('v','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
++
|
||||
++
|
||||
|
||||
|
||||
@@ -27,6 +27,20 @@ SHOW TABLES;
|
||||
|
||||
SHOW FULL TABLES;
|
||||
|
||||
-- psql: \dv
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('v','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
-- SQLNESS REPLACE (\s\d+\s) ID
|
||||
-- SQLNESS REPLACE (\s[\-0-9T:\.]{15,}) DATETIME
|
||||
-- SQLNESS REPLACE [\u0020\-]+
|
||||
@@ -50,3 +64,17 @@ DROP TABLE test_table;
|
||||
SELECT * FROM test_view LIMIT 10;
|
||||
|
||||
SHOW TABLES;
|
||||
|
||||
-- psql: \dv
|
||||
SELECT n.nspname as "Schema",
|
||||
c.relname as "Name",
|
||||
CASE c.relkind WHEN 'r' THEN 'table' WHEN 'v' THEN 'view' WHEN 'm' THEN 'materialized view' WHEN 'i' THEN 'index' WHEN 'S' THEN 'sequence' WHEN 't' THEN 'TOAST table' WHEN 'f' THEN 'foreign table' WHEN 'p' THEN 'partitioned table' WHEN 'I' THEN 'partitioned index' END as "Type",
|
||||
pg_catalog.pg_get_userbyid(c.relowner) as "Owner"
|
||||
FROM pg_catalog.pg_class c
|
||||
LEFT JOIN pg_catalog.pg_namespace n ON n.oid = c.relnamespace
|
||||
WHERE c.relkind IN ('v','')
|
||||
AND n.nspname <> 'pg_catalog'
|
||||
AND n.nspname !~ '^pg_toast'
|
||||
AND n.nspname <> 'information_schema'
|
||||
AND pg_catalog.pg_table_is_visible(c.oid)
|
||||
ORDER BY 1,2;
|
||||
|
||||
Reference in New Issue
Block a user