2017-04-01 09:19:00 +00:00
|
|
|
#include <Storages/IStorage.h>
|
2021-09-15 19:35:48 +00:00
|
|
|
#include <Processors/Sources/SourceFromSingleChunk.h>
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <QueryPipeline/BlockIO.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <Parsers/queryToString.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2017-11-01 14:34:05 +00:00
|
|
|
#include <TableFunctions/ITableFunction.h>
|
|
|
|
#include <TableFunctions/TableFunctionFactory.h>
|
2018-02-28 04:55:43 +00:00
|
|
|
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
2022-07-14 11:20:16 +00:00
|
|
|
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
2018-02-28 04:55:43 +00:00
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/InterpreterDescribeQuery.h>
|
2019-01-25 15:42:24 +00:00
|
|
|
#include <Interpreters/IdentifierSemantic.h>
|
2021-10-31 08:51:20 +00:00
|
|
|
#include <Access/Common/AccessFlags.h>
|
2017-11-01 14:34:05 +00:00
|
|
|
#include <Parsers/ASTIdentifier.h>
|
2018-02-28 04:55:43 +00:00
|
|
|
#include <Parsers/ASTFunction.h>
|
2017-11-01 14:34:05 +00:00
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2018-02-28 04:55:43 +00:00
|
|
|
#include <Parsers/TablePropertiesQueriesASTs.h>
|
2021-09-11 20:24:01 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2016-12-12 07:24:56 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
InterpreterDescribeQuery::InterpreterDescribeQuery(const ASTPtr & query_ptr_, ContextPtr context_)
|
|
|
|
: WithContext(context_)
|
|
|
|
, query_ptr(query_ptr_)
|
|
|
|
, settings(getContext()->getSettingsRef())
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
Block InterpreterDescribeQuery::getSampleBlock(bool include_subcolumns, bool include_virtuals, bool compact)
|
2016-12-12 07:24:56 +00:00
|
|
|
{
|
|
|
|
Block block;
|
|
|
|
|
|
|
|
ColumnWithTypeAndName col;
|
|
|
|
col.name = "name";
|
|
|
|
col.type = std::make_shared<DataTypeString>();
|
|
|
|
col.column = col.type->createColumn();
|
|
|
|
block.insert(col);
|
|
|
|
|
|
|
|
col.name = "type";
|
|
|
|
block.insert(col);
|
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
if (!compact)
|
|
|
|
{
|
|
|
|
col.name = "default_type";
|
|
|
|
block.insert(col);
|
2016-12-12 07:24:56 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
col.name = "default_expression";
|
|
|
|
block.insert(col);
|
2016-12-12 07:24:56 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
col.name = "comment";
|
|
|
|
block.insert(col);
|
2018-11-06 13:26:43 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
col.name = "codec_expression";
|
|
|
|
block.insert(col);
|
2018-12-25 10:04:38 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
col.name = "ttl_expression";
|
|
|
|
block.insert(col);
|
|
|
|
}
|
2019-04-15 09:30:45 +00:00
|
|
|
|
2021-09-11 20:24:01 +00:00
|
|
|
if (include_subcolumns)
|
|
|
|
{
|
|
|
|
col.name = "is_subcolumn";
|
|
|
|
col.type = std::make_shared<DataTypeUInt8>();
|
|
|
|
col.column = col.type->createColumn();
|
|
|
|
block.insert(col);
|
|
|
|
}
|
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
if (include_virtuals)
|
|
|
|
{
|
|
|
|
col.name = "is_virtual";
|
|
|
|
col.type = std::make_shared<DataTypeUInt8>();
|
|
|
|
col.column = col.type->createColumn();
|
|
|
|
block.insert(col);
|
|
|
|
}
|
|
|
|
|
2016-12-12 07:24:56 +00:00
|
|
|
return block;
|
|
|
|
}
|
|
|
|
|
2021-09-11 20:24:01 +00:00
|
|
|
BlockIO InterpreterDescribeQuery::execute()
|
2016-12-12 07:24:56 +00:00
|
|
|
{
|
2019-03-16 21:46:53 +00:00
|
|
|
const auto & ast = query_ptr->as<ASTDescribeQuery &>();
|
|
|
|
const auto & table_expression = ast.table_expression->as<ASTTableExpression &>();
|
2021-07-09 14:54:53 +00:00
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
if (table_expression.subquery)
|
2023-09-29 10:57:45 +00:00
|
|
|
fillColumnsFromSubquery(table_expression);
|
|
|
|
else if (table_expression.table_function)
|
|
|
|
fillColumnsFromTableFunction(table_expression);
|
|
|
|
else
|
|
|
|
fillColumnsFromTable(table_expression);
|
2022-07-14 11:20:16 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
Block sample_block = getSampleBlock(
|
|
|
|
settings.describe_include_subcolumns,
|
|
|
|
settings.describe_include_virtual_columns,
|
|
|
|
settings.describe_compact_output);
|
2022-07-14 11:20:16 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
|
|
|
|
|
|
|
for (const auto & column : columns)
|
|
|
|
addColumn(column, false, res_columns);
|
|
|
|
|
|
|
|
for (const auto & column : virtual_columns)
|
|
|
|
addColumn(column, true, res_columns);
|
|
|
|
|
|
|
|
if (settings.describe_include_subcolumns)
|
|
|
|
{
|
|
|
|
for (const auto & column : columns)
|
|
|
|
addSubcolumns(column, false, res_columns);
|
|
|
|
|
|
|
|
for (const auto & column : virtual_columns)
|
|
|
|
addSubcolumns(column, true, res_columns);
|
2018-02-28 04:55:43 +00:00
|
|
|
}
|
2023-09-29 10:57:45 +00:00
|
|
|
|
|
|
|
BlockIO res;
|
|
|
|
size_t num_rows = res_columns[0]->size();
|
|
|
|
auto source = std::make_shared<SourceFromSingleChunk>(sample_block, Chunk(std::move(res_columns), num_rows));
|
|
|
|
res.pipeline = QueryPipeline(std::move(source));
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
void InterpreterDescribeQuery::fillColumnsFromSubquery(const ASTTableExpression & table_expression)
|
|
|
|
{
|
|
|
|
NamesAndTypesList names_and_types;
|
|
|
|
auto select_query = table_expression.subquery->children.at(0);
|
|
|
|
auto current_context = getContext();
|
|
|
|
|
|
|
|
if (settings.allow_experimental_analyzer)
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
2023-09-29 10:57:45 +00:00
|
|
|
SelectQueryOptions select_query_options;
|
|
|
|
names_and_types = InterpreterSelectQueryAnalyzer(select_query, current_context, select_query_options).getSampleBlock().getNamesAndTypesList();
|
2020-10-14 12:19:29 +00:00
|
|
|
}
|
2017-11-01 14:34:05 +00:00
|
|
|
else
|
2016-12-12 07:24:56 +00:00
|
|
|
{
|
2023-09-29 10:57:45 +00:00
|
|
|
names_and_types = InterpreterSelectWithUnionQuery::getSampleBlock(select_query, current_context).getNamesAndTypesList();
|
2016-12-12 07:24:56 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
for (auto && [name, type] : names_and_types)
|
|
|
|
columns.emplace_back(std::move(name), std::move(type));
|
|
|
|
}
|
2021-12-01 02:58:24 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
void InterpreterDescribeQuery::fillColumnsFromTableFunction(const ASTTableExpression & table_expression)
|
|
|
|
{
|
|
|
|
auto current_context = getContext();
|
|
|
|
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, current_context);
|
|
|
|
auto column_descriptions = table_function_ptr->getActualTableStructure(getContext(), /*is_insert_query*/ true);
|
|
|
|
for (const auto & column : column_descriptions)
|
|
|
|
columns.emplace_back(column);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
if (settings.describe_include_virtual_columns)
|
2016-12-12 07:24:56 +00:00
|
|
|
{
|
2023-09-29 10:57:45 +00:00
|
|
|
auto table = table_function_ptr->execute(table_expression.table_function, getContext(), table_function_ptr->getName());
|
|
|
|
if (table)
|
|
|
|
{
|
|
|
|
for (const auto & column : table->getVirtuals())
|
|
|
|
{
|
|
|
|
if (!column_descriptions.has(column.name))
|
|
|
|
virtual_columns.emplace_back(column.name, column.type);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
void InterpreterDescribeQuery::fillColumnsFromTable(const ASTTableExpression & table_expression)
|
|
|
|
{
|
|
|
|
auto table_id = getContext()->resolveStorageID(table_expression.database_and_table_name);
|
|
|
|
getContext()->checkAccess(AccessType::SHOW_COLUMNS, table_id);
|
|
|
|
auto table = DatabaseCatalog::instance().getTable(table_id, getContext());
|
|
|
|
auto table_lock = table->lockForShare(getContext()->getInitialQueryId(), settings.lock_acquire_timeout);
|
2021-07-09 14:54:53 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
|
|
|
const auto & column_descriptions = metadata_snapshot->getColumns();
|
|
|
|
for (const auto & column : column_descriptions)
|
|
|
|
columns.emplace_back(column);
|
2023-09-28 12:23:13 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
if (settings.describe_include_virtual_columns)
|
|
|
|
{
|
|
|
|
for (const auto & column : table->getVirtuals())
|
|
|
|
{
|
|
|
|
if (!column_descriptions.has(column.name))
|
|
|
|
virtual_columns.emplace_back(column.name, column.type);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if (settings.describe_extend_object_types)
|
|
|
|
storage_snapshot = table->getStorageSnapshot(metadata_snapshot, getContext());
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
void InterpreterDescribeQuery::addColumn(const ColumnDescription & column, bool is_virtual, MutableColumns & res_columns)
|
|
|
|
{
|
|
|
|
size_t i = 0;
|
|
|
|
res_columns[i++]->insert(column.name);
|
|
|
|
|
|
|
|
auto type = storage_snapshot ? storage_snapshot->getConcreteType(column.name) : column.type;
|
|
|
|
if (settings.print_pretty_type_names)
|
|
|
|
res_columns[i++]->insert(type->getPrettyName());
|
|
|
|
else
|
|
|
|
res_columns[i++]->insert(type->getName());
|
|
|
|
|
|
|
|
if (!settings.describe_compact_output)
|
|
|
|
{
|
2019-03-14 15:20:51 +00:00
|
|
|
if (column.default_desc.expression)
|
2016-12-12 07:24:56 +00:00
|
|
|
{
|
2023-09-29 10:57:45 +00:00
|
|
|
res_columns[i++]->insert(toString(column.default_desc.kind));
|
|
|
|
res_columns[i++]->insert(queryToString(column.default_desc.expression));
|
2016-12-12 07:24:56 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2023-09-29 10:57:45 +00:00
|
|
|
res_columns[i++]->insertDefault();
|
|
|
|
res_columns[i++]->insertDefault();
|
2018-11-06 13:26:43 +00:00
|
|
|
}
|
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
res_columns[i++]->insert(column.comment);
|
2018-12-25 10:04:38 +00:00
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
if (column.codec)
|
2023-09-29 10:57:45 +00:00
|
|
|
res_columns[i++]->insert(queryToString(column.codec->as<ASTFunction>()->arguments));
|
2018-12-25 10:04:38 +00:00
|
|
|
else
|
2023-09-29 10:57:45 +00:00
|
|
|
res_columns[i++]->insertDefault();
|
2019-04-15 09:30:45 +00:00
|
|
|
|
|
|
|
if (column.ttl)
|
2023-09-29 10:57:45 +00:00
|
|
|
res_columns[i++]->insert(queryToString(column.ttl));
|
2019-04-15 09:30:45 +00:00
|
|
|
else
|
2023-09-29 10:57:45 +00:00
|
|
|
res_columns[i++]->insertDefault();
|
2021-09-11 20:24:01 +00:00
|
|
|
}
|
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
if (settings.describe_include_subcolumns)
|
|
|
|
res_columns[i++]->insertDefault();
|
|
|
|
|
|
|
|
if (settings.describe_include_virtual_columns)
|
|
|
|
res_columns[i++]->insert(is_virtual);
|
|
|
|
}
|
|
|
|
|
|
|
|
void InterpreterDescribeQuery::addSubcolumns(const ColumnDescription & column, bool is_virtual, MutableColumns & res_columns)
|
|
|
|
{
|
|
|
|
auto type = storage_snapshot ? storage_snapshot->getConcreteType(column.name) : column.type;
|
|
|
|
|
|
|
|
IDataType::forEachSubcolumn([&](const auto & path, const auto & name, const auto & data)
|
2021-09-11 20:24:01 +00:00
|
|
|
{
|
2023-09-29 10:57:45 +00:00
|
|
|
size_t i = 0;
|
|
|
|
res_columns[i++]->insert(Nested::concatenateName(column.name, name));
|
2021-12-01 02:58:24 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
if (settings.print_pretty_type_names)
|
|
|
|
res_columns[i++]->insert(data.type->getPrettyName());
|
|
|
|
else
|
|
|
|
res_columns[i++]->insert(data.type->getName());
|
|
|
|
|
|
|
|
if (!settings.describe_compact_output)
|
|
|
|
{
|
|
|
|
/// It's not trivial to calculate default expression for subcolumn.
|
|
|
|
/// So, leave it empty.
|
|
|
|
res_columns[i++]->insertDefault();
|
|
|
|
res_columns[i++]->insertDefault();
|
|
|
|
res_columns[i++]->insert(column.comment);
|
|
|
|
|
|
|
|
if (column.codec && ISerialization::isSpecialCompressionAllowed(path))
|
|
|
|
res_columns[i++]->insert(queryToString(column.codec->as<ASTFunction>()->arguments));
|
|
|
|
else
|
|
|
|
res_columns[i++]->insertDefault();
|
|
|
|
|
|
|
|
if (column.ttl)
|
|
|
|
res_columns[i++]->insert(queryToString(column.ttl));
|
|
|
|
else
|
|
|
|
res_columns[i++]->insertDefault();
|
2021-09-11 20:24:01 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
res_columns[i++]->insert(1U);
|
2021-09-11 20:24:01 +00:00
|
|
|
|
2023-09-29 10:57:45 +00:00
|
|
|
if (settings.describe_include_virtual_columns)
|
|
|
|
res_columns[i++]->insert(is_virtual);
|
|
|
|
|
|
|
|
}, ISerialization::SubstreamData(type->getDefaultSerialization()).withType(type));
|
2016-12-12 07:24:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|