mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
allow to include virtual columns into DESCRIBE query
This commit is contained in:
parent
e8d8c6dea5
commit
9b0e222cd2
@ -604,6 +604,8 @@ class IColumn;
|
||||
M(Bool, insert_null_as_default, true, "Insert DEFAULT values instead of NULL in INSERT SELECT (UNION ALL)", 0) \
|
||||
M(Bool, describe_extend_object_types, false, "Deduce concrete type of columns of type Object in DESCRIBE query", 0) \
|
||||
M(Bool, describe_include_subcolumns, false, "If true, subcolumns of all table columns will be included into result of DESCRIBE query", 0) \
|
||||
M(Bool, describe_include_virtual_columns, false, "If true, virtual columns of table will be included into result of DESCRIBE query", 0) \
|
||||
M(Bool, describe_compact_output, false, "If true, include only column names and types into result of DESCRIBE query", 0) \
|
||||
M(Bool, mutations_execute_nondeterministic_on_initiator, false, "If true nondeterministic function are executed on initiator and replaced to literals in UPDATE and DELETE queries", 0) \
|
||||
M(Bool, mutations_execute_subqueries_on_initiator, false, "If true scalar subqueries are executed on initiator and replaced to literals in UPDATE and DELETE queries", 0) \
|
||||
M(UInt64, mutations_max_literal_size_to_replace, 16384, "The maximum size of serialized literal in bytes to replace in UPDATE and DELETE queries", 0) \
|
||||
|
@ -21,7 +21,14 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
Block InterpreterDescribeQuery::getSampleBlock(bool include_subcolumns)
|
||||
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)
|
||||
{
|
||||
Block block;
|
||||
|
||||
@ -34,20 +41,23 @@ Block InterpreterDescribeQuery::getSampleBlock(bool include_subcolumns)
|
||||
col.name = "type";
|
||||
block.insert(col);
|
||||
|
||||
col.name = "default_type";
|
||||
block.insert(col);
|
||||
if (!compact)
|
||||
{
|
||||
col.name = "default_type";
|
||||
block.insert(col);
|
||||
|
||||
col.name = "default_expression";
|
||||
block.insert(col);
|
||||
col.name = "default_expression";
|
||||
block.insert(col);
|
||||
|
||||
col.name = "comment";
|
||||
block.insert(col);
|
||||
col.name = "comment";
|
||||
block.insert(col);
|
||||
|
||||
col.name = "codec_expression";
|
||||
block.insert(col);
|
||||
col.name = "codec_expression";
|
||||
block.insert(col);
|
||||
|
||||
col.name = "ttl_expression";
|
||||
block.insert(col);
|
||||
col.name = "ttl_expression";
|
||||
block.insert(col);
|
||||
}
|
||||
|
||||
if (include_subcolumns)
|
||||
{
|
||||
@ -57,141 +67,49 @@ Block InterpreterDescribeQuery::getSampleBlock(bool include_subcolumns)
|
||||
block.insert(col);
|
||||
}
|
||||
|
||||
if (include_virtuals)
|
||||
{
|
||||
col.name = "is_virtual";
|
||||
col.type = std::make_shared<DataTypeUInt8>();
|
||||
col.column = col.type->createColumn();
|
||||
block.insert(col);
|
||||
}
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
BlockIO InterpreterDescribeQuery::execute()
|
||||
{
|
||||
std::vector<ColumnDescription> columns;
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
|
||||
const auto & ast = query_ptr->as<ASTDescribeQuery &>();
|
||||
const auto & table_expression = ast.table_expression->as<ASTTableExpression &>();
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
|
||||
if (table_expression.subquery)
|
||||
{
|
||||
NamesAndTypesList names_and_types;
|
||||
auto select_query = table_expression.subquery->children.at(0);
|
||||
auto current_context = getContext();
|
||||
|
||||
if (settings.allow_experimental_analyzer)
|
||||
{
|
||||
SelectQueryOptions select_query_options;
|
||||
names_and_types = InterpreterSelectQueryAnalyzer(select_query, current_context, select_query_options).getSampleBlock().getNamesAndTypesList();
|
||||
}
|
||||
else
|
||||
{
|
||||
names_and_types = InterpreterSelectWithUnionQuery::getSampleBlock(select_query, current_context).getNamesAndTypesList();
|
||||
}
|
||||
|
||||
for (auto && [name, type] : names_and_types)
|
||||
{
|
||||
ColumnDescription description;
|
||||
description.name = std::move(name);
|
||||
description.type = std::move(type);
|
||||
columns.emplace_back(std::move(description));
|
||||
}
|
||||
}
|
||||
fillColumnsFromSubquery(table_expression);
|
||||
else if (table_expression.table_function)
|
||||
{
|
||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression.table_function, getContext());
|
||||
auto table_function_column_descriptions = table_function_ptr->getActualTableStructure(getContext(), /*is_insert_query*/ true);
|
||||
for (const auto & table_function_column_description : table_function_column_descriptions)
|
||||
columns.emplace_back(table_function_column_description);
|
||||
}
|
||||
fillColumnsFromTableFunction(table_expression);
|
||||
else
|
||||
{
|
||||
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);
|
||||
fillColumnsFromTable(table_expression);
|
||||
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
storage_snapshot = table->getStorageSnapshot(metadata_snapshot, getContext());
|
||||
auto metadata_column_descriptions = metadata_snapshot->getColumns();
|
||||
for (const auto & metadata_column_description : metadata_column_descriptions)
|
||||
columns.emplace_back(metadata_column_description);
|
||||
}
|
||||
Block sample_block = getSampleBlock(
|
||||
settings.describe_include_subcolumns,
|
||||
settings.describe_include_virtual_columns,
|
||||
settings.describe_compact_output);
|
||||
|
||||
bool extend_object_types = settings.describe_extend_object_types && storage_snapshot;
|
||||
bool include_subcolumns = settings.describe_include_subcolumns;
|
||||
|
||||
Block sample_block = getSampleBlock(include_subcolumns);
|
||||
MutableColumns res_columns = sample_block.cloneEmptyColumns();
|
||||
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
res_columns[0]->insert(column.name);
|
||||
addColumn(column, false, res_columns);
|
||||
|
||||
DataTypePtr type;
|
||||
if (extend_object_types)
|
||||
type = storage_snapshot->getConcreteType(column.name);
|
||||
else
|
||||
type = column.type;
|
||||
for (const auto & column : virtual_columns)
|
||||
addColumn(column, true, res_columns);
|
||||
|
||||
if (getContext()->getSettingsRef().print_pretty_type_names)
|
||||
res_columns[1]->insert(type->getPrettyName());
|
||||
else
|
||||
res_columns[1]->insert(type->getName());
|
||||
|
||||
if (column.default_desc.expression)
|
||||
{
|
||||
res_columns[2]->insert(toString(column.default_desc.kind));
|
||||
res_columns[3]->insert(queryToString(column.default_desc.expression));
|
||||
}
|
||||
else
|
||||
{
|
||||
res_columns[2]->insertDefault();
|
||||
res_columns[3]->insertDefault();
|
||||
}
|
||||
|
||||
res_columns[4]->insert(column.comment);
|
||||
|
||||
if (column.codec)
|
||||
res_columns[5]->insert(queryToString(column.codec->as<ASTFunction>()->arguments));
|
||||
else
|
||||
res_columns[5]->insertDefault();
|
||||
|
||||
if (column.ttl)
|
||||
res_columns[6]->insert(queryToString(column.ttl));
|
||||
else
|
||||
res_columns[6]->insertDefault();
|
||||
|
||||
if (include_subcolumns)
|
||||
res_columns[7]->insertDefault();
|
||||
}
|
||||
|
||||
if (include_subcolumns)
|
||||
if (settings.describe_include_subcolumns)
|
||||
{
|
||||
for (const auto & column : columns)
|
||||
{
|
||||
auto type = extend_object_types ? storage_snapshot->getConcreteType(column.name) : column.type;
|
||||
addSubcolumns(column, false, res_columns);
|
||||
|
||||
IDataType::forEachSubcolumn([&](const auto & path, const auto & name, const auto & data)
|
||||
{
|
||||
res_columns[0]->insert(Nested::concatenateName(column.name, name));
|
||||
res_columns[1]->insert(data.type->getName());
|
||||
|
||||
/// It's not trivial to calculate default expression for subcolumn.
|
||||
/// So, leave it empty.
|
||||
res_columns[2]->insertDefault();
|
||||
res_columns[3]->insertDefault();
|
||||
res_columns[4]->insert(column.comment);
|
||||
|
||||
if (column.codec && ISerialization::isSpecialCompressionAllowed(path))
|
||||
res_columns[5]->insert(queryToString(column.codec->as<ASTFunction>()->arguments));
|
||||
else
|
||||
res_columns[5]->insertDefault();
|
||||
|
||||
if (column.ttl)
|
||||
res_columns[6]->insert(queryToString(column.ttl));
|
||||
else
|
||||
res_columns[6]->insertDefault();
|
||||
|
||||
res_columns[7]->insert(1u);
|
||||
}, ISerialization::SubstreamData(type->getDefaultSerialization()).withType(type));
|
||||
}
|
||||
for (const auto & column : virtual_columns)
|
||||
addSubcolumns(column, true, res_columns);
|
||||
}
|
||||
|
||||
BlockIO res;
|
||||
@ -202,4 +120,155 @@ BlockIO InterpreterDescribeQuery::execute()
|
||||
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)
|
||||
{
|
||||
SelectQueryOptions select_query_options;
|
||||
names_and_types = InterpreterSelectQueryAnalyzer(select_query, current_context, select_query_options).getSampleBlock().getNamesAndTypesList();
|
||||
}
|
||||
else
|
||||
{
|
||||
names_and_types = InterpreterSelectWithUnionQuery::getSampleBlock(select_query, current_context).getNamesAndTypesList();
|
||||
}
|
||||
|
||||
for (auto && [name, type] : names_and_types)
|
||||
columns.emplace_back(std::move(name), std::move(type));
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
if (settings.describe_include_virtual_columns)
|
||||
{
|
||||
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);
|
||||
|
||||
auto metadata_snapshot = table->getInMemoryMetadataPtr();
|
||||
const auto & column_descriptions = metadata_snapshot->getColumns();
|
||||
for (const auto & column : column_descriptions)
|
||||
columns.emplace_back(column);
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
if (column.default_desc.expression)
|
||||
{
|
||||
res_columns[i++]->insert(toString(column.default_desc.kind));
|
||||
res_columns[i++]->insert(queryToString(column.default_desc.expression));
|
||||
}
|
||||
else
|
||||
{
|
||||
res_columns[i++]->insertDefault();
|
||||
res_columns[i++]->insertDefault();
|
||||
}
|
||||
|
||||
res_columns[i++]->insert(column.comment);
|
||||
|
||||
if (column.codec)
|
||||
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();
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
size_t i = 0;
|
||||
res_columns[i++]->insert(Nested::concatenateName(column.name, name));
|
||||
|
||||
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();
|
||||
}
|
||||
|
||||
res_columns[i++]->insert(1U);
|
||||
|
||||
if (settings.describe_include_virtual_columns)
|
||||
res_columns[i++]->insert(is_virtual);
|
||||
|
||||
}, ISerialization::SubstreamData(type->getDefaultSerialization()).withType(type));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,25 +1,41 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/IInterpreter.h>
|
||||
#include <Storages/ColumnsDescription.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct ASTTableExpression;
|
||||
|
||||
/** Return names, types and other information about columns in specified table.
|
||||
*/
|
||||
class InterpreterDescribeQuery : public IInterpreter, WithContext
|
||||
{
|
||||
public:
|
||||
InterpreterDescribeQuery(const ASTPtr & query_ptr_, ContextPtr context_) : WithContext(context_), query_ptr(query_ptr_) {}
|
||||
InterpreterDescribeQuery(const ASTPtr & query_ptr_, ContextPtr context_);
|
||||
|
||||
BlockIO execute() override;
|
||||
|
||||
static Block getSampleBlock(bool include_subcolumns);
|
||||
static Block getSampleBlock(bool include_subcolumns, bool include_virtuals, bool compact);
|
||||
|
||||
private:
|
||||
void fillColumnsFromSubquery(const ASTTableExpression & table_expression);
|
||||
void fillColumnsFromTableFunction(const ASTTableExpression & table_expression);
|
||||
void fillColumnsFromTable(const ASTTableExpression & table_expression);
|
||||
|
||||
void addColumn(const ColumnDescription & column, bool is_virtual, MutableColumns & res_columns);
|
||||
void addSubcolumns(const ColumnDescription & column, bool is_virtual, MutableColumns & res_columns);
|
||||
|
||||
ASTPtr query_ptr;
|
||||
const Settings & settings;
|
||||
|
||||
std::vector<ColumnDescription> columns;
|
||||
std::vector<ColumnDescription> virtual_columns;
|
||||
StorageSnapshotPtr storage_snapshot;
|
||||
};
|
||||
|
||||
|
||||
|
206
tests/queries/0_stateless/02890_describe_table_options.reference
Normal file
206
tests/queries/0_stateless/02890_describe_table_options.reference
Normal file
@ -0,0 +1,206 @@
|
||||
-- { echoOn }
|
||||
|
||||
SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 0;
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐
|
||||
│ id │ UInt64 │ │ │ index column │ │ │
|
||||
│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │
|
||||
│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │
|
||||
└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
┌─name─┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┐
|
||||
│ id │ UInt64 │ │ │ index column │ │ │
|
||||
│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │
|
||||
│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │
|
||||
└──────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┘
|
||||
SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 1;
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐
|
||||
│ id │ UInt64 │ │ │ index column │ │ │ 0 │
|
||||
│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │
|
||||
│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │
|
||||
│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │
|
||||
│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │
|
||||
└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
┌─name──────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┐
|
||||
│ id │ UInt64 │ │ │ index column │ │ │ 0 │
|
||||
│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │
|
||||
│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │
|
||||
│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │
|
||||
│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │
|
||||
└───────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┘
|
||||
SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 0;
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
┌─name─────────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐
|
||||
│ id │ UInt64 │ │ │ index column │ │ │ 0 │
|
||||
│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │
|
||||
│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │
|
||||
│ _part_index │ UInt64 │ │ │ │ │ │ 1 │
|
||||
│ _part_uuid │ UUID │ │ │ │ │ │ 1 │
|
||||
│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │
|
||||
│ _partition_value │ UInt8 │ │ │ │ │ │ 1 │
|
||||
│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │
|
||||
│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │
|
||||
│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │
|
||||
│ _block_number │ UInt64 │ │ │ │ │ │ 1 │
|
||||
└──────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_virtual─┐
|
||||
│ id │ UInt64 │ │ │ index column │ │ │ 0 │
|
||||
│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │
|
||||
│ _table │ LowCardinality(String) │ │ │ │ │ │ 1 │
|
||||
│ _part │ LowCardinality(String) │ │ │ │ │ │ 1 │
|
||||
│ _part_index │ UInt64 │ │ │ │ │ │ 1 │
|
||||
│ _part_uuid │ UUID │ │ │ │ │ │ 1 │
|
||||
│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 1 │
|
||||
│ _sample_factor │ Float64 │ │ │ │ │ │ 1 │
|
||||
│ _part_offset │ UInt64 │ │ │ │ │ │ 1 │
|
||||
│ _row_exists │ UInt8 │ │ │ │ │ │ 1 │
|
||||
│ _block_number │ UInt64 │ │ │ │ │ │ 1 │
|
||||
│ _shard_num │ UInt32 │ │ │ │ │ │ 1 │
|
||||
└────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴────────────┘
|
||||
SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1;
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
┌─name─────────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐
|
||||
│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │
|
||||
│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │
|
||||
│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _partition_value │ UInt8 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │
|
||||
│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │
|
||||
│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │
|
||||
└──────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
┌─name───────────┬─type──────────────────────┬─default_type─┬─default_expression─┬─comment──────┬─codec_expression─┬─ttl_expression─┬─is_subcolumn─┬─is_virtual─┐
|
||||
│ id │ UInt64 │ │ │ index column │ │ │ 0 │ 0 │
|
||||
│ arr │ Array(UInt64) │ DEFAULT │ [10, 20] │ │ ZSTD(1) │ │ 0 │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ DEFAULT │ ('foo', 0) │ │ ZSTD(1) │ │ 0 │ 0 │
|
||||
│ _table │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _part │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _part_index │ UInt64 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _part_uuid │ UUID │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _partition_id │ LowCardinality(String) │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _sample_factor │ Float64 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _part_offset │ UInt64 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _row_exists │ UInt8 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _block_number │ UInt64 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ _shard_num │ UInt32 │ │ │ │ │ │ 0 │ 1 │
|
||||
│ arr.size0 │ UInt64 │ │ │ │ │ │ 1 │ 0 │
|
||||
│ t.a │ String │ │ │ │ ZSTD(1) │ │ 1 │ 0 │
|
||||
│ t.b │ UInt64 │ │ │ │ ZSTD(1) │ │ 1 │ 0 │
|
||||
└────────────────┴───────────────────────────┴──────────────┴────────────────────┴──────────────┴──────────────────┴────────────────┴──────────────┴────────────┘
|
||||
SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 0;
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
┌─name─┬─type──────────────────────┐
|
||||
│ id │ UInt64 │
|
||||
│ arr │ Array(UInt64) │
|
||||
│ t │ Tuple(a String, b UInt64) │
|
||||
└──────┴───────────────────────────┘
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
┌─name─┬─type──────────────────────┐
|
||||
│ id │ UInt64 │
|
||||
│ arr │ Array(UInt64) │
|
||||
│ t │ Tuple(a String, b UInt64) │
|
||||
└──────┴───────────────────────────┘
|
||||
SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 1;
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
┌─name──────┬─type──────────────────────┬─is_subcolumn─┐
|
||||
│ id │ UInt64 │ 0 │
|
||||
│ arr │ Array(UInt64) │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ 0 │
|
||||
│ arr.size0 │ UInt64 │ 1 │
|
||||
│ t.a │ String │ 1 │
|
||||
│ t.b │ UInt64 │ 1 │
|
||||
└───────────┴───────────────────────────┴──────────────┘
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
┌─name──────┬─type──────────────────────┬─is_subcolumn─┐
|
||||
│ id │ UInt64 │ 0 │
|
||||
│ arr │ Array(UInt64) │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ 0 │
|
||||
│ arr.size0 │ UInt64 │ 1 │
|
||||
│ t.a │ String │ 1 │
|
||||
│ t.b │ UInt64 │ 1 │
|
||||
└───────────┴───────────────────────────┴──────────────┘
|
||||
SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 0;
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
┌─name─────────────┬─type──────────────────────┬─is_virtual─┐
|
||||
│ id │ UInt64 │ 0 │
|
||||
│ arr │ Array(UInt64) │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ 0 │
|
||||
│ _part │ LowCardinality(String) │ 1 │
|
||||
│ _part_index │ UInt64 │ 1 │
|
||||
│ _part_uuid │ UUID │ 1 │
|
||||
│ _partition_id │ LowCardinality(String) │ 1 │
|
||||
│ _partition_value │ UInt8 │ 1 │
|
||||
│ _sample_factor │ Float64 │ 1 │
|
||||
│ _part_offset │ UInt64 │ 1 │
|
||||
│ _row_exists │ UInt8 │ 1 │
|
||||
│ _block_number │ UInt64 │ 1 │
|
||||
└──────────────────┴───────────────────────────┴────────────┘
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
┌─name───────────┬─type──────────────────────┬─is_virtual─┐
|
||||
│ id │ UInt64 │ 0 │
|
||||
│ arr │ Array(UInt64) │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ 0 │
|
||||
│ _table │ LowCardinality(String) │ 1 │
|
||||
│ _part │ LowCardinality(String) │ 1 │
|
||||
│ _part_index │ UInt64 │ 1 │
|
||||
│ _part_uuid │ UUID │ 1 │
|
||||
│ _partition_id │ LowCardinality(String) │ 1 │
|
||||
│ _sample_factor │ Float64 │ 1 │
|
||||
│ _part_offset │ UInt64 │ 1 │
|
||||
│ _row_exists │ UInt8 │ 1 │
|
||||
│ _block_number │ UInt64 │ 1 │
|
||||
│ _shard_num │ UInt32 │ 1 │
|
||||
└────────────────┴───────────────────────────┴────────────┘
|
||||
SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1;
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
┌─name─────────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐
|
||||
│ id │ UInt64 │ 0 │ 0 │
|
||||
│ arr │ Array(UInt64) │ 0 │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │
|
||||
│ _part │ LowCardinality(String) │ 0 │ 1 │
|
||||
│ _part_index │ UInt64 │ 0 │ 1 │
|
||||
│ _part_uuid │ UUID │ 0 │ 1 │
|
||||
│ _partition_id │ LowCardinality(String) │ 0 │ 1 │
|
||||
│ _partition_value │ UInt8 │ 0 │ 1 │
|
||||
│ _sample_factor │ Float64 │ 0 │ 1 │
|
||||
│ _part_offset │ UInt64 │ 0 │ 1 │
|
||||
│ _row_exists │ UInt8 │ 0 │ 1 │
|
||||
│ _block_number │ UInt64 │ 0 │ 1 │
|
||||
│ arr.size0 │ UInt64 │ 1 │ 0 │
|
||||
│ t.a │ String │ 1 │ 0 │
|
||||
│ t.b │ UInt64 │ 1 │ 0 │
|
||||
└──────────────────┴───────────────────────────┴──────────────┴────────────┘
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
┌─name───────────┬─type──────────────────────┬─is_subcolumn─┬─is_virtual─┐
|
||||
│ id │ UInt64 │ 0 │ 0 │
|
||||
│ arr │ Array(UInt64) │ 0 │ 0 │
|
||||
│ t │ Tuple(a String, b UInt64) │ 0 │ 0 │
|
||||
│ _table │ LowCardinality(String) │ 0 │ 1 │
|
||||
│ _part │ LowCardinality(String) │ 0 │ 1 │
|
||||
│ _part_index │ UInt64 │ 0 │ 1 │
|
||||
│ _part_uuid │ UUID │ 0 │ 1 │
|
||||
│ _partition_id │ LowCardinality(String) │ 0 │ 1 │
|
||||
│ _sample_factor │ Float64 │ 0 │ 1 │
|
||||
│ _part_offset │ UInt64 │ 0 │ 1 │
|
||||
│ _row_exists │ UInt8 │ 0 │ 1 │
|
||||
│ _block_number │ UInt64 │ 0 │ 1 │
|
||||
│ _shard_num │ UInt32 │ 0 │ 1 │
|
||||
│ arr.size0 │ UInt64 │ 1 │ 0 │
|
||||
│ t.a │ String │ 1 │ 0 │
|
||||
│ t.b │ UInt64 │ 1 │ 0 │
|
||||
└────────────────┴───────────────────────────┴──────────────┴────────────┘
|
54
tests/queries/0_stateless/02890_describe_table_options.sql
Normal file
54
tests/queries/0_stateless/02890_describe_table_options.sql
Normal file
@ -0,0 +1,54 @@
|
||||
DROP TABLE IF EXISTS t_describe_options;
|
||||
|
||||
CREATE TABLE t_describe_options (
|
||||
id UInt64 COMMENT 'index column',
|
||||
arr Array(UInt64) DEFAULT [10, 20] CODEC(ZSTD),
|
||||
t Tuple(a String, b UInt64) DEFAULT ('foo', 0) CODEC(ZSTD))
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id;
|
||||
|
||||
-- { echoOn }
|
||||
|
||||
SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 0;
|
||||
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SET describe_compact_output = 0, describe_include_virtual_columns = 0, describe_include_subcolumns = 1;
|
||||
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 0;
|
||||
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SET describe_compact_output = 0, describe_include_virtual_columns = 1, describe_include_subcolumns = 1;
|
||||
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 0;
|
||||
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SET describe_compact_output = 1, describe_include_virtual_columns = 0, describe_include_subcolumns = 1;
|
||||
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 0;
|
||||
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
SET describe_compact_output = 1, describe_include_virtual_columns = 1, describe_include_subcolumns = 1;
|
||||
|
||||
DESCRIBE TABLE t_describe_options FORMAT PrettyCompactNoEscapes;
|
||||
DESCRIBE remote(default, currentDatabase(), t_describe_options) FORMAT PrettyCompactNoEscapes;
|
||||
|
||||
-- { echoOff }
|
||||
|
||||
DROP TABLE t_describe_options;
|
Loading…
Reference in New Issue
Block a user