mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge pull request #66832 from joelynch/joelynch/table-comments
Ensure COMMENT clause works for all table engines
This commit is contained in:
commit
6f4aa977d7
@ -21,7 +21,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster]
|
||||
name2 [type2] [NULL|NOT NULL] [DEFAULT|MATERIALIZED|EPHEMERAL|ALIAS expr2] [COMMENT 'comment for column'] [compression_codec] [TTL expr2],
|
||||
...
|
||||
) ENGINE = engine
|
||||
COMMENT 'comment for table'
|
||||
[COMMENT 'comment for table']
|
||||
```
|
||||
|
||||
Creates a table named `table_name` in the `db` database or the current database if `db` is not set, with the structure specified in brackets and the `engine` engine.
|
||||
@ -626,11 +626,6 @@ SELECT * FROM base.t1;
|
||||
|
||||
You can add a comment to the table when you creating it.
|
||||
|
||||
:::note
|
||||
The comment clause is supported by all table engines except [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) and [EmbeddedRocksDB](../../../engines/table-engines/integrations/embedded-rocksdb.md).
|
||||
:::
|
||||
|
||||
|
||||
**Syntax**
|
||||
|
||||
``` sql
|
||||
|
@ -16,6 +16,7 @@ Syntax:
|
||||
CREATE [OR REPLACE] VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster_name]
|
||||
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
|
||||
AS SELECT ...
|
||||
[COMMENT 'comment']
|
||||
```
|
||||
|
||||
Normal views do not store any data. They just perform a read from another table on each access. In other words, a normal view is nothing more than a saved query. When reading from a view, this saved query is used as a subquery in the [FROM](../../../sql-reference/statements/select/from.md) clause.
|
||||
@ -57,6 +58,7 @@ SELECT * FROM view(column1=value1, column2=value2 ...)
|
||||
CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]name] [ENGINE = engine] [POPULATE]
|
||||
[DEFINER = { user | CURRENT_USER }] [SQL SECURITY { DEFINER | INVOKER | NONE }]
|
||||
AS SELECT ...
|
||||
[COMMENT 'comment']
|
||||
```
|
||||
|
||||
:::tip
|
||||
@ -161,6 +163,7 @@ RANDOMIZE FOR interval
|
||||
DEPENDS ON [db.]name [, [db.]name [, ...]]
|
||||
[TO[db.]name] [(columns)] [ENGINE = engine] [EMPTY]
|
||||
AS SELECT ...
|
||||
[COMMENT 'comment']
|
||||
```
|
||||
where `interval` is a sequence of simple intervals:
|
||||
```sql
|
||||
@ -267,7 +270,10 @@ This is an experimental feature that may change in backwards-incompatible ways i
|
||||
:::
|
||||
|
||||
``` sql
|
||||
CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [INNER ENGINE engine] [ENGINE engine] [WATERMARK strategy] [ALLOWED_LATENESS interval_function] [POPULATE] AS SELECT ... GROUP BY time_window_function
|
||||
CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [INNER ENGINE engine] [ENGINE engine] [WATERMARK strategy] [ALLOWED_LATENESS interval_function] [POPULATE]
|
||||
AS SELECT ...
|
||||
GROUP BY time_window_function
|
||||
[COMMENT 'comment']
|
||||
```
|
||||
|
||||
Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table to reduce latency and can push the processing result to a specified table or push notifications using the WATCH query.
|
||||
|
@ -154,6 +154,7 @@ StoragePtr DatabaseSQLite::fetchTable(const String & table_name, ContextPtr loca
|
||||
table_name,
|
||||
ColumnsDescription{*columns},
|
||||
ConstraintsDescription{},
|
||||
/* comment = */ "",
|
||||
local_context);
|
||||
|
||||
return storage;
|
||||
|
@ -1207,6 +1207,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
if (!select_p.parse(pos, select, expected))
|
||||
return false;
|
||||
|
||||
auto comment = parseComment(pos, expected);
|
||||
|
||||
auto query = std::make_shared<ASTCreateQuery>();
|
||||
node = query;
|
||||
@ -1225,6 +1226,8 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
|
||||
query->children.push_back(query->database);
|
||||
if (query->table)
|
||||
query->children.push_back(query->table);
|
||||
if (comment)
|
||||
query->set(query->comment, comment);
|
||||
|
||||
query->set(query->columns_list, columns_list);
|
||||
|
||||
|
@ -418,8 +418,11 @@ namespace
|
||||
}
|
||||
|
||||
StorageKafka::StorageKafka(
|
||||
const StorageID & table_id_, ContextPtr context_,
|
||||
const ColumnsDescription & columns_, std::unique_ptr<KafkaSettings> kafka_settings_,
|
||||
const StorageID & table_id_,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment,
|
||||
std::unique_ptr<KafkaSettings> kafka_settings_,
|
||||
const String & collection_name_)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
@ -451,6 +454,7 @@ StorageKafka::StorageKafka(
|
||||
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
setVirtuals(createVirtuals(kafka_settings->kafka_handle_error_mode));
|
||||
|
||||
@ -1317,7 +1321,7 @@ void registerStorageKafka(StorageFactory & factory)
|
||||
"See https://clickhouse.com/docs/en/engines/table-engines/integrations/kafka/#configuration");
|
||||
}
|
||||
|
||||
return std::make_shared<StorageKafka>(args.table_id, args.getContext(), args.columns, std::move(kafka_settings), collection_name);
|
||||
return std::make_shared<StorageKafka>(args.table_id, args.getContext(), args.columns, args.comment, std::move(kafka_settings), collection_name);
|
||||
};
|
||||
|
||||
factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, });
|
||||
|
@ -40,6 +40,7 @@ public:
|
||||
const StorageID & table_id_,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment,
|
||||
std::unique_ptr<KafkaSettings> kafka_settings_,
|
||||
const String & collection_name_);
|
||||
|
||||
|
@ -49,6 +49,7 @@ StorageNATS::StorageNATS(
|
||||
const StorageID & table_id_,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment,
|
||||
std::unique_ptr<NATSSettings> nats_settings_,
|
||||
LoadingStrictnessLevel mode)
|
||||
: IStorage(table_id_)
|
||||
@ -87,6 +88,7 @@ StorageNATS::StorageNATS(
|
||||
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
setVirtuals(createVirtuals(nats_settings->nats_handle_error_mode));
|
||||
|
||||
@ -760,7 +762,7 @@ void registerStorageNATS(StorageFactory & factory)
|
||||
if (!nats_settings->nats_subjects.changed)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `nats_subjects` setting");
|
||||
|
||||
return std::make_shared<StorageNATS>(args.table_id, args.getContext(), args.columns, std::move(nats_settings), args.mode);
|
||||
return std::make_shared<StorageNATS>(args.table_id, args.getContext(), args.columns, args.comment, std::move(nats_settings), args.mode);
|
||||
};
|
||||
|
||||
factory.registerStorage("NATS", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, });
|
||||
|
@ -23,6 +23,7 @@ public:
|
||||
const StorageID & table_id_,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment,
|
||||
std::unique_ptr<NATSSettings> nats_settings_,
|
||||
LoadingStrictnessLevel mode);
|
||||
|
||||
|
@ -571,6 +571,7 @@ void registerStorageMaterializedPostgreSQL(StorageFactory & factory)
|
||||
StorageInMemoryMetadata metadata;
|
||||
metadata.setColumns(args.columns);
|
||||
metadata.setConstraints(args.constraints);
|
||||
metadata.setComment(args.comment);
|
||||
|
||||
if (args.mode <= LoadingStrictnessLevel::CREATE
|
||||
&& !args.getLocalContext()->getSettingsRef().allow_experimental_materialized_postgresql_table)
|
||||
|
@ -70,6 +70,7 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
const StorageID & table_id_,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment,
|
||||
std::unique_ptr<RabbitMQSettings> rabbitmq_settings_,
|
||||
LoadingStrictnessLevel mode)
|
||||
: IStorage(table_id_)
|
||||
@ -145,6 +146,7 @@ StorageRabbitMQ::StorageRabbitMQ(
|
||||
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
setVirtuals(createVirtuals(rabbitmq_settings->rabbitmq_handle_error_mode));
|
||||
|
||||
@ -1288,7 +1290,7 @@ void registerStorageRabbitMQ(StorageFactory & factory)
|
||||
if (!rabbitmq_settings->rabbitmq_format.changed)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "You must specify `rabbitmq_format` setting");
|
||||
|
||||
return std::make_shared<StorageRabbitMQ>(args.table_id, args.getContext(), args.columns, std::move(rabbitmq_settings), args.mode);
|
||||
return std::make_shared<StorageRabbitMQ>(args.table_id, args.getContext(), args.columns, args.comment, std::move(rabbitmq_settings), args.mode);
|
||||
};
|
||||
|
||||
factory.registerStorage("RabbitMQ", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, });
|
||||
|
@ -26,6 +26,7 @@ public:
|
||||
const StorageID & table_id_,
|
||||
ContextPtr context_,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment,
|
||||
std::unique_ptr<RabbitMQSettings> rabbitmq_settings_,
|
||||
LoadingStrictnessLevel mode);
|
||||
|
||||
|
@ -691,6 +691,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|
||||
StorageInMemoryMetadata metadata;
|
||||
metadata.setColumns(args.columns);
|
||||
metadata.setConstraints(args.constraints);
|
||||
metadata.setComment(args.comment);
|
||||
|
||||
if (!args.storage_def->primary_key)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageEmbeddedRocksDB must require one column in primary key");
|
||||
|
@ -77,7 +77,8 @@ StorageExecutable::StorageExecutable(
|
||||
const ExecutableSettings & settings_,
|
||||
const std::vector<ASTPtr> & input_queries_,
|
||||
const ColumnsDescription & columns,
|
||||
const ConstraintsDescription & constraints)
|
||||
const ConstraintsDescription & constraints,
|
||||
const String & comment)
|
||||
: IStorage(table_id_)
|
||||
, settings(settings_)
|
||||
, input_queries(input_queries_)
|
||||
@ -86,6 +87,7 @@ StorageExecutable::StorageExecutable(
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns);
|
||||
storage_metadata.setConstraints(constraints);
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
|
||||
ShellCommandSourceCoordinator::Configuration configuration
|
||||
@ -237,7 +239,7 @@ void registerStorageExecutable(StorageFactory & factory)
|
||||
settings.loadFromQuery(*args.storage_def);
|
||||
|
||||
auto global_context = args.getContext()->getGlobalContext();
|
||||
return std::make_shared<StorageExecutable>(args.table_id, format, settings, input_queries, columns, constraints);
|
||||
return std::make_shared<StorageExecutable>(args.table_id, format, settings, input_queries, columns, constraints, args.comment);
|
||||
};
|
||||
|
||||
StorageFactory::StorageFeatures storage_features;
|
||||
@ -255,4 +257,3 @@ void registerStorageExecutable(StorageFactory & factory)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -22,7 +22,8 @@ public:
|
||||
const ExecutableSettings & settings,
|
||||
const std::vector<ASTPtr> & input_queries,
|
||||
const ColumnsDescription & columns,
|
||||
const ConstraintsDescription & constraints);
|
||||
const ConstraintsDescription & constraints,
|
||||
const String & comment);
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
|
@ -1289,6 +1289,7 @@ StoragePtr create(const StorageFactory::Arguments & args)
|
||||
StorageInMemoryMetadata metadata;
|
||||
metadata.setColumns(args.columns);
|
||||
metadata.setConstraints(args.constraints);
|
||||
metadata.setComment(args.comment);
|
||||
|
||||
if (!args.storage_def->primary_key)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "StorageKeeperMap requires one column in primary key");
|
||||
|
@ -50,6 +50,7 @@ StorageSQLite::StorageSQLite(
|
||||
const String & remote_table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const String & comment,
|
||||
ContextPtr context_)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
@ -71,6 +72,7 @@ StorageSQLite::StorageSQLite(
|
||||
|
||||
storage_metadata.setConstraints(constraints_);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
storage_metadata.setComment(comment);
|
||||
}
|
||||
|
||||
|
||||
@ -211,7 +213,7 @@ void registerStorageSQLite(StorageFactory & factory)
|
||||
auto sqlite_db = openSQLiteDB(database_path, args.getContext(), /* throw_on_error */ args.mode <= LoadingStrictnessLevel::CREATE);
|
||||
|
||||
return std::make_shared<StorageSQLite>(args.table_id, sqlite_db, database_path,
|
||||
table_name, args.columns, args.constraints, args.getContext());
|
||||
table_name, args.columns, args.constraints, args.comment, args.getContext());
|
||||
},
|
||||
{
|
||||
.supports_schema_inference = true,
|
||||
|
@ -27,6 +27,7 @@ public:
|
||||
const String & remote_table_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
const String & comment,
|
||||
ContextPtr context_);
|
||||
|
||||
std::string getName() const override { return "SQLite"; }
|
||||
|
@ -1188,6 +1188,7 @@ StorageWindowView::StorageWindowView(
|
||||
ContextPtr context_,
|
||||
const ASTCreateQuery & query,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment,
|
||||
LoadingStrictnessLevel mode)
|
||||
: IStorage(table_id_)
|
||||
, WithContext(context_->getGlobalContext())
|
||||
@ -1206,6 +1207,7 @@ StorageWindowView::StorageWindowView(
|
||||
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
storage_metadata.setComment(comment);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
|
||||
/// If the target table is not set, use inner target table
|
||||
@ -1764,7 +1766,7 @@ void registerStorageWindowView(StorageFactory & factory)
|
||||
"Experimental WINDOW VIEW feature "
|
||||
"is not enabled (the setting 'allow_experimental_window_view')");
|
||||
|
||||
return std::make_shared<StorageWindowView>(args.table_id, args.getLocalContext(), args.query, args.columns, args.mode);
|
||||
return std::make_shared<StorageWindowView>(args.table_id, args.getLocalContext(), args.query, args.columns, args.comment, args.mode);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -111,6 +111,7 @@ public:
|
||||
ContextPtr context_,
|
||||
const ASTCreateQuery & query,
|
||||
const ColumnsDescription & columns_,
|
||||
const String & comment,
|
||||
LoadingStrictnessLevel mode);
|
||||
|
||||
String getName() const override { return "WindowView"; }
|
||||
|
@ -170,7 +170,14 @@ StoragePtr TableFunctionExecutable::executeImpl(const ASTPtr & /*ast_function*/,
|
||||
if (settings_query != nullptr)
|
||||
settings.applyChanges(settings_query->as<ASTSetQuery>()->changes);
|
||||
|
||||
auto storage = std::make_shared<StorageExecutable>(storage_id, format, settings, input_queries, getActualTableStructure(context, is_insert_query), ConstraintsDescription{});
|
||||
auto storage = std::make_shared<StorageExecutable>(
|
||||
storage_id,
|
||||
format,
|
||||
settings,
|
||||
input_queries,
|
||||
getActualTableStructure(context, is_insert_query),
|
||||
ConstraintsDescription{},
|
||||
/* comment = */ "");
|
||||
storage->startup();
|
||||
return storage;
|
||||
}
|
||||
|
@ -57,7 +57,7 @@ StoragePtr TableFunctionSQLite::executeImpl(const ASTPtr & /*ast_function*/,
|
||||
sqlite_db,
|
||||
database_path,
|
||||
remote_table_name,
|
||||
cached_columns, ConstraintsDescription{}, context);
|
||||
cached_columns, ConstraintsDescription{}, /* comment = */ "", context);
|
||||
|
||||
storage->startup();
|
||||
return storage;
|
||||
|
@ -1,4 +1,8 @@
|
||||
t1 this is a temtorary table
|
||||
t1 this is a temporary table
|
||||
t2 this is a MergeTree table
|
||||
t3 this is a Log table
|
||||
CREATE TABLE default.t1\n(\n `n` Int8\n)\nENGINE = Memory\nCOMMENT \'this is a temtorary table\'
|
||||
t4 this is a Kafka table
|
||||
t5 this is a EmbeddedRocksDB table
|
||||
t6 this is a Executable table
|
||||
t7 this is a WindowView table
|
||||
CREATE TABLE default.t1\n(\n `n` Int8\n)\nENGINE = Memory\nCOMMENT \'this is a temporary table\'
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-parallel
|
||||
-- Tags: no-parallel, no-fasttest
|
||||
|
||||
DROP TABLE IF EXISTS t1;
|
||||
DROP TABLE IF EXISTS t2;
|
||||
@ -9,7 +9,7 @@ CREATE TABLE t1
|
||||
`n` Int8
|
||||
)
|
||||
ENGINE = Memory
|
||||
COMMENT 'this is a temtorary table';
|
||||
COMMENT 'this is a temporary table';
|
||||
|
||||
CREATE TABLE t2
|
||||
(
|
||||
@ -26,14 +26,57 @@ CREATE TABLE t3
|
||||
ENGINE = Log
|
||||
COMMENT 'this is a Log table';
|
||||
|
||||
CREATE TABLE t4
|
||||
(
|
||||
`n` Int8
|
||||
)
|
||||
ENGINE = Kafka
|
||||
SETTINGS
|
||||
kafka_broker_list = 'localhost:10000',
|
||||
kafka_topic_list = 'test',
|
||||
kafka_group_name = 'test',
|
||||
kafka_format = 'JSONEachRow'
|
||||
COMMENT 'this is a Kafka table';
|
||||
|
||||
CREATE TABLE t5
|
||||
(
|
||||
`n` Int8
|
||||
)
|
||||
ENGINE = EmbeddedRocksDB
|
||||
PRIMARY KEY n
|
||||
COMMENT 'this is a EmbeddedRocksDB table';
|
||||
|
||||
CREATE TABLE t6
|
||||
(
|
||||
`n` Int8
|
||||
)
|
||||
ENGINE = Executable('script.py', TabSeparated)
|
||||
COMMENT 'this is a Executable table';
|
||||
|
||||
SET allow_experimental_window_view = 1;
|
||||
-- New analyzer doesn't support WindowView tables
|
||||
SET allow_experimental_analyzer = 0;
|
||||
|
||||
CREATE WINDOW VIEW t7
|
||||
(
|
||||
`n` Int8
|
||||
)
|
||||
ENGINE MergeTree
|
||||
ORDER BY n
|
||||
AS SELECT 1
|
||||
GROUP BY tumble(now(), toIntervalDay('1'))
|
||||
COMMENT 'this is a WindowView table';
|
||||
|
||||
SET allow_experimental_analyzer = 1;
|
||||
|
||||
SELECT
|
||||
name,
|
||||
comment
|
||||
FROM system.tables
|
||||
WHERE name IN ('t1', 't2', 't3') AND database = currentDatabase() order by name;
|
||||
WHERE name IN ('t1', 't2', 't3', 't4', 't5', 't6', 't7')
|
||||
AND database = currentDatabase() order by name;
|
||||
|
||||
SHOW CREATE TABLE t1;
|
||||
|
||||
DROP TABLE t1;
|
||||
DROP TABLE t2;
|
||||
DROP TABLE t3;
|
||||
DROP TABLE t1, t2, t3, t4, t5, t6;
|
||||
DROP VIEW t7;
|
||||
|
Loading…
Reference in New Issue
Block a user