mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
* Add StorageSystemISTables.cpp/.h * Another attempt * Columns and Views * Add information schema db and fix information schema 'tables' table * fix build * remove copy-paste, add views to system tables * add test * fix * fix_tests Co-authored-by: Damir Petrov <petrovdamir2235@gmail.com> Co-authored-by: Damir Petrov <0442a403@verstehen.sas.yp-c.yandex.net> Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
This commit is contained in:
parent
fc4f119417
commit
341a6c51d6
@ -8,6 +8,7 @@
|
||||
#include <Poco/NullChannel.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <Storages/System/attachInformationSchemaTables.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include <Interpreters/loadMetadata.h>
|
||||
@ -179,20 +180,18 @@ void LocalServer::tryInitPath()
|
||||
}
|
||||
|
||||
|
||||
static void attachSystemTables(ContextPtr context)
|
||||
static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const String & database_name)
|
||||
{
|
||||
DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(DatabaseCatalog::SYSTEM_DATABASE);
|
||||
DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(database_name);
|
||||
if (!system_database)
|
||||
{
|
||||
/// TODO: add attachTableDelayed into DatabaseMemory to speedup loading
|
||||
system_database = std::make_shared<DatabaseMemory>(DatabaseCatalog::SYSTEM_DATABASE, context);
|
||||
DatabaseCatalog::instance().attachDatabase(DatabaseCatalog::SYSTEM_DATABASE, system_database);
|
||||
system_database = std::make_shared<DatabaseMemory>(database_name, context);
|
||||
DatabaseCatalog::instance().attachDatabase(database_name, system_database);
|
||||
}
|
||||
|
||||
attachSystemTablesLocal(*system_database);
|
||||
return system_database;
|
||||
}
|
||||
|
||||
|
||||
int LocalServer::main(const std::vector<std::string> & /*args*/)
|
||||
try
|
||||
{
|
||||
@ -303,14 +302,18 @@ try
|
||||
fs::create_directories(fs::path(path) / "data/");
|
||||
fs::create_directories(fs::path(path) / "metadata/");
|
||||
loadMetadataSystem(global_context);
|
||||
attachSystemTables(global_context);
|
||||
attachSystemTablesLocal(*createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
|
||||
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
|
||||
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
|
||||
loadMetadata(global_context);
|
||||
DatabaseCatalog::instance().loadDatabases();
|
||||
LOG_DEBUG(log, "Loaded metadata.");
|
||||
}
|
||||
else if (!config().has("no-system-tables"))
|
||||
{
|
||||
attachSystemTables(global_context);
|
||||
attachSystemTablesLocal(*createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE));
|
||||
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA));
|
||||
attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
|
||||
}
|
||||
|
||||
processQueries();
|
||||
|
@ -56,6 +56,7 @@
|
||||
#include <Access/AccessControlManager.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <Storages/System/attachInformationSchemaTables.h>
|
||||
#include <AggregateFunctions/registerAggregateFunctions.h>
|
||||
#include <Functions/registerFunctions.h>
|
||||
#include <TableFunctions/registerTableFunctions.h>
|
||||
@ -1131,6 +1132,8 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
global_context->setSystemZooKeeperLogAfterInitializationIfNeeded();
|
||||
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
|
||||
attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper);
|
||||
attachInformationSchema(global_context, *database_catalog.getDatabase(DatabaseCatalog::INFORMATION_SCHEMA));
|
||||
attachInformationSchema(global_context, *database_catalog.getDatabase(DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE));
|
||||
/// Firstly remove partially dropped databases, to avoid race with MaterializedMySQLSyncThread,
|
||||
/// that may execute DROP before loadMarkedAsDroppedTables() in background,
|
||||
/// and so loadMarkedAsDroppedTables() will find it and try to add, and UUID will overlap.
|
||||
|
@ -119,8 +119,10 @@ namespace
|
||||
AccessRights res = access;
|
||||
res.modifyFlags(modifier);
|
||||
|
||||
/// Anyone has access to the "system" database.
|
||||
/// Anyone has access to the "system" and "information_schema" database.
|
||||
res.grant(AccessType::SELECT, DatabaseCatalog::SYSTEM_DATABASE);
|
||||
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA);
|
||||
res.grant(AccessType::SELECT, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -128,5 +128,8 @@
|
||||
/// Default limit on recursion depth of recursive descend parser.
|
||||
#define DBMS_DEFAULT_MAX_PARSER_DEPTH 1000
|
||||
|
||||
/// Default limit on query size.
|
||||
#define DBMS_DEFAULT_MAX_QUERY_SIZE 262144
|
||||
|
||||
/// Max depth of hierarchical dictionary
|
||||
#define DBMS_HIERARCHICAL_DICTIONARY_MAX_DEPTH 1000
|
||||
|
@ -48,7 +48,7 @@ class IColumn;
|
||||
M(MaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.", 0) \
|
||||
M(UInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \
|
||||
M(UInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).", 0) \
|
||||
M(UInt64, max_query_size, 262144, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \
|
||||
M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \
|
||||
M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \
|
||||
M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \
|
||||
M(Milliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.", 0) \
|
||||
|
@ -123,6 +123,8 @@ class DatabaseCatalog : boost::noncopyable, WithMutableContext
|
||||
public:
|
||||
static constexpr const char * TEMPORARY_DATABASE = "_temporary_and_external_tables";
|
||||
static constexpr const char * SYSTEM_DATABASE = "system";
|
||||
static constexpr const char * INFORMATION_SCHEMA = "information_schema";
|
||||
static constexpr const char * INFORMATION_SCHEMA_UPPERCASE = "INFORMATION_SCHEMA";
|
||||
|
||||
static DatabaseCatalog & init(ContextMutablePtr global_context_);
|
||||
static DatabaseCatalog & instance();
|
||||
|
@ -47,6 +47,12 @@ static void executeCreateQuery(
|
||||
interpreter.execute();
|
||||
}
|
||||
|
||||
static bool isSystemOrInformationSchema(const String & database_name)
|
||||
{
|
||||
return database_name == DatabaseCatalog::SYSTEM_DATABASE ||
|
||||
database_name == DatabaseCatalog::INFORMATION_SCHEMA ||
|
||||
database_name == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE;
|
||||
}
|
||||
|
||||
static void loadDatabase(
|
||||
ContextMutablePtr context,
|
||||
@ -116,7 +122,7 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam
|
||||
if (fs::path(current_file).extension() == ".sql")
|
||||
{
|
||||
String db_name = fs::path(current_file).stem();
|
||||
if (db_name != DatabaseCatalog::SYSTEM_DATABASE)
|
||||
if (!isSystemOrInformationSchema(db_name))
|
||||
databases.emplace(unescapeForFileName(db_name), fs::path(path) / db_name);
|
||||
}
|
||||
|
||||
@ -142,7 +148,7 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam
|
||||
if (current_file.at(0) == '.')
|
||||
continue;
|
||||
|
||||
if (current_file == DatabaseCatalog::SYSTEM_DATABASE)
|
||||
if (isSystemOrInformationSchema(current_file))
|
||||
continue;
|
||||
|
||||
databases.emplace(unescapeForFileName(current_file), it->path().string());
|
||||
@ -171,25 +177,31 @@ void loadMetadata(ContextMutablePtr context, const String & default_database_nam
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void loadMetadataSystem(ContextMutablePtr context)
|
||||
static void loadSystemDatabaseImpl(ContextMutablePtr context, const String & database_name, const String & default_engine)
|
||||
{
|
||||
String path = context->getPath() + "metadata/" + DatabaseCatalog::SYSTEM_DATABASE;
|
||||
String path = context->getPath() + "metadata/" + database_name;
|
||||
String metadata_file = path + ".sql";
|
||||
if (fs::exists(fs::path(path)) || fs::exists(fs::path(metadata_file)))
|
||||
{
|
||||
/// 'has_force_restore_data_flag' is true, to not fail on loading query_log table, if it is corrupted.
|
||||
loadDatabase(context, DatabaseCatalog::SYSTEM_DATABASE, path, true);
|
||||
loadDatabase(context, database_name, path, true);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Initialize system database manually
|
||||
String database_create_query = "CREATE DATABASE ";
|
||||
database_create_query += DatabaseCatalog::SYSTEM_DATABASE;
|
||||
database_create_query += " ENGINE=Atomic";
|
||||
executeCreateQuery(database_create_query, context, DatabaseCatalog::SYSTEM_DATABASE, "<no file>", true);
|
||||
database_create_query += database_name;
|
||||
database_create_query += " ENGINE=";
|
||||
database_create_query += default_engine;
|
||||
executeCreateQuery(database_create_query, context, database_name, "<no file>", true);
|
||||
}
|
||||
}
|
||||
|
||||
void loadMetadataSystem(ContextMutablePtr context)
|
||||
{
|
||||
loadSystemDatabaseImpl(context, DatabaseCatalog::SYSTEM_DATABASE, "Atomic");
|
||||
loadSystemDatabaseImpl(context, DatabaseCatalog::INFORMATION_SCHEMA, "Memory");
|
||||
loadSystemDatabaseImpl(context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE, "Memory");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -10,7 +10,8 @@ namespace DB
|
||||
/// You should first load system database, then attach system tables that you need into it, then load other databases.
|
||||
void loadMetadataSystem(ContextMutablePtr context);
|
||||
|
||||
/// Load tables from databases and add them to context. Database 'system' is ignored. Use separate function to load system tables.
|
||||
/// Load tables from databases and add them to context. Database 'system' and 'information_schema' is ignored.
|
||||
/// Use separate function to load system tables.
|
||||
void loadMetadata(ContextMutablePtr context, const String & default_database_name = {});
|
||||
|
||||
}
|
||||
|
@ -2,6 +2,8 @@
|
||||
# You can also regenerate it manually this way:
|
||||
# execute_process(COMMAND "${CMAKE_CURRENT_SOURCE_DIR}/StorageSystemContributors.sh")
|
||||
|
||||
include(${ClickHouse_SOURCE_DIR}/cmake/embed_binary.cmake)
|
||||
|
||||
set (CONFIG_BUILD "${CMAKE_CURRENT_BINARY_DIR}/StorageSystemBuildOptions.generated.cpp")
|
||||
get_property (BUILD_COMPILE_DEFINITIONS DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY COMPILE_DEFINITIONS)
|
||||
get_property (BUILD_INCLUDE_DIRECTORIES DIRECTORY ${ClickHouse_SOURCE_DIR} PROPERTY INCLUDE_DIRECTORIES)
|
||||
@ -41,7 +43,16 @@ list (APPEND storages_system_sources ${GENERATED_TIMEZONES_SRC})
|
||||
# Overlength strings
|
||||
set_source_files_properties(${GENERATED_LICENSES_SRC} PROPERTIES COMPILE_FLAGS -w)
|
||||
|
||||
clickhouse_embed_binaries(
|
||||
TARGET information_schema_metadata
|
||||
RESOURCE_DIR "${CMAKE_CURRENT_SOURCE_DIR}/InformationSchema/"
|
||||
RESOURCES schemata.sql tables.sql views.sql columns.sql
|
||||
)
|
||||
|
||||
add_library(clickhouse_storages_system ${storages_system_headers} ${storages_system_sources})
|
||||
|
||||
add_dependencies(clickhouse_storages_system information_schema_metadata)
|
||||
|
||||
target_link_libraries(clickhouse_storages_system PRIVATE
|
||||
dbms
|
||||
common
|
||||
@ -49,4 +60,5 @@ target_link_libraries(clickhouse_storages_system PRIVATE
|
||||
clickhouse_common_zookeeper
|
||||
clickhouse_parsers
|
||||
Poco::JSON
|
||||
INTERFACE "-Wl,${WHOLE_ARCHIVE} $<TARGET_FILE:information_schema_metadata> -Wl,${NO_WHOLE_ARCHIVE}"
|
||||
)
|
||||
|
74
src/Storages/System/InformationSchema/columns.sql
Normal file
74
src/Storages/System/InformationSchema/columns.sql
Normal file
@ -0,0 +1,74 @@
|
||||
ATTACH VIEW columns
|
||||
(
|
||||
`table_catalog` String,
|
||||
`table_schema` String,
|
||||
`table_name` String,
|
||||
`column_name` String,
|
||||
`ordinal_position` UInt64,
|
||||
`column_default` String,
|
||||
`is_nullable` UInt8,
|
||||
`data_type` String,
|
||||
`character_maximum_length` Nullable(UInt64),
|
||||
`character_octet_length` Nullable(UInt64),
|
||||
`numeric_precision` Nullable(UInt64),
|
||||
`numeric_precision_radix` Nullable(UInt64),
|
||||
`numeric_scale` Nullable(UInt64),
|
||||
`datetime_precision` Nullable(UInt64),
|
||||
`character_set_catalog` Nullable(String),
|
||||
`character_set_schema` Nullable(String),
|
||||
`character_set_name` Nullable(String),
|
||||
`collation_catalog` Nullable(String),
|
||||
`collation_schema` Nullable(String),
|
||||
`collation_name` Nullable(String),
|
||||
`domain_catalog` Nullable(String),
|
||||
`domain_schema` Nullable(String),
|
||||
`domain_name` Nullable(String),
|
||||
`TABLE_CATALOG` String ALIAS table_catalog,
|
||||
`TABLE_SCHEMA` String ALIAS table_schema,
|
||||
`TABLE_NAME` String ALIAS table_name,
|
||||
`COLUMN_NAME` String ALIAS column_name,
|
||||
`ORDINAL_POSITION` UInt64 ALIAS ordinal_position,
|
||||
`COLUMN_DEFAULT` String ALIAS column_default,
|
||||
`IS_NULLABLE` UInt8 ALIAS is_nullable,
|
||||
`DATA_TYPE` String ALIAS data_type,
|
||||
`CHARACTER_MAXIMUM_LENGTH` Nullable(UInt64) ALIAS character_maximum_length,
|
||||
`CHARACTER_OCTET_LENGTH` Nullable(UInt64) ALIAS character_octet_length,
|
||||
`NUMERIC_PRECISION` Nullable(UInt64) ALIAS numeric_precision,
|
||||
`NUMERIC_PRECISION_RADIX` Nullable(UInt64) ALIAS numeric_precision_radix,
|
||||
`NUMERIC_SCALE` Nullable(UInt64) ALIAS numeric_scale,
|
||||
`DATETIME_PRECISION` Nullable(UInt64) ALIAS datetime_precision,
|
||||
`CHARACTER_SET_CATALOG` Nullable(String) ALIAS character_set_catalog,
|
||||
`CHARACTER_SET_SCHEMA` Nullable(String) ALIAS character_set_schema,
|
||||
`CHARACTER_SET_NAME` Nullable(String) ALIAS character_set_name,
|
||||
`COLLATION_CATALOG` Nullable(String) ALIAS collation_catalog,
|
||||
`COLLATION_SCHEMA` Nullable(String) ALIAS collation_schema,
|
||||
`COLLATION_NAME` Nullable(String) ALIAS collation_name,
|
||||
`DOMAIN_CATALOG` Nullable(String) ALIAS domain_catalog,
|
||||
`DOMAIN_SCHEMA` Nullable(String) ALIAS domain_schema,
|
||||
`DOMAIN_NAME` Nullable(String) ALIAS domain_name
|
||||
) AS
|
||||
SELECT
|
||||
database AS table_catalog,
|
||||
database AS table_schema,
|
||||
table AS table_name,
|
||||
name AS column_name,
|
||||
position AS ordinal_position,
|
||||
default_expression AS column_default,
|
||||
type LIKE 'Nullable(%)' AS is_nullable,
|
||||
type AS data_type,
|
||||
character_octet_length AS character_maximum_length,
|
||||
character_octet_length,
|
||||
numeric_precision,
|
||||
numeric_precision_radix,
|
||||
numeric_scale,
|
||||
datetime_precision,
|
||||
NULL AS character_set_catalog,
|
||||
NULL AS character_set_schema,
|
||||
NULL AS character_set_name,
|
||||
NULL AS collation_catalog,
|
||||
NULL AS collation_schema,
|
||||
NULL AS collation_name,
|
||||
NULL AS domain_catalog,
|
||||
NULL AS domain_schema,
|
||||
NULL AS domain_name
|
||||
FROM system.columns
|
26
src/Storages/System/InformationSchema/schemata.sql
Normal file
26
src/Storages/System/InformationSchema/schemata.sql
Normal file
@ -0,0 +1,26 @@
|
||||
ATTACH VIEW schemata
|
||||
(
|
||||
`catalog_name` String,
|
||||
`schema_name` String,
|
||||
`schema_owner` String,
|
||||
`default_character_set_catalog` Nullable(String),
|
||||
`default_character_set_schema` Nullable(String),
|
||||
`default_character_set_name` Nullable(String),
|
||||
`sql_path` Nullable(String),
|
||||
`CATALOG_NAME` String ALIAS catalog_name,
|
||||
`SCHEMA_NAME` String ALIAS schema_name,
|
||||
`SCHEMA_OWNER` String ALIAS schema_owner,
|
||||
`DEFAULT_CHARACTER_SET_CATALOG` Nullable(String) ALIAS default_character_set_catalog,
|
||||
`DEFAULT_CHARACTER_SET_SCHEMA` Nullable(String) ALIAS default_character_set_schema,
|
||||
`DEFAULT_CHARACTER_SET_NAME` Nullable(String) ALIAS default_character_set_name,
|
||||
`SQL_PATH` Nullable(String) ALIAS sql_path
|
||||
) AS
|
||||
SELECT
|
||||
name AS catalog_name,
|
||||
name AS schema_name,
|
||||
'default' AS schema_owner,
|
||||
NULL AS default_character_set_catalog,
|
||||
NULL AS default_character_set_schema,
|
||||
NULL AS default_character_set_name,
|
||||
NULL AS sql_path
|
||||
FROM system.databases
|
17
src/Storages/System/InformationSchema/tables.sql
Normal file
17
src/Storages/System/InformationSchema/tables.sql
Normal file
@ -0,0 +1,17 @@
|
||||
ATTACH VIEW tables
|
||||
(
|
||||
`table_catalog` String,
|
||||
`table_schema` String,
|
||||
`table_name` String,
|
||||
`table_type` Enum8('BASE TABLE' = 1, 'VIEW' = 2, 'FOREIGN TABLE' = 3, 'LOCAL TEMPORARY' = 4, 'SYSTEM VIEW' = 5),
|
||||
`TABLE_CATALOG` String ALIAS table_catalog,
|
||||
`TABLE_SCHEMA` String ALIAS table_schema,
|
||||
`TABLE_NAME` String ALIAS table_name,
|
||||
`TABLE_TYPE` Enum8('BASE TABLE' = 1, 'VIEW' = 2, 'FOREIGN TABLE' = 3, 'LOCAL TEMPORARY' = 4, 'SYSTEM VIEW' = 5) ALIAS table_type
|
||||
) AS
|
||||
SELECT
|
||||
database AS table_catalog,
|
||||
database AS table_schema,
|
||||
name AS table_name,
|
||||
multiIf(is_temporary, 4, engine like '%View', 2, engine LIKE 'System%', 5, has_own_data = 0, 3, 1) AS table_type
|
||||
FROM system.tables
|
36
src/Storages/System/InformationSchema/views.sql
Normal file
36
src/Storages/System/InformationSchema/views.sql
Normal file
@ -0,0 +1,36 @@
|
||||
ATTACH VIEW views
|
||||
(
|
||||
`table_catalog` String,
|
||||
`table_schema` String,
|
||||
`table_name` String,
|
||||
`view_definition` String,
|
||||
`check_option` String,
|
||||
`is_updatable` Enum8('NO' = 0, 'YES' = 1),
|
||||
`is_insertable_into` Enum8('NO' = 0, 'YES' = 1),
|
||||
`is_trigger_updatable` Enum8('NO' = 0, 'YES' = 1),
|
||||
`is_trigger_deletable` Enum8('NO' = 0, 'YES' = 1),
|
||||
`is_trigger_insertable_into` Enum8('NO' = 0, 'YES' = 1),
|
||||
`TABLE_CATALOG` String ALIAS table_catalog,
|
||||
`TABLE_SCHEMA` String ALIAS table_schema,
|
||||
`TABLE_NAME` String ALIAS table_name,
|
||||
`VIEW_DEFINITION` String ALIAS view_definition,
|
||||
`CHECK_OPTION` String ALIAS check_option,
|
||||
`IS_UPDATABLE` Enum8('NO' = 0, 'YES' = 1) ALIAS is_updatable,
|
||||
`IS_INSERTABLE_INTO` Enum8('NO' = 0, 'YES' = 1) ALIAS is_insertable_into,
|
||||
`IS_TRIGGER_UPDATABLE` Enum8('NO' = 0, 'YES' = 1) ALIAS is_trigger_updatable,
|
||||
`IS_TRIGGER_DELETABLE` Enum8('NO' = 0, 'YES' = 1) ALIAS is_trigger_deletable,
|
||||
`IS_TRIGGER_INSERTABLE_INTO` Enum8('NO' = 0, 'YES' = 1) ALIAS is_trigger_insertable_into
|
||||
) AS
|
||||
SELECT
|
||||
database AS table_catalog,
|
||||
database AS table_schema,
|
||||
name AS table_name,
|
||||
as_select AS view_definition,
|
||||
'NONE' AS check_option,
|
||||
0 AS is_updatable,
|
||||
engine = 'MaterializedView' AS is_insertable_into,
|
||||
0 AS is_trigger_updatable,
|
||||
0 AS is_trigger_deletable,
|
||||
0 AS is_trigger_insertable_into
|
||||
FROM system.tables
|
||||
WHERE engine LIKE '%View'
|
@ -5,6 +5,8 @@
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
@ -44,6 +46,12 @@ StorageSystemColumns::StorageSystemColumns(const StorageID & table_id_)
|
||||
{ "is_in_primary_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "is_in_sampling_key", std::make_shared<DataTypeUInt8>() },
|
||||
{ "compression_codec", std::make_shared<DataTypeString>() },
|
||||
{ "character_octet_length", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
|
||||
{ "numeric_precision", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
|
||||
{ "numeric_precision_radix", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
|
||||
{ "numeric_scale", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
|
||||
{ "datetime_precision", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>()) },
|
||||
|
||||
}));
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
@ -218,6 +226,60 @@ protected:
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
/// character_octet_length makes sense for FixedString only
|
||||
DataTypePtr not_nullable_type = removeNullable(column.type);
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if (isFixedString(not_nullable_type))
|
||||
res_columns[res_index++]->insert(not_nullable_type->getSizeOfValueInMemory());
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
/// numeric_precision
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if (isInteger(not_nullable_type))
|
||||
res_columns[res_index++]->insert(not_nullable_type->getSizeOfValueInMemory() * 8); /// radix is 2
|
||||
else if (isDecimal(not_nullable_type))
|
||||
res_columns[res_index++]->insert(getDecimalPrecision(*not_nullable_type)); /// radix is 10
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
/// numeric_precision_radix
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if (isInteger(not_nullable_type))
|
||||
res_columns[res_index++]->insert(2);
|
||||
else if (isDecimal(not_nullable_type))
|
||||
res_columns[res_index++]->insert(10);
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
/// numeric_scale
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if (isInteger(not_nullable_type))
|
||||
res_columns[res_index++]->insert(0);
|
||||
else if (isDecimal(not_nullable_type))
|
||||
res_columns[res_index++]->insert(getDecimalScale(*not_nullable_type));
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
/// datetime_precision
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if (isDateTime64(not_nullable_type))
|
||||
res_columns[res_index++]->insert(assert_cast<const DataTypeDateTime64 &>(*not_nullable_type).getScale());
|
||||
else if (isDateOrDate32(not_nullable_type) || isDateTime(not_nullable_type) || isDateTime64(not_nullable_type))
|
||||
res_columns[res_index++]->insert(0);
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
++rows_count;
|
||||
}
|
||||
}
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
class Context;
|
||||
|
||||
/// system.replicated_fetches table. Takes data from context.getReplicatedFetchList()
|
||||
class StorageSystemReplicatedFetches final : public shared_ptr_helper<StorageSystemReplicatedFetches>, public IStorageSystemOneBlock<StorageSystemReplicatedFetches >
|
||||
class StorageSystemReplicatedFetches final : public shared_ptr_helper<StorageSystemReplicatedFetches>, public IStorageSystemOneBlock<StorageSystemReplicatedFetches>
|
||||
{
|
||||
friend struct shared_ptr_helper<StorageSystemReplicatedFetches>;
|
||||
public:
|
||||
|
@ -47,6 +47,7 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
|
||||
{"dependencies_table", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
{"create_table_query", std::make_shared<DataTypeString>()},
|
||||
{"engine_full", std::make_shared<DataTypeString>()},
|
||||
{"as_select", std::make_shared<DataTypeString>()},
|
||||
{"partition_key", std::make_shared<DataTypeString>()},
|
||||
{"sorting_key", std::make_shared<DataTypeString>()},
|
||||
{"primary_key", std::make_shared<DataTypeString>()},
|
||||
@ -57,6 +58,7 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
|
||||
{"lifetime_rows", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
|
||||
{"lifetime_bytes", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
|
||||
{"comment", std::make_shared<DataTypeString>()},
|
||||
{"has_own_data", std::make_shared<DataTypeUInt8>()},
|
||||
}));
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
}
|
||||
@ -209,6 +211,10 @@ protected:
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(table.second->getName());
|
||||
|
||||
// as_select
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
|
||||
// partition_key
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
@ -248,6 +254,10 @@ protected:
|
||||
// comment
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
|
||||
// has_own_data
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
}
|
||||
|
||||
@ -355,15 +365,15 @@ protected:
|
||||
res_columns[res_index++]->insert(dependencies_table_name_array);
|
||||
}
|
||||
|
||||
if (columns_mask[src_index] || columns_mask[src_index + 1])
|
||||
if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2])
|
||||
{
|
||||
ASTPtr ast = database->tryGetCreateTableQuery(table_name, context);
|
||||
auto * ast_create = ast ? ast->as<ASTCreateQuery>() : nullptr;
|
||||
|
||||
if (ast && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil)
|
||||
if (ast_create && !context->getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil)
|
||||
{
|
||||
auto & create = ast->as<ASTCreateQuery &>();
|
||||
create.uuid = UUIDHelpers::Nil;
|
||||
create.to_inner_uuid = UUIDHelpers::Nil;
|
||||
ast_create->uuid = UUIDHelpers::Nil;
|
||||
ast_create->to_inner_uuid = UUIDHelpers::Nil;
|
||||
}
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
@ -373,24 +383,28 @@ protected:
|
||||
{
|
||||
String engine_full;
|
||||
|
||||
if (ast)
|
||||
if (ast_create && ast_create->storage)
|
||||
{
|
||||
const auto & ast_create = ast->as<ASTCreateQuery &>();
|
||||
if (ast_create.storage)
|
||||
{
|
||||
engine_full = queryToString(*ast_create.storage);
|
||||
engine_full = queryToString(*ast_create->storage);
|
||||
|
||||
static const char * const extra_head = " ENGINE = ";
|
||||
if (startsWith(engine_full, extra_head))
|
||||
engine_full = engine_full.substr(strlen(extra_head));
|
||||
}
|
||||
static const char * const extra_head = " ENGINE = ";
|
||||
if (startsWith(engine_full, extra_head))
|
||||
engine_full = engine_full.substr(strlen(extra_head));
|
||||
}
|
||||
|
||||
res_columns[res_index++]->insert(engine_full);
|
||||
}
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
String as_select;
|
||||
if (ast_create && ast_create->select)
|
||||
as_select = queryToString(*ast_create->select);
|
||||
res_columns[res_index++]->insert(as_select);
|
||||
}
|
||||
}
|
||||
else
|
||||
src_index += 2;
|
||||
src_index += 3;
|
||||
|
||||
StorageMetadataPtr metadata_snapshot;
|
||||
if (table)
|
||||
@ -483,6 +497,14 @@ protected:
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
if (table)
|
||||
res_columns[res_index++]->insert(table->storesDataOnDisk());
|
||||
else
|
||||
res_columns[res_index++]->insertDefault();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
57
src/Storages/System/attachInformationSchemaTables.cpp
Normal file
57
src/Storages/System/attachInformationSchemaTables.cpp
Normal file
@ -0,0 +1,57 @@
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
#include <Storages/System/attachInformationSchemaTables.h>
|
||||
#include <Storages/System/attachSystemTablesImpl.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <common/getResource.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// View structures are taken from http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt
|
||||
|
||||
static void createInformationSchemaView(ContextMutablePtr context, IDatabase & database, const String & view_name)
|
||||
{
|
||||
try
|
||||
{
|
||||
assert(database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA ||
|
||||
database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE);
|
||||
if (database.getEngineName() != "Memory")
|
||||
return;
|
||||
bool is_uppercase = database.getDatabaseName() == DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE;
|
||||
|
||||
String metadata_resource_name = view_name + ".sql";
|
||||
auto attach_query = getResource(metadata_resource_name);
|
||||
if (attach_query.empty())
|
||||
return;
|
||||
|
||||
ParserCreateQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, attach_query.data(), attach_query.data() + attach_query.size(),
|
||||
"Attach query from embedded resource " + metadata_resource_name,
|
||||
DBMS_DEFAULT_MAX_QUERY_SIZE, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
||||
|
||||
auto & ast_create = ast->as<ASTCreateQuery &>();
|
||||
assert(view_name == ast_create.table);
|
||||
if (is_uppercase)
|
||||
ast_create.table = Poco::toUpper(view_name);
|
||||
|
||||
StoragePtr view = createTableFromAST(ast_create, database.getDatabaseName(),
|
||||
database.getTableDataPath(ast_create), context, true).second;
|
||||
|
||||
database.createTable(context, ast_create.table, view, ast);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
void attachInformationSchema(ContextMutablePtr context, IDatabase & information_schema_database)
|
||||
{
|
||||
createInformationSchemaView(context, information_schema_database, "schemata");
|
||||
createInformationSchemaView(context, information_schema_database, "tables");
|
||||
createInformationSchemaView(context, information_schema_database, "views");
|
||||
createInformationSchemaView(context, information_schema_database, "columns");
|
||||
}
|
||||
|
||||
}
|
11
src/Storages/System/attachInformationSchemaTables.h
Normal file
11
src/Storages/System/attachInformationSchemaTables.h
Normal file
@ -0,0 +1,11 @@
|
||||
#pragma once
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IDatabase;
|
||||
|
||||
void attachInformationSchema(ContextMutablePtr context, IDatabase & information_schema_database);
|
||||
|
||||
}
|
@ -9,6 +9,7 @@ namespace DB
|
||||
template<typename StorageT, typename... StorageArgs>
|
||||
void attach(IDatabase & system_database, const String & table_name, StorageArgs && ... args)
|
||||
{
|
||||
assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE);
|
||||
if (system_database.getUUID() == UUIDHelpers::Nil)
|
||||
{
|
||||
/// Attach to Ordinary database
|
||||
|
@ -218,6 +218,7 @@ SRCS(
|
||||
System/StorageSystemWarnings.cpp
|
||||
System/StorageSystemZeros.cpp
|
||||
System/StorageSystemZooKeeper.cpp
|
||||
System/attachInformationSchemaTables.cpp
|
||||
System/attachSystemTables.cpp
|
||||
TTLDescription.cpp
|
||||
VirtualColumnUtils.cpp
|
||||
|
@ -13,6 +13,7 @@ name CHAR
|
||||
a TINYINT
|
||||
Result:
|
||||
tables 1
|
||||
tables 1
|
||||
Columns:
|
||||
a
|
||||
b
|
||||
|
@ -407,24 +407,24 @@ def test_php_client(started_cluster, php_container):
|
||||
code, (stdout, stderr) = php_container.exec_run(
|
||||
'php -f test.php {host} {port} default 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True)
|
||||
assert code == 0
|
||||
assert stdout.decode() == 'tables\n'
|
||||
assert stdout.decode() == 'tables\ntables\n'
|
||||
|
||||
code, (stdout, stderr) = php_container.exec_run(
|
||||
'php -f test_ssl.php {host} {port} default 123'.format(host=started_cluster.get_instance_ip('node'), port=server_port), demux=True)
|
||||
assert code == 0
|
||||
assert stdout.decode() == 'tables\n'
|
||||
assert stdout.decode() == 'tables\ntables\n'
|
||||
|
||||
code, (stdout, stderr) = php_container.exec_run(
|
||||
'php -f test.php {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port),
|
||||
demux=True)
|
||||
assert code == 0
|
||||
assert stdout.decode() == 'tables\n'
|
||||
assert stdout.decode() == 'tables\ntables\n'
|
||||
|
||||
code, (stdout, stderr) = php_container.exec_run(
|
||||
'php -f test_ssl.php {host} {port} user_with_double_sha1 abacaba'.format(host=started_cluster.get_instance_ip('node'), port=server_port),
|
||||
demux=True)
|
||||
assert code == 0
|
||||
assert stdout.decode() == 'tables\n'
|
||||
assert stdout.decode() == 'tables\ntables\n'
|
||||
|
||||
|
||||
def test_mysqljs_client(started_cluster, nodejs_container):
|
||||
|
@ -399,9 +399,9 @@ def test_consumption_of_show_tables():
|
||||
instance.query("SHOW QUOTA"))
|
||||
|
||||
def test_consumption_of_show_databases():
|
||||
assert instance.query("SHOW DATABASES") == "default\nsystem\n"
|
||||
assert instance.query("SHOW DATABASES") == "INFORMATION_SCHEMA\ndefault\ninformation_schema\nsystem\n"
|
||||
assert re.match(
|
||||
"myQuota\\tdefault\\t.*\\t31556952\\t1\\t1000\\t1\\t500\\t0\\t500\\t0\\t\\\\N\\t2\\t\\\\N.*",
|
||||
"myQuota\\tdefault\\t.*\\t31556952\\t1\\t1000\\t1\\t500\\t0\\t500\\t0\\t\\\\N\\t4\\t\\\\N.*",
|
||||
instance.query("SHOW QUOTA"))
|
||||
|
||||
def test_consumption_of_show_clusters():
|
||||
|
27
tests/queries/0_stateless/01161_all_system_tables.sh
Executable file
27
tests/queries/0_stateless/01161_all_system_tables.sh
Executable file
@ -0,0 +1,27 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
THREADS=8
|
||||
RAND=$(($RANDOM))
|
||||
LIMIT=10000
|
||||
|
||||
function run_selects()
|
||||
{
|
||||
thread_num=$1
|
||||
readarray -t tables_arr < <(${CLICKHOUSE_CLIENT} -q "SELECT database || '.' || name FROM system.tables
|
||||
WHERE database in ('system', 'information_schema', 'INFORMATION_SCHEMA') and name!='zookeeper'
|
||||
AND sipHash64(name || toString($RAND)) % $THREADS = $thread_num")
|
||||
|
||||
for t in "${tables_arr[@]}"
|
||||
do
|
||||
${CLICKHOUSE_CLIENT} -q "SELECT * FROM $t LIMIT $LIMIT FORMAT Null" # Suppress style check: database=$CLICKHOUSE_DATABASEs
|
||||
done
|
||||
}
|
||||
|
||||
for ((i=0; i<THREADS; i++)) do
|
||||
run_selects "$i" &
|
||||
done
|
||||
wait
|
27
tests/queries/0_stateless/01161_information_schema.reference
Normal file
27
tests/queries/0_stateless/01161_information_schema.reference
Normal file
@ -0,0 +1,27 @@
|
||||
columns
|
||||
schemata
|
||||
tables
|
||||
views
|
||||
COLUMNS
|
||||
SCHEMATA
|
||||
TABLES
|
||||
VIEWS
|
||||
INFORMATION_SCHEMA INFORMATION_SCHEMA default \N \N \N \N
|
||||
information_schema information_schema default \N \N \N \N
|
||||
default default mv VIEW
|
||||
default default t FOREIGN TABLE
|
||||
default default v VIEW
|
||||
tmp LOCAL TEMPORARY
|
||||
default default mv SELECT * FROM system.one NONE NO YES NO NO NO
|
||||
default default v SELECT n, f FROM default.t NONE NO NO NO NO NO
|
||||
default default mv dummy 1 0 UInt8 \N \N 8 2 0 \N \N \N \N \N \N \N \N \N \N
|
||||
default default t n 1 0 UInt64 \N \N 64 2 0 \N \N \N \N \N \N \N \N \N \N
|
||||
default default t f 2 0 Float32 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N
|
||||
default default t s 3 0 String \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N
|
||||
default default t fs 4 0 FixedString(42) 42 42 \N \N \N \N \N \N \N \N \N \N \N \N \N
|
||||
default default t d 5 0 Decimal(9, 6) \N \N 9 10 6 \N \N \N \N \N \N \N \N \N \N
|
||||
default default v n 1 1 Nullable(Int32) \N \N 32 2 0 \N \N \N \N \N \N \N \N \N \N
|
||||
default default v f 2 0 Float64 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N
|
||||
tmp d 1 0 Date \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N
|
||||
tmp dt 2 0 DateTime \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N
|
||||
tmp dtms 3 0 DateTime64(3) \N \N \N \N \N 3 \N \N \N \N \N \N \N \N \N
|
19
tests/queries/0_stateless/01161_information_schema.sql
Normal file
19
tests/queries/0_stateless/01161_information_schema.sql
Normal file
@ -0,0 +1,19 @@
|
||||
show tables from information_schema;
|
||||
SHOW TABLES FROM INFORMATION_SCHEMA;
|
||||
|
||||
create table t (n UInt64, f Float32, s String, fs FixedString(42), d Decimal(9, 6)) engine=Memory;
|
||||
create view v (n Nullable(Int32), f Float64) as select n, f from t;
|
||||
create materialized view mv engine=Null as select * from system.one;
|
||||
create temporary table tmp (d Date, dt DateTime, dtms DateTime64(3));
|
||||
|
||||
-- FIXME #28687
|
||||
select * from information_schema.schemata where schema_name ilike 'information_schema';
|
||||
-- SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (TABLE_SCHEMA=currentDatabase() OR TABLE_SCHEMA='') AND TABLE_NAME NOT LIKE '%inner%';
|
||||
SELECT * FROM INFORMATION_SCHEMA.TABLES WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%';
|
||||
select * from information_schema.views where table_schema=currentDatabase();
|
||||
-- SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (TABLE_SCHEMA=currentDatabase() OR TABLE_SCHEMA='') AND TABLE_NAME NOT LIKE '%inner%';
|
||||
SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE (table_schema=currentDatabase() OR table_schema='') AND table_name NOT LIKE '%inner%';
|
||||
|
||||
drop table t;
|
||||
drop view v;
|
||||
drop view mv;
|
@ -1,8 +1,8 @@
|
||||
test_01602a CREATE TEMPORARY TABLE test_01602a (`x` UInt32) ENGINE = Memory Memory Memory 1
|
||||
test_01602b CREATE TEMPORARY TABLE test_01602b (`y` Float64, `z` String) ENGINE = Memory Memory Memory 1
|
||||
test_01602a x UInt32 1 0 0 0 0 0 0 0
|
||||
test_01602b y Float64 1 0 0 0 0 0 0 0
|
||||
test_01602b z String 2 0 0 0 0 0 0 0
|
||||
test_01602a x UInt32 1 0 0 0 0 0 0 0 \N 32 2 0 \N
|
||||
test_01602b y Float64 1 0 0 0 0 0 0 0 \N \N \N \N \N
|
||||
test_01602b z String 2 0 0 0 0 0 0 0 \N \N \N \N \N
|
||||
CREATE TEMPORARY TABLE test_01602a\n(\n `x` UInt32\n)\nENGINE = Memory
|
||||
CREATE TEMPORARY TABLE test_01602b\n(\n `y` Float64,\n `z` String\n)\nENGINE = Memory
|
||||
0
|
||||
|
Loading…
Reference in New Issue
Block a user