Merge pull request #28665 from kssenii/revert

Revert  #28082
This commit is contained in:
Kseniia Sumarokova 2021-09-07 09:43:02 +03:00 committed by GitHub
commit 8e8b15edd7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 1 additions and 118 deletions

View File

@ -617,12 +617,6 @@ 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)

View File

@ -175,7 +175,6 @@ 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);

View File

@ -1,68 +0,0 @@
#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>()},
{"main_dependency_database", std::make_shared<DataTypeString>()},
{"main_dependency_table", std::make_shared<DataTypeString>()},
{"view_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);
}
}
}
}

View File

@ -1,24 +0,0 @@
#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();
};
}

View File

@ -44,7 +44,6 @@
#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>
@ -96,7 +95,6 @@ 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");

View File

@ -1 +0,0 @@
02015_db materialized_view 02015_db view_source_tb Materialized

View File

@ -1,14 +0,0 @@
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;

View File

@ -512,7 +512,6 @@
"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
"02015_system_views"
"02002_row_level_filter_bug" ///create user
]
}