2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <DataTypes/DataTypeDateTime.h>
|
2020-03-29 08:06:07 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataStreams/OneBlockInputStream.h>
|
|
|
|
#include <Storages/System/StorageSystemTables.h>
|
2017-11-20 05:22:54 +00:00
|
|
|
#include <Storages/VirtualColumnUtils.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Databases/IDatabase.h>
|
2020-03-07 17:37:38 +00:00
|
|
|
#include <Access/ContextAccess.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2018-01-30 17:47:04 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
|
|
|
#include <Parsers/queryToString.h>
|
|
|
|
#include <Common/typeid_cast.h>
|
2018-06-05 19:46:49 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.h>
|
2018-01-25 17:50:57 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2018-08-10 10:15:59 +00:00
|
|
|
#include <DataTypes/DataTypeArray.h>
|
2019-12-12 08:57:25 +00:00
|
|
|
#include <Disks/DiskSpaceMonitor.h>
|
2020-01-24 18:25:21 +00:00
|
|
|
#include <Processors/Sources/SourceWithProgress.h>
|
|
|
|
#include <Processors/Pipe.h>
|
2012-05-08 11:19:00 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-01-30 17:47:04 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2019-03-28 23:17:22 +00:00
|
|
|
extern const int TABLE_IS_DROPPED;
|
2018-01-30 17:47:04 +00:00
|
|
|
}
|
|
|
|
|
2018-02-08 14:14:08 +00:00
|
|
|
|
2015-01-21 04:30:33 +00:00
|
|
|
StorageSystemTables::StorageSystemTables(const std::string & name_)
|
2019-12-03 16:25:32 +00:00
|
|
|
: IStorage({"system", name_})
|
2018-01-25 14:42:39 +00:00
|
|
|
{
|
2018-03-13 14:18:11 +00:00
|
|
|
setColumns(ColumnsDescription(
|
2018-02-08 21:15:46 +00:00
|
|
|
{
|
2017-10-11 14:41:49 +00:00
|
|
|
{"database", std::make_shared<DataTypeString>()},
|
|
|
|
{"name", std::make_shared<DataTypeString>()},
|
2018-02-08 21:15:46 +00:00
|
|
|
{"engine", std::make_shared<DataTypeString>()},
|
|
|
|
{"is_temporary", std::make_shared<DataTypeUInt8>()},
|
2019-05-11 20:44:43 +00:00
|
|
|
{"data_paths", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
2018-02-21 19:26:59 +00:00
|
|
|
{"metadata_path", std::make_shared<DataTypeString>()},
|
2018-07-07 18:58:48 +00:00
|
|
|
{"metadata_modification_time", std::make_shared<DataTypeDateTime>()},
|
2018-08-12 02:55:01 +00:00
|
|
|
{"dependencies_database", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
|
|
|
{"dependencies_table", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
2018-07-07 18:58:48 +00:00
|
|
|
{"create_table_query", std::make_shared<DataTypeString>()},
|
2018-11-08 17:45:41 +00:00
|
|
|
{"engine_full", std::make_shared<DataTypeString>()},
|
|
|
|
{"partition_key", std::make_shared<DataTypeString>()},
|
2018-11-27 17:07:10 +00:00
|
|
|
{"sorting_key", std::make_shared<DataTypeString>()},
|
|
|
|
{"primary_key", std::make_shared<DataTypeString>()},
|
|
|
|
{"sampling_key", std::make_shared<DataTypeString>()},
|
2019-05-24 19:03:07 +00:00
|
|
|
{"storage_policy", std::make_shared<DataTypeString>()},
|
2020-03-29 08:06:07 +00:00
|
|
|
{"total_rows", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
|
2020-03-29 08:58:39 +00:00
|
|
|
{"total_bytes", std::make_shared<DataTypeNullable>(std::make_shared<DataTypeUInt64>())},
|
2018-03-13 14:18:11 +00:00
|
|
|
}));
|
2012-05-08 11:19:00 +00:00
|
|
|
}
|
|
|
|
|
2015-01-21 04:30:33 +00:00
|
|
|
|
2017-12-16 00:49:03 +00:00
|
|
|
static ColumnPtr getFilteredDatabases(const ASTPtr & query, const Context & context)
|
2015-01-21 04:30:33 +00:00
|
|
|
{
|
2017-12-16 00:49:03 +00:00
|
|
|
MutableColumnPtr column = ColumnString::create();
|
2020-02-10 13:10:17 +00:00
|
|
|
for (const auto & db : DatabaseCatalog::instance().getDatabases())
|
2017-12-16 00:49:03 +00:00
|
|
|
column->insert(db.first);
|
2016-08-04 23:35:07 +00:00
|
|
|
|
2018-11-24 01:48:06 +00:00
|
|
|
Block block { ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), "database") };
|
2017-04-01 07:20:54 +00:00
|
|
|
VirtualColumnUtils::filterBlockWithQuery(query, block, context);
|
2017-12-16 00:49:03 +00:00
|
|
|
return block.getByPosition(0).column;
|
2013-02-06 11:26:35 +00:00
|
|
|
}
|
|
|
|
|
2019-10-08 00:31:50 +00:00
|
|
|
/// Avoid heavy operation on tables if we only queried columns that we can get without table object.
|
|
|
|
/// Otherwise it will require table initialization for Lazy database.
|
2019-10-08 00:29:41 +00:00
|
|
|
static bool needLockStructure(const DatabasePtr & database, const Block & header)
|
2019-10-05 10:21:04 +00:00
|
|
|
{
|
2019-10-02 12:58:13 +00:00
|
|
|
if (database->getEngineName() != "Lazy")
|
|
|
|
return true;
|
|
|
|
|
2019-10-03 14:18:17 +00:00
|
|
|
static const std::set<std::string> columns_without_lock = { "database", "name", "metadata_modification_time" };
|
2019-10-08 00:29:41 +00:00
|
|
|
for (const auto & column : header.getColumnsWithTypeAndName())
|
2019-10-05 10:21:04 +00:00
|
|
|
{
|
|
|
|
if (columns_without_lock.find(column.name) == columns_without_lock.end())
|
2019-10-02 12:58:13 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
2012-05-08 11:19:00 +00:00
|
|
|
|
2020-01-24 18:25:21 +00:00
|
|
|
class TablesBlockSource : public SourceWithProgress
|
2012-05-08 11:19:00 +00:00
|
|
|
{
|
2018-08-25 16:07:48 +00:00
|
|
|
public:
|
2020-01-24 18:25:21 +00:00
|
|
|
TablesBlockSource(
|
2019-08-03 11:02:40 +00:00
|
|
|
std::vector<UInt8> columns_mask_,
|
2020-01-24 18:25:21 +00:00
|
|
|
Block header,
|
2019-08-03 11:02:40 +00:00
|
|
|
UInt64 max_block_size_,
|
|
|
|
ColumnPtr databases_,
|
|
|
|
const Context & context_)
|
2020-01-24 18:25:21 +00:00
|
|
|
: SourceWithProgress(std::move(header))
|
|
|
|
, columns_mask(std::move(columns_mask_))
|
2019-10-11 13:21:52 +00:00
|
|
|
, max_block_size(max_block_size_)
|
|
|
|
, databases(std::move(databases_))
|
|
|
|
, context(context_) {}
|
2018-08-25 16:07:48 +00:00
|
|
|
|
|
|
|
String getName() const override { return "Tables"; }
|
|
|
|
|
|
|
|
protected:
|
2020-01-24 18:25:21 +00:00
|
|
|
Chunk generate() override
|
2018-08-25 16:07:48 +00:00
|
|
|
{
|
|
|
|
if (done)
|
|
|
|
return {};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-01-24 18:25:21 +00:00
|
|
|
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2020-03-07 17:37:38 +00:00
|
|
|
const auto access = context.getAccess();
|
|
|
|
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
|
2020-01-24 16:20:36 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
size_t rows_count = 0;
|
|
|
|
while (rows_count < max_block_size)
|
|
|
|
{
|
|
|
|
if (tables_it && !tables_it->isValid())
|
|
|
|
++database_idx;
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
while (database_idx < databases->size() && (!tables_it || !tables_it->isValid()))
|
|
|
|
{
|
|
|
|
database_name = databases->getDataAt(database_idx).toString();
|
2020-02-10 13:10:17 +00:00
|
|
|
database = DatabaseCatalog::instance().tryGetDatabase(database_name);
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
if (!database)
|
2018-08-25 16:07:48 +00:00
|
|
|
{
|
|
|
|
/// Database was deleted just now or the user has no access.
|
|
|
|
++database_idx;
|
|
|
|
continue;
|
|
|
|
}
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
break;
|
|
|
|
}
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2018-11-26 00:56:50 +00:00
|
|
|
/// This is for temporary tables. They are output in single block regardless to max_block_size.
|
2018-08-25 16:07:48 +00:00
|
|
|
if (database_idx >= databases->size())
|
|
|
|
{
|
|
|
|
if (context.hasSessionContext())
|
|
|
|
{
|
|
|
|
Tables external_tables = context.getSessionContext().getExternalTables();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-09 03:14:24 +00:00
|
|
|
for (auto & table : external_tables)
|
2018-08-25 16:07:48 +00:00
|
|
|
{
|
|
|
|
size_t src_index = 0;
|
|
|
|
size_t res_index = 0;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// database
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// name
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insert(table.first);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// engine
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insert(table.second->getName());
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// is_temporary
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
2018-10-22 08:54:54 +00:00
|
|
|
res_columns[res_index++]->insert(1u);
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// data_paths
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// metadata_path
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// metadata_modification_time
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// dependencies_database
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// dependencies_table
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// create_table_query
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// engine_full
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insert(table.second->getName());
|
2018-11-08 17:45:41 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// partition_key
|
2018-11-08 17:45:41 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// sorting_key
|
2018-11-08 17:45:41 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// primary_key
|
2018-11-08 17:45:41 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2018-11-11 15:44:21 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// sampling_key
|
2018-11-11 15:44:21 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2019-05-24 19:03:07 +00:00
|
|
|
|
2020-03-29 14:51:42 +00:00
|
|
|
// storage_policy
|
2019-05-24 19:03:07 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2020-03-29 08:06:07 +00:00
|
|
|
|
|
|
|
// total_rows
|
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2020-03-29 08:58:39 +00:00
|
|
|
|
|
|
|
// total_bytes
|
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insertDefault();
|
2018-08-25 16:07:48 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-24 18:25:21 +00:00
|
|
|
UInt64 num_rows = res_columns.at(0)->size();
|
2020-01-27 13:00:18 +00:00
|
|
|
done = true;
|
2020-01-24 18:25:21 +00:00
|
|
|
return Chunk(std::move(res_columns), num_rows);
|
2018-08-25 16:07:48 +00:00
|
|
|
}
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2020-03-07 17:37:38 +00:00
|
|
|
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name);
|
2020-01-24 16:20:36 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
if (!tables_it || !tables_it->isValid())
|
2019-10-17 13:05:12 +00:00
|
|
|
tables_it = database->getTablesWithDictionaryTablesIterator(context);
|
2019-10-05 10:21:04 +00:00
|
|
|
|
2020-01-24 18:25:21 +00:00
|
|
|
const bool need_lock_structure = needLockStructure(database, getPort().getHeader());
|
2019-10-05 10:21:04 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next())
|
2018-08-10 10:15:59 +00:00
|
|
|
{
|
2018-08-25 16:07:48 +00:00
|
|
|
auto table_name = tables_it->name();
|
2020-03-07 17:37:38 +00:00
|
|
|
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
|
2020-01-24 16:20:36 +00:00
|
|
|
continue;
|
2019-03-28 23:17:22 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
StoragePtr table = nullptr;
|
2019-03-28 23:17:22 +00:00
|
|
|
TableStructureReadLockHolder lock;
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
2019-10-02 12:58:13 +00:00
|
|
|
if (need_lock_structure)
|
2019-10-01 13:07:00 +00:00
|
|
|
{
|
2020-03-23 13:03:15 +00:00
|
|
|
table = tables_it->table();
|
2020-04-01 12:43:09 +00:00
|
|
|
lock = table->lockStructureForShare(false, context.getCurrentQueryId());
|
2019-10-01 13:07:00 +00:00
|
|
|
}
|
2019-03-28 23:17:22 +00:00
|
|
|
}
|
|
|
|
catch (const Exception & e)
|
|
|
|
{
|
|
|
|
if (e.code() == ErrorCodes::TABLE_IS_DROPPED)
|
|
|
|
continue;
|
|
|
|
throw;
|
|
|
|
}
|
2019-02-08 14:10:26 +00:00
|
|
|
|
|
|
|
++rows_count;
|
2018-08-10 10:15:59 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
size_t src_index = 0;
|
|
|
|
size_t res_index = 0;
|
2018-08-12 02:55:01 +00:00
|
|
|
|
|
|
|
if (columns_mask[src_index++])
|
2018-08-25 16:07:48 +00:00
|
|
|
res_columns[res_index++]->insert(database_name);
|
2018-08-10 10:15:59 +00:00
|
|
|
|
2018-08-12 02:55:01 +00:00
|
|
|
if (columns_mask[src_index++])
|
2018-08-25 16:07:48 +00:00
|
|
|
res_columns[res_index++]->insert(table_name);
|
2018-08-10 10:15:59 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
2019-10-01 13:07:00 +00:00
|
|
|
{
|
2020-03-23 13:03:15 +00:00
|
|
|
assert(table != nullptr);
|
2019-03-28 19:13:59 +00:00
|
|
|
res_columns[res_index++]->insert(table->getName());
|
2019-10-01 13:07:00 +00:00
|
|
|
}
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2018-07-07 18:58:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
2018-11-08 17:45:41 +00:00
|
|
|
res_columns[res_index++]->insert(0u); // is_temporary
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2018-07-07 18:58:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
2019-04-28 14:49:41 +00:00
|
|
|
{
|
2020-03-23 13:03:15 +00:00
|
|
|
assert(table != nullptr);
|
2019-05-11 18:00:43 +00:00
|
|
|
Array table_paths_array;
|
2019-04-28 14:49:41 +00:00
|
|
|
auto paths = table->getDataPaths();
|
2019-05-11 18:00:43 +00:00
|
|
|
table_paths_array.reserve(paths.size());
|
2019-05-09 17:53:34 +00:00
|
|
|
for (const String & path : paths)
|
2019-05-11 18:00:43 +00:00
|
|
|
table_paths_array.push_back(path);
|
|
|
|
res_columns[res_index++]->insert(table_paths_array);
|
2019-04-28 14:49:41 +00:00
|
|
|
}
|
2018-08-25 16:07:48 +00:00
|
|
|
|
|
|
|
if (columns_mask[src_index++])
|
2019-10-10 20:47:47 +00:00
|
|
|
res_columns[res_index++]->insert(database->getObjectMetadataPath(table_name));
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
2019-11-06 16:05:04 +00:00
|
|
|
res_columns[res_index++]->insert(static_cast<UInt64>(database->getObjectMetadataModificationTime(table_name)));
|
2018-08-25 16:07:48 +00:00
|
|
|
|
|
|
|
{
|
|
|
|
Array dependencies_table_name_array;
|
|
|
|
Array dependencies_database_name_array;
|
|
|
|
if (columns_mask[src_index] || columns_mask[src_index + 1])
|
2018-01-30 17:47:04 +00:00
|
|
|
{
|
2020-02-12 16:54:26 +00:00
|
|
|
const auto dependencies = DatabaseCatalog::instance().getDependencies(StorageID(database_name, table_name));
|
2018-01-30 17:47:04 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
dependencies_table_name_array.reserve(dependencies.size());
|
|
|
|
dependencies_database_name_array.reserve(dependencies.size());
|
|
|
|
for (const auto & dependency : dependencies)
|
|
|
|
{
|
2019-12-05 11:42:13 +00:00
|
|
|
dependencies_table_name_array.push_back(dependency.table_name);
|
|
|
|
dependencies_database_name_array.push_back(dependency.database_name);
|
2018-01-30 17:47:04 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insert(dependencies_database_name_array);
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insert(dependencies_table_name_array);
|
|
|
|
}
|
2018-02-02 13:17:45 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index] || columns_mask[src_index + 1])
|
|
|
|
{
|
|
|
|
ASTPtr ast = database->tryGetCreateTableQuery(context, table_name);
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
res_columns[res_index++]->insert(ast ? queryToString(ast) : "");
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
{
|
|
|
|
String engine_full;
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
if (ast)
|
|
|
|
{
|
2019-03-15 17:09:14 +00:00
|
|
|
const auto & ast_create = ast->as<ASTCreateQuery &>();
|
|
|
|
if (ast_create.storage)
|
2018-08-25 16:07:48 +00:00
|
|
|
{
|
2019-03-15 17:09:14 +00:00
|
|
|
engine_full = queryToString(*ast_create.storage);
|
2018-08-25 16:07:48 +00:00
|
|
|
|
|
|
|
static const char * const extra_head = " ENGINE = ";
|
|
|
|
if (startsWith(engine_full, extra_head))
|
|
|
|
engine_full = engine_full.substr(strlen(extra_head));
|
|
|
|
}
|
|
|
|
}
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
res_columns[res_index++]->insert(engine_full);
|
|
|
|
}
|
|
|
|
}
|
2018-11-08 17:45:41 +00:00
|
|
|
else
|
2018-11-19 14:31:16 +00:00
|
|
|
src_index += 2;
|
2018-11-08 17:45:41 +00:00
|
|
|
|
|
|
|
ASTPtr expression_ptr;
|
|
|
|
if (columns_mask[src_index++])
|
|
|
|
{
|
2020-03-23 13:03:15 +00:00
|
|
|
assert(table != nullptr);
|
2019-02-08 14:10:26 +00:00
|
|
|
if ((expression_ptr = table->getPartitionKeyAST()))
|
2018-11-08 17:45:41 +00:00
|
|
|
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
|
|
|
else
|
|
|
|
res_columns[res_index++]->insertDefault();
|
|
|
|
}
|
|
|
|
|
2018-11-11 15:44:21 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
{
|
2020-03-23 13:03:15 +00:00
|
|
|
assert(table != nullptr);
|
2019-02-08 14:10:26 +00:00
|
|
|
if ((expression_ptr = table->getSortingKeyAST()))
|
2018-11-11 15:44:21 +00:00
|
|
|
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
|
|
|
else
|
|
|
|
res_columns[res_index++]->insertDefault();
|
|
|
|
}
|
|
|
|
|
2018-11-08 17:45:41 +00:00
|
|
|
if (columns_mask[src_index++])
|
|
|
|
{
|
2020-03-23 13:03:15 +00:00
|
|
|
assert(table != nullptr);
|
2019-02-08 14:10:26 +00:00
|
|
|
if ((expression_ptr = table->getPrimaryKeyAST()))
|
2018-11-08 17:45:41 +00:00
|
|
|
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
|
|
|
else
|
|
|
|
res_columns[res_index++]->insertDefault();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (columns_mask[src_index++])
|
|
|
|
{
|
2020-03-23 13:03:15 +00:00
|
|
|
assert(table != nullptr);
|
2019-02-08 14:10:26 +00:00
|
|
|
if ((expression_ptr = table->getSamplingKeyAST()))
|
2018-11-08 17:45:41 +00:00
|
|
|
res_columns[res_index++]->insert(queryToString(expression_ptr));
|
|
|
|
else
|
|
|
|
res_columns[res_index++]->insertDefault();
|
|
|
|
}
|
2019-05-24 19:03:07 +00:00
|
|
|
|
|
|
|
if (columns_mask[src_index++])
|
|
|
|
{
|
2020-03-23 13:03:15 +00:00
|
|
|
assert(table != nullptr);
|
2019-05-24 19:03:07 +00:00
|
|
|
auto policy = table->getStoragePolicy();
|
|
|
|
if (policy)
|
|
|
|
res_columns[res_index++]->insert(policy->getName());
|
|
|
|
else
|
|
|
|
res_columns[res_index++]->insertDefault();
|
|
|
|
}
|
2020-03-29 08:06:07 +00:00
|
|
|
|
|
|
|
if (columns_mask[src_index++])
|
|
|
|
{
|
|
|
|
assert(table != nullptr);
|
|
|
|
auto total_rows = table->totalRows();
|
|
|
|
if (total_rows)
|
|
|
|
res_columns[res_index++]->insert(*total_rows);
|
|
|
|
else
|
|
|
|
res_columns[res_index++]->insertDefault();
|
|
|
|
}
|
2020-03-29 08:58:39 +00:00
|
|
|
|
|
|
|
if (columns_mask[src_index++])
|
|
|
|
{
|
|
|
|
assert(table != nullptr);
|
|
|
|
auto total_bytes = table->totalBytes();
|
|
|
|
if (total_bytes)
|
|
|
|
res_columns[res_index++]->insert(*total_bytes);
|
|
|
|
else
|
|
|
|
res_columns[res_index++]->insertDefault();
|
|
|
|
}
|
2018-08-25 16:07:48 +00:00
|
|
|
}
|
|
|
|
}
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2020-01-24 18:25:21 +00:00
|
|
|
UInt64 num_rows = res_columns.at(0)->size();
|
|
|
|
return Chunk(std::move(res_columns), num_rows);
|
2018-08-25 16:07:48 +00:00
|
|
|
}
|
|
|
|
private:
|
|
|
|
std::vector<UInt8> columns_mask;
|
2019-02-10 16:55:12 +00:00
|
|
|
UInt64 max_block_size;
|
2018-08-25 16:07:48 +00:00
|
|
|
ColumnPtr databases;
|
|
|
|
size_t database_idx = 0;
|
2019-10-10 20:47:47 +00:00
|
|
|
DatabaseTablesIteratorPtr tables_it;
|
2018-08-25 16:07:48 +00:00
|
|
|
const Context context;
|
|
|
|
bool done = false;
|
|
|
|
DatabasePtr database;
|
|
|
|
std::string database_name;
|
|
|
|
};
|
2018-07-07 18:58:48 +00:00
|
|
|
|
|
|
|
|
2020-02-19 16:07:28 +00:00
|
|
|
Pipes StorageSystemTables::read(
|
2018-08-25 16:07:48 +00:00
|
|
|
const Names & column_names,
|
|
|
|
const SelectQueryInfo & query_info,
|
|
|
|
const Context & context,
|
2018-09-08 11:29:23 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
2019-02-18 23:38:44 +00:00
|
|
|
const size_t max_block_size,
|
2018-08-25 16:07:48 +00:00
|
|
|
const unsigned /*num_streams*/)
|
|
|
|
{
|
|
|
|
check(column_names);
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
/// Create a mask of what columns are needed in the result.
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
NameSet names_set(column_names.begin(), column_names.end());
|
2018-08-10 10:15:59 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
Block sample_block = getSampleBlock();
|
|
|
|
Block res_block;
|
2018-07-07 18:58:48 +00:00
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
std::vector<UInt8> columns_mask(sample_block.columns());
|
|
|
|
for (size_t i = 0, size = columns_mask.size(); i < size; ++i)
|
|
|
|
{
|
|
|
|
if (names_set.count(sample_block.getByPosition(i).name))
|
|
|
|
{
|
|
|
|
columns_mask[i] = 1;
|
|
|
|
res_block.insert(sample_block.getByPosition(i));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-08-25 16:07:48 +00:00
|
|
|
ColumnPtr filtered_databases_column = getFilteredDatabases(query_info.query, context);
|
2020-01-24 18:25:21 +00:00
|
|
|
|
|
|
|
Pipes pipes;
|
|
|
|
pipes.emplace_back(std::make_shared<TablesBlockSource>(
|
|
|
|
std::move(columns_mask), std::move(res_block), max_block_size, std::move(filtered_databases_column), context));
|
|
|
|
|
|
|
|
return pipes;
|
2018-01-30 17:47:04 +00:00
|
|
|
}
|
|
|
|
|
2012-05-08 11:19:00 +00:00
|
|
|
}
|