mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #28082 from zhongyuankai/add-system-of-table_views
Add system of table views
This commit is contained in:
commit
e5bcfba89e
@ -609,6 +609,12 @@ Dependencies DatabaseCatalog::getDependencies(const StorageID & from) const
|
||||
return Dependencies(iter->second.begin(), iter->second.end());
|
||||
}
|
||||
|
||||
ViewDependencies DatabaseCatalog::getViewDependencies() const
|
||||
{
|
||||
std::lock_guard lock{databases_mutex};
|
||||
return ViewDependencies(view_dependencies.begin(), view_dependencies.end());
|
||||
}
|
||||
|
||||
void
|
||||
DatabaseCatalog::updateDependency(const StorageID & old_from, const StorageID & old_where, const StorageID & new_from,
|
||||
const StorageID & new_where)
|
||||
|
@ -174,6 +174,7 @@ public:
|
||||
void addDependency(const StorageID & from, const StorageID & where);
|
||||
void removeDependency(const StorageID & from, const StorageID & where);
|
||||
Dependencies getDependencies(const StorageID & from) const;
|
||||
ViewDependencies getViewDependencies() const;
|
||||
|
||||
/// For Materialized and Live View
|
||||
void updateDependency(const StorageID & old_from, const StorageID & old_where,const StorageID & new_from, const StorageID & new_where);
|
||||
|
68
src/Storages/System/StorageSystemViews.cpp
Normal file
68
src/Storages/System/StorageSystemViews.cpp
Normal file
@ -0,0 +1,68 @@
|
||||
#include <Storages/System/StorageSystemViews.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/QueryViewsLog.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Storages/LiveView/StorageLiveView.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
|
||||
NamesAndTypesList StorageSystemViews::getNamesAndTypes()
|
||||
{
|
||||
auto view_type_datatype = std::make_shared<DataTypeEnum8>(DataTypeEnum8::Values{
|
||||
{"Default", static_cast<Int8>(QueryViewsLogElement::ViewType::DEFAULT)},
|
||||
{"Materialized", static_cast<Int8>(QueryViewsLogElement::ViewType::MATERIALIZED)},
|
||||
{"Live", static_cast<Int8>(QueryViewsLogElement::ViewType::LIVE)}});
|
||||
|
||||
return {
|
||||
{"database", std::make_shared<DataTypeString>()},
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"table_database", std::make_shared<DataTypeString>()},
|
||||
{"table", std::make_shared<DataTypeString>()},
|
||||
{"type", std::move(view_type_datatype)},
|
||||
};
|
||||
}
|
||||
|
||||
void StorageSystemViews::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
|
||||
{
|
||||
const auto access = context->getAccess();
|
||||
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
|
||||
|
||||
for (const auto & [table_id, view_ids] : DatabaseCatalog::instance().getViewDependencies())
|
||||
{
|
||||
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, table_id.database_name);
|
||||
|
||||
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, table_id.database_name, table_id.table_name))
|
||||
continue;
|
||||
|
||||
size_t col_num;
|
||||
for (const auto & view_id : view_ids)
|
||||
{
|
||||
auto view_ptr = DatabaseCatalog::instance().getTable(view_id, context);
|
||||
QueryViewsLogElement::ViewType type = QueryViewsLogElement::ViewType::DEFAULT;
|
||||
|
||||
if (typeid_cast<const StorageMaterializedView *>(view_ptr.get()))
|
||||
{
|
||||
type = QueryViewsLogElement::ViewType::MATERIALIZED;
|
||||
}
|
||||
else if (typeid_cast<const StorageLiveView *>(view_ptr.get()))
|
||||
{
|
||||
type = QueryViewsLogElement::ViewType::LIVE;
|
||||
}
|
||||
|
||||
col_num = 0;
|
||||
res_columns[col_num++]->insert(view_id.database_name);
|
||||
res_columns[col_num++]->insert(view_id.table_name);
|
||||
res_columns[col_num++]->insert(table_id.database_name);
|
||||
res_columns[col_num++]->insert(table_id.table_name);
|
||||
res_columns[col_num++]->insert(type);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
24
src/Storages/System/StorageSystemViews.h
Normal file
24
src/Storages/System/StorageSystemViews.h
Normal file
@ -0,0 +1,24 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/shared_ptr_helper.h>
|
||||
#include <Storages/System/IStorageSystemOneBlock.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class StorageSystemViews final : public shared_ptr_helper<StorageSystemViews>, public IStorageSystemOneBlock<StorageSystemViews>
|
||||
{
|
||||
friend struct shared_ptr_helper<StorageSystemViews>;
|
||||
protected:
|
||||
using IStorageSystemOneBlock::IStorageSystemOneBlock;
|
||||
|
||||
void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const override;
|
||||
|
||||
public:
|
||||
std::string getName() const override { return "SystemViews"; }
|
||||
|
||||
static NamesAndTypesList getNamesAndTypes();
|
||||
|
||||
};
|
||||
|
||||
}
|
@ -44,6 +44,7 @@
|
||||
#include <Storages/System/StorageSystemTableEngines.h>
|
||||
#include <Storages/System/StorageSystemTableFunctions.h>
|
||||
#include <Storages/System/StorageSystemTables.h>
|
||||
#include <Storages/System/StorageSystemViews.h>
|
||||
#include <Storages/System/StorageSystemZooKeeper.h>
|
||||
#include <Storages/System/StorageSystemContributors.h>
|
||||
#include <Storages/System/StorageSystemErrors.h>
|
||||
@ -95,6 +96,7 @@ void attachSystemTablesLocal(IDatabase & system_database)
|
||||
attach<StorageSystemZeros>(system_database, "zeros_mt", true);
|
||||
attach<StorageSystemDatabases>(system_database, "databases");
|
||||
attach<StorageSystemTables>(system_database, "tables");
|
||||
attach<StorageSystemViews>(system_database, "views");
|
||||
attach<StorageSystemColumns>(system_database, "columns");
|
||||
attach<StorageSystemFunctions>(system_database, "functions");
|
||||
attach<StorageSystemEvents>(system_database, "events");
|
||||
|
1
tests/queries/0_stateless/02015_system_views.reference
Normal file
1
tests/queries/0_stateless/02015_system_views.reference
Normal file
@ -0,0 +1 @@
|
||||
02015_db materialized_view 02015_db view_source_tb Materialized
|
14
tests/queries/0_stateless/02015_system_views.sql
Normal file
14
tests/queries/0_stateless/02015_system_views.sql
Normal file
@ -0,0 +1,14 @@
|
||||
DROP DATABASE IF EXISTS 02015_db;
|
||||
CREATE DATABASE IF NOT EXISTS 02015_db;
|
||||
|
||||
DROP TABLE IF EXISTS 02015_db.view_source_tb;
|
||||
CREATE TABLE IF NOT EXISTS 02015_db.view_source_tb (a UInt8, s String) ENGINE = MergeTree() ORDER BY a;
|
||||
|
||||
DROP TABLE IF EXISTS 02015_db.materialized_view;
|
||||
CREATE MATERIALIZED VIEW IF NOT EXISTS 02015_db.materialized_view ENGINE = ReplacingMergeTree() ORDER BY a AS SELECT * FROM 02015_db.view_source_tb;
|
||||
|
||||
SELECT * FROM system.views WHERE database='02015_db' and name = 'materialized_view';
|
||||
|
||||
DROP TABLE IF EXISTS 02015_db.materialized_view;
|
||||
DROP TABLE IF EXISTS 02015_db.view_source_tb;
|
||||
DROP DATABASE IF EXISTS 02015_db;
|
@ -512,6 +512,7 @@
|
||||
"01532_execute_merges_on_single_replica", /// static zk path
|
||||
"01530_drop_database_atomic_sync", /// creates database
|
||||
"02001_add_default_database_to_system_users", ///create user
|
||||
"02002_row_level_filter_bug" ///create user
|
||||
"02002_row_level_filter_bug", ///create user
|
||||
"02015_system_views"
|
||||
]
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user