diff --git a/docs/en/engines/table-engines/special/merge.md b/docs/en/engines/table-engines/special/merge.md index 1149711902de..299b1a4a458b 100644 --- a/docs/en/engines/table-engines/special/merge.md +++ b/docs/en/engines/table-engines/special/merge.md @@ -16,7 +16,7 @@ Reading is automatically parallelized. Writing to a table is not supported. When ## Creating a Table {#creating-a-table} ``` sql -CREATE TABLE ... Engine=Merge(db_name, tables_regexp) +CREATE TABLE ... Engine=Merge(db_name, tables_regexp [, table_to_write]) ``` ## Engine Parameters {#engine-parameters} @@ -35,6 +35,14 @@ CREATE TABLE ... Engine=Merge(db_name, tables_regexp) Regular expressions — [re2](https://github.com/google/re2) (supports a subset of PCRE), case-sensitive. See the notes about escaping symbols in regular expressions in the "match" section. +### table_to_write {#table_to_write} + +`table_to_write` - Table name to write during inserts into `Merge` table. +Possible values: + - `'db_name.table_name'` - insert into the specific table in the specific database. + - `'table_name'` - insert into table `db_name.table_name`. Allowed only when the first parameter `db_name` is not a regular expression. + - `auto` - insert into the last table passed to `tables_regexp` in lexicographical order. Allowed only when the first parameter `db_name` is not a regular expression. + ## Usage {#usage} When selecting tables to read, the `Merge` table itself is not selected, even if it matches the regex. This is to avoid loops. @@ -65,7 +73,7 @@ CREATE TABLE WatchLog_new(date Date, UserId Int64, EventType String, Cnt UInt64) ENGINE=MergeTree PARTITION BY date ORDER BY (UserId, EventType) SETTINGS index_granularity=8192; INSERT INTO WatchLog_new VALUES ('2018-01-02', 2, 'hit', 3); -CREATE TABLE WatchLog as WatchLog_old ENGINE=Merge(currentDatabase(), '^WatchLog'); +CREATE TABLE WatchLog as WatchLog_old ENGINE=Merge(currentDatabase(), '^WatchLog', 'WatchLog_new'); SELECT * FROM WatchLog; ``` @@ -79,6 +87,22 @@ SELECT * FROM WatchLog; └────────────┴────────┴───────────┴─────┘ ``` +Insert to table `WatchLog` is going into table `WatchLog_new` +```sql +INSERT INTO WatchLog VALUES ('2018-01-03', 3, 'hit', 3); + +SELECT * FROM WatchLog_New; +``` + +```text +┌───────date─┬─UserId─┬─EventType─┬─Cnt─┐ +│ 2018-01-02 │ 2 │ hit │ 3 │ +└────────────┴────────┴───────────┴─────┘ +┌───────date─┬─UserId─┬─EventType─┬─Cnt─┐ +│ 2018-01-03 │ 3 │ hit │ 3 │ +└────────────┴────────┴───────────┴─────┘ +``` + ## Virtual Columns {#virtual-columns} - `_table` — Contains the name of the table from which data was read. Type: [String](../../../sql-reference/data-types/string.md). diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index a527296a7267..fbfc9b89f1bc 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -41,6 +41,7 @@ #include #include #include +#include #include #include #include @@ -83,6 +84,9 @@ extern const int SAMPLING_NOT_SUPPORTED; extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; extern const int STORAGE_REQUIRES_PARAMETER; +extern const int UNKNOWN_TABLE; +extern const int ACCESS_DENIED; +extern const int TABLE_IS_READ_ONLY; } namespace @@ -143,6 +147,8 @@ StorageMerge::StorageMerge( const String & source_database_name_or_regexp_, bool database_is_regexp_, const DBToTableSetMap & source_databases_and_tables_, + const std::optional & table_to_write_, + bool table_to_write_auto_, ContextPtr context_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) @@ -151,6 +157,7 @@ StorageMerge::StorageMerge( database_is_regexp_, source_database_name_or_regexp_, {}, source_databases_and_tables_) + , table_to_write_auto(table_to_write_auto_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_.empty() @@ -159,6 +166,8 @@ StorageMerge::StorageMerge( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); setVirtuals(createVirtuals()); + if (!table_to_write_auto) + setTableToWrite(table_to_write_, source_database_name_or_regexp_, database_is_regexp_); } StorageMerge::StorageMerge( @@ -168,6 +177,8 @@ StorageMerge::StorageMerge( const String & source_database_name_or_regexp_, bool database_is_regexp_, const String & source_table_regexp_, + const std::optional & table_to_write_, + bool table_to_write_auto_, ContextPtr context_) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) @@ -176,6 +187,7 @@ StorageMerge::StorageMerge( database_is_regexp_, source_database_name_or_regexp_, source_table_regexp_, {}) + , table_to_write_auto(table_to_write_auto_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_.empty() @@ -184,6 +196,8 @@ StorageMerge::StorageMerge( storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); setVirtuals(createVirtuals()); + if (!table_to_write_auto) + setTableToWrite(table_to_write_, source_database_name_or_regexp_, database_is_regexp_); } StorageMerge::DatabaseTablesIterators StorageMerge::getDatabaseIterators(ContextPtr context_) const @@ -293,6 +307,29 @@ void StorageMerge::forEachTable(F && func) const }); } +template +void StorageMerge::forEachTableName(F && func) const +{ + auto database_table_iterators = database_name_or_regexp.getDatabaseIterators(getContext()); + + for (auto & iterator : database_table_iterators) + { + while (iterator->isValid()) + { + const auto & table = iterator->table(); + if (table.get() != this) + { + QualifiedTableName table_name; + table_name.database = iterator->databaseName(); + table_name.table = iterator->name(); + func(table_name); + } + + iterator->next(); + } + } +} + bool StorageMerge::isRemote() const { auto first_remote_table = traverseTablesUntil([](const StoragePtr & table) { return table && table->isRemote(); }); @@ -1702,6 +1739,77 @@ std::optional StorageMerge::totalRowsOrBytes(F && func) const return first_table ? std::nullopt : std::make_optional(total_rows_or_bytes); } +void StorageMerge::setTableToWrite( + const std::optional & table_to_write_, + const String & source_database_name_or_regexp_, + bool database_is_regexp_) +{ + if (!table_to_write_.has_value()) + { + table_to_write = std::nullopt; + return; + } + + auto qualified_name = QualifiedTableName::parseFromString(*table_to_write_); + + if (qualified_name.database.empty()) + { + if (database_is_regexp_) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Argument 'table_to_write' must contain database if 'db_name' is regular expression"); + + qualified_name.database = source_database_name_or_regexp_; + } + + table_to_write = qualified_name; +} + +SinkToStoragePtr StorageMerge::write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context_, + bool async_insert) +{ + const auto & access = context_->getAccess(); + + if (table_to_write_auto) + { + table_to_write = std::nullopt; + bool any_table_found = false; + forEachTableName([&](const auto & table_name) + { + any_table_found = true; + if (!table_to_write.has_value() || table_to_write->getFullName() < table_name.getFullName()) + { + if (access->isGranted(AccessType::INSERT, table_name.database, table_name.table)) + table_to_write = table_name; + } + }); + if (!table_to_write.has_value()) + { + if (any_table_found) + throw Exception(ErrorCodes::ACCESS_DENIED, "Not allowed to write in any suitable table for storage {}", getName()); + else + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Can't find any table to write for storage {}", getName()); + } + } + else + { + if (!table_to_write.has_value()) + throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Method write is not allowed in storage {} without described table to write", getName()); + + access->checkAccess(AccessType::INSERT, table_to_write->database, table_to_write->table); + } + + auto database = DatabaseCatalog::instance().getDatabase(table_to_write->database); + auto table = database->getTable(table_to_write->table, context_); + auto table_lock = table->lockForShare( + context_->getInitialQueryId(), + context_->getSettingsRef()[Setting::lock_acquire_timeout]); + auto sink = table->write(query, metadata_snapshot, context_, async_insert); + sink->addTableLock(table_lock); + return sink; +} + void registerStorageMerge(StorageFactory & factory) { factory.registerStorage("Merge", [](const StorageFactory::Arguments & args) @@ -1712,10 +1820,12 @@ void registerStorageMerge(StorageFactory & factory) ASTs & engine_args = args.engine_args; - if (engine_args.size() != 2) + size_t size = engine_args.size(); + + if (size < 2 || size > 3) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage Merge requires exactly 2 parameters - name " - "of source database and regexp for table names."); + "Storage Merge requires 2 or 3 parameters - name " + "of source database, regexp for table names, and optional table name for writing"); auto [is_regexp, database_ast] = StorageMerge::evaluateDatabaseName(engine_args[0], args.getLocalContext()); @@ -1727,8 +1837,24 @@ void registerStorageMerge(StorageFactory & factory) engine_args[1] = evaluateConstantExpressionAsLiteral(engine_args[1], args.getLocalContext()); String table_name_regexp = checkAndGetLiteralArgument(engine_args[1], "table_name_regexp"); + std::optional table_to_write = std::nullopt; + bool table_to_write_auto = false; + if (size == 3) + { + bool is_identifier = engine_args[2]->as(); + engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.getLocalContext()); + table_to_write = checkAndGetLiteralArgument(engine_args[2], "table_to_write"); + if (is_identifier && table_to_write == "auto") + { + if (is_regexp) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "RegExp for database with auto table_to_write is forbidden"); + table_to_write_auto = true; + } + } + return std::make_shared( - args.table_id, args.columns, args.comment, source_database_name_or_regexp, is_regexp, table_name_regexp, args.getLocalContext()); + args.table_id, args.columns, args.comment, source_database_name_or_regexp, is_regexp, + table_name_regexp, table_to_write, table_to_write_auto, args.getLocalContext()); }, { .supports_schema_inference = true diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index ac4ae17e006f..ef193dcdacec 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -30,6 +30,8 @@ class StorageMerge final : public IStorage, WithContext const String & source_database_name_or_regexp_, bool database_is_regexp_, const DBToTableSetMap & source_databases_and_tables_, + const std::optional & table_to_write_, + bool table_to_write_auto_, ContextPtr context_); StorageMerge( @@ -39,6 +41,8 @@ class StorageMerge final : public IStorage, WithContext const String & source_database_name_or_regexp_, bool database_is_regexp_, const String & source_table_regexp_, + const std::optional & table_to_write_, + bool table_to_write_auto_, ContextPtr context_); std::string getName() const override { return "Merge"; } @@ -70,6 +74,12 @@ class StorageMerge final : public IStorage, WithContext size_t max_block_size, size_t num_streams) override; + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + bool async_insert) override; + void checkAlterIsPossible(const AlterCommands & commands, ContextPtr context) const override; /// you need to add and remove columns in the sub-tables manually @@ -122,12 +132,18 @@ class StorageMerge final : public IStorage, WithContext DatabaseNameOrRegexp database_name_or_regexp; + std::optional table_to_write; + bool table_to_write_auto = false; + template StoragePtr traverseTablesUntil(F && predicate) const; template void forEachTable(F && func) const; + template + void forEachTableName(F && func) const; + template static StoragePtr traverseTablesUntilImpl(const ContextPtr & query_context, const IStorage * ignore_self, const DatabaseNameOrRegexp & database_name_or_regexp, F && predicate); @@ -149,6 +165,11 @@ class StorageMerge final : public IStorage, WithContext template std::optional totalRowsOrBytes(F && func) const; + void setTableToWrite( + const std::optional & table_to_write_, + const String & source_database_name_or_regexp_, + bool database_is_regexp_); + friend class ReadFromMerge; }; diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 396e96e94b51..443a31693ff5 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -141,6 +141,7 @@ ColumnsDescription TableFunctionMerge::getActualTableStructure(ContextPtr contex StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const { + std::optional table_to_write = std::nullopt; auto res = std::make_shared( StorageID(getDatabaseName(), table_name), ColumnsDescription{}, @@ -148,6 +149,8 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & /*ast_function*/, Cont source_database_name_or_regexp, database_is_regexp, source_table_regexp, + table_to_write, + false, context); res->startup(); diff --git a/tests/queries/0_stateless/03373_write_to_merge_table.reference b/tests/queries/0_stateless/03373_write_to_merge_table.reference new file mode 100644 index 000000000000..0ecbe836033f --- /dev/null +++ b/tests/queries/0_stateless/03373_write_to_merge_table.reference @@ -0,0 +1,44 @@ +2 1 +2 2 +2 3 +1 1 +2 1 +2 2 +2 3 +1 1 +2 1 +2 2 +2 3 +1 1 +2 1 +2 2 +2 3 +1 1 +2 1 +2 2 +2 3 +1 1 +2 1 +2 2 +2 3 +4 +2 1 +2 2 +2 3 +3 1 +4 +2 1 +2 2 +2 3 +3 1 +1 +3 2 +4 +2 1 +2 2 +2 3 +3 1 +0 +2 +3 2 +3 3 diff --git a/tests/queries/0_stateless/03373_write_to_merge_table.sql b/tests/queries/0_stateless/03373_write_to_merge_table.sql new file mode 100644 index 000000000000..a358c5f2f0de --- /dev/null +++ b/tests/queries/0_stateless/03373_write_to_merge_table.sql @@ -0,0 +1,78 @@ +DROP TABLE IF EXISTS test03373_table_1; +DROP TABLE IF EXISTS test03373_table_2; +DROP TABLE IF EXISTS test03373_table_3; +DROP TABLE IF EXISTS test03373_table_4; +DROP TABLE IF EXISTS test03373_merge_ro; +DROP TABLE IF EXISTS test03373_merge_wr_1; +DROP TABLE IF EXISTS test03373_merge_wr_2; +DROP TABLE IF EXISTS test03373_merge_wr_3; +DROP TABLE IF EXISTS test03373_merge_wr_auto; + +CREATE TABLE test03373_table_1 (key UInt32, value UInt32) ENGINE=MergeTree() ORDER BY key; +CREATE TABLE test03373_table_2 (key UInt32, value UInt32) ENGINE=MergeTree() ORDER BY key; + +CREATE TABLE test03373_merge_ro (key UInt32, value UInt32) ENGINE=Merge({CLICKHOUSE_DATABASE:String}, 'test03373_table_\d+'); + +CREATE TABLE test03373_merge_wr_1 (key UInt32, value UInt32) ENGINE=Merge({CLICKHOUSE_DATABASE:String}, 'test03373_table_\d+', test03373_table_2); +CREATE TABLE test03373_merge_wr_2 (key UInt32, value UInt32) ENGINE=Merge({CLICKHOUSE_DATABASE:String}, 'test03373_table_\d+', {CLICKHOUSE_DATABASE:Identifier}.test03373_table_2); +CREATE TABLE test03373_merge_wr_fail (key UInt32, value UInt32) ENGINE=Merge(REGEXP({CLICKHOUSE_DATABASE:String}), 'test03373_table_\d+', test03373_table_2); -- { serverError BAD_ARGUMENTS } +CREATE TABLE test03373_merge_wr_3 (key UInt32, value UInt32) ENGINE=Merge(REGEXP({CLICKHOUSE_DATABASE:String}), 'test03373_table_\d+', {CLICKHOUSE_DATABASE:Identifier}.test03373_table_2); + +CREATE TABLE test03373_merge_wr_auto (key UInt32, value UInt32) ENGINE=Merge({CLICKHOUSE_DATABASE:String}, 'test03373_table_\d+', auto); +CREATE TABLE test03373_merge_wr_auto_fail (key UInt32, value UInt32) ENGINE=Merge(REGEXP({CLICKHOUSE_DATABASE:String}), 'test03373_table_\d+', auto); -- { serverError BAD_ARGUMENTS } + +INSERT INTO test03373_table_1 VALUES (1,1); + +INSERT INTO test03373_merge_ro VALUES (1,2); -- { serverError TABLE_IS_READ_ONLY } + +INSERT INTO test03373_merge_wr_1 VALUES (2,1); +INSERT INTO test03373_merge_wr_2 VALUES (2,2); +INSERT INTO test03373_merge_wr_3 VALUES (2,3); + +OPTIMIZE TABLE test03373_table_1 FINAL; +OPTIMIZE TABLE test03373_table_2 FINAL; + +SELECT * FROM test03373_table_2 ORDER BY key, value; + +SELECT * FROM test03373_merge_ro ORDER BY key, value; +SELECT * FROM test03373_merge_wr_1 ORDER BY key, value; +SELECT * FROM test03373_merge_wr_2 ORDER BY key, value; +SELECT * FROM test03373_merge_wr_3 ORDER BY key, value; + +SELECT * FROM test03373_merge_wr_auto ORDER BY key, value; + +-- insert into test03373_table_2 +INSERT INTO test03373_merge_wr_auto VALUES (3,1); +OPTIMIZE TABLE test03373_table_2 FINAL; +SELECT count() FROM test03373_table_2; +SELECT * FROM test03373_table_2 ORDER BY key, value; + +CREATE TABLE test03373_table_4 (key UInt32, value UInt32) ENGINE=MergeTree() ORDER BY key; +-- insert into test03373_table_4 +INSERT INTO test03373_merge_wr_auto VALUES (3,2); +OPTIMIZE TABLE test03373_table_4 FINAL; +SELECT count() FROM test03373_table_2; +SELECT * FROM test03373_table_2 ORDER BY key, value; +SELECT count() FROM test03373_table_4; +SELECT * FROM test03373_table_4 ORDER BY key, value; + +CREATE TABLE test03373_table_3 (key UInt32, value UInt32) ENGINE=MergeTree() ORDER BY key; +-- insert into test03373_table_4 +INSERT INTO test03373_merge_wr_auto VALUES (3,3); +OPTIMIZE TABLE test03373_table_4 FINAL; +SELECT count() FROM test03373_table_2; +SELECT * FROM test03373_table_2 ORDER BY key, value; +SELECT count() FROM test03373_table_3; +SELECT * FROM test03373_table_3 ORDER BY key, value; +SELECT count() FROM test03373_table_4; +SELECT * FROM test03373_table_4 ORDER BY key, value; + +DROP TABLE IF EXISTS test03373_table_1; +DROP TABLE IF EXISTS test03373_table_2; +DROP TABLE IF EXISTS test03373_table_3; +DROP TABLE IF EXISTS test03373_table_4; +DROP TABLE IF EXISTS test03373_merge_ro; +DROP TABLE IF EXISTS test03373_merge_wr_1; +DROP TABLE IF EXISTS test03373_merge_wr_2; +DROP TABLE IF EXISTS test03373_merge_wr_3; +DROP TABLE IF EXISTS test03373_merge_wr_auto; diff --git a/tests/queries/0_stateless/03409_write_to_merge_table_grants.reference b/tests/queries/0_stateless/03409_write_to_merge_table_grants.reference new file mode 100644 index 000000000000..c45696d6b5f6 --- /dev/null +++ b/tests/queries/0_stateless/03409_write_to_merge_table_grants.reference @@ -0,0 +1,6 @@ +*** Select from test03409_table_1 *** +10 +*** Select from test03409_table_2 *** +8 +11 +*** Select from test03409_table_3 *** diff --git a/tests/queries/0_stateless/03409_write_to_merge_table_grants.sh b/tests/queries/0_stateless/03409_write_to_merge_table_grants.sh new file mode 100755 index 000000000000..a86bf66508e8 --- /dev/null +++ b/tests/queries/0_stateless/03409_write_to_merge_table_grants.sh @@ -0,0 +1,66 @@ +#!/usr/bin/env bash +# Tag no-parallel: create user + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +CLICKHOUSE_USER="user_03409_$CLICKHOUSE_DATABASE" + +$CLICKHOUSE_CLIENT --multiline -q """ + CREATE TABLE test03409_table_1 (key UInt32) Engine=MergeTree() ORDER BY key; + CREATE TABLE test03409_table_2 (key UInt32) Engine=MergeTree() ORDER BY key; + CREATE TABLE test03409_table_3 (key UInt32) Engine=MergeTree() ORDER BY key; + CREATE TABLE test03409_merge_ro (key UInt32) Engine=Merge($CLICKHOUSE_DATABASE, 'test03409_table_\d+'); + CREATE TABLE test03409_merge_rw (key UInt32) Engine=Merge($CLICKHOUSE_DATABASE, 'test03409_table_\d+', 'test03409_table_1'); + CREATE TABLE test03409_merge_auto (key UInt32) Engine=Merge($CLICKHOUSE_DATABASE, 'test03409_table_\d+', auto); + CREATE TABLE test03409_merge_no_table (key UInt32) Engine=Merge($CLICKHOUSE_DATABASE, 'test03409_notable_\d+', 'test03409_notable_1'); + CREATE TABLE test03409_merge_no_table_auto (key UInt32) Engine=Merge($CLICKHOUSE_DATABASE, 'test03409_notable_\d+', auto); + DROP USER IF EXISTS $CLICKHOUSE_USER; + CREATE USER $CLICKHOUSE_USER IDENTIFIED WITH plaintext_password BY 'user_03409_password'; + GRANT INSERT ON $CLICKHOUSE_DATABASE.test03409_merge_ro TO $CLICKHOUSE_USER; + GRANT INSERT ON $CLICKHOUSE_DATABASE.test03409_merge_rw TO $CLICKHOUSE_USER; + GRANT INSERT ON $CLICKHOUSE_DATABASE.test03409_merge_auto TO $CLICKHOUSE_USER; + GRANT INSERT ON $CLICKHOUSE_DATABASE.test03409_merge_no_table TO $CLICKHOUSE_USER; + GRANT INSERT ON $CLICKHOUSE_DATABASE.test03409_merge_no_table_auto TO $CLICKHOUSE_USER; +""" + +$CLICKHOUSE_CLIENT --multiline --user $CLICKHOUSE_USER --password user_03409_password -q """ + INSERT INTO test03409_merge_ro VALUES (1); -- { serverError TABLE_IS_READ_ONLY } + INSERT INTO test03409_merge_rw VALUES (2); -- { serverError ACCESS_DENIED } + INSERT INTO test03409_merge_auto VALUES (3); -- { serverError ACCESS_DENIED } + INSERT INTO test03409_merge_no_table VALUES (4); -- { serverError ACCESS_DENIED } + INSERT INTO test03409_merge_no_table_auto VALUES (5); -- { serverError UNKNOWN_TABLE } +""" + +$CLICKHOUSE_CLIENT --multiline -q """ + GRANT INSERT ON $CLICKHOUSE_DATABASE.test03409_table_2 TO $CLICKHOUSE_USER; +""" + +$CLICKHOUSE_CLIENT --multiline --user $CLICKHOUSE_USER --password user_03409_password -q """ + INSERT INTO test03409_merge_ro VALUES (6); -- { serverError TABLE_IS_READ_ONLY } + INSERT INTO test03409_merge_rw VALUES (7); -- { serverError ACCESS_DENIED } + INSERT INTO test03409_merge_auto VALUES (8); +""" + +$CLICKHOUSE_CLIENT --multiline -q """ + GRANT INSERT ON $CLICKHOUSE_DATABASE.test03409_table_1 TO $CLICKHOUSE_USER; +""" + +$CLICKHOUSE_CLIENT --multiline --user $CLICKHOUSE_USER --password user_03409_password -q """ + INSERT INTO test03409_merge_ro VALUES (9); -- { serverError TABLE_IS_READ_ONLY } + INSERT INTO test03409_merge_rw VALUES (10); + INSERT INTO test03409_merge_auto VALUES (11); +""" + +$CLICKHOUSE_CLIENT --multiline -q """ + OPTIMIZE TABLE test03409_table_1 FINAL; + OPTIMIZE TABLE test03409_table_2 FINAL; + OPTIMIZE TABLE test03409_table_3 FINAL; + SELECT '*** Select from test03409_table_1 ***'; + SELECT * FROM test03409_table_1 ORDER BY key FORMAT CSV; + SELECT '*** Select from test03409_table_2 ***'; + SELECT * FROM test03409_table_2 ORDER BY key FORMAT CSV; + SELECT '*** Select from test03409_table_3 ***'; + SELECT * FROM test03409_table_3 ORDER BY key FORMAT CSV; +"""